feat: add ssts releated system table (#6924)

* feat: add InformationExtension.inspect_datanode for datanode inspection

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* aggregate results from all datanodes

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* fix fmt

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* feat: add ssts releated system table

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* update sst entry

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* address comments

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* fix sqlness

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

* fix sqlness

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>

---------

Signed-off-by: Zhenchi <zhongzc_arch@outlook.com>
This commit is contained in:
Zhenchi
2025-09-23 19:06:00 +08:00
committed by GitHub
parent 4507736528
commit 80c8ab42b0
10 changed files with 415 additions and 5 deletions

View File

@@ -137,21 +137,24 @@ impl DataSource for SystemTableDataSource {
&self,
request: ScanRequest,
) -> std::result::Result<SendableRecordBatchStream, BoxedError> {
let projection = request.projection.clone();
let projected_schema = match &projection {
let projected_schema = match &request.projection {
Some(projection) => self.try_project(projection)?,
None => self.table.schema(),
};
let projection = request.projection.clone();
let stream = self
.table
.to_stream(request)
.map_err(BoxedError::new)
.context(TablesRecordBatchSnafu)
.map_err(BoxedError::new)?
.map(move |batch| match &projection {
Some(p) => batch.and_then(|b| b.try_project(p)),
None => batch,
.map(move |batch| match (&projection, batch) {
// Some tables (e.g., inspect tables) already honor projection in their inner stream;
// others ignore it and return full rows. We will only apply projection here if the
// inner batch width doesn't match the projection size.
(Some(p), Ok(b)) if b.num_columns() != p.len() => b.try_project(p),
(_, res) => res,
});
let stream = RecordBatchStreamWrapper {

View File

@@ -24,6 +24,7 @@ pub mod region_peers;
mod region_statistics;
mod runtime_metrics;
pub mod schemata;
mod ssts;
mod table_constraints;
mod table_names;
pub mod tables;
@@ -66,6 +67,9 @@ use crate::system_schema::information_schema::partitions::InformationSchemaParti
use crate::system_schema::information_schema::region_peers::InformationSchemaRegionPeers;
use crate::system_schema::information_schema::runtime_metrics::InformationSchemaMetrics;
use crate::system_schema::information_schema::schemata::InformationSchemaSchemata;
use crate::system_schema::information_schema::ssts::{
InformationSchemaSstsManifest, InformationSchemaSstsStorage,
};
use crate::system_schema::information_schema::table_constraints::InformationSchemaTableConstraints;
use crate::system_schema::information_schema::tables::InformationSchemaTables;
use crate::system_schema::memory_table::MemoryTable;
@@ -253,6 +257,12 @@ impl SystemSchemaProviderInner for InformationSchemaProvider {
.process_manager
.as_ref()
.map(|p| Arc::new(InformationSchemaProcessList::new(p.clone())) as _),
SSTS_MANIFEST => Some(Arc::new(InformationSchemaSstsManifest::new(
self.catalog_manager.clone(),
)) as _),
SSTS_STORAGE => Some(Arc::new(InformationSchemaSstsStorage::new(
self.catalog_manager.clone(),
)) as _),
_ => None,
}
}
@@ -324,6 +334,14 @@ impl InformationSchemaProvider {
REGION_STATISTICS.to_string(),
self.build_table(REGION_STATISTICS).unwrap(),
);
tables.insert(
SSTS_MANIFEST.to_string(),
self.build_table(SSTS_MANIFEST).unwrap(),
);
tables.insert(
SSTS_STORAGE.to_string(),
self.build_table(SSTS_STORAGE).unwrap(),
);
}
tables.insert(TABLES.to_string(), self.build_table(TABLES).unwrap());

View File

@@ -0,0 +1,142 @@
// 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, Weak};
use common_catalog::consts::{
INFORMATION_SCHEMA_SSTS_MANIFEST_TABLE_ID, INFORMATION_SCHEMA_SSTS_STORAGE_TABLE_ID,
};
use common_error::ext::BoxedError;
use common_recordbatch::SendableRecordBatchStream;
use common_recordbatch::adapter::AsyncRecordBatchStreamAdapter;
use datatypes::schema::SchemaRef;
use snafu::ResultExt;
use store_api::sst_entry::{ManifestSstEntry, StorageSstEntry};
use store_api::storage::{ScanRequest, TableId};
use crate::CatalogManager;
use crate::error::{ProjectSchemaSnafu, Result};
use crate::information_schema::{
DatanodeInspectKind, DatanodeInspectRequest, InformationTable, SSTS_MANIFEST, SSTS_STORAGE,
};
use crate::system_schema::utils;
/// Information schema table for sst manifest.
pub struct InformationSchemaSstsManifest {
schema: SchemaRef,
catalog_manager: Weak<dyn CatalogManager>,
}
impl InformationSchemaSstsManifest {
pub(super) fn new(catalog_manager: Weak<dyn CatalogManager>) -> Self {
Self {
schema: ManifestSstEntry::schema(),
catalog_manager,
}
}
}
impl InformationTable for InformationSchemaSstsManifest {
fn table_id(&self) -> TableId {
INFORMATION_SCHEMA_SSTS_MANIFEST_TABLE_ID
}
fn table_name(&self) -> &'static str {
SSTS_MANIFEST
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = if let Some(p) = &request.projection {
Arc::new(self.schema.try_project(p).context(ProjectSchemaSnafu)?)
} else {
self.schema.clone()
};
let info_ext = utils::information_extension(&self.catalog_manager)?;
let req = DatanodeInspectRequest {
kind: DatanodeInspectKind::SstManifest,
scan: request,
};
let future = async move {
info_ext
.inspect_datanode(req)
.await
.map_err(BoxedError::new)
.context(common_recordbatch::error::ExternalSnafu)
};
Ok(Box::pin(AsyncRecordBatchStreamAdapter::new(
schema,
Box::pin(future),
)))
}
}
/// Information schema table for sst storage.
pub struct InformationSchemaSstsStorage {
schema: SchemaRef,
catalog_manager: Weak<dyn CatalogManager>,
}
impl InformationSchemaSstsStorage {
pub(super) fn new(catalog_manager: Weak<dyn CatalogManager>) -> Self {
Self {
schema: StorageSstEntry::schema(),
catalog_manager,
}
}
}
impl InformationTable for InformationSchemaSstsStorage {
fn table_id(&self) -> TableId {
INFORMATION_SCHEMA_SSTS_STORAGE_TABLE_ID
}
fn table_name(&self) -> &'static str {
SSTS_STORAGE
}
fn schema(&self) -> SchemaRef {
self.schema.clone()
}
fn to_stream(&self, request: ScanRequest) -> Result<SendableRecordBatchStream> {
let schema = if let Some(p) = &request.projection {
Arc::new(self.schema.try_project(p).context(ProjectSchemaSnafu)?)
} else {
self.schema.clone()
};
let info_ext = utils::information_extension(&self.catalog_manager)?;
let req = DatanodeInspectRequest {
kind: DatanodeInspectKind::SstStorage,
scan: request,
};
let future = async move {
info_ext
.inspect_datanode(req)
.await
.map_err(BoxedError::new)
.context(common_recordbatch::error::ExternalSnafu)
};
Ok(Box::pin(AsyncRecordBatchStreamAdapter::new(
schema,
Box::pin(future),
)))
}
}

View File

@@ -48,3 +48,5 @@ pub const FLOWS: &str = "flows";
pub const PROCEDURE_INFO: &str = "procedure_info";
pub const REGION_STATISTICS: &str = "region_statistics";
pub const PROCESS_LIST: &str = "process_list";
pub const SSTS_MANIFEST: &str = "ssts_manifest";
pub const SSTS_STORAGE: &str = "ssts_storage";

View File

@@ -104,6 +104,10 @@ pub const INFORMATION_SCHEMA_PROCEDURE_INFO_TABLE_ID: u32 = 34;
pub const INFORMATION_SCHEMA_REGION_STATISTICS_TABLE_ID: u32 = 35;
/// id for information_schema.process_list
pub const INFORMATION_SCHEMA_PROCESS_LIST_TABLE_ID: u32 = 36;
/// id for information_schema.ssts_manifest
pub const INFORMATION_SCHEMA_SSTS_MANIFEST_TABLE_ID: u32 = 37;
/// id for information_schema.ssts_storage
pub const INFORMATION_SCHEMA_SSTS_STORAGE_TABLE_ID: u32 = 38;
// ----- End of information_schema tables -----

View File

@@ -0,0 +1,153 @@
DESC TABLE information_schema.ssts_manifest;
+------------------+---------------------+-----+------+---------+---------------+
| Column | Type | Key | Null | Default | Semantic Type |
+------------------+---------------------+-----+------+---------+---------------+
| table_dir | String | | NO | | FIELD |
| region_id | UInt64 | | NO | | FIELD |
| table_id | UInt32 | | NO | | FIELD |
| region_number | UInt32 | | NO | | FIELD |
| region_group | UInt8 | | NO | | FIELD |
| region_sequence | UInt32 | | NO | | FIELD |
| file_id | String | | NO | | FIELD |
| level | UInt8 | | NO | | FIELD |
| file_path | String | | NO | | FIELD |
| file_size | UInt64 | | NO | | FIELD |
| index_file_path | String | | YES | | FIELD |
| index_file_size | UInt64 | | YES | | FIELD |
| num_rows | UInt64 | | NO | | FIELD |
| num_row_groups | UInt64 | | NO | | FIELD |
| min_ts | TimestampNanosecond | | YES | | FIELD |
| max_ts | TimestampNanosecond | | YES | | FIELD |
| sequence | UInt64 | | YES | | FIELD |
| origin_region_id | UInt64 | | NO | | FIELD |
| node_id | UInt64 | | YES | | FIELD |
| visible | Boolean | | NO | | FIELD |
+------------------+---------------------+-----+------+---------+---------------+
DESC TABLE information_schema.ssts_storage;
+------------------+----------------------+-----+------+---------+---------------+
| Column | Type | Key | Null | Default | Semantic Type |
+------------------+----------------------+-----+------+---------+---------------+
| file_path | String | | NO | | FIELD |
| file_size | UInt64 | | YES | | FIELD |
| last_modified_ms | TimestampMillisecond | | YES | | FIELD |
| node_id | UInt64 | | YES | | FIELD |
+------------------+----------------------+-----+------+---------+---------------+
CREATE TABLE sst_case (
a INT PRIMARY KEY,
b STRING SKIPPING INDEX,
ts TIMESTAMP TIME INDEX,
)
PARTITION ON COLUMNS (a) (
a < 1000,
a >= 1000 AND a < 2000,
a >= 2000
);
Affected Rows: 0
INSERT INTO sst_case VALUES
(500, 'a', 1),
(1500, 'b', 2),
(2500, 'c', 3);
Affected Rows: 3
ADMIN FLUSH_TABLE('sst_case');
+-------------------------------+
| ADMIN FLUSH_TABLE('sst_case') |
+-------------------------------+
| 0 |
+-------------------------------+
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{3,9}) <DATETIME>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
SELECT * FROM information_schema.ssts_manifest order by file_path;
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+-------------------------+-------------------------+----------+------------------+---------+---------+
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
SELECT * FROM information_schema.ssts_storage order by file_path;
+---------------------------------------------------------------------------------------------+-----------+------------------+---------+
| file_path | file_size | last_modified_ms | node_id |
+---------------------------------------------------------------------------------------------+-----------+------------------+---------+
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
+---------------------------------------------------------------------------------------------+-----------+------------------+---------+
INSERT INTO sst_case VALUES
(24, 'foo', 100),
(124, 'bar', 200),
(1024, 'baz', 300);
Affected Rows: 3
ADMIN FLUSH_TABLE('sst_case');
+-------------------------------+
| ADMIN FLUSH_TABLE('sst_case') |
+-------------------------------+
| 0 |
+-------------------------------+
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{3,9}) <DATETIME>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
SELECT * FROM information_schema.ssts_manifest order by file_path;
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| table_dir | region_id | table_id | region_number | region_group | region_sequence | file_id | level | file_path | file_size | index_file_path | index_file_size | num_rows | num_row_groups | min_ts | max_ts | sequence | origin_region_id | node_id | visible |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+-------------------------+-------------------------+----------+------------------+---------+---------+
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
| data/greptime/public/<TABLE_ID>/ |<NUM>|<NUM>|<NUM>|<NUM>|<NUM>| <UUID> |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet |<NUM>| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin |<NUM>|<NUM>|<NUM>| <DATETIME> | <DATETIME> |<NUM>|<NUM>|<NUM>| true |
+----------------------------+---------------+----------+---------------+--------------+-----------------+--------------------------------------+-------+----------------------------------------------------------------------------------------+-----------+---------------------------------------------------------------------------------------------+-----------------+----------+----------------+-------------------------+-------------------------+----------+------------------+---------+---------+
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
SELECT * FROM information_schema.ssts_storage order by file_path;
+---------------------------------------------------------------------------------------------+-----------+------------------+---------+
| file_path | file_size | last_modified_ms | node_id |
+---------------------------------------------------------------------------------------------+-----------+------------------+---------+
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/<UUID>.parquet | | |<NUM>|
| data/greptime/public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>/index/<UUID>.puffin | | |<NUM>|
+---------------------------------------------------------------------------------------------+-----------+------------------+---------+
DROP TABLE sst_case;
Affected Rows: 0

View File

@@ -0,0 +1,54 @@
DESC TABLE information_schema.ssts_manifest;
DESC TABLE information_schema.ssts_storage;
CREATE TABLE sst_case (
a INT PRIMARY KEY,
b STRING SKIPPING INDEX,
ts TIMESTAMP TIME INDEX,
)
PARTITION ON COLUMNS (a) (
a < 1000,
a >= 1000 AND a < 2000,
a >= 2000
);
INSERT INTO sst_case VALUES
(500, 'a', 1),
(1500, 'b', 2),
(2500, 'c', 3);
ADMIN FLUSH_TABLE('sst_case');
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{3,9}) <DATETIME>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
SELECT * FROM information_schema.ssts_manifest order by file_path;
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
SELECT * FROM information_schema.ssts_storage order by file_path;
INSERT INTO sst_case VALUES
(24, 'foo', 100),
(124, 'bar', 200),
(1024, 'baz', 300);
ADMIN FLUSH_TABLE('sst_case');
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (\d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}\.\d{3,9}) <DATETIME>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
-- SQLNESS REPLACE (/public/\d+) /public/<TABLE_ID>
SELECT * FROM information_schema.ssts_manifest order by file_path;
-- SQLNESS REPLACE (\s+\d+\s+) <NUM>
-- SQLNESS REPLACE ([0-9a-fA-F]{8}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{4}-[0-9a-fA-F]{12}) <UUID>
-- SQLNESS REPLACE (/public/\d+/\d+_\d+) /public/<TABLE_ID>/<REGION_ID>_<REGION_NUMBER>
SELECT * FROM information_schema.ssts_storage order by file_path;
DROP TABLE sst_case;

View File

@@ -56,6 +56,8 @@ SHOW TABLES;
| schema_privileges |
| schemata |
| session_status |
| ssts_manifest |
| ssts_storage |
| table_constraints |
| table_privileges |
| tables |
@@ -105,6 +107,8 @@ SHOW FULL TABLES;
| schema_privileges | LOCAL TEMPORARY |
| schemata | LOCAL TEMPORARY |
| session_status | LOCAL TEMPORARY |
| ssts_manifest | LOCAL TEMPORARY |
| ssts_storage | LOCAL TEMPORARY |
| table_constraints | LOCAL TEMPORARY |
| table_privileges | LOCAL TEMPORARY |
| tables | LOCAL TEMPORARY |
@@ -148,6 +152,8 @@ SHOW TABLE STATUS;
|schema_privileges||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||
|schemata||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||
|session_status||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||
|ssts_manifest||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||
|ssts_storage||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||
|table_constraints||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||
|table_privileges||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||
|tables||11|Fixed|0|0|0|0|0|0|0|DATETIME|||utf8_bin|0|||

View File

@@ -43,6 +43,8 @@ order by table_schema, table_name;
|greptime|information_schema|schema_privileges|LOCALTEMPORARY|22|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|greptime|information_schema|schemata|LOCALTEMPORARY|15|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|greptime|information_schema|session_status|LOCALTEMPORARY|26|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|greptime|information_schema|ssts_manifest|LOCALTEMPORARY|37|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|greptime|information_schema|ssts_storage|LOCALTEMPORARY|38|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|greptime|information_schema|table_constraints|LOCALTEMPORARY|30|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|greptime|information_schema|table_privileges|LOCALTEMPORARY|23|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
|greptime|information_schema|tables|LOCALTEMPORARY|3|0|0|0|0|0||11|Fixed|0|0|0|DATETIME|||utf8_bin|0|||Y|
@@ -383,6 +385,30 @@ select * from information_schema.columns order by table_schema, table_name, colu
| greptime | information_schema | schemata | sql_path | 5 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | session_status | variable_name | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | session_status | variable_value | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | file_id | 7 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | file_path | 9 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | file_size | 10 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | index_file_path | 11 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | Yes | string | | |
| greptime | information_schema | ssts_manifest | index_file_size | 12 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | level | 8 | | | 3 | 0 | | | | | | select,insert | | UInt8 | tinyint unsigned | FIELD | | No | tinyint unsigned | | |
| greptime | information_schema | ssts_manifest | max_ts | 16 | | | | | 9 | | | | | select,insert | | TimestampNanosecond | timestamp(9) | FIELD | | Yes | timestamp(9) | | |
| greptime | information_schema | ssts_manifest | min_ts | 15 | | | | | 9 | | | | | select,insert | | TimestampNanosecond | timestamp(9) | FIELD | | Yes | timestamp(9) | | |
| greptime | information_schema | ssts_manifest | node_id | 19 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_row_groups | 14 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | num_rows | 13 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | origin_region_id | 18 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | region_group | 5 | | | 3 | 0 | | | | | | select,insert | | UInt8 | tinyint unsigned | FIELD | | No | tinyint unsigned | | |
| greptime | information_schema | ssts_manifest | region_id | 2 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | No | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | region_number | 4 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | ssts_manifest | region_sequence | 6 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | ssts_manifest | sequence | 17 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_manifest | table_dir | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_manifest | table_id | 3 | | | 10 | 0 | | | | | | select,insert | | UInt32 | int unsigned | FIELD | | No | int unsigned | | |
| greptime | information_schema | ssts_manifest | visible | 20 | | | | | | | | | | select,insert | | Boolean | boolean | FIELD | | No | boolean | | |
| greptime | information_schema | ssts_storage | file_path | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | ssts_storage | file_size | 2 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | ssts_storage | last_modified_ms | 3 | | | | | 3 | | | | | select,insert | | TimestampMillisecond | timestamp(3) | FIELD | | Yes | timestamp(3) | | |
| greptime | information_schema | ssts_storage | node_id | 4 | | | 20 | 0 | | | | | | select,insert | | UInt64 | bigint unsigned | FIELD | | Yes | bigint unsigned | | |
| greptime | information_schema | table_constraints | constraint_catalog | 1 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | table_constraints | constraint_name | 3 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |
| greptime | information_schema | table_constraints | constraint_schema | 2 | 2147483647 | 2147483647 | | | | utf8 | utf8_bin | | | select,insert | | String | string | FIELD | | No | string | | |

View File

@@ -117,6 +117,8 @@ SELECT * FROM INFORMATION_SCHEMA.TABLES ORDER BY TABLE_NAME, TABLE_TYPE;
|greptime|information_schema|schema_privileges|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|greptime|information_schema|schemata|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|greptime|information_schema|session_status|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|greptime|information_schema|ssts_manifest|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|greptime|information_schema|ssts_storage|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|greptime|information_schema|table_constraints|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|greptime|information_schema|table_privileges|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|
|greptime|information_schema|tables|LOCALTEMPORARY|ID|ID|ID|ID|ID|ID||ID|Fixed|ID|ID|ID|DATETIME|||utf8_bin|ID|||Y|