feat: adds runtime_metrics (#3127)

* feat: adds runtime_metrics

* fix: comment

* feat: refactor metrics table

* chore: ensure build_info and runtime_metrics only avaiable in greptime catalog

* feat: adds timestamp column
This commit is contained in:
dennis zhuang
2024-01-10 18:51:30 +08:00
committed by GitHub
parent 1b0e39a7f2
commit fd3f23ea15
9 changed files with 444 additions and 164 deletions

1
Cargo.lock generated
View File

@@ -1206,6 +1206,7 @@ dependencies = [
"datatypes",
"futures",
"futures-util",
"itertools 0.10.5",
"lazy_static",
"log-store",
"meta-client",

View File

@@ -30,6 +30,7 @@ datafusion.workspace = true
datatypes.workspace = true
futures = "0.3"
futures-util.workspace = true
itertools.workspace = true
lazy_static.workspace = true
meta-client.workspace = true
moka = { workspace = true, features = ["future"] }

View File

@@ -16,6 +16,7 @@ mod columns;
mod key_column_usage;
mod memory_table;
mod predicate;
mod runtime_metrics;
mod schemata;
mod table_names;
mod tables;
@@ -23,7 +24,7 @@ mod tables;
use std::collections::HashMap;
use std::sync::{Arc, Weak};
use common_catalog::consts::{self, INFORMATION_SCHEMA_NAME};
use common_catalog::consts::{self, DEFAULT_CATALOG_NAME, INFORMATION_SCHEMA_NAME};
use common_error::ext::BoxedError;
use common_recordbatch::{RecordBatchStreamWrapper, SendableRecordBatchStream};
use datatypes::schema::SchemaRef;
@@ -46,6 +47,7 @@ use self::columns::InformationSchemaColumns;
use crate::error::Result;
use crate::information_schema::key_column_usage::InformationSchemaKeyColumnUsage;
use crate::information_schema::memory_table::{get_schema_columns, MemoryTable};
use crate::information_schema::runtime_metrics::InformationSchemaMetrics;
use crate::information_schema::schemata::InformationSchemaSchemata;
use crate::information_schema::tables::InformationSchemaTables;
use crate::CatalogManager;
@@ -56,7 +58,6 @@ lazy_static! {
ENGINES,
COLUMN_PRIVILEGES,
COLUMN_STATISTICS,
BUILD_INFO,
CHARACTER_SETS,
COLLATIONS,
COLLATION_CHARACTER_SET_APPLICABILITY,
@@ -142,6 +143,21 @@ impl InformationSchemaProvider {
fn build_tables(&mut self) {
let mut tables = HashMap::new();
// Carefully consider the tables that may expose sensitive cluster configurations,
// authentication details, and other critical information.
// Only put these tables under `greptime` catalog to prevent info leak.
if self.catalog_name == DEFAULT_CATALOG_NAME {
tables.insert(
RUNTIME_METRICS.to_string(),
self.build_table(RUNTIME_METRICS).unwrap(),
);
tables.insert(
BUILD_INFO.to_string(),
self.build_table(BUILD_INFO).unwrap(),
);
}
tables.insert(TABLES.to_string(), self.build_table(TABLES).unwrap());
tables.insert(SCHEMATA.to_string(), self.build_table(SCHEMATA).unwrap());
tables.insert(COLUMNS.to_string(), self.build_table(COLUMNS).unwrap());
@@ -209,6 +225,7 @@ impl InformationSchemaProvider {
self.catalog_name.clone(),
self.catalog_manager.clone(),
)) as _),
RUNTIME_METRICS => Some(Arc::new(InformationSchemaMetrics::new())),
_ => None,
}
}

View File

@@ -0,0 +1,250 @@
// Copyright 2023 Greptime Team
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
use std::sync::Arc;
use arrow_schema::SchemaRef as ArrowSchemaRef;
use common_catalog::consts::INFORMATION_SCHEMA_RUNTIME_METRICS_TABLE_ID;
use common_error::ext::BoxedError;
use common_query::physical_plan::TaskContext;
use common_recordbatch::adapter::RecordBatchStreamAdapter;
use common_recordbatch::{RecordBatch, SendableRecordBatchStream};
use common_time::util::current_time_millis;
use datafusion::physical_plan::stream::RecordBatchStreamAdapter as DfRecordBatchStreamAdapter;
use datafusion::physical_plan::streaming::PartitionStream as DfPartitionStream;
use datafusion::physical_plan::SendableRecordBatchStream as DfSendableRecordBatchStream;
use datatypes::prelude::{ConcreteDataType, MutableVector};
use datatypes::scalars::ScalarVectorBuilder;
use datatypes::schema::{ColumnSchema, Schema, SchemaRef};
use datatypes::vectors::{
ConstantVector, Float64VectorBuilder, StringVector, StringVectorBuilder,
TimestampMillisecondVector, VectorRef,
};
use itertools::Itertools;
use snafu::ResultExt;
use store_api::storage::{ScanRequest, TableId};
use super::{InformationTable, RUNTIME_METRICS};
use crate::error::{CreateRecordBatchSnafu, InternalSnafu, Result};
pub(super) struct InformationSchemaMetrics {
schema: SchemaRef,
}
const METRIC_NAME: &str = "metric_name";
const METRIC_VALUE: &str = "value";
const METRIC_LABELS: &str = "labels";
const NODE: &str = "node";
const NODE_TYPE: &str = "node_type";
const TIMESTAMP: &str = "timestamp";
/// The `information_schema.runtime_metrics` virtual table.
/// It provides the GreptimeDB runtime metrics for the users by SQL.
impl InformationSchemaMetrics {
pub(super) fn new() -> Self {
Self {
schema: Self::schema(),
}
}
fn schema() -> SchemaRef {
Arc::new(Schema::new(vec![
ColumnSchema::new(METRIC_NAME, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(METRIC_VALUE, ConcreteDataType::float64_datatype(), false),
ColumnSchema::new(METRIC_LABELS, ConcreteDataType::string_datatype(), true),
ColumnSchema::new(NODE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(NODE_TYPE, ConcreteDataType::string_datatype(), false),
ColumnSchema::new(
TIMESTAMP,
ConcreteDataType::timestamp_millisecond_datatype(),
false,
),
]))
}
fn builder(&self) -> InformationSchemaMetricsBuilder {
InformationSchemaMetricsBuilder::new(self.schema.clone())
}
}
impl InformationTable for InformationSchemaMetrics {
fn table_id(&self) -> TableId {
INFORMATION_SCHEMA_RUNTIME_METRICS_TABLE_ID
}
fn table_name(&self) -> &'static str {
RUNTIME_METRICS
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
let stream = Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_metrics(Some(request))
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
));
Ok(Box::pin(
RecordBatchStreamAdapter::try_new(stream)
.map_err(BoxedError::new)
.context(InternalSnafu)?,
))
}
}
struct InformationSchemaMetricsBuilder {
schema: SchemaRef,
metric_names: StringVectorBuilder,
metric_values: Float64VectorBuilder,
metric_labels: StringVectorBuilder,
}
impl InformationSchemaMetricsBuilder {
fn new(schema: SchemaRef) -> Self {
Self {
schema,
metric_names: StringVectorBuilder::with_capacity(42),
metric_values: Float64VectorBuilder::with_capacity(42),
metric_labels: StringVectorBuilder::with_capacity(42),
}
}
fn add_metric(&mut self, metric_name: &str, labels: String, metric_value: f64) {
self.metric_names.push(Some(metric_name));
self.metric_values.push(Some(metric_value));
self.metric_labels.push(Some(&labels));
}
async fn make_metrics(&mut self, _request: Option<ScanRequest>) -> Result<RecordBatch> {
let metric_families = prometheus::gather();
let write_request =
common_telemetry::metric::convert_metric_to_write_request(metric_families, None, 0);
for ts in write_request.timeseries {
//Safety: always has `__name__` label
let metric_name = ts
.labels
.iter()
.find_map(|label| {
if label.name == "__name__" {
Some(label.value.clone())
} else {
None
}
})
.unwrap();
self.add_metric(
&metric_name,
ts.labels
.into_iter()
.filter_map(|label| {
if label.name == "__name__" {
None
} else {
Some(format!("{}={}", label.name, label.value))
}
})
.join(", "),
// Safety: always has a sample
ts.samples[0].value,
);
}
self.finish()
}
fn finish(&mut self) -> Result<RecordBatch> {
let rows_num = self.metric_names.len();
let unknowns = Arc::new(ConstantVector::new(
Arc::new(StringVector::from(vec!["unknown"])),
rows_num,
));
let timestamps = Arc::new(ConstantVector::new(
Arc::new(TimestampMillisecondVector::from_slice([
current_time_millis(),
])),
rows_num,
));
let columns: Vec<VectorRef> = vec![
Arc::new(self.metric_names.finish()),
Arc::new(self.metric_values.finish()),
Arc::new(self.metric_labels.finish()),
// TODO(dennis): supports node and node_type for cluster
unknowns.clone(),
unknowns,
timestamps,
];
RecordBatch::new(self.schema.clone(), columns).context(CreateRecordBatchSnafu)
}
}
impl DfPartitionStream for InformationSchemaMetrics {
fn schema(&self) -> &ArrowSchemaRef {
self.schema.arrow_schema()
}
fn execute(&self, _: Arc<TaskContext>) -> DfSendableRecordBatchStream {
let schema = self.schema.arrow_schema().clone();
let mut builder = self.builder();
Box::pin(DfRecordBatchStreamAdapter::new(
schema,
futures::stream::once(async move {
builder
.make_metrics(None)
.await
.map(|x| x.into_df_record_batch())
.map_err(Into::into)
}),
))
}
}
#[cfg(test)]
mod tests {
use common_recordbatch::RecordBatches;
use super::*;
#[tokio::test]
async fn test_make_metrics() {
let metrics = InformationSchemaMetrics::new();
let stream = metrics.to_stream(ScanRequest::default()).unwrap();
let batches = RecordBatches::try_collect(stream).await.unwrap();
let result_literal = batches.pretty_print().unwrap();
assert!(result_literal.contains(METRIC_NAME));
assert!(result_literal.contains(METRIC_VALUE));
assert!(result_literal.contains(METRIC_LABELS));
assert!(result_literal.contains(NODE));
assert!(result_literal.contains(NODE_TYPE));
assert!(result_literal.contains(TIMESTAMP));
}
}

View File

@@ -38,3 +38,4 @@ pub const TABLE_PRIVILEGES: &str = "table_privileges";
pub const TRIGGERS: &str = "triggers";
pub const GLOBAL_STATUS: &str = "global_status";
pub const SESSION_STATUS: &str = "session_status";
pub const RUNTIME_METRICS: &str = "runtime_metrics";

View File

@@ -80,6 +80,8 @@ pub const INFORMATION_SCHEMA_TRIGGERS_TABLE_ID: u32 = 24;
pub const INFORMATION_SCHEMA_GLOBAL_STATUS_TABLE_ID: u32 = 25;
/// id for information_schema.SESSION_STATUS
pub const INFORMATION_SCHEMA_SESSION_STATUS_TABLE_ID: u32 = 26;
/// id for information_schema.RUNTIME_METRICS
pub const INFORMATION_SCHEMA_RUNTIME_METRICS_TABLE_ID: u32 = 27;
/// ----- End of information_schema tables -----
pub const MITO_ENGINE: &str = "mito";

View File

@@ -20,7 +20,6 @@ show tables;
+---------------------------------------+
| Tables |
+---------------------------------------+
| build_info |
| character_sets |
| check_constraints |
| collation_character_set_applicability |

View File

@@ -12,7 +12,6 @@ order by table_schema, table_name;
+---------------+--------------------+---------------------------------------+-----------------+----------+-------------+
| table_catalog | table_schema | table_name | table_type | table_id | engine |
+---------------+--------------------+---------------------------------------+-----------------+----------+-------------+
| greptime | information_schema | build_info | LOCAL TEMPORARY | 8 | |
| greptime | information_schema | character_sets | LOCAL TEMPORARY | 9 | |
| greptime | information_schema | check_constraints | LOCAL TEMPORARY | 12 | |
| greptime | information_schema | collation_character_set_applicability | LOCAL TEMPORARY | 11 | |
@@ -39,259 +38,254 @@ order by table_schema, table_name;
| greptime | public | numbers | LOCAL TEMPORARY | 2 | test_engine |
+---------------+--------------------+---------------------------------------+-----------------+----------+-------------+
select * from information_schema.columns order by table_schema, table_name;
select * from information_schema.columns order by table_schema, table_name, column_name;
+---------------+--------------------+---------------------------------------+-----------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+
| table_catalog | table_schema | table_name | column_name | data_type | semantic_type | column_default | is_nullable | column_type | column_comment |
+---------------+--------------------+---------------------------------------+-----------------------------------+-----------+---------------+----------------+-------------+-------------+----------------+
| greptime | information_schema | build_info | git_branch | String | FIELD | | No | String | |
| greptime | information_schema | build_info | git_commit | String | FIELD | | No | String | |
| greptime | information_schema | build_info | git_commit_short | String | FIELD | | No | String | |
| greptime | information_schema | build_info | git_dirty | String | FIELD | | No | String | |
| greptime | information_schema | build_info | pkg_version | String | FIELD | | No | String | |
| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | character_sets | character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | character_sets | default_collate_name | String | FIELD | | No | String | |
| greptime | information_schema | character_sets | description | String | FIELD | | No | String | |
| greptime | information_schema | character_sets | maxlen | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | check_constraints | check_clause | String | FIELD | | No | String | |
| greptime | information_schema | check_constraints | constraint_catalog | String | FIELD | | No | String | |
| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | |
| greptime | information_schema | check_constraints | constraint_name | String | FIELD | | No | String | |
| greptime | information_schema | check_constraints | constraint_schema | String | FIELD | | No | String | |
| greptime | information_schema | collation_character_set_applicability | character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | collation_character_set_applicability | collation_name | String | FIELD | | No | String | |
| greptime | information_schema | collations | character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | collations | collation_name | String | FIELD | | No | String | |
| greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | collations | id | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | collations | is_default | String | FIELD | | No | String | |
| greptime | information_schema | collations | is_compiled | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | |
| greptime | information_schema | collations | is_default | String | FIELD | | No | String | |
| greptime | information_schema | collations | sortlen | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | column_privileges | column_name | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | grantee | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | is_grantable | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | privilege_type | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | table_name | String | FIELD | | No | String | |
| greptime | information_schema | column_privileges | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | |
| greptime | information_schema | column_statistics | histogram | String | FIELD | | No | String | |
| greptime | information_schema | column_statistics | schema_name | String | FIELD | | No | String | |
| greptime | information_schema | column_statistics | table_name | String | FIELD | | No | String | |
| greptime | information_schema | column_statistics | column_name | String | FIELD | | No | String | |
| greptime | information_schema | columns | column_type | String | FIELD | | No | String | |
| greptime | information_schema | columns | column_comment | String | FIELD | | Yes | String | |
| greptime | information_schema | columns | table_name | String | FIELD | | No | String | |
| greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | |
| greptime | information_schema | columns | column_default | String | FIELD | | Yes | String | |
| greptime | information_schema | columns | column_name | String | FIELD | | No | String | |
| greptime | information_schema | columns | column_type | String | FIELD | | No | String | |
| greptime | information_schema | columns | data_type | String | FIELD | | No | String | |
| greptime | information_schema | columns | is_nullable | String | FIELD | | No | String | |
| greptime | information_schema | columns | semantic_type | String | FIELD | | No | String | |
| greptime | information_schema | columns | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | columns | table_name | String | FIELD | | No | String | |
| greptime | information_schema | columns | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | columns | data_type | String | FIELD | | No | String | |
| greptime | information_schema | columns | column_name | String | FIELD | | No | String | |
| greptime | information_schema | engines | savepoints | String | FIELD | | No | String | |
| greptime | information_schema | engines | xa | String | FIELD | | No | String | |
| greptime | information_schema | engines | transactions | String | FIELD | | No | String | |
| greptime | information_schema | engines | comment | String | FIELD | | No | String | |
| greptime | information_schema | engines | support | String | FIELD | | No | String | |
| greptime | information_schema | engines | engine | String | FIELD | | No | String | |
| greptime | information_schema | events | event_name | String | FIELD | | No | String | |
| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | database_collation | String | FIELD | | No | String | |
| greptime | information_schema | events | collation_connection | String | FIELD | | No | String | |
| greptime | information_schema | engines | savepoints | String | FIELD | | No | String | |
| greptime | information_schema | engines | support | String | FIELD | | No | String | |
| greptime | information_schema | engines | transactions | String | FIELD | | No | String | |
| greptime | information_schema | engines | xa | String | FIELD | | No | String | |
| greptime | information_schema | events | character_set_client | String | FIELD | | No | String | |
| greptime | information_schema | events | originator | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | events | event_catalog | String | FIELD | | No | String | |
| greptime | information_schema | events | event_schema | String | FIELD | | No | String | |
| greptime | information_schema | events | event_comment | String | FIELD | | No | String | |
| greptime | information_schema | events | collation_connection | String | FIELD | | No | String | |
| greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | database_collation | String | FIELD | | No | String | |
| greptime | information_schema | events | definer | String | FIELD | | No | String | |
| greptime | information_schema | events | time_zone | String | FIELD | | No | String | |
| greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | event_body | String | FIELD | | No | String | |
| greptime | information_schema | events | event_catalog | String | FIELD | | No | String | |
| greptime | information_schema | events | event_comment | String | FIELD | | No | String | |
| greptime | information_schema | events | event_definition | String | FIELD | | No | String | |
| greptime | information_schema | events | event_name | String | FIELD | | No | String | |
| greptime | information_schema | events | event_schema | String | FIELD | | No | String | |
| greptime | information_schema | events | event_type | String | FIELD | | No | String | |
| greptime | information_schema | events | execute_at | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | events | interval_field | String | FIELD | | No | String | |
| greptime | information_schema | events | interval_value | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | last_executed | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | on_completion | String | FIELD | | No | String | |
| greptime | information_schema | events | originator | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | events | sql_mode | String | FIELD | | No | String | |
| greptime | information_schema | events | starts | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | ends | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | status | String | FIELD | | No | String | |
| greptime | information_schema | events | on_completion | String | FIELD | | No | String | |
| greptime | information_schema | events | created | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | events | last_altered | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | file_name | String | FIELD | | No | String | |
| greptime | information_schema | files | free_extents | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | checksum | String | FIELD | | No | String | |
| greptime | information_schema | files | update_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | create_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | status | String | FIELD | | No | String | |
| greptime | information_schema | files | file_id | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | events | time_zone | String | FIELD | | No | String | |
| greptime | information_schema | files | autoextend_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | avg_row_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | check_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | checksum | String | FIELD | | No | String | |
| greptime | information_schema | files | create_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | creation_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | data_free | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | data_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | deleted_rows | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | engine | String | FIELD | | No | String | |
| greptime | information_schema | files | extent_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | extra | String | FIELD | | No | String | |
| greptime | information_schema | files | file_id | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | file_name | String | FIELD | | No | String | |
| greptime | information_schema | files | file_type | String | FIELD | | No | String | |
| greptime | information_schema | files | tablespace_name | String | FIELD | | No | String | |
| greptime | information_schema | files | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | files | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | files | table_name | String | FIELD | | No | String | |
| greptime | information_schema | files | free_extents | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | fulltext_keys | String | FIELD | | No | String | |
| greptime | information_schema | files | index_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | initial_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | last_access_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | last_update_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | logfile_group_name | String | FIELD | | No | String | |
| greptime | information_schema | files | logfile_group_number | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | engine | String | FIELD | | No | String | |
| greptime | information_schema | files | fulltext_keys | String | FIELD | | No | String | |
| greptime | information_schema | files | deleted_rows | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | update_count | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | extra | String | FIELD | | No | String | |
| greptime | information_schema | files | total_extents | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | extent_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | initial_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | maximum_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | autoextend_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | creation_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | last_update_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | last_access_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | recover_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | transaction_counter | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | version | String | FIELD | | No | String | |
| greptime | information_schema | files | row_format | String | FIELD | | No | String | |
| greptime | information_schema | files | table_rows | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | avg_row_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | data_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | max_data_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | index_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | data_free | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | maximum_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | recover_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | row_format | String | FIELD | | No | String | |
| greptime | information_schema | files | status | String | FIELD | | No | String | |
| greptime | information_schema | files | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | files | table_name | String | FIELD | | No | String | |
| greptime | information_schema | files | table_rows | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | files | tablespace_name | String | FIELD | | No | String | |
| greptime | information_schema | files | total_extents | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | transaction_counter | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | update_count | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | files | update_time | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | files | version | String | FIELD | | No | String | |
| greptime | information_schema | global_status | variable_name | String | FIELD | | No | String | |
| greptime | information_schema | global_status | variable_value | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | column_name | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | constraint_catalog | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | constraint_name | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | constraint_schema | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | ordinal_position | UInt32 | FIELD | | No | UInt32 | |
| greptime | information_schema | key_column_usage | position_in_unique_constraint | UInt32 | FIELD | | Yes | UInt32 | |
| greptime | information_schema | key_column_usage | referenced_column_name | String | FIELD | | Yes | String | |
| greptime | information_schema | key_column_usage | referenced_table_name | String | FIELD | | Yes | String | |
| greptime | information_schema | key_column_usage | referenced_table_schema | String | FIELD | | Yes | String | |
| greptime | information_schema | key_column_usage | position_in_unique_constraint | UInt32 | FIELD | | Yes | UInt32 | |
| greptime | information_schema | key_column_usage | constraint_catalog | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | constraint_schema | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | constraint_name | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | table_name | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | column_name | String | FIELD | | No | String | |
| greptime | information_schema | key_column_usage | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | optimizer_trace | insufficient_privileges | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | optimizer_trace | missing_bytes_beyond_max_mem_size | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | optimizer_trace | trace | String | FIELD | | No | String | |
| greptime | information_schema | optimizer_trace | query | String | FIELD | | No | String | |
| greptime | information_schema | parameters | specific_catalog | String | FIELD | | No | String | |
| greptime | information_schema | optimizer_trace | trace | String | FIELD | | No | String | |
| greptime | information_schema | parameters | character_maximum_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | routine_type | String | FIELD | | No | String | |
| greptime | information_schema | parameters | dtd_identifier | String | FIELD | | No | String | |
| greptime | information_schema | parameters | collation_name | String | FIELD | | No | String | |
| greptime | information_schema | parameters | character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | parameters | datetime_precision | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | numeric_scale | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | numeric_precision | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | character_octet_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | specific_schema | String | FIELD | | No | String | |
| greptime | information_schema | parameters | specific_name | String | FIELD | | No | String | |
| greptime | information_schema | parameters | character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | parameters | collation_name | String | FIELD | | No | String | |
| greptime | information_schema | parameters | data_type | String | FIELD | | No | String | |
| greptime | information_schema | parameters | datetime_precision | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | dtd_identifier | String | FIELD | | No | String | |
| greptime | information_schema | parameters | numeric_precision | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | numeric_scale | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | ordinal_position | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | parameters | parameter_mode | String | FIELD | | No | String | |
| greptime | information_schema | parameters | parameter_name | String | FIELD | | No | String | |
| greptime | information_schema | parameters | data_type | String | FIELD | | No | String | |
| greptime | information_schema | parameters | routine_type | String | FIELD | | No | String | |
| greptime | information_schema | parameters | specific_catalog | String | FIELD | | No | String | |
| greptime | information_schema | parameters | specific_name | String | FIELD | | No | String | |
| greptime | information_schema | parameters | specific_schema | String | FIELD | | No | String | |
| greptime | information_schema | profiling | block_ops_in | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | block_ops_out | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | messages_sent | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | source_line | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | source_file | String | FIELD | | No | String | |
| greptime | information_schema | profiling | source_function | String | FIELD | | No | String | |
| greptime | information_schema | profiling | swaps | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | page_faults_minor | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | page_faults_major | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | context_involuntary | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | context_voluntary | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | cpu_system | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | cpu_user | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | duration | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | messages_received | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | messages_sent | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | page_faults_major | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | page_faults_minor | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | query_id | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | seq | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | source_file | String | FIELD | | No | String | |
| greptime | information_schema | profiling | source_function | String | FIELD | | No | String | |
| greptime | information_schema | profiling | source_line | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | state | String | FIELD | | No | String | |
| greptime | information_schema | profiling | duration | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | cpu_user | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | cpu_system | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | context_voluntary | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | context_involuntary | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | profiling | block_ops_in | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | referential_constraints | referenced_table_name | String | FIELD | | No | String | |
| greptime | information_schema | profiling | swaps | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | referential_constraints | constraint_catalog | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | constraint_name | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | table_name | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | constraint_schema | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | delete_rule | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | update_rule | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | match_option | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | referenced_table_name | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | table_name | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | unique_constraint_catalog | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | unique_constraint_name | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | unique_constraint_schema | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | unique_constraint_catalog | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | constraint_catalog | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | constraint_schema | String | FIELD | | No | String | |
| greptime | information_schema | routines | sql_mode | String | FIELD | | No | String | |
| greptime | information_schema | routines | security_type | String | FIELD | | No | String | |
| greptime | information_schema | routines | database_collation | String | FIELD | | No | String | |
| greptime | information_schema | routines | data_type | String | FIELD | | No | String | |
| greptime | information_schema | routines | character_set_client | String | FIELD | | No | String | |
| greptime | information_schema | routines | definer | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_comment | String | FIELD | | No | String | |
| greptime | information_schema | routines | last_altered | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | routines | created | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | routines | specific_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_catalog | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_schema | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_type | String | FIELD | | No | String | |
| greptime | information_schema | routines | collation_connection | String | FIELD | | No | String | |
| greptime | information_schema | referential_constraints | update_rule | String | FIELD | | No | String | |
| greptime | information_schema | routines | character_maximum_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | routines | character_octet_length | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | routines | character_set_client | String | FIELD | | No | String | |
| greptime | information_schema | routines | character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | collation_connection | String | FIELD | | No | String | |
| greptime | information_schema | routines | collation_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | created | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | routines | data_type | String | FIELD | | No | String | |
| greptime | information_schema | routines | database_collation | String | FIELD | | No | String | |
| greptime | information_schema | routines | datetime_precision | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | routines | definer | String | FIELD | | No | String | |
| greptime | information_schema | routines | dtd_identifier | String | FIELD | | No | String | |
| greptime | information_schema | routines | external_language | String | FIELD | | No | String | |
| greptime | information_schema | routines | external_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | is_deterministic | String | FIELD | | No | String | |
| greptime | information_schema | routines | last_altered | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | routines | numeric_precision | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | routines | numeric_scale | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | routines | datetime_precision | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | routines | character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | collation_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | dtd_identifier | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_body | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_definition | String | FIELD | | No | String | |
| greptime | information_schema | routines | external_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | external_language | String | FIELD | | No | String | |
| greptime | information_schema | routines | parameter_style | String | FIELD | | No | String | |
| greptime | information_schema | routines | is_deterministic | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_body | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_catalog | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_comment | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_definition | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_schema | String | FIELD | | No | String | |
| greptime | information_schema | routines | routine_type | String | FIELD | | No | String | |
| greptime | information_schema | routines | security_type | String | FIELD | | No | String | |
| greptime | information_schema | routines | specific_name | String | FIELD | | No | String | |
| greptime | information_schema | routines | sql_data_access | String | FIELD | | No | String | |
| greptime | information_schema | routines | sql_mode | String | FIELD | | No | String | |
| greptime | information_schema | routines | sql_path | String | FIELD | | No | String | |
| greptime | information_schema | schema_privileges | is_grantable | String | FIELD | | No | String | |
| greptime | information_schema | schema_privileges | grantee | String | FIELD | | No | String | |
| greptime | information_schema | schema_privileges | is_grantable | String | FIELD | | No | String | |
| greptime | information_schema | schema_privileges | privilege_type | String | FIELD | | No | String | |
| greptime | information_schema | schema_privileges | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | schema_privileges | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | |
| greptime | information_schema | schema_privileges | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | schemata | catalog_name | String | FIELD | | No | String | |
| greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | |
| greptime | information_schema | schemata | default_character_set_name | String | FIELD | | No | String | |
| greptime | information_schema | schemata | default_collation_name | String | FIELD | | No | String | |
| greptime | information_schema | session_status | variable_value | String | FIELD | | No | String | |
| greptime | information_schema | schemata | schema_name | String | FIELD | | No | String | |
| greptime | information_schema | schemata | sql_path | String | FIELD | | Yes | String | |
| greptime | information_schema | session_status | variable_name | String | FIELD | | No | String | |
| greptime | information_schema | session_status | variable_value | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | grantee | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | table_name | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | privilege_type | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | is_grantable | String | FIELD | | No | String | |
| greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | tables | table_name | String | FIELD | | No | String | |
| greptime | information_schema | tables | table_type | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | privilege_type | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | table_name | String | FIELD | | No | String | |
| greptime | information_schema | table_privileges | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | tables | engine | String | FIELD | | Yes | String | |
| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | tables | table_catalog | String | FIELD | | No | String | |
| greptime | information_schema | tables | table_id | UInt32 | FIELD | | Yes | UInt32 | |
| greptime | information_schema | triggers | event_manipulation | String | FIELD | | No | String | |
| greptime | information_schema | triggers | event_object_schema | String | FIELD | | No | String | |
| greptime | information_schema | triggers | event_object_table | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_order | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | tables | table_name | String | FIELD | | No | String | |
| greptime | information_schema | tables | table_schema | String | FIELD | | No | String | |
| greptime | information_schema | tables | table_type | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_condition | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_statement | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_order | Int64 | FIELD | | No | Int64 | |
| greptime | information_schema | triggers | action_orientation | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_timing | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_reference_old_table | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_reference_new_row | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_reference_new_table | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_reference_old_row | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_reference_new_row | String | FIELD | | No | String | |
| greptime | information_schema | triggers | created | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | triggers | sql_mode | String | FIELD | | No | String | |
| greptime | information_schema | triggers | definer | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_reference_old_table | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_statement | String | FIELD | | No | String | |
| greptime | information_schema | triggers | action_timing | String | FIELD | | No | String | |
| greptime | information_schema | triggers | character_set_client | String | FIELD | | No | String | |
| greptime | information_schema | triggers | collation_connection | String | FIELD | | No | String | |
| greptime | information_schema | triggers | created | DateTime | FIELD | | No | DateTime | |
| greptime | information_schema | triggers | database_collation | String | FIELD | | No | String | |
| greptime | information_schema | triggers | trigger_catalog | String | FIELD | | No | String | |
| greptime | information_schema | triggers | definer | String | FIELD | | No | String | |
| greptime | information_schema | triggers | event_manipulation | String | FIELD | | No | String | |
| greptime | information_schema | triggers | event_object_catalog | String | FIELD | | No | String | |
| greptime | information_schema | triggers | event_object_schema | String | FIELD | | No | String | |
| greptime | information_schema | triggers | event_object_table | String | FIELD | | No | String | |
| greptime | information_schema | triggers | sql_mode | String | FIELD | | No | String | |
| greptime | information_schema | triggers | trigger_catalog | String | FIELD | | No | String | |
| greptime | information_schema | triggers | trigger_name | String | FIELD | | No | String | |
| greptime | information_schema | triggers | trigger_schema | String | FIELD | | No | String | |
| greptime | public | numbers | number | UInt32 | TAG | | No | UInt32 | |
@@ -615,6 +609,19 @@ select * from CHECK_CONSTRAINTS;
+--------------------+-------------------+-----------------+--------------+
+--------------------+-------------------+-----------------+--------------+
desc table RUNTIME_METRICS;
+-------------+----------------------+-----+------+---------+---------------+
| Column | Type | Key | Null | Default | Semantic Type |
+-------------+----------------------+-----+------+---------+---------------+
| metric_name | String | | NO | | FIELD |
| value | Float64 | | NO | | FIELD |
| labels | String | | YES | | FIELD |
| node | String | | NO | | FIELD |
| node_type | String | | NO | | FIELD |
| timestamp | TimestampMillisecond | | NO | | FIELD |
+-------------+----------------------+-----+------+---------+---------------+
use public;
Affected Rows: 0

View File

@@ -7,7 +7,7 @@ from information_schema.tables
where table_name != 'scripts'
order by table_schema, table_name;
select * from information_schema.columns order by table_schema, table_name;
select * from information_schema.columns order by table_schema, table_name, column_name;
create
database my_db;
@@ -115,4 +115,6 @@ desc table CHECK_CONSTRAINTS;
select * from CHECK_CONSTRAINTS;
desc table RUNTIME_METRICS;
use public;