This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-2.0
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-2.0 by this push:
     new f67dd5698b [feature](information_schema)add metadata_name_ids for 
quickly get catlogs,db,table and add profiling table in order to Compatible 
with mysql (#24059)
f67dd5698b is described below

commit f67dd5698b9c2c40c3c13db9d1297375bb91dad1
Author: daidai <2017501...@qq.com>
AuthorDate: Fri Sep 8 09:25:07 2023 +0800

    [feature](information_schema)add metadata_name_ids for quickly get 
catlogs,db,table and add profiling table in order to Compatible with mysql 
(#24059)
    
    ## Proposed changes
    add information_schema.metadata_name_idsfor quickly get catlogs,db,table.
    
    1. table  struct :
    ```mysql
    mysql> desc  internal.information_schema.metadata_name_ids;
    +---------------+--------------+------+-------+---------+-------+
    | Field         | Type         | Null | Key   | Default | Extra |
    +---------------+--------------+------+-------+---------+-------+
    | CATALOG_ID    | BIGINT       | Yes  | false | NULL    |       |
    | CATALOG_NAME  | VARCHAR(512) | Yes  | false | NULL    |       |
    | DATABASE_ID   | BIGINT       | Yes  | false | NULL    |       |
    | DATABASE_NAME | VARCHAR(64)  | Yes  | false | NULL    |       |
    | TABLE_ID      | BIGINT       | Yes  | false | NULL    |       |
    | TABLE_NAME    | VARCHAR(64)  | Yes  | false | NULL    |       |
    +---------------+--------------+------+-------+---------+-------+
    6 rows in set (0.00 sec)
    
    mysql> select * from internal.information_schema.metadata_name_ids where 
CATALOG_NAME="hive1" limit 1 \G;
    *************************** 1. row ***************************
       CATALOG_ID: 113008
     CATALOG_NAME: hive1
      DATABASE_ID: 113042
    DATABASE_NAME: ssb1_parquet
         TABLE_ID: 114009
       TABLE_NAME: dates
    1 row in set (0.07 sec)
    ```
    
    2. when you create / drop catalog , need not refresh catalog .
    ```mysql
    mysql> select count(*) from internal.information_schema.metadata_name_ids\G;
    *************************** 1. row ***************************
    count(*): 21301
    1 row in set (0.34 sec)
    
    mysql> drop catalog hive2;
    Query OK, 0 rows affected (0.01 sec)
    
    mysql> select count(*) from internal.information_schema.metadata_name_ids\G;
    *************************** 1. row ***************************
    count(*): 10665
    1 row in set (0.04 sec)
    
    mysql> create catalog hive3 ...
    mysql> select count(*) from internal.information_schema.metadata_name_ids\G;
    *************************** 1. row ***************************
    count(*): 21301
    1 row in set (0.32 sec)
    ```
    
    3. create / drop table , need not refresh catalog .
    ```mysql
    mysql> CREATE TABLE IF NOT EXISTS demo.example_tbl ... ;
    
    mysql> select count(*) from internal.information_schema.metadata_name_ids\G;
    *************************** 1. row ***************************
    count(*): 10666
    1 row in set (0.04 sec)
    
    mysql> drop table demo.example_tbl;
    Query OK, 0 rows affected (0.01 sec)
    
    mysql> select count(*) from internal.information_schema.metadata_name_ids\G;
    *************************** 1. row ***************************
    count(*): 10665
    1 row in set (0.04 sec)
    
    ```
    
    4. you can set query time , prevent queries from taking too long .
    ```
    
    fe.conf :  query_metadata_name_ids_timeout
    
    the time used to obtain all tables in one database
    
    ```
    5. add information_schema.profiling in order to Compatible with  mysql
    
    ```mysql
    mysql> select * from information_schema.profiling;
    Empty set (0.07 sec)
    
    mysql> set profiling=1;
    Query OK, 0 rows affected (0.01 sec)
    ```
---
 be/src/exec/schema_scanner.cpp                     |   6 +
 be/src/exec/schema_scanner/schema_helper.cpp       |   8 +
 be/src/exec/schema_scanner/schema_helper.h         |   4 +
 .../schema_metadata_name_ids_scanner.cpp           | 244 +++++++++++++++++++++
 .../schema_metadata_name_ids_scanner.h             |  54 +++++
 .../schema_scanner/schema_profiling_scanner.cpp    | 102 +++++++++
 .../exec/schema_scanner/schema_profiling_scanner.h |  46 ++++
 .../main/java/org/apache/doris/common/Config.java  |   6 +
 .../org/apache/doris/analysis/SchemaTableType.java |   5 +-
 .../java/org/apache/doris/catalog/SchemaTable.java |  97 +++++---
 .../org/apache/doris/datasource/CatalogIf.java     |   5 +
 .../apache/doris/datasource/ExternalCatalog.java   |   9 +
 .../apache/doris/datasource/InternalCatalog.java   |   5 +
 .../java/org/apache/doris/qe/SessionVariable.java  |   8 +
 .../apache/doris/service/FrontendServiceImpl.java  | 131 +++++++++--
 gensrc/thrift/Descriptors.thrift                   |   4 +-
 gensrc/thrift/FrontendService.thrift               |  15 +-
 .../query_p0/system/test_metadata_name_ids.out     |  14 ++
 .../data/query_p0/system/test_query_sys_tables.out |   1 +
 .../query_p0/system/test_metadata_name_ids.groovy  |  47 ++++
 .../query_p0/system/test_query_sys_tables.groovy   |   1 +
 21 files changed, 760 insertions(+), 52 deletions(-)

diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp
index dce7cc5c0e..9733558284 100644
--- a/be/src/exec/schema_scanner.cpp
+++ b/be/src/exec/schema_scanner.cpp
@@ -31,7 +31,9 @@
 #include "exec/schema_scanner/schema_columns_scanner.h"
 #include "exec/schema_scanner/schema_dummy_scanner.h"
 #include "exec/schema_scanner/schema_files_scanner.h"
+#include "exec/schema_scanner/schema_metadata_name_ids_scanner.h"
 #include "exec/schema_scanner/schema_partitions_scanner.h"
+#include "exec/schema_scanner/schema_profiling_scanner.h"
 #include "exec/schema_scanner/schema_rowsets_scanner.h"
 #include "exec/schema_scanner/schema_schema_privileges_scanner.h"
 #include "exec/schema_scanner/schema_schemata_scanner.h"
@@ -144,6 +146,10 @@ std::unique_ptr<SchemaScanner> 
SchemaScanner::create(TSchemaTableType::type type
         return SchemaPartitionsScanner::create_unique();
     case TSchemaTableType::SCH_ROWSETS:
         return SchemaRowsetsScanner::create_unique();
+    case TSchemaTableType::SCH_METADATA_NAME_IDS:
+        return SchemaMetadataNameIdsScanner::create_unique();
+    case TSchemaTableType::SCH_PROFILING:
+        return SchemaProfilingScanner::create_unique();
     default:
         return SchemaDummyScanner::create_unique();
         break;
diff --git a/be/src/exec/schema_scanner/schema_helper.cpp 
b/be/src/exec/schema_scanner/schema_helper.cpp
index 9dbc40886e..3184aed4d2 100644
--- a/be/src/exec/schema_scanner/schema_helper.cpp
+++ b/be/src/exec/schema_scanner/schema_helper.cpp
@@ -60,6 +60,14 @@ Status SchemaHelper::list_table_status(const std::string& 
ip, const int32_t port
                 client->listTableStatus(*result, request);
             });
 }
+Status SchemaHelper::list_table_metadata_name_ids(const std::string& ip, const 
int32_t port,
+                                                  const 
doris::TGetTablesParams& request,
+                                                  
TListTableMetadataNameIdsResult* result) {
+    return ThriftRpcHelper::rpc<FrontendServiceClient>(
+            ip, port, [&request, &result](FrontendServiceConnection& client) {
+                client->listTableMetadataNameIds(*result, request);
+            });
+}
 
 Status SchemaHelper::describe_table(const std::string& ip, const int32_t port,
                                     const TDescribeTableParams& request,
diff --git a/be/src/exec/schema_scanner/schema_helper.h 
b/be/src/exec/schema_scanner/schema_helper.h
index 72b7a9acf0..900f963f78 100644
--- a/be/src/exec/schema_scanner/schema_helper.h
+++ b/be/src/exec/schema_scanner/schema_helper.h
@@ -34,6 +34,7 @@ class TGetTablesParams;
 class TGetTablesResult;
 class TListPrivilegesResult;
 class TListTableStatusResult;
+class TListTableMetadataNameIdsResult;
 class TShowVariableRequest;
 class TShowVariableResult;
 
@@ -50,6 +51,9 @@ public:
     static Status list_table_status(const std::string& ip, const int32_t port,
                                     const TGetTablesParams& table_params,
                                     TListTableStatusResult* table_result);
+    static Status list_table_metadata_name_ids(const std::string& ip, const 
int32_t port,
+                                               const doris::TGetTablesParams& 
request,
+                                               
TListTableMetadataNameIdsResult* result);
 
     static Status describe_table(const std::string& ip, const int32_t port,
                                  const TDescribeTableParams& desc_params,
diff --git a/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp 
b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp
new file mode 100644
index 0000000000..f99d05dc27
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp
@@ -0,0 +1,244 @@
+// 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_metadata_name_ids_scanner.h"
+
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <stdint.h>
+
+#include <string>
+
+#include "common/status.h"
+#include "exec/schema_scanner/schema_helper.h"
+#include "runtime/decimalv2_value.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
+#include "util/timezone_utils.h"
+#include "vec/common/string_ref.h"
+#include "vec/runtime/vdatetime_value.h"
+
+namespace doris {
+class RuntimeState;
+
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
+std::vector<SchemaScanner::ColumnDesc> 
SchemaMetadataNameIdsScanner::_s_tbls_columns = {
+        //   name,       type,          size,     is_null
+        {"CATALOG_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"CATALOG_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"DATABASE_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"DATABASE_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+        {"TABLE_ID", TYPE_BIGINT, sizeof(int64_t), true},
+        {"TABLE_NAME", TYPE_VARCHAR, sizeof(StringRef), true},
+};
+
+SchemaMetadataNameIdsScanner::SchemaMetadataNameIdsScanner()
+        : SchemaScanner(_s_tbls_columns, 
TSchemaTableType::SCH_METADATA_NAME_IDS), _db_index(0) {}
+
+SchemaMetadataNameIdsScanner::~SchemaMetadataNameIdsScanner() {}
+
+Status SchemaMetadataNameIdsScanner::start(RuntimeState* state) {
+    if (!_is_init) {
+        return Status::InternalError("used before initialized.");
+    }
+    SCOPED_TIMER(_get_db_timer);
+    TGetDbsParams db_params;
+    if (nullptr != _param->db) {
+        db_params.__set_pattern(*(_param->db));
+    }
+    if (nullptr != _param->catalog) {
+        db_params.__set_catalog(*(_param->catalog));
+    }
+    if (nullptr != _param->current_user_ident) {
+        db_params.__set_current_user_ident(*(_param->current_user_ident));
+    } else {
+        if (nullptr != _param->user) {
+            db_params.__set_user(*(_param->user));
+        }
+        if (nullptr != _param->user_ip) {
+            db_params.__set_user_ip(*(_param->user_ip));
+        }
+    }
+    db_params.__set_get_null_catalog(true);
+    if (nullptr != _param->ip && 0 != _param->port) {
+        RETURN_IF_ERROR(
+                SchemaHelper::get_db_names(*(_param->ip), _param->port, 
db_params, &_db_result));
+    } else {
+        return Status::InternalError("IP or port doesn't exists");
+    }
+    return Status::OK();
+}
+
+Status SchemaMetadataNameIdsScanner::_get_new_table() {
+    SCOPED_TIMER(_get_table_timer);
+    if (_db_result.db_ids[_db_index] == -1 &&
+        _db_result.dbs[_db_index] == "NULL") { //catalog is empty.
+        _db_index++;
+        _table_result.tables.clear();
+        _table_result.tables.push_back(TTableMetadataNameIds());
+
+        return Status::OK();
+    }
+    TGetTablesParams table_params;
+    table_params.__set_db(_db_result.dbs[_db_index]);
+    if (_db_result.__isset.catalogs) {
+        table_params.__set_catalog(_db_result.catalogs[_db_index]);
+    }
+    _db_index++;
+    if (nullptr != _param->wild) {
+        table_params.__set_pattern(*(_param->wild));
+    }
+    if (nullptr != _param->current_user_ident) {
+        table_params.__set_current_user_ident(*(_param->current_user_ident));
+    } else {
+        if (nullptr != _param->user) {
+            table_params.__set_user(*(_param->user));
+        }
+        if (nullptr != _param->user_ip) {
+            table_params.__set_user_ip(*(_param->user_ip));
+        }
+    }
+
+    if (nullptr != _param->ip && 0 != _param->port) {
+        
RETURN_IF_ERROR(SchemaHelper::list_table_metadata_name_ids(*(_param->ip), 
_param->port,
+                                                                   
table_params, &_table_result));
+    } else {
+        return Status::InternalError("IP or port doesn't exists");
+    }
+    return Status::OK();
+}
+
+Status SchemaMetadataNameIdsScanner::_fill_block_impl(vectorized::Block* 
block) {
+    SCOPED_TIMER(_fill_block_timer);
+    auto table_num = _table_result.tables.size();
+    if (table_num == 0) { //database is null
+        table_num = 1;
+        _table_result.tables.push_back(TTableMetadataNameIds());
+    }
+    std::vector<void*> null_datas(table_num, nullptr);
+    std::vector<void*> datas(table_num);
+
+    // catalog_id
+    {
+        int64_t srcs[table_num];
+        if (_db_result.__isset.catalog_ids) {
+            int64_t id = _db_result.catalog_ids[_db_index - 1];
+            for (int i = 0; i < table_num; ++i) {
+                srcs[i] = id;
+                datas[i] = srcs + i;
+            }
+            fill_dest_column_for_range(block, 0, datas);
+        } else {
+            fill_dest_column_for_range(block, 0, null_datas);
+        }
+    }
+
+    // catalog_name
+    {
+        if (_db_result.__isset.catalogs) {
+            std::string catalog_name = _db_result.catalogs[_db_index - 1];
+            StringRef str_slot = StringRef(catalog_name.c_str(), 
catalog_name.size());
+            for (int i = 0; i < table_num; ++i) {
+                datas[i] = &str_slot;
+            }
+            fill_dest_column_for_range(block, 1, datas);
+        } else {
+            fill_dest_column_for_range(block, 1, null_datas);
+        }
+    }
+
+    // database_id
+    {
+        int64_t srcs[table_num];
+        if (_db_result.__isset.db_ids) {
+            int64_t id = _db_result.db_ids[_db_index - 1];
+            for (int i = 0; i < table_num; ++i) {
+                srcs[i] = id;
+                datas[i] = srcs + i;
+            }
+            fill_dest_column_for_range(block, 2, datas);
+        } else {
+            fill_dest_column_for_range(block, 2, null_datas);
+        }
+    }
+
+    // database_name
+    {
+        if (_db_result.__isset.dbs) {
+            std::string db_name = 
SchemaHelper::extract_db_name(_db_result.dbs[_db_index - 1]);
+            StringRef str_slot = StringRef(db_name.c_str(), db_name.size());
+            for (int i = 0; i < table_num; ++i) {
+                datas[i] = &str_slot;
+            }
+            fill_dest_column_for_range(block, 3, datas);
+        } else {
+            fill_dest_column_for_range(block, 3, null_datas);
+        }
+    }
+    //     table_id
+    {
+        int64_t srcs[table_num];
+        for (int i = 0; i < table_num; ++i) {
+            if (_table_result.tables[i].__isset.id) {
+                srcs[i] = _table_result.tables[i].id;
+                datas[i] = &srcs;
+            } else {
+                datas[i] = nullptr;
+            }
+        }
+        fill_dest_column_for_range(block, 4, datas);
+    }
+
+    //table_name
+    {
+        StringRef strs[table_num];
+        for (int i = 0; i < table_num; ++i) {
+            if (_table_result.tables[i].__isset.name) {
+                const std::string* src = &_table_result.tables[i].name;
+                strs[i] = StringRef(src->c_str(), src->size());
+                datas[i] = strs + i;
+            } else {
+                datas[i] = nullptr;
+            }
+        }
+        fill_dest_column_for_range(block, 5, datas);
+    }
+
+    return Status::OK();
+}
+
+Status SchemaMetadataNameIdsScanner::get_next_block(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 (_db_index < _db_result.dbs.size()) {
+        RETURN_IF_ERROR(_get_new_table());
+    } else {
+        *eos = true;
+        return Status::OK();
+    }
+    *eos = false;
+    return _fill_block_impl(block);
+}
+
+} // namespace doris
diff --git a/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h 
b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h
new file mode 100644
index 0000000000..9981d441d8
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h
@@ -0,0 +1,54 @@
+// 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 SchemaMetadataNameIdsScanner : public SchemaScanner {
+    ENABLE_FACTORY_CREATOR(SchemaMetadataNameIdsScanner);
+
+public:
+    SchemaMetadataNameIdsScanner();
+    ~SchemaMetadataNameIdsScanner() override;
+
+    Status start(RuntimeState* state) override;
+    Status get_next_block(vectorized::Block* block, bool* eos) override;
+
+private:
+    Status _get_new_table();
+    Status _fill_block_impl(vectorized::Block* block);
+
+    int _db_index;
+    TGetDbsResult _db_result;
+    TListTableMetadataNameIdsResult _table_result;
+    static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns;
+};
+
+} // namespace doris
diff --git a/be/src/exec/schema_scanner/schema_profiling_scanner.cpp 
b/be/src/exec/schema_scanner/schema_profiling_scanner.cpp
new file mode 100644
index 0000000000..d2bd8b256f
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_profiling_scanner.cpp
@@ -0,0 +1,102 @@
+// 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_profiling_scanner.h"
+
+#include <gen_cpp/Descriptors_types.h>
+#include <gen_cpp/FrontendService_types.h>
+#include <stdint.h>
+
+#include "exec/schema_scanner/schema_helper.h"
+#include "runtime/define_primitive_type.h"
+#include "util/runtime_profile.h"
+#include "vec/common/string_ref.h"
+
+namespace doris {
+class RuntimeState;
+namespace vectorized {
+class Block;
+} // namespace vectorized
+
+std::vector<SchemaScanner::ColumnDesc> SchemaProfilingScanner::_s_tbls_columns 
= {
+        //   name,       type,          size,     is_null
+        {"QUERY_ID", TYPE_INT, sizeof(int), false},
+        {"SEQ", TYPE_INT, sizeof(int), false},
+        {"STATE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"DURATION", TYPE_DOUBLE, sizeof(double), false},
+        {"CPU_USER", TYPE_DOUBLE, sizeof(double), true},
+        {"CPU_SYSTEM", TYPE_DOUBLE, sizeof(double), true},
+        {"CONTEXT_VOLUNTARY", TYPE_INT, sizeof(int), true},
+        {"CONTEXT_INVOLUNTARY", TYPE_INT, sizeof(int), true},
+        {"BLOCK_OPS_IN", TYPE_INT, sizeof(int), true},
+        {"BLOCK_OPS_OUT", TYPE_INT, sizeof(int), true},
+        {"MESSAGES_SENT", TYPE_INT, sizeof(int), true},
+        {"MESSAGES_RECEIVED", TYPE_INT, sizeof(int), true},
+        {"PAGE_FAULTS_MAJOR", TYPE_INT, sizeof(int), true},
+        {"PAGE_FAULTS_MINOR", TYPE_INT, sizeof(int), true},
+        {"SWAPS", TYPE_INT, sizeof(int), true},
+        {"SOURCE_FUNCTION", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"SOURCE_FILE", TYPE_VARCHAR, sizeof(StringRef), false},
+        {"SOURCE_LINE", TYPE_INT, sizeof(int), true},
+};
+
+SchemaProfilingScanner::SchemaProfilingScanner()
+        : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_PROFILING) {}
+
+SchemaProfilingScanner::~SchemaProfilingScanner() {}
+
+Status SchemaProfilingScanner::start(RuntimeState* state) {
+    if (!_is_init) {
+        return Status::InternalError("used before initialized.");
+    }
+    SCOPED_TIMER(_get_db_timer);
+    TGetDbsParams db_params;
+    if (nullptr != _param->db) {
+        db_params.__set_pattern(*(_param->db));
+    }
+    if (nullptr != _param->catalog) {
+        db_params.__set_catalog(*(_param->catalog));
+    }
+    if (nullptr != _param->current_user_ident) {
+        db_params.__set_current_user_ident(*(_param->current_user_ident));
+    } else {
+        if (nullptr != _param->user) {
+            db_params.__set_user(*(_param->user));
+        }
+        if (nullptr != _param->user_ip) {
+            db_params.__set_user_ip(*(_param->user_ip));
+        }
+    }
+
+    if (nullptr == _param->ip || 0 == _param->port) {
+        return Status::InternalError("IP or port doesn't exists");
+    }
+    return Status::OK();
+}
+
+Status SchemaProfilingScanner::get_next_block(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.");
+    }
+    *eos = true;
+    return Status::OK();
+}
+
+} // namespace doris
diff --git a/be/src/exec/schema_scanner/schema_profiling_scanner.h 
b/be/src/exec/schema_scanner/schema_profiling_scanner.h
new file mode 100644
index 0000000000..5399cb14eb
--- /dev/null
+++ b/be/src/exec/schema_scanner/schema_profiling_scanner.h
@@ -0,0 +1,46 @@
+// 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 SchemaProfilingScanner : public SchemaScanner {
+    ENABLE_FACTORY_CREATOR(SchemaProfilingScanner);
+
+public:
+    SchemaProfilingScanner();
+    ~SchemaProfilingScanner() override;
+
+    Status start(RuntimeState* state) override;
+    Status get_next_block(vectorized::Block* block, bool* eos) override;
+
+    static std::vector<SchemaScanner::ColumnDesc> _s_tbls_columns;
+};
+
+} // namespace doris
diff --git a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java 
b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
index b95139d3b1..9ca45ffd2a 100644
--- a/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
+++ b/fe/fe-common/src/main/java/org/apache/doris/common/Config.java
@@ -2109,5 +2109,11 @@ public class Config extends ConfigBase {
             "temporary config filed, will make all olap tables enable light 
schema change"
     })
     public static boolean enable_convert_light_weight_schema_change = true;
+    @ConfField(mutable = true, masterOnly = false, description = {
+            "查询information_schema.metadata_name_ids表时,获取一个数据库中所有表用的时间",
+            "When querying the information_schema.metadata_name_ids table,"
+                    + " the time used to obtain all tables in one database"
+    })
+    public static long query_metadata_name_ids_timeout = 3;
 
 }
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 f1e6031d23..10c958c47b 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
@@ -68,8 +68,9 @@ public enum SchemaTableType {
     SCH_CREATE_TABLE("CREATE_TABLE", "CREATE_TABLE", 
TSchemaTableType.SCH_CREATE_TABLE),
     SCH_INVALID("NULL", "NULL", TSchemaTableType.SCH_INVALID),
     SCH_ROWSETS("ROWSETS", "ROWSETS", TSchemaTableType.SCH_ROWSETS),
-    SCH_PARAMETERS("PARAMETERS", "PARAMETERS", 
TSchemaTableType.SCH_PARAMETERS);
-
+    SCH_PARAMETERS("PARAMETERS", "PARAMETERS", 
TSchemaTableType.SCH_PARAMETERS),
+    SCH_METADATA_NAME_IDS("METADATA_NAME_IDS", "METADATA_NAME_IDS", 
TSchemaTableType.SCH_METADATA_NAME_IDS),
+    SCH_PROFILING("PROFILING", "PROFILING", TSchemaTableType.SCH_PROFILING);
     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 d5035dbd52..7215cf0fc7 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
@@ -250,10 +250,10 @@ public class SchemaTable extends Table {
             // Compatible with mysql for mysqldump
             .put("column_statistics",
                     new SchemaTable(SystemIdGenerator.getNextId(), 
"column_statistics", TableType.SCHEMA,
-                    builder().column("SCHEMA_NAME", 
ScalarType.createVarchar(64))
-                            .column("TABLE_NAME", ScalarType.createVarchar(64))
-                            .column("COLUMN_NAME", 
ScalarType.createVarchar(64))
-                            .column("HISTOGRAM", 
ScalarType.createJsonbType()).build()))
+                            builder().column("SCHEMA_NAME", 
ScalarType.createVarchar(64))
+                                    .column("TABLE_NAME", 
ScalarType.createVarchar(64))
+                                    .column("COLUMN_NAME", 
ScalarType.createVarchar(64))
+                                    .column("HISTOGRAM", 
ScalarType.createJsonbType()).build()))
             .put("files",
                     new SchemaTable(SystemIdGenerator.getNextId(), "files", 
TableType.SCHEMA,
                             builder().column("FILE_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
@@ -384,37 +384,66 @@ public class SchemaTable extends Table {
                                     .column("COLLATION_CONNECTION", 
ScalarType.createVarchar(32))
                                     .column("DATABASE_COLLATION", 
ScalarType.createVarchar(32)).build()))
             .put("rowsets", new SchemaTable(SystemIdGenerator.getNextId(), 
"rowsets", TableType.SCHEMA,
-                            builder().column("BACKEND_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("ROWSET_ID", 
ScalarType.createVarchar(64))
-                                    .column("TABLET_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("ROWSET_NUM_ROWS", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("TXN_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("NUM_SEGMENTS", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("START_VERSION", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("END_VERSION", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("INDEX_DISK_SIZE", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("DATA_DISK_SIZE", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("CREATION_TIME", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .column("NEWEST_WRITE_TIMESTAMP", 
ScalarType.createType(PrimitiveType.BIGINT))
-                                    .build()))
+                    builder().column("BACKEND_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("ROWSET_ID", ScalarType.createVarchar(64))
+                            .column("TABLET_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("ROWSET_NUM_ROWS", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("TXN_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("NUM_SEGMENTS", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("START_VERSION", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("END_VERSION", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("INDEX_DISK_SIZE", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("DATA_DISK_SIZE", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("CREATION_TIME", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("NEWEST_WRITE_TIMESTAMP", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .build()))
             .put("parameters", new SchemaTable(SystemIdGenerator.getNextId(), 
"parameters", TableType.SCHEMA,
-                            builder().column("SPECIFIC_CATALOG", 
ScalarType.createVarchar(64))
-                                    .column("SPECIFIC_SCHEMA", 
ScalarType.createVarchar(64))
-                                    .column("SPECIFIC_NAME", 
ScalarType.createVarchar(64))
-                                    .column("ORDINAL_POSITION", 
ScalarType.createVarchar(77))
-                                    .column("PARAMETER_MODE", 
ScalarType.createVarchar(77))
-                                    .column("PARAMETER_NAME", 
ScalarType.createVarchar(77))
-                                    .column("DATA_TYPE", 
ScalarType.createVarchar(64))
-                                    .column("CHARACTER_OCTET_LENGTH", 
ScalarType.createVarchar(64))
-                                    .column("NUMERIC_PRECISION", 
ScalarType.createVarchar(512))
-                                    .column("NUMERIC_SCALE", 
ScalarType.createVarchar(64))
-                                    .column("DATETIME_PRECISION", 
ScalarType.createVarchar(64))
-                                    .column("CHARACTER_SET_NAME", 
ScalarType.createVarchar(256))
-                                    .column("COLLATION_NAME", 
ScalarType.createVarchar(64))
-                                    .column("DTD_IDENTIFIER", 
ScalarType.createVarchar(64))
-                                    .column("ROUTINE_TYPE", 
ScalarType.createVarchar(64))
-                                    .column("DATA_TYPEDTD_IDENDS", 
ScalarType.createVarchar(64))
-                                    .build()))
+                    builder().column("SPECIFIC_CATALOG", 
ScalarType.createVarchar(64))
+                            .column("SPECIFIC_SCHEMA", 
ScalarType.createVarchar(64))
+                            .column("SPECIFIC_NAME", 
ScalarType.createVarchar(64))
+                            .column("ORDINAL_POSITION", 
ScalarType.createVarchar(77))
+                            .column("PARAMETER_MODE", 
ScalarType.createVarchar(77))
+                            .column("PARAMETER_NAME", 
ScalarType.createVarchar(77))
+                            .column("DATA_TYPE", ScalarType.createVarchar(64))
+                            .column("CHARACTER_OCTET_LENGTH", 
ScalarType.createVarchar(64))
+                            .column("NUMERIC_PRECISION", 
ScalarType.createVarchar(512))
+                            .column("NUMERIC_SCALE", 
ScalarType.createVarchar(64))
+                            .column("DATETIME_PRECISION", 
ScalarType.createVarchar(64))
+                            .column("CHARACTER_SET_NAME", 
ScalarType.createVarchar(256))
+                            .column("COLLATION_NAME", 
ScalarType.createVarchar(64))
+                            .column("DTD_IDENTIFIER", 
ScalarType.createVarchar(64))
+                            .column("ROUTINE_TYPE", 
ScalarType.createVarchar(64))
+                            .column("DATA_TYPEDTD_IDENDS", 
ScalarType.createVarchar(64))
+                            .build()))
+            .put("metadata_name_ids", new 
SchemaTable(SystemIdGenerator.getNextId(),
+                        "metadata_name_ids", TableType.SCHEMA,
+                    builder().column("CATALOG_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("CATALOG_NAME", 
ScalarType.createVarchar(FN_REFLEN))
+                            .column("DATABASE_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("DATABASE_NAME", 
ScalarType.createVarchar(NAME_CHAR_LEN))
+                            .column("TABLE_ID", 
ScalarType.createType(PrimitiveType.BIGINT))
+                            .column("TABLE_NAME", 
ScalarType.createVarchar(NAME_CHAR_LEN))
+                            .build()))
+            .put("profiling", new SchemaTable(SystemIdGenerator.getNextId(), 
"profiling", TableType.SCHEMA,
+                    builder().column("QUERY_ID", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("SEQ", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("STATE", ScalarType.createVarchar(30))
+                            .column("DURATION", 
ScalarType.createType(PrimitiveType.DOUBLE))
+                            .column("CPU_USER", 
ScalarType.createType(PrimitiveType.DOUBLE))
+                            .column("CPU_SYSTEM", 
ScalarType.createType(PrimitiveType.DOUBLE))
+                            .column("CONTEXT_VOLUNTARY", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("CONTEXT_INVOLUNTARY", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("BLOCK_OPS_IN", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("BLOCK_OPS_OUT", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("MESSAGES_SENT", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("MESSAGES_RECEIVED", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("PAGE_FAULTS_MAJOR", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("PAGE_FAULTS_MINOR", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("SWAPS", 
ScalarType.createType(PrimitiveType.INT))
+                            .column("SOURCE_FUNCTION", 
ScalarType.createVarchar(30))
+                            .column("SOURCE_FILE", 
ScalarType.createVarchar(20))
+                            .column("SOURCE_LINE", 
ScalarType.createType(PrimitiveType.INT))
+                            .build()))
             .build();
 
     protected SchemaTable(long id, String name, TableType type, List<Column> 
baseSchema) {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java 
b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java
index 8e3304e54a..b84c769eac 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/CatalogIf.java
@@ -30,6 +30,7 @@ import com.google.common.collect.Lists;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
@@ -165,4 +166,8 @@ public interface CatalogIf<T extends DatabaseIf> {
         log.setProps(getProperties());
         return log;
     }
+
+    // Return a copy of all db collection.
+    @SuppressWarnings({"rawtypes", "unchecked"})
+    public Collection<DatabaseIf> getAllDbs();
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java 
b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
index 8f322cfd73..c202ec6f56 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/ExternalCatalog.java
@@ -18,6 +18,7 @@
 package org.apache.doris.datasource;
 
 import org.apache.doris.catalog.Column;
+import org.apache.doris.catalog.DatabaseIf;
 import org.apache.doris.catalog.Env;
 import org.apache.doris.catalog.Resource;
 import org.apache.doris.catalog.external.EsExternalDatabase;
@@ -55,6 +56,8 @@ import java.io.DataInput;
 import java.io.DataOutput;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
@@ -578,4 +581,10 @@ public abstract class ExternalCatalog
         }
         return ret;
     }
+
+    @Override
+    public Collection<DatabaseIf> getAllDbs() {
+        makeSureInitialized();
+        return new HashSet<>(idToDb.values());
+    }
 }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java 
b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
index cef2bcf676..02e737d26c 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/datasource/InternalCatalog.java
@@ -3099,4 +3099,9 @@ public class InternalCatalog implements 
CatalogIf<Database> {
     public ConcurrentHashMap<Long, Database> getIdToDb() {
         return new ConcurrentHashMap<>(idToDb);
     }
+
+    @Override
+    public Collection<DatabaseIf> getAllDbs() {
+        return new HashSet<>(idToDb.values());
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java 
b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
index af364fe11d..2ecfc369a5 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java
@@ -65,6 +65,7 @@ public class SessionVariable implements Serializable, 
Writable {
     public static final String EXEC_MEM_LIMIT = "exec_mem_limit";
     public static final String SCAN_QUEUE_MEM_LIMIT = "scan_queue_mem_limit";
     public static final String QUERY_TIMEOUT = "query_timeout";
+
     public static final String MAX_EXECUTION_TIME = "max_execution_time";
     public static final String INSERT_TIMEOUT = "insert_timeout";
     public static final String ENABLE_PROFILE = "enable_profile";
@@ -132,6 +133,9 @@ public class SessionVariable implements Serializable, 
Writable {
     public static final String DEFAULT_STORAGE_ENGINE = 
"default_storage_engine";
     public static final String DEFAULT_TMP_STORAGE_ENGINE = 
"default_tmp_storage_engine";
 
+    // Compatible with  mysql
+    public static final String PROFILLING = "profiling";
+
     public static final String DIV_PRECISION_INCREMENT = 
"div_precision_increment";
 
     // see comment of `doris_max_scan_key_num` and 
`max_pushdown_conditions_per_column` in BE config
@@ -714,6 +718,9 @@ public class SessionVariable implements Serializable, 
Writable {
         return beNumberForTest;
     }
 
+    @VariableMgr.VarAttr(name = PROFILLING)
+    public boolean profiling = false;
+
     public void setBeNumberForTest(int beNumberForTest) {
         this.beNumberForTest = beNumberForTest;
     }
@@ -1266,6 +1273,7 @@ public class SessionVariable implements Serializable, 
Writable {
         return insertTimeoutS;
     }
 
+
     public void setInsertTimeoutS(int insertTimeoutS) {
         this.insertTimeoutS = insertTimeoutS;
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java 
b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
index 7c200f8024..a6bca2dc0a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/service/FrontendServiceImpl.java
@@ -128,6 +128,7 @@ import org.apache.doris.thrift.TGetTabletReplicaInfosResult;
 import org.apache.doris.thrift.TInitExternalCtlMetaRequest;
 import org.apache.doris.thrift.TInitExternalCtlMetaResult;
 import org.apache.doris.thrift.TListPrivilegesResult;
+import org.apache.doris.thrift.TListTableMetadataNameIdsResult;
 import org.apache.doris.thrift.TListTableStatusResult;
 import org.apache.doris.thrift.TLoadTxn2PCRequest;
 import org.apache.doris.thrift.TLoadTxn2PCResult;
@@ -166,6 +167,7 @@ import 
org.apache.doris.thrift.TStreamLoadMultiTablePutResult;
 import org.apache.doris.thrift.TStreamLoadPutRequest;
 import org.apache.doris.thrift.TStreamLoadPutResult;
 import org.apache.doris.thrift.TTableIndexQueryStats;
+import org.apache.doris.thrift.TTableMetadataNameIds;
 import org.apache.doris.thrift.TTableQueryStats;
 import org.apache.doris.thrift.TTableStatus;
 import org.apache.doris.thrift.TUpdateExportTaskStatusRequest;
@@ -192,13 +194,19 @@ import java.time.Instant;
 import java.time.ZoneId;
 import java.time.ZonedDateTime;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.function.IntSupplier;
@@ -311,8 +319,11 @@ public class FrontendServiceImpl implements 
FrontendService.Iface {
         LOG.debug("get db request: {}", params);
         TGetDbsResult result = new TGetDbsResult();
 
-        List<String> dbs = Lists.newArrayList();
-        List<String> catalogs = Lists.newArrayList();
+        List<String> dbNames = Lists.newArrayList();
+        List<String> catalogNames = Lists.newArrayList();
+        List<Long> dbIds = Lists.newArrayList();
+        List<Long> catalogIds = Lists.newArrayList();
+
         PatternMatcher matcher = null;
         if (params.isSetPattern()) {
             try {
@@ -332,40 +343,51 @@ public class FrontendServiceImpl implements 
FrontendService.Iface {
                     .getCatalogOrException(params.catalog, catalog -> new 
TException("Unknown catalog " + catalog)));
         }
         for (CatalogIf catalog : catalogIfs) {
-            List<String> dbNames;
+            Collection<DatabaseIf> dbs = new HashSet<DatabaseIf>();
             try {
-                dbNames = catalog.getDbNamesOrEmpty();
+                dbs = catalog.getAllDbs();
             } catch (Exception e) {
                 LOG.warn("failed to get database names for catalog {}", 
catalog.getName(), e);
                 // Some external catalog may fail to get databases due to 
wrong connection info.
-                // So continue here to get databases of other catalogs.
+            }
+            LOG.debug("get db size: {}, in catalog: {}", dbs.size(), 
catalog.getName());
+            if (dbs.isEmpty() && params.isSetGetNullCatalog() && 
params.get_null_catalog) {
+                catalogNames.add(catalog.getName());
+                dbNames.add("NULL");
+                catalogIds.add(catalog.getId());
+                dbIds.add(-1L);
+                continue;
+            }
+            if (dbs.isEmpty()) {
                 continue;
             }
-            LOG.debug("get db names: {}, in catalog: {}", dbNames, 
catalog.getName());
-
             UserIdentity currentUser = null;
             if (params.isSetCurrentUserIdent()) {
                 currentUser = 
UserIdentity.fromThrift(params.current_user_ident);
             } else {
                 currentUser = 
UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip);
             }
-            for (String fullName : dbNames) {
+            for (DatabaseIf db : dbs) {
+                String fullName = db.getFullName();
                 if (!env.getAccessManager().checkDbPriv(currentUser, fullName, 
PrivPredicate.SHOW)) {
                     continue;
                 }
 
-                final String db = 
ClusterNamespace.getNameFromFullName(fullName);
-                if (matcher != null && !matcher.match(db)) {
+                if (matcher != null && 
!matcher.match(ClusterNamespace.getNameFromFullName(fullName))) {
                     continue;
                 }
 
-                catalogs.add(catalog.getName());
-                dbs.add(fullName);
+                catalogNames.add(catalog.getName());
+                dbNames.add(fullName);
+                catalogIds.add(catalog.getId());
+                dbIds.add(db.getId());
             }
         }
 
-        result.setDbs(dbs);
-        result.setCatalogs(catalogs);
+        result.setDbs(dbNames);
+        result.setCatalogs(catalogNames);
+        result.setCatalogIds(catalogIds);
+        result.setDbIds(dbIds);
         return result;
     }
 
@@ -666,6 +688,87 @@ public class FrontendServiceImpl implements 
FrontendService.Iface {
         return result;
     }
 
+    public TListTableMetadataNameIdsResult 
listTableMetadataNameIds(TGetTablesParams params) throws TException {
+
+        LOG.debug("get list simple table request: {}", params);
+
+        TListTableMetadataNameIdsResult result = new 
TListTableMetadataNameIdsResult();
+        List<TTableMetadataNameIds> tablesResult = Lists.newArrayList();
+        result.setTables(tablesResult);
+
+        UserIdentity currentUser;
+        if (params.isSetCurrentUserIdent()) {
+            currentUser = UserIdentity.fromThrift(params.current_user_ident);
+        } else {
+            currentUser = 
UserIdentity.createAnalyzedUserIdentWithIp(params.user, params.user_ip);
+        }
+
+        String catalogName;
+        if (params.isSetCatalog()) {
+            catalogName = params.catalog;
+        } else {
+            catalogName = InternalCatalog.INTERNAL_CATALOG_NAME;
+        }
+
+        PatternMatcher matcher = null;
+        if (params.isSetPattern()) {
+            try {
+                matcher = 
PatternMatcher.createMysqlPattern(params.getPattern(),
+                    CaseSensibility.TABLE.getCaseSensibility());
+            } catch (PatternMatcherException e) {
+                throw new TException("Pattern is in bad format " + 
params.getPattern());
+            }
+        }
+        PatternMatcher finalMatcher = matcher;
+
+
+        ExecutorService executor = Executors.newSingleThreadExecutor();
+        Future<?> future = executor.submit(() -> {
+
+            CatalogIf catalog = 
Env.getCurrentEnv().getCatalogMgr().getCatalog(catalogName);
+            if (catalog != null) {
+                DatabaseIf db = catalog.getDbNullable(params.db);
+                if (db != null) {
+                    List<TableIf> tables = db.getTables();
+                    for (TableIf table : tables) {
+                        if 
(!Env.getCurrentEnv().getAccessManager().checkTblPriv(currentUser, params.db,
+                                table.getName(), PrivPredicate.SHOW)) {
+                            continue;
+                        }
+                        table.readLock();
+                        try {
+                            if (finalMatcher != null && 
!finalMatcher.match(table.getName())) {
+                                continue;
+                            }
+                            TTableMetadataNameIds status = new 
TTableMetadataNameIds();
+                            status.setName(table.getName());
+                            status.setId(table.getId());
+
+                            tablesResult.add(status);
+                        } finally {
+                            table.readUnlock();
+                        }
+                    }
+                }
+            }
+        });
+        try {
+            if (catalogName.equals(InternalCatalog.INTERNAL_CATALOG_NAME)) {
+                future.get();
+            } else {
+                future.get(Config.query_metadata_name_ids_timeout, 
TimeUnit.SECONDS);
+            }
+        } catch (TimeoutException e) {
+            future.cancel(true);
+            LOG.info("From catalog:{},db:{} get tables timeout.", catalogName, 
params.db);
+        } catch (InterruptedException | ExecutionException e) {
+            future.cancel(true);
+        } finally {
+            executor.shutdown();
+        }
+        return result;
+    }
+
     @Override
     public TListPrivilegesResult listTablePrivilegeStatus(TGetTablesParams 
params) throws TException {
         LOG.debug("get list table privileges request: {}", params);
diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift
index 40c89c6748..9e68c6fe09 100644
--- a/gensrc/thrift/Descriptors.thrift
+++ b/gensrc/thrift/Descriptors.thrift
@@ -115,7 +115,9 @@ enum TSchemaTableType {
     SCH_ROWSETS,
     SCH_BACKENDS,
     SCH_COLUMN_STATISTICS,
-    SCH_PARAMETERS;
+    SCH_PARAMETERS,
+    SCH_METADATA_NAME_IDS,
+    SCH_PROFILING;
 }
 
 enum THdfsCompression {
diff --git a/gensrc/thrift/FrontendService.thrift 
b/gensrc/thrift/FrontendService.thrift
index 14bf4ac1ae..d792323867 100644
--- a/gensrc/thrift/FrontendService.thrift
+++ b/gensrc/thrift/FrontendService.thrift
@@ -308,12 +308,15 @@ struct TGetDbsParams {
   3: optional string user_ip    // deprecated
   4: optional Types.TUserIdentity current_user_ident // to replace the user 
and user ip
   5: optional string catalog
+  6: optional bool get_null_catalog  //if catalog is empty , get dbName 
="NULL" and dbId = -1.
 }
 
-// getDbNames returns a list of database names and catalog names
+// getDbNames returns a list of database names , database ids and catalog 
names ,catalog ids
 struct TGetDbsResult {
   1: optional list<string> dbs
   2: optional list<string> catalogs
+  3: optional list<i64> db_ids
+  4: optional list<i64> catalog_ids
 }
 
 // Arguments to getTableNames, which returns a list of tables that match an
@@ -351,6 +354,15 @@ struct TListTableStatusResult {
     1: required list<TTableStatus> tables
 }
 
+struct TTableMetadataNameIds {
+    1: optional string name
+    2: optional i64 id 
+}
+
+struct TListTableMetadataNameIdsResult {
+    1: optional list<TTableMetadataNameIds> tables 
+}
+
 // getTableNames returns a list of unqualified table names
 struct TGetTablesResult {
   1: list<string> tables
@@ -1096,6 +1108,7 @@ service FrontendService {
     TMasterOpResult forward(1: TMasterOpRequest params)
 
     TListTableStatusResult listTableStatus(1: TGetTablesParams params)
+    TListTableMetadataNameIdsResult listTableMetadataNameIds(1: 
TGetTablesParams params)
     TListPrivilegesResult listTablePrivilegeStatus(1: TGetTablesParams params)
     TListPrivilegesResult listSchemaPrivilegeStatus(1: TGetTablesParams params)
     TListPrivilegesResult listUserPrivilegeStatus(1: TGetTablesParams params)
diff --git a/regression-test/data/query_p0/system/test_metadata_name_ids.out 
b/regression-test/data/query_p0/system/test_metadata_name_ids.out
new file mode 100644
index 0000000000..4dc532f4d2
--- /dev/null
+++ b/regression-test/data/query_p0/system/test_metadata_name_ids.out
@@ -0,0 +1,14 @@
+-- This file is automatically generated. You should know what you did if you 
want to edit this
+-- !desc --
+CATALOG_ID     BIGINT  Yes     false   \N      
+CATALOG_NAME   VARCHAR(512)    Yes     false   \N      
+DATABASE_ID    BIGINT  Yes     false   \N      
+DATABASE_NAME  VARCHAR(64)     Yes     false   \N      
+TABLE_ID       BIGINT  Yes     false   \N      
+TABLE_NAME     VARCHAR(64)     Yes     false   \N      
+
+-- !select2 --
+internal       demo    test_metadata_name_ids
+
+-- !select3 --
+
diff --git a/regression-test/data/query_p0/system/test_query_sys_tables.out 
b/regression-test/data/query_p0/system/test_query_sys_tables.out
index 73300030c8..7b7ce0583d 100644
--- a/regression-test/data/query_p0/system/test_query_sys_tables.out
+++ b/regression-test/data/query_p0/system/test_query_sys_tables.out
@@ -57,3 +57,4 @@ test_view
 
 -- !sql --
 
+-- !sql --
diff --git 
a/regression-test/suites/query_p0/system/test_metadata_name_ids.groovy 
b/regression-test/suites/query_p0/system/test_metadata_name_ids.groovy
new file mode 100644
index 0000000000..ddce7c6a21
--- /dev/null
+++ b/regression-test/suites/query_p0/system/test_metadata_name_ids.groovy
@@ -0,0 +1,47 @@
+// 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_metadata_name_ids", "p0" ) {
+    
+
+       def tableName = "internal.information_schema.metadata_name_ids"
+    qt_desc """ desc  ${tableName} """
+
+
+       sql """ create database if not exists demo; """
+       sql """ use demo ; """ 
+       
+       sql """ create table if not exists test_metadata_name_ids  (
+        a int ,
+        b varchar(30)
+    )
+    DUPLICATE KEY(`a`)
+    DISTRIBUTED BY HASH(`a`) BUCKETS 10
+    PROPERTIES (
+        "replication_allocation" = "tag.location.default: 1"
+    ); 
+       """
+       qt_select2 """ select CATALOG_NAME,DATABASE_NAME,TABLE_NAME from 
${tableName}
+               where CATALOG_NAME="internal" and DATABASE_NAME ="demo" and 
TABLE_NAME="test_metadata_name_ids";""" 
+
+       sql """ drop table test_metadata_name_ids """ 
+
+       qt_select3 """ select CATALOG_NAME,DATABASE_NAME,TABLE_NAME from 
${tableName}
+               where CATALOG_NAME="internal" and DATABASE_NAME ="demo" and 
TABLE_NAME="test_metadata_name_ids";""" 
+
+
+}
diff --git 
a/regression-test/suites/query_p0/system/test_query_sys_tables.groovy 
b/regression-test/suites/query_p0/system/test_query_sys_tables.groovy
index f74edcd524..c848f91ebc 100644
--- a/regression-test/suites/query_p0/system/test_query_sys_tables.groovy
+++ b/regression-test/suites/query_p0/system/test_query_sys_tables.groovy
@@ -206,4 +206,5 @@ suite("test_query_sys_tables", "query,p0") {
     qt_sql "select * from key_column_usage"
     qt_sql "select * from triggers"
     qt_sql "select * from parameters"
+    qt_sql "select * from profiling"
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org
For additional commands, e-mail: commits-h...@doris.apache.org

Reply via email to