From e6e33d7551a4f1ce2e477dfb3b76d313c687304c Mon Sep 17 00:00:00 2001 From: BiteTheDDDDt Date: Tue, 5 Sep 2023 19:16:12 +0800 Subject: [PATCH] Revert "[feature](information_schema)add metadata_name_ids for quickly get catlogs,db,table and add profiling table in order to Compatible with mysql (#22702) (#23753)" This reverts commit 39c339c7ee802dfc1c0f0ef5e08a54e92e675392. --- 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_profiling_scanner.cpp | 102 -------- .../schema_scanner/schema_profiling_scanner.h | 46 ---- be/src/pipeline/pipeline_task.cpp | 9 +- .../java/org/apache/doris/common/Config.java | 10 - .../doris/analysis/SchemaTableType.java | 5 +- .../org/apache/doris/catalog/SchemaTable.java | 97 +++---- .../doris/datasource/ExternalCatalog.java | 6 - .../org/apache/doris/qe/SessionVariable.java | 8 - .../doris/service/FrontendServiceImpl.java | 131 +--------- gensrc/thrift/Descriptors.thrift | 4 +- gensrc/thrift/FrontendService.thrift | 15 +- .../system/test_metadata_name_ids.out | 14 - .../query_p0/system/test_query_sys_tables.out | 1 - .../system/test_metadata_name_ids.groovy | 47 ---- .../system/test_query_sys_tables.groovy | 1 - 20 files changed, 60 insertions(+), 752 deletions(-) delete mode 100644 be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp delete mode 100644 be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h delete mode 100644 be/src/exec/schema_scanner/schema_profiling_scanner.cpp delete mode 100644 be/src/exec/schema_scanner/schema_profiling_scanner.h delete mode 100644 regression-test/data/query_p0/system/test_metadata_name_ids.out delete mode 100644 regression-test/suites/query_p0/system/test_metadata_name_ids.groovy diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index 9733558284a8fd..dce7cc5c0e66ed 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -31,9 +31,7 @@ #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" @@ -146,10 +144,6 @@ std::unique_ptr 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 3184aed4d2e4f5..9dbc40886e1bc2 100644 --- a/be/src/exec/schema_scanner/schema_helper.cpp +++ b/be/src/exec/schema_scanner/schema_helper.cpp @@ -60,14 +60,6 @@ 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( - 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 900f963f7893cc..72b7a9acf056c1 100644 --- a/be/src/exec/schema_scanner/schema_helper.h +++ b/be/src/exec/schema_scanner/schema_helper.h @@ -34,7 +34,6 @@ class TGetTablesParams; class TGetTablesResult; class TListPrivilegesResult; class TListTableStatusResult; -class TListTableMetadataNameIdsResult; class TShowVariableRequest; class TShowVariableResult; @@ -51,9 +50,6 @@ class SchemaHelper { 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 deleted file mode 100644 index f99d05dc276bc3..00000000000000 --- a/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.cpp +++ /dev/null @@ -1,244 +0,0 @@ -// 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 -#include -#include - -#include - -#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 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 null_datas(table_num, nullptr); - std::vector 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 deleted file mode 100644 index 9981d441d856aa..00000000000000 --- a/be/src/exec/schema_scanner/schema_metadata_name_ids_scanner.h +++ /dev/null @@ -1,54 +0,0 @@ -// 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 - -#include - -#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 _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 deleted file mode 100644 index d2bd8b256fced1..00000000000000 --- a/be/src/exec/schema_scanner/schema_profiling_scanner.cpp +++ /dev/null @@ -1,102 +0,0 @@ -// 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 -#include -#include - -#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 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 deleted file mode 100644 index 5399cb14eb43f5..00000000000000 --- a/be/src/exec/schema_scanner/schema_profiling_scanner.h +++ /dev/null @@ -1,46 +0,0 @@ -// 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 - -#include - -#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 _s_tbls_columns; -}; - -} // namespace doris diff --git a/be/src/pipeline/pipeline_task.cpp b/be/src/pipeline/pipeline_task.cpp index 0462fb3aaad04e..afcd876f8bcb22 100644 --- a/be/src/pipeline/pipeline_task.cpp +++ b/be/src/pipeline/pipeline_task.cpp @@ -51,7 +51,14 @@ PipelineTask::PipelineTask(PipelinePtr& pipeline, uint32_t index, RuntimeState* _operators(pipeline->_operators), _source(_operators.front()), _root(_operators.back()), - _sink(sink) { + _sink(sink), + _prepared(false), + _opened(false), + _state(state), + _cur_state(PipelineTaskState::NOT_READY), + _data_state(SourceState::DEPEND_ON_SOURCE), + _fragment_context(fragment_context), + _parent_profile(parent_profile) { _pipeline_task_watcher.start(); } 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 4a7270d06becf6..3ca13c6437bd0d 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 @@ -2100,14 +2100,4 @@ public class Config extends ConfigBase { @ConfField public static boolean forbid_running_alter_job = false; - - @ConfField - public static int table_stats_health_threshold = 80; - @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 10c958c47b0bb6..f1e6031d23dd0b 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,9 +68,8 @@ 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_METADATA_NAME_IDS("METADATA_NAME_IDS", "METADATA_NAME_IDS", TSchemaTableType.SCH_METADATA_NAME_IDS), - SCH_PROFILING("PROFILING", "PROFILING", TSchemaTableType.SCH_PROFILING); + SCH_PARAMETERS("PARAMETERS", "PARAMETERS", TSchemaTableType.SCH_PARAMETERS); + 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 7215cf0fc7634f..d5035dbd52d3b9 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,66 +384,37 @@ 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())) - .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())) + 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())) .build(); protected SchemaTable(long id, String name, TableType type, List baseSchema) { 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 73c11b37c9cacc..8f322cfd7353f3 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 @@ -578,10 +578,4 @@ public boolean useSelfSplitter() { } return ret; } - - @Override - public Collection getAllDbs() { - makeSureInitialized(); - 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 2ecfc369a55a61..af364fe11dbae8 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,7 +65,6 @@ 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"; @@ -133,9 +132,6 @@ 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 @@ -718,9 +714,6 @@ public int getBeNumberForTest() { return beNumberForTest; } - @VariableMgr.VarAttr(name = PROFILLING) - public boolean profiling = false; - public void setBeNumberForTest(int beNumberForTest) { this.beNumberForTest = beNumberForTest; } @@ -1273,7 +1266,6 @@ public int getInsertTimeoutS() { 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 a6bca2dc0a4af7..7c200f80247065 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,7 +128,6 @@ 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; @@ -167,7 +166,6 @@ 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; @@ -194,19 +192,13 @@ 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; @@ -319,11 +311,8 @@ public TGetDbsResult getDbNames(TGetDbsParams params) throws TException { LOG.debug("get db request: {}", params); TGetDbsResult result = new TGetDbsResult(); - List dbNames = Lists.newArrayList(); - List catalogNames = Lists.newArrayList(); - List dbIds = Lists.newArrayList(); - List catalogIds = Lists.newArrayList(); - + List dbs = Lists.newArrayList(); + List catalogs = Lists.newArrayList(); PatternMatcher matcher = null; if (params.isSetPattern()) { try { @@ -343,51 +332,40 @@ public TGetDbsResult getDbNames(TGetDbsParams params) throws TException { .getCatalogOrException(params.catalog, catalog -> new TException("Unknown catalog " + catalog))); } for (CatalogIf catalog : catalogIfs) { - Collection dbs = new HashSet(); + List dbNames; try { - dbs = catalog.getAllDbs(); + dbNames = catalog.getDbNamesOrEmpty(); } 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. - } - 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()) { + // So continue here to get databases of other catalogs. 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 (DatabaseIf db : dbs) { - String fullName = db.getFullName(); + for (String fullName : dbNames) { if (!env.getAccessManager().checkDbPriv(currentUser, fullName, PrivPredicate.SHOW)) { continue; } - if (matcher != null && !matcher.match(ClusterNamespace.getNameFromFullName(fullName))) { + final String db = ClusterNamespace.getNameFromFullName(fullName); + if (matcher != null && !matcher.match(db)) { continue; } - catalogNames.add(catalog.getName()); - dbNames.add(fullName); - catalogIds.add(catalog.getId()); - dbIds.add(db.getId()); + catalogs.add(catalog.getName()); + dbs.add(fullName); } } - result.setDbs(dbNames); - result.setCatalogs(catalogNames); - result.setCatalogIds(catalogIds); - result.setDbIds(dbIds); + result.setDbs(dbs); + result.setCatalogs(catalogs); return result; } @@ -688,87 +666,6 @@ public TListTableStatusResult listTableStatus(TGetTablesParams params) throws TE return result; } - public TListTableMetadataNameIdsResult listTableMetadataNameIds(TGetTablesParams params) throws TException { - - LOG.debug("get list simple table request: {}", params); - - TListTableMetadataNameIdsResult result = new TListTableMetadataNameIdsResult(); - List 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 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 9e68c6fe09e5ad..40c89c6748630d 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -115,9 +115,7 @@ enum TSchemaTableType { SCH_ROWSETS, SCH_BACKENDS, SCH_COLUMN_STATISTICS, - SCH_PARAMETERS, - SCH_METADATA_NAME_IDS, - SCH_PROFILING; + SCH_PARAMETERS; } enum THdfsCompression { diff --git a/gensrc/thrift/FrontendService.thrift b/gensrc/thrift/FrontendService.thrift index 20a3107e69c8af..6a005450cc5ef0 100644 --- a/gensrc/thrift/FrontendService.thrift +++ b/gensrc/thrift/FrontendService.thrift @@ -308,15 +308,12 @@ 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 , database ids and catalog names ,catalog ids +// getDbNames returns a list of database names and catalog names struct TGetDbsResult { 1: optional list dbs 2: optional list catalogs - 3: optional list db_ids - 4: optional list catalog_ids } // Arguments to getTableNames, which returns a list of tables that match an @@ -354,15 +351,6 @@ struct TListTableStatusResult { 1: required list tables } -struct TTableMetadataNameIds { - 1: optional string name - 2: optional i64 id -} - -struct TListTableMetadataNameIdsResult { - 1: optional list tables -} - // getTableNames returns a list of unqualified table names struct TGetTablesResult { 1: list tables @@ -1107,7 +1095,6 @@ 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 deleted file mode 100644 index 4dc532f4d21ec2..00000000000000 --- a/regression-test/data/query_p0/system/test_metadata_name_ids.out +++ /dev/null @@ -1,14 +0,0 @@ --- 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 7b7ce0583de21d..73300030c84d65 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,4 +57,3 @@ 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 deleted file mode 100644 index ddce7c6a218a64..00000000000000 --- a/regression-test/suites/query_p0/system/test_metadata_name_ids.groovy +++ /dev/null @@ -1,47 +0,0 @@ -// 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 c848f91ebca26f..f74edcd52406ff 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,5 +206,4 @@ 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