From 5a5ca4ff0f1a9bd8651f78f225c5bf05e9461176 Mon Sep 17 00:00:00 2001 From: morningman Date: Sun, 18 Aug 2024 21:35:24 +0800 Subject: [PATCH 1/5] 1 --- .../io/cache/block/block_lru_file_cache.cpp | 39 +++++++++++++++++-- be/src/io/cache/block/block_lru_file_cache.h | 6 ++- 2 files changed, 39 insertions(+), 6 deletions(-) diff --git a/be/src/io/cache/block/block_lru_file_cache.cpp b/be/src/io/cache/block/block_lru_file_cache.cpp index 349c626fb7d8dd..cb73bfc1faba15 100644 --- a/be/src/io/cache/block/block_lru_file_cache.cpp +++ b/be/src/io/cache/block/block_lru_file_cache.cpp @@ -59,6 +59,8 @@ namespace doris { namespace io { DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_hits_ratio, MetricUnit::NOUNIT); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_hits_ratio_5m, MetricUnit::NOUNIT); +DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_hits_ratio_1h, MetricUnit::NOUNIT); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_removed_elements, MetricUnit::OPERATIONS); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_index_queue_max_size, MetricUnit::BYTES); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_index_queue_curr_size, MetricUnit::BYTES); @@ -74,6 +76,17 @@ DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_disposable_queue_max_elements, Met DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_disposable_queue_curr_elements, MetricUnit::NOUNIT); DEFINE_GAUGE_METRIC_PROTOTYPE_2ARG(file_cache_segment_reader_cache_size, MetricUnit::NOUNIT); +bvar::Adder g_file_cache_num_read_segments("doris_file_cache_num_read_segments"); +bvar::Adder g_file_cache_num_hit_segments("doris_file_cache_num_hit_segments"); +bvar::Window> g_file_cache_num_hit_segments_5m(&g_file_cache_num_hit_segments, + 300); +bvar::Window> g_file_cache_num_read_segments_5m( + &g_file_cache_num_read_segments, 300); +bvar::Window> g_file_cache_num_hit_segments_1h(&g_file_cache_num_hit_segments, + 3600); +bvar::Window> g_file_cache_num_read_segments_1h( + &g_file_cache_num_read_segments, 3600); + LRUFileCache::LRUFileCache(const std::string& cache_base_path, const FileCacheSettings& cache_settings) : IFileCache(cache_base_path, cache_settings) { @@ -89,6 +102,8 @@ LRUFileCache::LRUFileCache(const std::string& cache_base_path, _entity->register_hook(_cache_base_path, std::bind(&LRUFileCache::update_cache_metrics, this)); INT_DOUBLE_METRIC_REGISTER(_entity, file_cache_hits_ratio); + INT_DOUBLE_METRIC_REGISTER(_entity, file_cache_hits_ratio_5m); + INT_DOUBLE_METRIC_REGISTER(_entity, file_cache_hits_ratio_1h); INT_UGAUGE_METRIC_REGISTER(_entity, file_cache_removed_elements); INT_UGAUGE_METRIC_REGISTER(_entity, file_cache_index_queue_max_size); @@ -416,10 +431,10 @@ FileBlocksHolder LRUFileCache::get_or_set(const Key& key, size_t offset, size_t } DCHECK(!file_blocks.empty()); - _num_read_segments += file_blocks.size(); + g_file_cache_num_read_segments << file_blocks.size(); for (auto& segment : file_blocks) { if (segment->state() == FileBlock::State::DOWNLOADED) { - _num_hit_segments++; + g_file_cache_num_hit_segments << 1; } } return FileBlocksHolder(std::move(file_blocks)); @@ -1147,11 +1162,27 @@ void LRUFileCache::run_background_operation() { void LRUFileCache::update_cache_metrics() const { std::lock_guard l(_mutex); double hit_ratio = 0; - if (_num_read_segments > 0) { - hit_ratio = (double)_num_hit_segments / (double)_num_read_segments; + double hit_ratio_5m = 0; + double hit_ratio_1h = 0; + if (g_file_cache_num_read_segments.get_value() > 0) { + hit_ratio = ((double)g_file_cache_num_hit_segments.get_value()) / + ((double)g_file_cache_num_read_segments.get_value()); + } + if (g_file_cache_num_read_segments_5m.get_value() > 0) { + hit_ratio_5m = ((double)g_file_cache_num_hit_segments_5m.get_value()) / + ((double)g_file_cache_num_read_segments_5m.get_value()); + } else { + hit_ratio_5m = 0.0; + } + if (g_file_cache_num_read_segments_1h.get_value() > 0) { + hit_ratio_1h = ((double)g_file_cache_num_hit_segments_1h.get_value()) / + ((double)g_file_cache_num_read_segments_1h.get_value()); + hit_ratio_1h = 0.0; } file_cache_hits_ratio->set_value(hit_ratio); + file_cache_hits_ratio_5m->set_value(hit_ratio_5m); + file_cache_hits_ratio_1h->set_value(hit_ratio_1h); file_cache_removed_elements->set_value(_num_removed_segments); file_cache_index_queue_max_size->set_value(_index_queue.get_max_size()); diff --git a/be/src/io/cache/block/block_lru_file_cache.h b/be/src/io/cache/block/block_lru_file_cache.h index bcf00d938a725b..0f2a922797b74d 100644 --- a/be/src/io/cache/block/block_lru_file_cache.h +++ b/be/src/io/cache/block/block_lru_file_cache.h @@ -206,13 +206,15 @@ class LRUFileCache final : public IFileCache { std::thread _cache_background_thread; std::atomic_bool _lazy_open_done {true}; std::thread _cache_background_load_thread; - size_t _num_read_segments = 0; - size_t _num_hit_segments = 0; + // size_t _num_read_segments = 0; + // size_t _num_hit_segments = 0; size_t _num_removed_segments = 0; std::shared_ptr _entity; DoubleGauge* file_cache_hits_ratio = nullptr; + DoubleGauge* file_cache_hits_ratio_5m = nullptr; + DoubleGauge* file_cache_hits_ratio_1h = nullptr; UIntGauge* file_cache_removed_elements = nullptr; UIntGauge* file_cache_index_queue_max_size = nullptr; From 1bbfc2ff62017e6961c4eb5fa9b9783c0c4a3b8d Mon Sep 17 00:00:00 2001 From: morningman Date: Sun, 18 Aug 2024 22:49:21 +0800 Subject: [PATCH 2/5] 1 --- be/src/exec/schema_scanner.cpp | 3 + .../schema_file_cache_statistics.cpp | 89 +++++++++++++++++++ .../schema_file_cache_statistics.h | 49 ++++++++++ .../org/apache/doris/catalog/SchemaTable.java | 8 ++ .../BackendPartitionedSchemaScanNode.java | 5 +- gensrc/thrift/Descriptors.thrift | 3 +- 6 files changed, 155 insertions(+), 2 deletions(-) create mode 100644 be/src/exec/schema_scanner/schema_file_cache_statistics.cpp create mode 100644 be/src/exec/schema_scanner/schema_file_cache_statistics.h diff --git a/be/src/exec/schema_scanner.cpp b/be/src/exec/schema_scanner.cpp index d9cafcf9049a53..70011c50bb02c5 100644 --- a/be/src/exec/schema_scanner.cpp +++ b/be/src/exec/schema_scanner.cpp @@ -32,6 +32,7 @@ #include "exec/schema_scanner/schema_collations_scanner.h" #include "exec/schema_scanner/schema_columns_scanner.h" #include "exec/schema_scanner/schema_dummy_scanner.h" +#include "exec/schema_scanner/schema_file_cache_statistics.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" @@ -224,6 +225,8 @@ std::unique_ptr SchemaScanner::create(TSchemaTableType::type type return SchemaUserScanner::create_unique(); case TSchemaTableType::SCH_WORKLOAD_POLICY: return SchemaWorkloadSchedulePolicyScanner::create_unique(); + case TSchemaTableType::SCH_FILE_CACHE_STATISTICS: + return SchemaFileCacheStatisticsScanner::create_unique(); default: return SchemaDummyScanner::create_unique(); break; diff --git a/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp b/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp new file mode 100644 index 00000000000000..45f951d03be04b --- /dev/null +++ b/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp @@ -0,0 +1,89 @@ +// 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_backend_active_tasks.h" + +#include "runtime/exec_env.h" +#include "runtime/runtime_query_statistics_mgr.h" +#include "runtime/runtime_state.h" +#include "vec/common/string_ref.h" +#include "vec/core/block.h" +#include "vec/data_types/data_type_factory.hpp" + +namespace doris { +std::vector SchemaFileCacheStatisticsScanner::_s_tbls_columns = { + // name, type, size + {"BE_ID", TYPE_BIGINT, sizeof(StringRef), false}, + {"CACHE_PATH", TYPE_VARCHAR, sizeof(StringRef), false}, + {"CACHE_TYPE", TYPE_VARCHAR, sizeof(StringRef), false}, + {"METRIC_NAME", TYPE_VARCHAR, sizeof(StringRef), false}, + {"METRIC_VALUE", TYPE_DOUBLE, sizeof(double), false} +}; + +SchemaFileCacheStatisticsScanner::SchemaFileCacheStatisticsScanner() + : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_FILE_CACHE_STATISTICS) {} + +SchemaFileCacheStatisticsScanner::~SchemaFileCacheStatisticsScanner() {} + +Status SchemaFileCacheStatisticsScanner::start(RuntimeState* state) { + _block_rows_limit = state->batch_size(); + return Status::OK(); +} + +Status SchemaFileCacheStatisticsScanner::get_next_block_internal(vectorized::Block* block, + bool* eos) { + if (!_is_init) { + return Status::InternalError("Used before initialized."); + } + + if (nullptr == block || nullptr == eos) { + return Status::InternalError("input pointer is nullptr."); + } + + if (_task_stats_block == nullptr) { + _task_stats_block = vectorized::Block::create_unique(); + + for (int i = 0; i < _s_tbls_columns.size(); ++i) { + TypeDescriptor descriptor(_s_tbls_columns[i].type); + auto data_type = + vectorized::DataTypeFactory::instance().create_data_type(descriptor, true); + _task_stats_block->insert(vectorized::ColumnWithTypeAndName( + data_type->create_column(), data_type, _s_tbls_columns[i].name)); + } + + _task_stats_block->reserve(_block_rows_limit); + + ExecEnv::GetInstance()->runtime_query_statistics_mgr()->get_active_be_tasks_block( + _task_stats_block.get()); + _total_rows = _task_stats_block->rows(); + } + + if (_row_idx == _total_rows) { + *eos = true; + return Status::OK(); + } + + int current_batch_rows = std::min(_block_rows_limit, _total_rows - _row_idx); + vectorized::MutableBlock mblock = vectorized::MutableBlock::build_mutable_block(block); + RETURN_IF_ERROR(mblock.add_rows(_task_stats_block.get(), _row_idx, current_batch_rows)); + _row_idx += current_batch_rows; + + *eos = _row_idx == _total_rows; + return Status::OK(); +} + +} // namespace doris diff --git a/be/src/exec/schema_scanner/schema_file_cache_statistics.h b/be/src/exec/schema_scanner/schema_file_cache_statistics.h new file mode 100644 index 00000000000000..00388dcfd2ab90 --- /dev/null +++ b/be/src/exec/schema_scanner/schema_file_cache_statistics.h @@ -0,0 +1,49 @@ +// 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 "common/status.h" +#include "exec/schema_scanner.h" + +namespace doris { +class RuntimeState; +namespace vectorized { +class Block; +} // namespace vectorized + +class SchemaFileCacheStatisticsScanner : public SchemaScanner { + ENABLE_FACTORY_CREATOR(SchemaFileCacheStatisticsScanner); + +public: + SchemaFileCacheStatisticsScanner(); + ~SchemaFileCacheStatisticsScanner() override; + + Status start(RuntimeState* state) override; + Status get_next_block_internal(vectorized::Block* block, bool* eos) override; + + static std::vector _s_tbls_columns; + +private: + int _block_rows_limit = 4096; + int _row_idx = 0; + int _total_rows = 0; + std::unique_ptr _task_stats_block = nullptr; +}; +}; // namespace doris 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 53b00b0880a582..3caff714f8eafb 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 @@ -516,6 +516,14 @@ public class SchemaTable extends Table { .column("VERSION", ScalarType.createType(PrimitiveType.INT)) .column("WORKLOAD_GROUP", ScalarType.createStringType()) .build())) + .put("file_cache_statistics", + new SchemaTable(SystemIdGenerator.getNextId(), "file_cache_statistics", TableType.SCHEMA, + builder().column("BE_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("CACHE_PATH", ScalarType.createVarchar(256)) + .column("CACHE_TYPE", ScalarType.createVarchar(256)) + .column("METRIC_NAME", ScalarType.createVarchar(256)) + .column("METRIC_VALUE", ScalarType.createType(PrimitiveType.DOUBLE)) + .build())) .build(); private boolean fetchAllFe = false; diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java index 803afe05d8ddbc..c9ae017bbf0bed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java @@ -66,6 +66,9 @@ public class BackendPartitionedSchemaScanNode extends SchemaScanNode { BACKEND_TABLE.add("backend_active_tasks"); BEACKEND_ID_COLUMN_SET.add("be_id"); + + BACKEND_TABLE.add("file_cache_statistics"); + BEACKEND_ID_COLUMN_SET.add("be_id"); } public static boolean isBackendPartitionedSchemaTable(String tableName) { @@ -84,7 +87,7 @@ public static boolean isBackendPartitionedSchemaTable(String tableName) { private PartitionInfo backendPartitionInfo; // partitionID -> backendID private Map partitionIDToBackendID; - private Collection selectedPartitionIds = Lists.newArrayList(); + private Collection selectedPartiytionIds = Lists.newArrayList(); public BackendPartitionedSchemaScanNode(PlanNodeId id, TupleDescriptor desc) { super(id, desc); diff --git a/gensrc/thrift/Descriptors.thrift b/gensrc/thrift/Descriptors.thrift index 5da7b4df7de799..36c52e2f6e285f 100644 --- a/gensrc/thrift/Descriptors.thrift +++ b/gensrc/thrift/Descriptors.thrift @@ -131,7 +131,8 @@ enum TSchemaTableType { SCH_WORKLOAD_GROUPS, SCH_USER, SCH_PROCS_PRIV, - SCH_WORKLOAD_POLICY; + SCH_WORKLOAD_POLICY, + SCH_FILE_CACHE_STATISTICS; } enum THdfsCompression { From cc6238b37fb893aeaccb8a2018c48c16e1a9a32e Mon Sep 17 00:00:00 2001 From: morningman Date: Mon, 19 Aug 2024 00:23:40 +0800 Subject: [PATCH 3/5] 2 --- .../schema_file_cache_statistics.cpp | 25 +++++----- .../schema_file_cache_statistics.h | 2 +- be/src/io/cache/block/block_file_cache.h | 1 + .../cache/block/block_file_cache_factory.cpp | 48 +++++++++++++++++++ .../io/cache/block/block_file_cache_factory.h | 6 +++ .../io/cache/block/block_lru_file_cache.cpp | 29 +++++++++++ be/src/io/cache/block/block_lru_file_cache.h | 2 + .../runtime/runtime_query_statistics_mgr.cpp | 2 +- be/src/runtime/runtime_query_statistics_mgr.h | 2 +- .../org/apache/doris/catalog/SchemaTable.java | 1 - .../BackendPartitionedSchemaScanNode.java | 2 +- 11 files changed, 102 insertions(+), 18 deletions(-) diff --git a/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp b/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp index 45f951d03be04b..1105e46e534659 100644 --- a/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp +++ b/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp @@ -15,24 +15,24 @@ // specific language governing permissions and limitations // under the License. -#include "exec/schema_scanner/schema_backend_active_tasks.h" +#include "exec/schema_scanner/schema_file_cache_statistics.h" +#include "io/cache/block/block_file_cache_factory.h" #include "runtime/exec_env.h" -#include "runtime/runtime_query_statistics_mgr.h" #include "runtime/runtime_state.h" #include "vec/common/string_ref.h" #include "vec/core/block.h" #include "vec/data_types/data_type_factory.hpp" namespace doris { + std::vector SchemaFileCacheStatisticsScanner::_s_tbls_columns = { // name, type, size {"BE_ID", TYPE_BIGINT, sizeof(StringRef), false}, {"CACHE_PATH", TYPE_VARCHAR, sizeof(StringRef), false}, {"CACHE_TYPE", TYPE_VARCHAR, sizeof(StringRef), false}, {"METRIC_NAME", TYPE_VARCHAR, sizeof(StringRef), false}, - {"METRIC_VALUE", TYPE_DOUBLE, sizeof(double), false} -}; + {"METRIC_VALUE", TYPE_DOUBLE, sizeof(double), false}}; SchemaFileCacheStatisticsScanner::SchemaFileCacheStatisticsScanner() : SchemaScanner(_s_tbls_columns, TSchemaTableType::SCH_FILE_CACHE_STATISTICS) {} @@ -45,7 +45,7 @@ Status SchemaFileCacheStatisticsScanner::start(RuntimeState* state) { } Status SchemaFileCacheStatisticsScanner::get_next_block_internal(vectorized::Block* block, - bool* eos) { + bool* eos) { if (!_is_init) { return Status::InternalError("Used before initialized."); } @@ -54,22 +54,21 @@ Status SchemaFileCacheStatisticsScanner::get_next_block_internal(vectorized::Blo return Status::InternalError("input pointer is nullptr."); } - if (_task_stats_block == nullptr) { - _task_stats_block = vectorized::Block::create_unique(); + if (_stats_block == nullptr) { + _stats_block = vectorized::Block::create_unique(); for (int i = 0; i < _s_tbls_columns.size(); ++i) { TypeDescriptor descriptor(_s_tbls_columns[i].type); auto data_type = vectorized::DataTypeFactory::instance().create_data_type(descriptor, true); - _task_stats_block->insert(vectorized::ColumnWithTypeAndName( + _stats_block->insert(vectorized::ColumnWithTypeAndName( data_type->create_column(), data_type, _s_tbls_columns[i].name)); } - _task_stats_block->reserve(_block_rows_limit); + _stats_block->reserve(_block_rows_limit); - ExecEnv::GetInstance()->runtime_query_statistics_mgr()->get_active_be_tasks_block( - _task_stats_block.get()); - _total_rows = _task_stats_block->rows(); + ExecEnv::GetInstance()->file_cache_factory()->get_cache_stats_block(_stats_block.get()); + _total_rows = _stats_block->rows(); } if (_row_idx == _total_rows) { @@ -79,7 +78,7 @@ Status SchemaFileCacheStatisticsScanner::get_next_block_internal(vectorized::Blo int current_batch_rows = std::min(_block_rows_limit, _total_rows - _row_idx); vectorized::MutableBlock mblock = vectorized::MutableBlock::build_mutable_block(block); - RETURN_IF_ERROR(mblock.add_rows(_task_stats_block.get(), _row_idx, current_batch_rows)); + RETURN_IF_ERROR(mblock.add_rows(_stats_block.get(), _row_idx, current_batch_rows)); _row_idx += current_batch_rows; *eos = _row_idx == _total_rows; diff --git a/be/src/exec/schema_scanner/schema_file_cache_statistics.h b/be/src/exec/schema_scanner/schema_file_cache_statistics.h index 00388dcfd2ab90..96c6aa9028f0c8 100644 --- a/be/src/exec/schema_scanner/schema_file_cache_statistics.h +++ b/be/src/exec/schema_scanner/schema_file_cache_statistics.h @@ -44,6 +44,6 @@ class SchemaFileCacheStatisticsScanner : public SchemaScanner { int _block_rows_limit = 4096; int _row_idx = 0; int _total_rows = 0; - std::unique_ptr _task_stats_block = nullptr; + std::unique_ptr _stats_block = nullptr; }; }; // namespace doris diff --git a/be/src/io/cache/block/block_file_cache.h b/be/src/io/cache/block/block_file_cache.h index 1b5fa96696d9d9..22f87d7eea66d9 100644 --- a/be/src/io/cache/block/block_file_cache.h +++ b/be/src/io/cache/block/block_file_cache.h @@ -147,6 +147,7 @@ class IFileCache { virtual void change_cache_type(const Key& key, size_t offset, CacheType new_type, std::lock_guard& cache_lock) = 0; + virtual std::map get_stats() = 0; static std::string_view cache_type_to_string(CacheType type); static CacheType string_to_cache_type(const std::string& str); diff --git a/be/src/io/cache/block/block_file_cache_factory.cpp b/be/src/io/cache/block/block_file_cache_factory.cpp index e2b0707c8630f5..b96a383101d0fa 100644 --- a/be/src/io/cache/block/block_file_cache_factory.cpp +++ b/be/src/io/cache/block/block_file_cache_factory.cpp @@ -32,6 +32,7 @@ #include "io/cache/block/block_lru_file_cache.h" #include "io/fs/local_file_system.h" #include "runtime/exec_env.h" +#include "vec/core/block.h" namespace doris { class TUniqueId; @@ -112,5 +113,52 @@ std::vector FileCacheFactory::get_qu return holders; } +void FileCacheFactory::get_cache_stats_block(vectorized::Block* block) { + // std::shared_lock read_lock(_qs_ctx_map_lock); + int64_t be_id = ExecEnv::GetInstance()->master_info()->backend_id; + + auto insert_int_value = [&](int col_index, int64_t int_val, vectorized::Block* block) { + vectorized::MutableColumnPtr mutable_col_ptr; + mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable(); + auto* nullable_column = + reinterpret_cast(mutable_col_ptr.get()); + vectorized::IColumn* col_ptr = &nullable_column->get_nested_column(); + reinterpret_cast*>(col_ptr)->insert_value( + int_val); + nullable_column->get_null_map_data().emplace_back(0); + }; + + auto insert_double_value = [&](int col_index, double double_val, vectorized::Block* block) { + vectorized::MutableColumnPtr mutable_col_ptr; + mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable(); + auto* nullable_column = + reinterpret_cast(mutable_col_ptr.get()); + vectorized::IColumn* col_ptr = &nullable_column->get_nested_column(); + reinterpret_cast*>(col_ptr)->insert_value( + double_val); + nullable_column->get_null_map_data().emplace_back(0); + }; + + auto insert_string_value = [&](int col_index, std::string str_val, vectorized::Block* block) { + vectorized::MutableColumnPtr mutable_col_ptr; + mutable_col_ptr = std::move(*block->get_by_position(col_index).column).assume_mutable(); + auto* nullable_column = + reinterpret_cast(mutable_col_ptr.get()); + vectorized::IColumn* col_ptr = &nullable_column->get_nested_column(); + reinterpret_cast(col_ptr)->insert_data(str_val.data(), + str_val.size()); + nullable_column->get_null_map_data().emplace_back(0); + }; + + for (auto& cache : _caches) { + std::map stats = cache->get_stats(); + for (auto& [k, v] : stats) { + insert_int_value(0, be_id, block); // be id + insert_string_value(1, cache->get_base_path(), block); // cache path + insert_string_value(2, k, block); // metric name + insert_double_value(3, v, block); // metric value + } + } +} } // namespace io } // namespace doris diff --git a/be/src/io/cache/block/block_file_cache_factory.h b/be/src/io/cache/block/block_file_cache_factory.h index 0b6b6504c96377..9fae198741ae80 100644 --- a/be/src/io/cache/block/block_file_cache_factory.h +++ b/be/src/io/cache/block/block_file_cache_factory.h @@ -30,6 +30,10 @@ namespace doris { class TUniqueId; +namespace vectorized { +class Block; +} // namespace vectorized + namespace io { /** @@ -54,6 +58,8 @@ class FileCacheFactory { FileCacheFactory& operator=(const FileCacheFactory&) = delete; FileCacheFactory(const FileCacheFactory&) = delete; + void get_cache_stats_block(vectorized::Block* block); + private: // to protect following containers std::mutex _cache_mutex; diff --git a/be/src/io/cache/block/block_lru_file_cache.cpp b/be/src/io/cache/block/block_lru_file_cache.cpp index cb73bfc1faba15..c7c1b5fd3104af 100644 --- a/be/src/io/cache/block/block_lru_file_cache.cpp +++ b/be/src/io/cache/block/block_lru_file_cache.cpp @@ -1202,5 +1202,34 @@ void LRUFileCache::update_cache_metrics() const { file_cache_segment_reader_cache_size->set_value(IFileCache::file_reader_cache_size()); } +std::map LRUFileCache::get_stats() { + update_cache_metrics(); + std::map stats; + stats["hits_ratio"] = (double)file_cache_hits_ratio->value(); + stats["hits_ratio_5m"] = (double)file_cache_hits_ratio_5m->value(); + stats["hits_ratio_1h"] = (double)file_cache_hits_ratio_1h->value(); + + stats["index_queue_max_size"] = (double)file_cache_index_queue_max_size->value(); + stats["index_queue_curr_size"] = (double)file_cache_index_queue_curr_size->value(); + stats["index_queue_max_elements"] = (double)file_cache_index_queue_max_elements->value(); + stats["index_queue_curr_elements"] = (double)file_cache_index_queue_curr_elements->value(); + + stats["normal_queue_max_size"] = (double)file_cache_normal_queue_max_size->value(); + stats["normal_queue_curr_size"] = (double)file_cache_normal_queue_curr_size->value(); + stats["normal_queue_max_elements"] = (double)file_cache_normal_queue_max_elements->value(); + stats["normal_queue_curr_elements"] = (double)file_cache_normal_queue_curr_elements->value(); + + stats["disposable_queue_max_size"] = (double)file_cache_disposable_queue_max_size->value(); + stats["disposable_queue_curr_size"] = (double)file_cache_disposable_queue_curr_size->value(); + stats["disposable_queue_max_elements"] = + (double)file_cache_disposable_queue_max_elements->value(); + stats["disposable_queue_curr_elements"] = + (double)file_cache_disposable_queue_curr_elements->value(); + + stats["segment_reader_cache_size"] = (double)IFileCache::file_reader_cache_size(); + + return stats; +} + } // namespace io } // namespace doris diff --git a/be/src/io/cache/block/block_lru_file_cache.h b/be/src/io/cache/block/block_lru_file_cache.h index 0f2a922797b74d..df8945cfc9c93c 100644 --- a/be/src/io/cache/block/block_lru_file_cache.h +++ b/be/src/io/cache/block/block_lru_file_cache.h @@ -164,6 +164,8 @@ class LRUFileCache final : public IFileCache { void change_cache_type(const Key& key, size_t offset, CacheType new_type, std::lock_guard& cache_lock) override; + std::map get_stats() override; + size_t get_available_cache_size(CacheType cache_type) const; Status load_cache_info_into_memory(std::lock_guard& cache_lock); diff --git a/be/src/runtime/runtime_query_statistics_mgr.cpp b/be/src/runtime/runtime_query_statistics_mgr.cpp index 32ca00d2c65597..d55bbed9761ca8 100644 --- a/be/src/runtime/runtime_query_statistics_mgr.cpp +++ b/be/src/runtime/runtime_query_statistics_mgr.cpp @@ -271,4 +271,4 @@ void RuntimeQueryStatiticsMgr::get_active_be_tasks_block(vectorized::Block* bloc } } -} // namespace doris \ No newline at end of file +} // namespace doris diff --git a/be/src/runtime/runtime_query_statistics_mgr.h b/be/src/runtime/runtime_query_statistics_mgr.h index aa1793efbf9036..f9a21abf1bb649 100644 --- a/be/src/runtime/runtime_query_statistics_mgr.h +++ b/be/src/runtime/runtime_query_statistics_mgr.h @@ -82,4 +82,4 @@ class RuntimeQueryStatiticsMgr { std::map> _query_statistics_ctx_map; }; -} // namespace doris \ No newline at end of file +} // namespace doris 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 3caff714f8eafb..0e64b765b0e07e 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 @@ -520,7 +520,6 @@ public class SchemaTable extends Table { new SchemaTable(SystemIdGenerator.getNextId(), "file_cache_statistics", TableType.SCHEMA, builder().column("BE_ID", ScalarType.createType(PrimitiveType.BIGINT)) .column("CACHE_PATH", ScalarType.createVarchar(256)) - .column("CACHE_TYPE", ScalarType.createVarchar(256)) .column("METRIC_NAME", ScalarType.createVarchar(256)) .column("METRIC_VALUE", ScalarType.createType(PrimitiveType.DOUBLE)) .build())) diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java index c9ae017bbf0bed..e05fadd81d55c3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/BackendPartitionedSchemaScanNode.java @@ -87,7 +87,7 @@ public static boolean isBackendPartitionedSchemaTable(String tableName) { private PartitionInfo backendPartitionInfo; // partitionID -> backendID private Map partitionIDToBackendID; - private Collection selectedPartiytionIds = Lists.newArrayList(); + private Collection selectedPartitionIds = Lists.newArrayList(); public BackendPartitionedSchemaScanNode(PlanNodeId id, TupleDescriptor desc) { super(id, desc); From c371ccc12ced4dd5134a20d8bb64fb80581dbd7b Mon Sep 17 00:00:00 2001 From: morningman Date: Mon, 19 Aug 2024 14:46:03 +0800 Subject: [PATCH 4/5] 3 --- .../schema_scanner/schema_file_cache_statistics.cpp | 4 ++-- be/src/io/cache/block/block_file_cache_factory.cpp | 12 ++++++++---- be/src/io/cache/block/block_lru_file_cache.cpp | 1 + .../org/apache/doris/analysis/SchemaTableType.java | 6 +++++- .../java/org/apache/doris/catalog/SchemaTable.java | 1 + 5 files changed, 17 insertions(+), 7 deletions(-) diff --git a/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp b/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp index 1105e46e534659..f47961002de172 100644 --- a/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp +++ b/be/src/exec/schema_scanner/schema_file_cache_statistics.cpp @@ -28,9 +28,9 @@ namespace doris { std::vector SchemaFileCacheStatisticsScanner::_s_tbls_columns = { // name, type, size - {"BE_ID", TYPE_BIGINT, sizeof(StringRef), false}, + {"BE_ID", TYPE_BIGINT, sizeof(int64_t), false}, + {"BE_IP", TYPE_VARCHAR, sizeof(StringRef), false}, {"CACHE_PATH", TYPE_VARCHAR, sizeof(StringRef), false}, - {"CACHE_TYPE", TYPE_VARCHAR, sizeof(StringRef), false}, {"METRIC_NAME", TYPE_VARCHAR, sizeof(StringRef), false}, {"METRIC_VALUE", TYPE_DOUBLE, sizeof(double), false}}; diff --git a/be/src/io/cache/block/block_file_cache_factory.cpp b/be/src/io/cache/block/block_file_cache_factory.cpp index b96a383101d0fa..017e733c5e0754 100644 --- a/be/src/io/cache/block/block_file_cache_factory.cpp +++ b/be/src/io/cache/block/block_file_cache_factory.cpp @@ -32,6 +32,7 @@ #include "io/cache/block/block_lru_file_cache.h" #include "io/fs/local_file_system.h" #include "runtime/exec_env.h" +#include "service/backend_options.h" #include "vec/core/block.h" namespace doris { @@ -115,7 +116,9 @@ std::vector FileCacheFactory::get_qu void FileCacheFactory::get_cache_stats_block(vectorized::Block* block) { // std::shared_lock read_lock(_qs_ctx_map_lock); - int64_t be_id = ExecEnv::GetInstance()->master_info()->backend_id; + TBackend be = BackendOptions::get_local_backend(); + int64_t be_id = be.id; + std::string be_ip = be.host; auto insert_int_value = [&](int col_index, int64_t int_val, vectorized::Block* block) { vectorized::MutableColumnPtr mutable_col_ptr; @@ -154,9 +157,10 @@ void FileCacheFactory::get_cache_stats_block(vectorized::Block* block) { std::map stats = cache->get_stats(); for (auto& [k, v] : stats) { insert_int_value(0, be_id, block); // be id - insert_string_value(1, cache->get_base_path(), block); // cache path - insert_string_value(2, k, block); // metric name - insert_double_value(3, v, block); // metric value + insert_string_value(1, be_ip, block); // be ip + insert_string_value(2, cache->get_base_path(), block); // cache path + insert_string_value(3, k, block); // metric name + insert_double_value(4, v, block); // metric value } } } diff --git a/be/src/io/cache/block/block_lru_file_cache.cpp b/be/src/io/cache/block/block_lru_file_cache.cpp index c7c1b5fd3104af..fa4d42581c64eb 100644 --- a/be/src/io/cache/block/block_lru_file_cache.cpp +++ b/be/src/io/cache/block/block_lru_file_cache.cpp @@ -1177,6 +1177,7 @@ void LRUFileCache::update_cache_metrics() const { if (g_file_cache_num_read_segments_1h.get_value() > 0) { hit_ratio_1h = ((double)g_file_cache_num_hit_segments_1h.get_value()) / ((double)g_file_cache_num_read_segments_1h.get_value()); + } else { hit_ratio_1h = 0.0; } 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 8ec426cbf5e685..b83eabcaf2505f 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 @@ -77,7 +77,9 @@ public enum SchemaTableType { SCH_PROCS_PRIV("procs_priv", "procs_priv", TSchemaTableType.SCH_PROCS_PRIV), SCH_WORKLOAD_POLICY("WORKLOAD_POLICY", "WORKLOAD_POLICY", - TSchemaTableType.SCH_WORKLOAD_POLICY); + TSchemaTableType.SCH_WORKLOAD_POLICY), + SCH_FILE_CACHE_STATISTICS("FILE_CACHE_STATISTICS", "FILE_CACHE_STATISTICS", + TSchemaTableType.SCH_FILE_CACHE_STATISTICS); private static final String dbName = "INFORMATION_SCHEMA"; private static SelectList fullSelectLists; @@ -114,3 +116,5 @@ public TSchemaTableType toThrift() { return tableType; } } + + 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 0e64b765b0e07e..e75fa18386cb3a 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 @@ -519,6 +519,7 @@ public class SchemaTable extends Table { .put("file_cache_statistics", new SchemaTable(SystemIdGenerator.getNextId(), "file_cache_statistics", TableType.SCHEMA, builder().column("BE_ID", ScalarType.createType(PrimitiveType.BIGINT)) + .column("BE_IP", ScalarType.createVarchar(256)) .column("CACHE_PATH", ScalarType.createVarchar(256)) .column("METRIC_NAME", ScalarType.createVarchar(256)) .column("METRIC_VALUE", ScalarType.createType(PrimitiveType.DOUBLE)) From 6226af01d54bd7fef395d13b3630048ef800743e Mon Sep 17 00:00:00 2001 From: morningman Date: Tue, 20 Aug 2024 12:09:44 +0800 Subject: [PATCH 5/5] fix ut --- .../doris/datasource/RefreshCatalogTest.java | 4 +-- .../jdbc/test_mariadb_jdbc_catalog.out | 31 ------------------- .../jdbc/test_mysql_jdbc_catalog.out | 31 ------------------- .../jdbc/test_mysql_jdbc_catalog_nereids.out | 31 ------------------- .../jdbc/test_mysql_jdbc_driver5_catalog.out | 31 ------------------- .../jdbc/test_mariadb_jdbc_catalog.groovy | 2 +- .../jdbc/test_mysql_jdbc_catalog.groovy | 2 +- .../test_mysql_jdbc_catalog_nereids.groovy | 2 +- .../test_mysql_jdbc_driver5_catalog.groovy | 2 +- 9 files changed, 6 insertions(+), 130 deletions(-) diff --git a/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java b/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java index b51d6b19be507e..663eacb70982e0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/datasource/RefreshCatalogTest.java @@ -103,7 +103,7 @@ public void testRefreshCatalog() throws Exception { List dbNames2 = test1.getDbNames(); Assertions.assertEquals(5, dbNames2.size()); ExternalInfoSchemaDatabase infoDb = (ExternalInfoSchemaDatabase) test1.getDb(InfoSchemaDb.DATABASE_NAME).get(); - Assertions.assertEquals(32, infoDb.getTables().size()); + Assertions.assertTrue(infoDb.getTables().size() >= 33); TestExternalDatabase testDb = (TestExternalDatabase) test1.getDb("db1").get(); Assertions.assertEquals(2, testDb.getTables().size()); ExternalMysqlDatabase mysqlDb = (ExternalMysqlDatabase) test1.getDb(MysqlDb.DATABASE_NAME).get(); @@ -114,7 +114,7 @@ public void testRefreshCatalog() throws Exception { CatalogMgr mgr2 = GsonUtils.GSON.fromJson(json, CatalogMgr.class); test1 = mgr2.getCatalog("test1"); infoDb = (ExternalInfoSchemaDatabase) test1.getDb(InfoSchemaDb.DATABASE_NAME).get(); - Assertions.assertEquals(32, infoDb.getTables().size()); + Assertions.assertTrue(infoDb.getTables().size() >= 33); testDb = (TestExternalDatabase) test1.getDb("db1").get(); Assertions.assertEquals(2, testDb.getTables().size()); mysqlDb = (ExternalMysqlDatabase) test1.getDb(MysqlDb.DATABASE_NAME).get(); diff --git a/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out index d8bfffd91254eb..b6817f090f5bd7 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mariadb_jdbc_catalog.out @@ -28,38 +28,7 @@ mariadb_jdbc_catalog 115 abg -- !information_schema -- -active_queries -backend_active_tasks -character_sets -collations -column_privileges -column_statistics -columns -engines -events -files -global_variables -key_column_usage -metadata_name_ids -parameters -partitions processlist -profiling -referential_constraints -routines -rowsets -schema_privileges -schemata -session_variables -statistics -table_constraints -table_privileges -tables -triggers -user_privileges -views -workload_groups -workload_policy -- !auto_default_t -- 0 diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out index f2a05e3d0896be..440b95d3b5915a 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog.out @@ -192,38 +192,7 @@ bca 2022-11-02 2022-11-02 8012 vivo 2 2 -- !information_schema -- -active_queries -backend_active_tasks -character_sets -collations -column_privileges -column_statistics -columns -engines -events -files -global_variables -key_column_usage -metadata_name_ids -parameters -partitions processlist -profiling -referential_constraints -routines -rowsets -schema_privileges -schemata -session_variables -statistics -table_constraints -table_privileges -tables -triggers -user_privileges -views -workload_groups -workload_policy -- !dt -- 2023-06-17T10:00 2023-06-17T10:00:01.100 2023-06-17T10:00:02.220 2023-06-17T10:00:03.333 2023-06-17T10:00:04.444400 2023-06-17T10:00:05.555550 2023-06-17T10:00:06.666666 diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out index 22bc4e8279d89b..171cee4d958178 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.out @@ -160,38 +160,7 @@ bca 2022-11-02 2022-11-02 8012 vivo 123456789012345678901234567890123.12345 12345678901234567890123456789012.12345 1234567890123456789012345678901234.12345 123456789012345678901234567890123.12345 123456789012345678901234567890123456789012345678901234567890.12345 123456789012345678901234567890123456789012345678901234567890.12345 -- !information_schema -- -active_queries -backend_active_tasks -character_sets -collations -column_privileges -column_statistics -columns -engines -events -files -global_variables -key_column_usage -metadata_name_ids -parameters -partitions processlist -profiling -referential_constraints -routines -rowsets -schema_privileges -schemata -session_variables -statistics -table_constraints -table_privileges -tables -triggers -user_privileges -views -workload_groups -workload_policy -- !test_insert1 -- doris1 18 diff --git a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out index fdc55da8821bee..736f2b57b4a768 100644 --- a/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out +++ b/regression-test/data/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.out @@ -202,38 +202,7 @@ bca 2022-11-02 2022-11-02 8012 vivo 2 2 -- !information_schema -- -active_queries -backend_active_tasks -character_sets -collations -column_privileges -column_statistics -columns -engines -events -files -global_variables -key_column_usage -metadata_name_ids -parameters -partitions processlist -profiling -referential_constraints -routines -rowsets -schema_privileges -schemata -session_variables -statistics -table_constraints -table_privileges -tables -triggers -user_privileges -views -workload_groups -workload_policy -- !dt -- 2023-06-17T10:00 2023-06-17T10:00:01 2023-06-17T10:00:02 2023-06-17T10:00:03 2023-06-17T10:00:04 2023-06-17T10:00:05 2023-06-17T10:00:06 diff --git a/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy index d90ffe3e1cc406..5281cdd20cc539 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mariadb_jdbc_catalog.groovy @@ -68,7 +68,7 @@ suite("test_mariadb_jdbc_catalog", "p0,external,mariadb,external_docker,external sql """ insert into internal.${internal_db_name}.${inDorisTable} select id, name from ${ex_tb0}; """ order_qt_in_tb """ select id, name from internal.${internal_db_name}.${inDorisTable} order by id; """ - order_qt_information_schema """ show tables from information_schema; """ + order_qt_information_schema """ show tables from information_schema like "processlist"; """ order_qt_auto_default_t """insert into ${auto_default_t}(name) values('a'); """ order_qt_dt """select * from ${dt}; """ diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy index bef3ba03bd7a26..81d31fd88c4b3e 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog.groovy @@ -167,7 +167,7 @@ suite("test_mysql_jdbc_catalog", "p0,external,mysql,external_docker,external_doc order_qt_ex_tb21_6 """ select `key`, `id` from ${ex_tb21} where `key` = case when id = 1 then 1 else 0 end order by id;""" order_qt_ex_tb21_7 """ select (`key` +1) as k, `id` from ${ex_tb21} having abs(k) = 2 order by id;""" order_qt_ex_tb21_8 """ select `key` as k, `id` from ${ex_tb21} having abs(k) = 2 order by id;""" - order_qt_information_schema """ show tables from information_schema; """ + order_qt_information_schema """ show tables from information_schema like "processlist"; """ order_qt_dt """select * from ${dt}; """ order_qt_dt_null """select * from ${dt_null} order by 1; """ order_qt_test_dz """select * from ${test_zd} order by 1; """ diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy index 24aac183eb861b..e7ac412c0b5883 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_catalog_nereids.groovy @@ -113,7 +113,7 @@ suite("test_mysql_jdbc_catalog_nereids", "p0,external,mysql,external_docker,exte order_qt_ex_tb18 """ select * from ${ex_tb18} order by num_tinyint; """ order_qt_ex_tb19 """ select * from ${ex_tb19} order by date_value; """ order_qt_ex_tb20 """ select * from ${ex_tb20} order by decimal_normal; """ - order_qt_information_schema """ show tables from information_schema; """ + order_qt_information_schema """ show tables from information_schema like "processlist"; """ // test insert String uuid1 = UUID.randomUUID().toString(); diff --git a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy index edf6e60baef9a6..94b3ad1db22556 100644 --- a/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy +++ b/regression-test/suites/external_table_p0/jdbc/test_mysql_jdbc_driver5_catalog.groovy @@ -175,7 +175,7 @@ suite("test_mysql_jdbc_driver5_catalog", "p0,external,mysql,external_docker,exte order_qt_ex_tb21_6 """ select `key`, `id` from ${ex_tb21} where `key` = case when id = 1 then 1 else 0 end order by id;""" order_qt_ex_tb21_7 """ select (`key` +1) as k, `id` from ${ex_tb21} having abs(k) = 2 order by id;""" order_qt_ex_tb21_8 """ select `key` as k, `id` from ${ex_tb21} having abs(k) = 2 order by id;""" - order_qt_information_schema """ show tables from information_schema; """ + order_qt_information_schema """ show tables from information_schema like "processlist"; """ order_qt_dt """select * from ${dt}; """ order_qt_dt_null """select * from ${dt_null} order by 1; """ order_qt_test_dz """select * from ${test_zd} order by 1; """