This is an automated email from the ASF dual-hosted git repository.
morningman pushed a commit to branch branch-2.1
in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-2.1 by this push:
new 173aafc86f0 [Enhancement] add information_schema.table_properties
#38745 (#38746) (#39886)
173aafc86f0 is described below
commit 173aafc86f04d6207ab46561800639cc2c860158
Author: Mingyu Chen <[email protected]>
AuthorDate: Tue Aug 27 17:22:19 2024 +0800
[Enhancement] add information_schema.table_properties #38745 (#38746)
(#39886)
bp #38746
---------
Co-authored-by: Vallish Pai <[email protected]>
---
be/src/exec/schema_scanner.cpp | 3 +
.../schema_table_properties_scanner.cpp | 171 ++++++++++++++++++
.../schema_table_properties_scanner.h | 55 ++++++
.../org/apache/doris/analysis/SchemaTableType.java | 5 +-
.../java/org/apache/doris/catalog/SchemaTable.java | 14 +-
.../doris/tablefunction/MetadataGenerator.java | 102 +++++++++++
gensrc/thrift/FrontendService.thrift | 3 +
.../data/query_p0/system/test_table_properties.out | 200 +++++++++++++++++++++
.../query_p0/system/test_table_properties.groovy | 124 +++++++++++++
9 files changed, 671 insertions(+), 6 deletions(-)
diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp
index 4dd04d1558e..c0b98650c3e 100644
--- a/be/src/exec/schema_scanner.cpp
+++ b/be/src/exec/schema_scanner.cpp
@@ -43,6 +43,7 @@
#include "exec/schema_scanner/schema_schema_privileges_scanner.h"
#include "exec/schema_scanner/schema_schemata_scanner.h"
#include "exec/schema_scanner/schema_table_privileges_scanner.h"
+#include "exec/schema_scanner/schema_table_properties_scanner.h"
#include "exec/schema_scanner/schema_tables_scanner.h"
#include "exec/schema_scanner/schema_user_privileges_scanner.h"
#include "exec/schema_scanner/schema_user_scanner.h"
@@ -233,6 +234,8 @@ std::unique_ptr<SchemaScanner>
SchemaScanner::create(TSchemaTableType::type type
return SchemaWorkloadGroupPrivilegesScanner::create_unique();
case TSchemaTableType::SCH_WORKLOAD_GROUP_RESOURCE_USAGE:
return SchemaBackendWorkloadGroupResourceUsage::create_unique();
+ case TSchemaTableType::SCH_TABLE_PROPERTIES:
+ return SchemaTablePropertiesScanner::create_unique();
default:
return SchemaDummyScanner::create_unique();
break;
diff --git a/be/src/exec/schema_scanner/schema_table_properties_scanner.cpp
b/be/src/exec/schema_scanner/schema_table_properties_scanner.cpp
new file mode 100644
index 00000000000..749113da1b5
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_table_properties_scanner.cpp
@@ -0,0 +1,171 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you 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.
+
+#include "exec/schema_scanner/schema_table_properties_scanner.h"
+
+#include "exec/schema_scanner/schema_helper.h"
+#include "runtime/client_cache.h"
+#include "runtime/exec_env.h"
+#include "runtime/runtime_state.h"
+#include "util/thrift_rpc_helper.h"
+#include "vec/common/string_ref.h"
+#include "vec/core/block.h"
+#include "vec/data_types/data_type_factory.hpp"
+
+namespace doris {
+std::vector<SchemaScanner::ColumnDesc>
SchemaTablePropertiesScanner::_s_tbls_columns = {
+ {"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringRef), true},
+ {"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringRef), true},
+ {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+ {"PROPERTY_NAME", TYPE_STRING, sizeof(StringRef), true},
+ {"PROPERTY_VALUE", TYPE_STRING, sizeof(StringRef), true},
+};
+
+SchemaTablePropertiesScanner::SchemaTablePropertiesScanner()
+ : SchemaScanner(_s_tbls_columns,
TSchemaTableType::SCH_TABLE_PROPERTIES) {}
+
+Status SchemaTablePropertiesScanner::start(RuntimeState* state) {
+ if (!_is_init) {
+ return Status::InternalError("used before initialized.");
+ }
+
+ // first get the all the database specific to current catalog
+ SCOPED_TIMER(_get_db_timer);
+ TGetDbsParams db_params;
+
+ if (_param->common_param->catalog) {
+ db_params.__set_catalog(*(_param->common_param->catalog));
+ }
+ if (_param->common_param->current_user_ident) {
+
db_params.__set_current_user_ident(*(_param->common_param->current_user_ident));
+ }
+
+ if (_param->common_param->ip && 0 != _param->common_param->port) {
+ RETURN_IF_ERROR(SchemaHelper::get_db_names(
+ *(_param->common_param->ip), _param->common_param->port,
db_params, &_db_result));
+ } else {
+ return Status::InternalError("IP or port doesn't exists");
+ }
+ _block_rows_limit = state->batch_size();
+ _rpc_timeout_ms = state->execution_timeout() * 1000;
+
+ return Status::OK();
+}
+
+Status SchemaTablePropertiesScanner::get_onedb_info_from_fe(int64_t dbId) {
+ TNetworkAddress master_addr =
ExecEnv::GetInstance()->master_info()->network_address;
+
+ TSchemaTableRequestParams schema_table_request_params;
+ for (int i = 0; i < _s_tbls_columns.size(); i++) {
+ schema_table_request_params.__isset.columns_name = true;
+
schema_table_request_params.columns_name.emplace_back(_s_tbls_columns[i].name);
+ }
+
+
schema_table_request_params.__set_current_user_ident(*_param->common_param->current_user_ident);
+ schema_table_request_params.__set_catalog(*_param->common_param->catalog);
+ schema_table_request_params.__set_dbId(dbId);
+
+ TFetchSchemaTableDataRequest request;
+ request.__set_schema_table_name(TSchemaTableName::TABLE_PROPERTIES);
+ request.__set_schema_table_params(schema_table_request_params);
+
+ TFetchSchemaTableDataResult result;
+
+ RETURN_IF_ERROR(ThriftRpcHelper::rpc<FrontendServiceClient>(
+ master_addr.hostname, master_addr.port,
+ [&request, &result](FrontendServiceConnection& client) {
+ client->fetchSchemaTableData(result, request);
+ },
+ _rpc_timeout_ms));
+
+ Status status(Status::create(result.status));
+ if (!status.ok()) {
+ LOG(WARNING) << "fetch table options from FE failed, errmsg=" <<
status;
+ return status;
+ }
+ std::vector<TRow> result_data = result.data_batch;
+
+ _tableproperties_block = vectorized::Block::create_unique();
+ for (int i = 0; i < _s_tbls_columns.size(); ++i) {
+ TypeDescriptor descriptor(_s_tbls_columns[i].type);
+ auto data_type =
vectorized::DataTypeFactory::instance().create_data_type(descriptor, true);
+ _tableproperties_block->insert(vectorized::ColumnWithTypeAndName(
+ data_type->create_column(), data_type,
_s_tbls_columns[i].name));
+ }
+ _tableproperties_block->reserve(_block_rows_limit);
+ if (result_data.size() > 0) {
+ int col_size = result_data[0].column_value.size();
+ if (col_size != _s_tbls_columns.size()) {
+ return Status::InternalError<false>("table options schema is not
match for FE and BE");
+ }
+ }
+
+ for (int i = 0; i < result_data.size(); i++) {
+ TRow row = result_data[i];
+ for (int j = 0; j < _s_tbls_columns.size(); j++) {
+ RETURN_IF_ERROR(insert_block_column(
+ row.column_value[j], j, _tableproperties_block.get(),
_s_tbls_columns[j].type));
+ }
+ }
+ return Status::OK();
+}
+
+bool SchemaTablePropertiesScanner::check_and_mark_eos(bool* eos) const {
+ if (_row_idx == _total_rows) {
+ *eos = true;
+ if (_db_index < _db_result.db_ids.size()) {
+ *eos = false;
+ }
+ return true;
+ }
+ return false;
+}
+
+Status
SchemaTablePropertiesScanner::get_next_block_internal(vectorized::Block* block,
bool* eos) {
+ if (!_is_init) {
+ return Status::InternalError("Used before initialized.");
+ }
+
+ if (nullptr == block || nullptr == eos) {
+ return Status::InternalError("input pointer is nullptr.");
+ }
+
+ if ((_tableproperties_block == nullptr) || (_row_idx == _total_rows)) {
+ if (_db_index < _db_result.db_ids.size()) {
+
RETURN_IF_ERROR(get_onedb_info_from_fe(_db_result.db_ids[_db_index]));
+ _row_idx = 0; // reset row index so that it start filling for next
block.
+ _total_rows = _tableproperties_block->rows();
+ _db_index++;
+ }
+ }
+
+ if (check_and_mark_eos(eos)) {
+ return Status::OK();
+ }
+
+ int current_batch_rows = std::min(_block_rows_limit, _total_rows -
_row_idx);
+ vectorized::MutableBlock mblock =
vectorized::MutableBlock::build_mutable_block(block);
+ RETURN_IF_ERROR(mblock.add_rows(_tableproperties_block.get(), _row_idx,
current_batch_rows));
+ _row_idx += current_batch_rows;
+
+ if (!check_and_mark_eos(eos)) {
+ *eos = false;
+ }
+ return Status::OK();
+}
+
+} // namespace doris
diff --git a/be/src/exec/schema_scanner/schema_table_properties_scanner.h
b/be/src/exec/schema_scanner/schema_table_properties_scanner.h
new file mode 100644
index 00000000000..0820fee9628
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_table_properties_scanner.h
@@ -0,0 +1,55 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you 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.
+
+#pragma once
+#include <gen_cpp/FrontendService_types.h>
+
+#include <vector>
+
+#include "common/status.h"
+#include "exec/schema_scanner.h"
+
+namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
+class SchemaTablePropertiesScanner : public SchemaScanner {
+ ENABLE_FACTORY_CREATOR(SchemaTablePropertiesScanner);
+
+public:
+ SchemaTablePropertiesScanner();
+ ~SchemaTablePropertiesScanner() override = default;
+
+ Status start(RuntimeState* state) override;
+ Status get_next_block_internal(vectorized::Block* block, bool* eos)
override;
+
+ static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns;
+
+private:
+ Status get_onedb_info_from_fe(int64_t dbId);
+ bool check_and_mark_eos(bool* eos) const;
+ int _block_rows_limit = 4096;
+ int _row_idx = 0;
+ int _total_rows = 0;
+ int _db_index = 0;
+ TGetDbsResult _db_result;
+ std::unique_ptr<vectorized::Block> _tableproperties_block = nullptr;
+ int _rpc_timeout_ms = 3000;
+};
+}; // namespace doris
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
index e2f618c8178..b33659fbc66 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/SchemaTableType.java
@@ -82,9 +82,10 @@ public enum SchemaTableType {
TSchemaTableType.SCH_FILE_CACHE_STATISTICS),
SCH_WORKLOAD_GROUP_PRIVILEGES("WORKLOAD_GROUP_PRIVILEGES",
"WORKLOAD_GROUP_PRIVILEGES",
TSchemaTableType.SCH_WORKLOAD_GROUP_PRIVILEGES),
-
SCH_WORKLOAD_GROUP_RESOURCE_USAGE("WORKLOAD_GROUP_RESOURCE_USAGE",
- "WORKLOAD_GROUP_RESOURCE_USAGE",
TSchemaTableType.SCH_WORKLOAD_GROUP_RESOURCE_USAGE);
+ "WORKLOAD_GROUP_RESOURCE_USAGE",
TSchemaTableType.SCH_WORKLOAD_GROUP_RESOURCE_USAGE),
+ SCH_TABLE_PROPERTIES("TABLE_PROPERTIES", "TABLE_PROPERTIES",
+ TSchemaTableType.SCH_TABLE_PROPERTIES);
private static final String dbName = "INFORMATION_SCHEMA";
private static SelectList fullSelectLists;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
index 6162304a5de..b106ab9d796 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/SchemaTable.java
@@ -532,8 +532,7 @@ public class SchemaTable extends Table {
.column("WORKLOAD_GROUP_NAME",
ScalarType.createVarchar(256))
.column("PRIVILEGE_TYPE",
ScalarType.createVarchar(PRIVILEGE_TYPE_LEN))
.column("IS_GRANTABLE",
ScalarType.createVarchar(IS_GRANTABLE_LEN))
- .build())
- )
+ .build()))
.put("workload_group_resource_usage",
new SchemaTable(SystemIdGenerator.getNextId(),
"workload_group_resource_usage", TableType.SCHEMA,
builder().column("BE_ID",
ScalarType.createType(PrimitiveType.BIGINT))
@@ -542,8 +541,15 @@ public class SchemaTable extends Table {
.column("CPU_USAGE_PERCENT",
ScalarType.createType(PrimitiveType.DOUBLE))
.column("LOCAL_SCAN_BYTES_PER_SECOND",
ScalarType.createType(PrimitiveType.BIGINT))
.column("REMOTE_SCAN_BYTES_PER_SECOND",
ScalarType.createType(PrimitiveType.BIGINT))
- .build())
- )
+ .build()))
+ .put("table_properties",
+ new SchemaTable(SystemIdGenerator.getNextId(),
"table_properties", TableType.SCHEMA,
+ builder().column("TABLE_CATALOG",
ScalarType.createVarchar(NAME_CHAR_LEN))
+ .column("TABLE_SCHEMA",
ScalarType.createVarchar(NAME_CHAR_LEN))
+ .column("TABLE_NAME",
ScalarType.createVarchar(NAME_CHAR_LEN))
+ .column("PROPERTY_NAME",
ScalarType.createStringType())
+ .column("PROPERTY_VALUE",
ScalarType.createStringType())
+ .build()))
.build();
private boolean fetchAllFe = false;
diff --git
a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
index db50b155366..0c928fd838b 100644
---
a/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
+++
b/fe/fe-core/src/main/java/org/apache/doris/tablefunction/MetadataGenerator.java
@@ -27,6 +27,7 @@ import org.apache.doris.catalog.OlapTable;
import org.apache.doris.catalog.SchemaTable;
import org.apache.doris.catalog.Table;
import org.apache.doris.catalog.TableIf;
+import org.apache.doris.catalog.TableProperty;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.ClientPool;
import org.apache.doris.common.Pair;
@@ -36,6 +37,7 @@ import org.apache.doris.common.proc.PartitionsProcDir;
import org.apache.doris.common.util.NetUtils;
import org.apache.doris.common.util.TimeUtils;
import org.apache.doris.datasource.CatalogIf;
+import org.apache.doris.datasource.ExternalCatalog;
import org.apache.doris.datasource.InternalCatalog;
import org.apache.doris.datasource.hive.HMSExternalCatalog;
import org.apache.doris.datasource.iceberg.IcebergMetadataCache;
@@ -108,6 +110,8 @@ public class MetadataGenerator {
private static final ImmutableMap<String, Integer>
WORKLOAD_GROUP_PRIVILEGES_COLUMN_TO_INDEX;
+ private static final ImmutableMap<String, Integer>
TABLE_PROPERTIES_COLUMN_TO_INDEX;
+
static {
ImmutableMap.Builder<String, Integer> activeQueriesbuilder = new
ImmutableMap.Builder();
List<Column> activeQueriesColList =
SchemaTable.TABLE_MAP.get("active_queries").getFullSchema();
@@ -141,6 +145,13 @@ public class MetadataGenerator {
wgPrivsBuilder.put(wgPrivsColList.get(i).getName().toLowerCase(),
i);
}
WORKLOAD_GROUP_PRIVILEGES_COLUMN_TO_INDEX = wgPrivsBuilder.build();
+
+ ImmutableMap.Builder<String, Integer> propertiesBuilder = new
ImmutableMap.Builder();
+ List<Column> propertiesColList =
SchemaTable.TABLE_MAP.get("table_properties").getFullSchema();
+ for (int i = 0; i < propertiesColList.size(); i++) {
+
propertiesBuilder.put(propertiesColList.get(i).getName().toLowerCase(), i);
+ }
+ TABLE_PROPERTIES_COLUMN_TO_INDEX = propertiesBuilder.build();
}
public static TFetchSchemaTableDataResult
getMetadataTable(TFetchSchemaTableDataRequest request) throws TException {
@@ -224,6 +235,10 @@ public class MetadataGenerator {
result = workloadGroupPrivsMetadataResult(schemaTableParams);
columnIndex = WORKLOAD_GROUP_PRIVILEGES_COLUMN_TO_INDEX;
break;
+ case TABLE_PROPERTIES:
+ result = tablePropertiesMetadataResult(schemaTableParams);
+ columnIndex = TABLE_PROPERTIES_COLUMN_TO_INDEX;
+ break;
default:
return errorResult("invalid schema table name.");
}
@@ -1013,4 +1028,91 @@ public class MetadataGenerator {
result.setStatus(new TStatus(TStatusCode.OK));
return result;
}
+
+ private static void tablePropertiesForInternalCatalog(UserIdentity
currentUserIdentity,
+ CatalogIf catalog, DatabaseIf database, List<TableIf> tables,
List<TRow> dataBatch) {
+ for (TableIf table : tables) {
+ if (!(table instanceof OlapTable)) {
+ continue;
+ }
+ if
(!Env.getCurrentEnv().getAccessManager().checkTblPriv(currentUserIdentity,
catalog.getName(),
+ database.getFullName(), table.getName(),
PrivPredicate.SHOW)) {
+ continue;
+ }
+ OlapTable olapTable = (OlapTable) table;
+ TableProperty property = olapTable.getTableProperty();
+ if (property == null) {
+ // if there is no properties, then write empty properties and
check next table.
+ TRow trow = new TRow();
+ trow.addToColumnValue(new
TCell().setStringVal(catalog.getName())); // TABLE_CATALOG
+ trow.addToColumnValue(new
TCell().setStringVal(database.getFullName())); // TABLE_SCHEMA
+ trow.addToColumnValue(new
TCell().setStringVal(table.getName())); // TABLE_NAME
+ trow.addToColumnValue(new TCell().setStringVal("")); //
PROPERTIES_NAME
+ trow.addToColumnValue(new TCell().setStringVal("")); //
PROPERTIES_VALUE
+ dataBatch.add(trow);
+ continue;
+ }
+
+ Map<String, String> propertiesMap = property.getProperties();
+ propertiesMap.forEach((key, value) -> {
+ TRow trow = new TRow();
+ trow.addToColumnValue(new
TCell().setStringVal(catalog.getName())); // TABLE_CATALOG
+ trow.addToColumnValue(new
TCell().setStringVal(database.getFullName())); // TABLE_SCHEMA
+ trow.addToColumnValue(new
TCell().setStringVal(table.getName())); // TABLE_NAME
+ trow.addToColumnValue(new TCell().setStringVal(key)); //
PROPERTIES_NAME
+ trow.addToColumnValue(new TCell().setStringVal(value)); //
PROPERTIES_VALUE
+ dataBatch.add(trow);
+ });
+ } // for table
+ }
+
+ private static void tablePropertiesForExternalCatalog(UserIdentity
currentUserIdentity,
+ CatalogIf catalog, DatabaseIf database, List<TableIf> tables,
List<TRow> dataBatch) {
+ for (TableIf table : tables) {
+ if
(!Env.getCurrentEnv().getAccessManager().checkTblPriv(currentUserIdentity,
catalog.getName(),
+ database.getFullName(), table.getName(),
PrivPredicate.SHOW)) {
+ continue;
+ }
+ // Currently for external catalog, we put properties as empty, can
extend in future
+ TRow trow = new TRow();
+ trow.addToColumnValue(new
TCell().setStringVal(catalog.getName())); // TABLE_CATALOG
+ trow.addToColumnValue(new
TCell().setStringVal(database.getFullName())); // TABLE_SCHEMA
+ trow.addToColumnValue(new TCell().setStringVal(table.getName()));
// TABLE_NAME
+ trow.addToColumnValue(new TCell().setStringVal("")); //
PROPERTIES_NAME
+ trow.addToColumnValue(new TCell().setStringVal("")); //
PROPERTIES_VALUE
+ dataBatch.add(trow);
+ } // for table
+ }
+
+ private static TFetchSchemaTableDataResult
tablePropertiesMetadataResult(TSchemaTableRequestParams params) {
+ if (!params.isSetCurrentUserIdent()) {
+ return errorResult("current user ident is not set.");
+ }
+
+ if (!params.isSetDbId()) {
+ return errorResult("current db id is not set.");
+ }
+
+ if (!params.isSetCatalog()) {
+ return errorResult("current catalog is not set.");
+ }
+
+ TUserIdentity tcurrentUserIdentity = params.getCurrentUserIdent();
+ UserIdentity currentUserIdentity =
UserIdentity.fromThrift(tcurrentUserIdentity);
+ TFetchSchemaTableDataResult result = new TFetchSchemaTableDataResult();
+ Long dbId = params.getDbId();
+ String clg = params.getCatalog();
+ CatalogIf catalog =
Env.getCurrentEnv().getCatalogMgr().getCatalog(clg);
+ List<TRow> dataBatch = Lists.newArrayList();
+ DatabaseIf database = catalog.getDbNullable(dbId);
+ List<TableIf> tables = database.getTables();
+ if (catalog instanceof InternalCatalog) {
+ tablePropertiesForInternalCatalog(currentUserIdentity, catalog,
database, tables, dataBatch);
+ } else if (catalog instanceof ExternalCatalog) {
+ tablePropertiesForExternalCatalog(currentUserIdentity, catalog,
database, tables, dataBatch);
+ }
+ result.setDataBatch(dataBatch);
+ result.setStatus(new TStatus(TStatusCode.OK));
+ return result;
+ }
}
diff --git a/gensrc/thrift/FrontendService.thrift
b/gensrc/thrift/FrontendService.thrift
index d7a71a9d1c5..c0176ef5135 100644
--- a/gensrc/thrift/FrontendService.thrift
+++ b/gensrc/thrift/FrontendService.thrift
@@ -956,6 +956,7 @@ enum TSchemaTableName {
WORKLOAD_SCHEDULE_POLICY = 5,
TABLE_OPTIONS = 6,
WORKLOAD_GROUP_PRIVILEGES = 7,
+ TABLE_PROPERTIES = 8,
}
struct TMetadataTableRequestParams {
@@ -976,6 +977,8 @@ struct TSchemaTableRequestParams {
1: optional list<string> columns_name
2: optional Types.TUserIdentity current_user_ident
3: optional bool replay_to_other_fe
+ 4: optional string catalog // use for table specific queries
+ 5: optional i64 dbId // used for table specific queries
}
struct TFetchSchemaTableDataRequest {
diff --git a/regression-test/data/query_p0/system/test_table_properties.out
b/regression-test/data/query_p0/system/test_table_properties.out
new file mode 100644
index 00000000000..f69fd14aea5
--- /dev/null
+++ b/regression-test/data/query_p0/system/test_table_properties.out
@@ -0,0 +1,200 @@
+-- This file is automatically generated. You should know what you did if you
want to edit this
+-- !select_check_1 --
+93
+
+-- !select_check_2 --
+internal test_table_properties_db duplicate_table _auto_bucket
false
+internal test_table_properties_db duplicate_table binlog.enable
false
+internal test_table_properties_db duplicate_table
binlog.max_bytes 9223372036854775807
+internal test_table_properties_db duplicate_table
binlog.max_history_nums 9223372036854775807
+internal test_table_properties_db duplicate_table
binlog.ttl_seconds 86400
+internal test_table_properties_db duplicate_table
compaction_policy size_based
+internal test_table_properties_db duplicate_table compression
LZ4F
+internal test_table_properties_db duplicate_table
data_sort.col_num 3
+internal test_table_properties_db duplicate_table
data_sort.sort_type LEXICAL
+internal test_table_properties_db duplicate_table
default.replication_allocation tag.location.default: 1
+internal test_table_properties_db duplicate_table
disable_auto_compaction false
+internal test_table_properties_db duplicate_table
enable_mow_light_delete false
+internal test_table_properties_db duplicate_table
enable_single_replica_compaction false
+internal test_table_properties_db duplicate_table
enable_unique_key_merge_on_write false
+internal test_table_properties_db duplicate_table
group_commit_data_bytes 134217728
+internal test_table_properties_db duplicate_table
group_commit_interval_ms 10000
+internal test_table_properties_db duplicate_table in_memory
false
+internal test_table_properties_db duplicate_table
inverted_index_storage_format V1
+internal test_table_properties_db duplicate_table is_being_synced
false
+internal test_table_properties_db duplicate_table
light_schema_change true
+internal test_table_properties_db duplicate_table
min_load_replica_num -1
+internal test_table_properties_db duplicate_table
row_store_page_size 16384
+internal test_table_properties_db duplicate_table
skip_write_index_on_load false
+internal test_table_properties_db duplicate_table storage_format
V2
+internal test_table_properties_db duplicate_table storage_medium
HDD
+internal test_table_properties_db duplicate_table
store_row_column false
+internal test_table_properties_db duplicate_table
time_series_compaction_empty_rowsets_threshold 5
+internal test_table_properties_db duplicate_table
time_series_compaction_file_count_threshold 2000
+internal test_table_properties_db duplicate_table
time_series_compaction_goal_size_mbytes 1024
+internal test_table_properties_db duplicate_table
time_series_compaction_level_threshold 1
+internal test_table_properties_db duplicate_table
time_series_compaction_time_threshold_seconds 3600
+internal test_table_properties_db listtable _auto_bucket
false
+internal test_table_properties_db listtable binlog.enable
false
+internal test_table_properties_db listtable
binlog.max_bytes 9223372036854775807
+internal test_table_properties_db listtable
binlog.max_history_nums 9223372036854775807
+internal test_table_properties_db listtable
binlog.ttl_seconds 86400
+internal test_table_properties_db listtable
compaction_policy size_based
+internal test_table_properties_db listtable compression
LZ4F
+internal test_table_properties_db listtable
data_sort.col_num 6
+internal test_table_properties_db listtable
data_sort.sort_type LEXICAL
+internal test_table_properties_db listtable
default.replication_allocation tag.location.default: 1
+internal test_table_properties_db listtable
disable_auto_compaction false
+internal test_table_properties_db listtable
enable_mow_light_delete false
+internal test_table_properties_db listtable
enable_single_replica_compaction false
+internal test_table_properties_db listtable
enable_unique_key_merge_on_write false
+internal test_table_properties_db listtable
group_commit_data_bytes 134217728
+internal test_table_properties_db listtable
group_commit_interval_ms 10000
+internal test_table_properties_db listtable in_memory
false
+internal test_table_properties_db listtable
inverted_index_storage_format V1
+internal test_table_properties_db listtable is_being_synced
false
+internal test_table_properties_db listtable
light_schema_change true
+internal test_table_properties_db listtable
min_load_replica_num -1
+internal test_table_properties_db listtable
row_store_page_size 16384
+internal test_table_properties_db listtable
skip_write_index_on_load false
+internal test_table_properties_db listtable storage_format
V2
+internal test_table_properties_db listtable storage_medium
HDD
+internal test_table_properties_db listtable
store_row_column false
+internal test_table_properties_db listtable
time_series_compaction_empty_rowsets_threshold 5
+internal test_table_properties_db listtable
time_series_compaction_file_count_threshold 2000
+internal test_table_properties_db listtable
time_series_compaction_goal_size_mbytes 1024
+internal test_table_properties_db listtable
time_series_compaction_level_threshold 1
+internal test_table_properties_db listtable
time_series_compaction_time_threshold_seconds 3600
+internal test_table_properties_db unique_table _auto_bucket
false
+internal test_table_properties_db unique_table binlog.enable
false
+internal test_table_properties_db unique_table
binlog.max_bytes 9223372036854775807
+internal test_table_properties_db unique_table
binlog.max_history_nums 9223372036854775807
+internal test_table_properties_db unique_table
binlog.ttl_seconds 86400
+internal test_table_properties_db unique_table
compaction_policy size_based
+internal test_table_properties_db unique_table compression
LZ4F
+internal test_table_properties_db unique_table
data_sort.col_num 2
+internal test_table_properties_db unique_table
data_sort.sort_type LEXICAL
+internal test_table_properties_db unique_table
default.replication_allocation tag.location.default: 1
+internal test_table_properties_db unique_table
disable_auto_compaction false
+internal test_table_properties_db unique_table
enable_mow_light_delete false
+internal test_table_properties_db unique_table
enable_single_replica_compaction false
+internal test_table_properties_db unique_table
enable_unique_key_merge_on_write true
+internal test_table_properties_db unique_table
group_commit_data_bytes 134217728
+internal test_table_properties_db unique_table
group_commit_interval_ms 10000
+internal test_table_properties_db unique_table in_memory
false
+internal test_table_properties_db unique_table
inverted_index_storage_format V1
+internal test_table_properties_db unique_table is_being_synced
false
+internal test_table_properties_db unique_table
light_schema_change true
+internal test_table_properties_db unique_table
min_load_replica_num -1
+internal test_table_properties_db unique_table
row_store_page_size 16384
+internal test_table_properties_db unique_table
skip_write_index_on_load false
+internal test_table_properties_db unique_table storage_format
V2
+internal test_table_properties_db unique_table storage_medium
HDD
+internal test_table_properties_db unique_table
store_row_column false
+internal test_table_properties_db unique_table
time_series_compaction_empty_rowsets_threshold 5
+internal test_table_properties_db unique_table
time_series_compaction_file_count_threshold 2000
+internal test_table_properties_db unique_table
time_series_compaction_goal_size_mbytes 1024
+internal test_table_properties_db unique_table
time_series_compaction_level_threshold 1
+internal test_table_properties_db unique_table
time_series_compaction_time_threshold_seconds 3600
+
+-- !select_check_3 --
+internal test_table_properties_db duplicate_table _auto_bucket
false
+internal test_table_properties_db duplicate_table binlog.enable
false
+internal test_table_properties_db duplicate_table
binlog.max_bytes 9223372036854775807
+internal test_table_properties_db duplicate_table
binlog.max_history_nums 9223372036854775807
+internal test_table_properties_db duplicate_table
binlog.ttl_seconds 86400
+internal test_table_properties_db duplicate_table
compaction_policy size_based
+internal test_table_properties_db duplicate_table compression
LZ4F
+internal test_table_properties_db duplicate_table
data_sort.col_num 3
+internal test_table_properties_db duplicate_table
data_sort.sort_type LEXICAL
+internal test_table_properties_db duplicate_table
default.replication_allocation tag.location.default: 1
+internal test_table_properties_db duplicate_table
disable_auto_compaction false
+internal test_table_properties_db duplicate_table
enable_mow_light_delete false
+internal test_table_properties_db duplicate_table
enable_single_replica_compaction false
+internal test_table_properties_db duplicate_table
enable_unique_key_merge_on_write false
+internal test_table_properties_db duplicate_table
group_commit_data_bytes 134217728
+internal test_table_properties_db duplicate_table
group_commit_interval_ms 10000
+internal test_table_properties_db duplicate_table in_memory
false
+internal test_table_properties_db duplicate_table
inverted_index_storage_format V1
+internal test_table_properties_db duplicate_table is_being_synced
false
+internal test_table_properties_db duplicate_table
light_schema_change true
+internal test_table_properties_db duplicate_table
min_load_replica_num -1
+internal test_table_properties_db duplicate_table
row_store_page_size 16384
+internal test_table_properties_db duplicate_table
skip_write_index_on_load false
+internal test_table_properties_db duplicate_table storage_format
V2
+internal test_table_properties_db duplicate_table storage_medium
HDD
+internal test_table_properties_db duplicate_table
store_row_column false
+internal test_table_properties_db duplicate_table
time_series_compaction_empty_rowsets_threshold 5
+internal test_table_properties_db duplicate_table
time_series_compaction_file_count_threshold 2000
+internal test_table_properties_db duplicate_table
time_series_compaction_goal_size_mbytes 1024
+internal test_table_properties_db duplicate_table
time_series_compaction_level_threshold 1
+internal test_table_properties_db duplicate_table
time_series_compaction_time_threshold_seconds 3600
+internal test_table_properties_db unique_table _auto_bucket
false
+internal test_table_properties_db unique_table binlog.enable
false
+internal test_table_properties_db unique_table
binlog.max_bytes 9223372036854775807
+internal test_table_properties_db unique_table
binlog.max_history_nums 9223372036854775807
+internal test_table_properties_db unique_table
binlog.ttl_seconds 86400
+internal test_table_properties_db unique_table
compaction_policy size_based
+internal test_table_properties_db unique_table compression
LZ4F
+internal test_table_properties_db unique_table
data_sort.col_num 2
+internal test_table_properties_db unique_table
data_sort.sort_type LEXICAL
+internal test_table_properties_db unique_table
default.replication_allocation tag.location.default: 1
+internal test_table_properties_db unique_table
disable_auto_compaction false
+internal test_table_properties_db unique_table
enable_mow_light_delete false
+internal test_table_properties_db unique_table
enable_single_replica_compaction false
+internal test_table_properties_db unique_table
enable_unique_key_merge_on_write true
+internal test_table_properties_db unique_table
group_commit_data_bytes 134217728
+internal test_table_properties_db unique_table
group_commit_interval_ms 10000
+internal test_table_properties_db unique_table in_memory
false
+internal test_table_properties_db unique_table
inverted_index_storage_format V1
+internal test_table_properties_db unique_table is_being_synced
false
+internal test_table_properties_db unique_table
light_schema_change true
+internal test_table_properties_db unique_table
min_load_replica_num -1
+internal test_table_properties_db unique_table
row_store_page_size 16384
+internal test_table_properties_db unique_table
skip_write_index_on_load false
+internal test_table_properties_db unique_table storage_format
V2
+internal test_table_properties_db unique_table storage_medium
HDD
+internal test_table_properties_db unique_table
store_row_column false
+internal test_table_properties_db unique_table
time_series_compaction_empty_rowsets_threshold 5
+internal test_table_properties_db unique_table
time_series_compaction_file_count_threshold 2000
+internal test_table_properties_db unique_table
time_series_compaction_goal_size_mbytes 1024
+internal test_table_properties_db unique_table
time_series_compaction_level_threshold 1
+internal test_table_properties_db unique_table
time_series_compaction_time_threshold_seconds 3600
+
+-- !select_check_4 --
+
+-- !select_check_5 --
+internal test_table_properties_db duplicate_table _auto_bucket
false
+internal test_table_properties_db duplicate_table binlog.enable
false
+internal test_table_properties_db duplicate_table
binlog.max_bytes 9223372036854775807
+internal test_table_properties_db duplicate_table
binlog.max_history_nums 9223372036854775807
+internal test_table_properties_db duplicate_table
binlog.ttl_seconds 86400
+internal test_table_properties_db duplicate_table
compaction_policy size_based
+internal test_table_properties_db duplicate_table compression
LZ4F
+internal test_table_properties_db duplicate_table
data_sort.col_num 3
+internal test_table_properties_db duplicate_table
data_sort.sort_type LEXICAL
+internal test_table_properties_db duplicate_table
default.replication_allocation tag.location.default: 1
+internal test_table_properties_db duplicate_table
disable_auto_compaction false
+internal test_table_properties_db duplicate_table
enable_mow_light_delete false
+internal test_table_properties_db duplicate_table
enable_single_replica_compaction false
+internal test_table_properties_db duplicate_table
enable_unique_key_merge_on_write false
+internal test_table_properties_db duplicate_table
group_commit_data_bytes 134217728
+internal test_table_properties_db duplicate_table
group_commit_interval_ms 10000
+internal test_table_properties_db duplicate_table in_memory
false
+internal test_table_properties_db duplicate_table
inverted_index_storage_format V1
+internal test_table_properties_db duplicate_table is_being_synced
false
+internal test_table_properties_db duplicate_table
light_schema_change true
+internal test_table_properties_db duplicate_table
min_load_replica_num -1
+internal test_table_properties_db duplicate_table
row_store_page_size 16384
+internal test_table_properties_db duplicate_table
skip_write_index_on_load false
+internal test_table_properties_db duplicate_table storage_format
V2
+internal test_table_properties_db duplicate_table storage_medium
HDD
+internal test_table_properties_db duplicate_table
store_row_column false
+internal test_table_properties_db duplicate_table
time_series_compaction_empty_rowsets_threshold 5
+internal test_table_properties_db duplicate_table
time_series_compaction_file_count_threshold 2000
+internal test_table_properties_db duplicate_table
time_series_compaction_goal_size_mbytes 1024
+internal test_table_properties_db duplicate_table
time_series_compaction_level_threshold 1
+internal test_table_properties_db duplicate_table
time_series_compaction_time_threshold_seconds 3600
+
+-- !select_check_6 --
+
diff --git
a/regression-test/suites/query_p0/system/test_table_properties.groovy
b/regression-test/suites/query_p0/system/test_table_properties.groovy
new file mode 100644
index 00000000000..7dd55d4fb78
--- /dev/null
+++ b/regression-test/suites/query_p0/system/test_table_properties.groovy
@@ -0,0 +1,124 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements. See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership. The ASF licenses this file
+// to you 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.
+
+suite("test_table_properties") {
+ def dbName = "test_table_properties_db"
+ sql "drop database if exists ${dbName}"
+ sql "CREATE DATABASE IF NOT EXISTS ${dbName}"
+ sql "use ${dbName}"
+
+ sql """
+ CREATE TABLE IF NOT EXISTS unique_table
+ (
+ `user_id` LARGEINT NOT NULL COMMENT "User ID",
+ `username` VARCHAR(50) NOT NULL COMMENT "Username",
+ `city` VARCHAR(20) COMMENT "User location city",
+ `age` SMALLINT COMMENT "User age",
+ `sex` TINYINT COMMENT "User gender",
+ `phone` LARGEINT COMMENT "User phone number",
+ `address` VARCHAR(500) COMMENT "User address",
+ `register_time` DATETIME COMMENT "User registration time"
+ )
+ UNIQUE KEY(`user_id`, `username`)
+ DISTRIBUTED BY HASH(`user_id`) BUCKETS 1
+ PROPERTIES (
+ "replication_allocation" = "tag.location.default: 1"
+ );
+ """
+ sql """
+ CREATE TABLE IF NOT EXISTS duplicate_table
+ (
+ `timestamp` DATETIME NOT NULL COMMENT "Log time",
+ `type` INT NOT NULL COMMENT "Log type",
+ `error_code` INT COMMENT "Error code",
+ `error_msg` VARCHAR(1024) COMMENT "Error detail message",
+ `op_id` BIGINT COMMENT "Operator ID",
+ `op_time` DATETIME COMMENT "Operation time"
+ )
+ DISTRIBUTED BY HASH(`type`) BUCKETS 1
+ PROPERTIES (
+ "replication_allocation" = "tag.location.default: 1"
+ );
+ """
+ sql """
+ CREATE TABLE IF NOT EXISTS listtable
+ (
+ `user_id` LARGEINT NOT NULL COMMENT "User id",
+ `date` DATE NOT NULL COMMENT "Data fill in date time",
+ `timestamp` DATETIME NOT NULL COMMENT "Timestamp of data being
poured",
+ `city` VARCHAR(20) COMMENT "The city where the user is located",
+ `age` SMALLINT COMMENT "User Age",
+ `sex` TINYINT COMMENT "User gender",
+ `last_visit_date` DATETIME REPLACE DEFAULT "1970-01-01 00:00:00"
COMMENT "User last visit time",
+ `cost` BIGINT SUM DEFAULT "0" COMMENT "Total user consumption",
+ `max_dwell_time` INT MAX DEFAULT "0" COMMENT "User maximum dwell
time",
+ `min_dwell_time` INT MIN DEFAULT "99999" COMMENT "User minimum dwell
time"
+ )
+ ENGINE=olap
+ AGGREGATE KEY(`user_id`, `date`, `timestamp`, `city`, `age`, `sex`)
+ PARTITION BY LIST(`city`)
+ (
+ PARTITION `p_cn` VALUES IN ("Beijing", "Shanghai", "Hong Kong"),
+ PARTITION `p_usa` VALUES IN ("New York", "San Francisco"),
+ PARTITION `p_jp` VALUES IN ("Tokyo")
+ )
+ DISTRIBUTED BY HASH(`user_id`) BUCKETS 16
+ PROPERTIES
+ (
+ "replication_num" = "1"
+ );
+ """
+
+ qt_select_check_1 """select count(*) from
information_schema.table_properties where table_schema=\"${dbName}\"; """
+ qt_select_check_2 """select * from information_schema.table_properties
where table_schema=\"${dbName}\" ORDER BY
TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,PROPERTY_NAME,PROPERTY_VALUE; """
+ sql """
+ drop table listtable;
+ """
+ qt_select_check_3 """select * from information_schema.table_properties
where table_schema=\"${dbName}\" ORDER BY
TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,PROPERTY_NAME,PROPERTY_VALUE; """
+
+ def user = "table_properties_user"
+ sql "DROP USER IF EXISTS ${user}"
+ sql "CREATE USER ${user} IDENTIFIED BY '123abc!@#'"
+ //cloud-mode
+ if (isCloudMode()) {
+ def clusters = sql " SHOW CLUSTERS; "
+ assertTrue(!clusters.isEmpty())
+ def validCluster = clusters[0][0]
+ sql """GRANT USAGE_PRIV ON CLUSTER ${validCluster} TO ${user}""";
+ }
+ sql "GRANT SELECT_PRIV ON information_schema.table_properties TO ${user}"
+
+ def tokens = context.config.jdbcUrl.split('/')
+ def url=tokens[0] + "//" + tokens[2] + "/" + "information_schema" + "?"
+
+ connect(user=user, password='123abc!@#', url=url) {
+ qt_select_check_4 """select * from information_schema.table_properties
ORDER BY TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,PROPERTY_NAME,PROPERTY_VALUE;
"""
+ }
+
+ sql "GRANT SELECT_PRIV ON ${dbName}.duplicate_table TO ${user}"
+ connect(user=user, password='123abc!@#', url=url) {
+ qt_select_check_5 """select * from information_schema.table_properties
ORDER BY TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,PROPERTY_NAME,PROPERTY_VALUE;
"""
+ }
+
+ sql "REVOKE SELECT_PRIV ON ${dbName}.duplicate_table FROM ${user}"
+ connect(user=user, password='123abc!@#', url=url) {
+ qt_select_check_6 """select * from information_schema.table_properties
ORDER BY TABLE_CATALOG,TABLE_SCHEMA,TABLE_NAME,PROPERTY_NAME,PROPERTY_VALUE;
"""
+ }
+
+
+
+}
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]