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 <morning...@163.com>
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 <vallish...@gmail.com>
---
 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: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org


Reply via email to