From ade7a44c7f36c36c7decf76110070131ec03b376 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 10 Feb 2023 10:34:31 +0800 Subject: [PATCH 01/23] add GeneratedColumnPlaceholderInputStream Signed-off-by: guo-shaoge --- .../GeneratedColumnPlaceholderInputStream.h | 97 +++++++++++++++++++ .../Debug/MockExecutor/TableScanBinder.cpp | 37 ++++++- dbms/src/Debug/MockExecutor/TableScanBinder.h | 1 + dbms/src/Debug/MockStorage.cpp | 2 +- dbms/src/Debug/MockStorage.h | 2 +- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 10 ++ .../Coprocessor/DAGStorageInterpreter.cpp | 12 +++ .../Flash/Coprocessor/DAGStorageInterpreter.h | 2 + .../Flash/Coprocessor/InterpreterUtils.cpp | 11 +++ dbms/src/Flash/Coprocessor/InterpreterUtils.h | 5 + dbms/src/Flash/Planner/PhysicalPlan.cpp | 1 + .../Planner/Plans/PhysicalMockTableScan.cpp | 3 + dbms/src/Flash/executeQuery.cpp | 7 ++ .../Flash/tests/gtest_generated_column.cpp | 58 +++++++++++ dbms/src/Flash/tests/gtest_interpreter.cpp | 19 +++- dbms/src/Storages/Transaction/TiDB.h | 3 +- dbms/src/TestUtils/ExecutorTestUtils.cpp | 49 ++++++++++ dbms/src/TestUtils/ExecutorTestUtils.h | 2 + 18 files changed, 316 insertions(+), 5 deletions(-) create mode 100644 dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h create mode 100644 dbms/src/Flash/tests/gtest_generated_column.cpp diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h new file mode 100644 index 00000000000..5e61f77d6b1 --- /dev/null +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h @@ -0,0 +1,97 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed 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 +#include +#include +#include +#include + +namespace DB +{ +class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputStream +{ +public: + GeneratedColumnPlaceholderBlockInputStream( + const BlockInputStreamPtr & input, + const std::vector> & generated_column_infos_, + const String & req_id_) + : generated_column_infos(generated_column_infos_) + , log(Logger::get(req_id_)) + { + children.push_back(input); + } + + String getName() const override { return NAME; } + Block getHeader() const override + { + Block block = children.back()->getHeader(); + insertColumns(block, /*insert_null=*/false); + return block; + } + + static String getColumnName(UInt64 col_index) + { + return "generated_column_" + std::to_string(col_index); + } +protected: + void readPrefix() override + { + RUNTIME_CHECK(!generated_column_infos.empty()); + // Validation check. + for (size_t i = 1; i < generated_column_infos.size(); ++i) + { + RUNTIME_CHECK(generated_column_infos[i].first > generated_column_infos[i - 1].first); + } + } + + Block readImpl() override + { + Block block = children.back()->read(); + insertColumns(block, /*insert_null=*/true); + return block; + } + +private: + void insertColumns(Block & block, bool insert_null) const + { + if (!block) + return; + + ColumnPtr nested_column = ColumnString::create(); + for (const auto & ele : generated_column_infos) + { + auto col_index = ele.first; + auto data_type = ele.second; + ColumnPtr null_map; + if (insert_null) + null_map = ColumnUInt8::create(block.rows(), 1); + else + null_map = ColumnUInt8::create(); + if (!data_type->isNullable()) + data_type = std::make_shared(data_type); + ColumnPtr column = ColumnNullable::create(nested_column, std::move(null_map)); + block.insert(col_index, ColumnWithTypeAndName{column, data_type, getColumnName(col_index)}); + } + } + + static constexpr auto NAME = "GeneratedColumnPlaceholder"; + const std::vector> generated_column_infos; + const LoggerPtr log; +}; +} // namespace DB diff --git a/dbms/src/Debug/MockExecutor/TableScanBinder.cpp b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp index e5319054bdc..556bac5bf90 100644 --- a/dbms/src/Debug/MockExecutor/TableScanBinder.cpp +++ b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp @@ -17,6 +17,8 @@ #include #include +#include + namespace DB::mock { bool TableScanBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t, const MPPInfo &, const Context &) @@ -52,12 +54,25 @@ TableID TableScanBinder::getTableId() const } void TableScanBinder::setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info) const +{ + auto names = splitQualifiedName(dag_column_info.first); + Int64 column_id = 0; + if (names.column_name == MutableSupport::tidb_pk_column_name) + column_id = -1; + else if (dag_column_info.second.hasGeneratedColumnFlag()) + column_id = dag_column_info.second.id; + else + column_id = table_info.getColumnID(names.column_name); + setTipbColumnInfo(ci, dag_column_info, column_id); +} + +void TableScanBinder::setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info, Int64 column_id) { auto names = splitQualifiedName(dag_column_info.first); if (names.column_name == MutableSupport::tidb_pk_column_name) ci->set_column_id(-1); else - ci->set_column_id(table_info.getColumnID(names.column_name)); + ci->set_column_id(column_id); ci->set_tp(dag_column_info.second.tp); ci->set_flag(dag_column_info.second.flag); ci->set_columnlen(dag_column_info.second.flen); @@ -120,6 +135,26 @@ ExecutorBinderPtr compileTableScan(size_t & executor_index, TableInfo & table_in ci.setNotNullFlag(); ts_output.emplace_back(std::make_pair(MutableSupport::tidb_pk_column_name, std::move(ci))); } + // if (random_gennerated_column) + int gen_col_num = 100; + auto ori_ts_output = ts_output; + size_t ori_output_index = 0; + for (size_t i = 0; i < ori_ts_output.size() + gen_col_num; ++i) + { + if (std::rand() % 2 == 0 || ori_output_index >= ori_ts_output.size()) + { + TiDB::ColumnInfo ci; + ci.tp = TiDB::TP::TypeLong; + ci.flag = TiDB::ColumnFlag::ColumnFlagGeneratedColumn; + String gen_column_name = fmt::format("gen_col_{}", i); + ci.name = gen_column_name; + ts_output.emplace_back(std::make_pair(db + "." + table_name + "." + gen_column_name, std::move(ci))); + } + else + { + ts_output.emplace_back(ori_ts_output[ori_output_index]); + } + } return std::make_shared(executor_index, ts_output, table_info); } diff --git a/dbms/src/Debug/MockExecutor/TableScanBinder.h b/dbms/src/Debug/MockExecutor/TableScanBinder.h index 6eaeae7a035..d40c85c398f 100644 --- a/dbms/src/Debug/MockExecutor/TableScanBinder.h +++ b/dbms/src/Debug/MockExecutor/TableScanBinder.h @@ -38,6 +38,7 @@ class TableScanBinder : public ExecutorBinder TableID getTableId() const; + static void setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info, Int64 column_id); private: TableInfo table_info; /// used by column pruner diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index 79be39bf0df..149882be636 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -427,7 +427,7 @@ void MockStorage::addTableInfo(const String & name, const MockColumnInfoVec & co table_infos[name] = table_info; } -TableInfo MockStorage::getTableInfo(const String & name) +TableInfo & MockStorage::getTableInfo(const String & name) { return table_infos[name]; } diff --git a/dbms/src/Debug/MockStorage.h b/dbms/src/Debug/MockStorage.h index e36858472d5..3c7b941db4c 100644 --- a/dbms/src/Debug/MockStorage.h +++ b/dbms/src/Debug/MockStorage.h @@ -105,7 +105,7 @@ class MockStorage /// for MPP Tasks, it will split data by partition num, then each MPP service will have a subset of mock data. ColumnsWithTypeAndName getColumnsForMPPTableScan(const TiDBTableScan & table_scan, Int64 partition_id, Int64 partition_num); - TableInfo getTableInfo(const String & name); + TableInfo & getTableInfo(const String & name); TableInfo getTableInfoForDeltaMerge(const String & name); size_t getTableScanConcurrencyHint(const TiDBTableScan & table_scan); diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 9fcc310d1bc..ee4f564a0e4 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -184,6 +184,16 @@ void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_s analyzer = std::make_unique(std::move(names_and_types), context); pipeline.streams.insert(pipeline.streams.end(), mock_table_scan_streams.begin(), mock_table_scan_streams.end()); } + std::vector> generated_column_infos; + for (Int64 i = 0; i < table_scan.getColumnSize(); i++) + { + if (table_scan.getColumns()[i].hasGeneratedColumnFlag()) + { + auto data_type = getDataTypeByColumnInfoForComputingLayer(table_scan.getColumns()[i]); + generated_column_infos.push_back(std::make_pair(i, data_type)); + } + } + executeGeneratedColumnPlaceholder(generated_column_infos, log, pipeline); } diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 6c2cf2a2860..bbc19365455 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -337,6 +338,8 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) executeCastAfterTableScan(remote_read_streams_start_index, pipeline); recordProfileStreams(pipeline, table_scan.getTableScanExecutorID()); + executeGeneratedColumnPlaceholder(generated_column_infos, log, pipeline); + /// handle filter conditions for local and remote table scan. if (filter_conditions.hasValue()) { @@ -940,6 +943,15 @@ std::tuple> DAGStorageIn auto const & ci = table_scan.getColumns()[i]; const ColumnID cid = ci.id; + if (ci.hasGeneratedColumnFlag()) + { + LOG_WARNING(log, "got generated column"); + auto data_type = getDataTypeByColumnInfoForComputingLayer(ci); + generated_column_infos.push_back(std::make_pair(i, data_type)); + source_columns_tmp.emplace_back(NameAndTypePair{GeneratedColumnPlaceholderBlockInputStream::getColumnName(i), data_type}); + continue; + } + LOG_WARNING(log, "no got generated column"); // Column ID -1 return the handle column String name; if (cid == TiDBPkColumnID) diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index 177a5558d7f..a07de68aabd 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -134,6 +134,8 @@ class DAGStorageInterpreter ManageableStoragePtr storage_for_logical_table; Names required_columns; NamesAndTypes source_columns; + // For generated column, just need a placeholder, and TiDB will fill this column. + std::vector> generated_column_infos; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index adfa0f08ac5..5fd559ec9b8 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -220,4 +221,14 @@ void executePushedDownFilter( stream->setExtraInfo("projection after push down filter"); } } + +void executeGeneratedColumnPlaceholder(const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline) +{ + if (generated_column_infos.empty()) + return; + pipeline.transform([&](auto & stream) { + stream = std::make_shared(stream, generated_column_infos, log->identifier()); + stream->setExtraInfo("generated column placeholder above table scan"); + }); +} } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index f842654d162..1cabc41572e 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -78,4 +78,9 @@ void executePushedDownFilter( DAGExpressionAnalyzer & analyzer, LoggerPtr log, DAGPipeline & pipeline); + +void executeGeneratedColumnPlaceholder( + const std::vector> & generated_column_infos, + LoggerPtr log, + DAGPipeline & pipeline); } // namespace DB diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index 8aabeef3885..ced4db70a2d 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -96,6 +96,7 @@ void PhysicalPlan::build(const tipb::DAGRequest * dag_request) void PhysicalPlan::buildTableScan(const String & executor_id, const tipb::Executor * executor) { + LOG_INFO(log, "gjt debug in PhysicalPlan::buildTableScan"); TiDBTableScan table_scan(executor, executor_id, dagContext()); if (unlikely(context.isTest())) pushBack(PhysicalMockTableScan::build(context, executor_id, log, table_scan)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index 4a1143686f5..cd5d4a71932 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -90,6 +90,7 @@ PhysicalPlanNodePtr PhysicalMockTableScan::build( const LoggerPtr & log, const TiDBTableScan & table_scan) { + LOG_INFO(log, "gjt debug in PhysicalMockTableScan::build("); assert(context.isTest()); auto [schema, mock_streams] = mockSchemaAndStreams(context, executor_id, log, table_scan); @@ -105,12 +106,14 @@ PhysicalPlanNodePtr PhysicalMockTableScan::build( void PhysicalMockTableScan::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) { + LOG_INFO(log, "gjt debug in PhysicalMockTableScan 2"); assert(pipeline.streams.empty()); pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); } void PhysicalMockTableScan::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) { + LOG_INFO(log, "gjt debug in PhysicalMockTableScan 3"); group_builder.init(mock_streams.size()); size_t i = 0; group_builder.transform([&](auto & builder) { diff --git a/dbms/src/Flash/executeQuery.cpp b/dbms/src/Flash/executeQuery.cpp index dba7f904082..817230273ff 100644 --- a/dbms/src/Flash/executeQuery.cpp +++ b/dbms/src/Flash/executeQuery.cpp @@ -153,14 +153,21 @@ BlockIO executeAsBlockIO(Context & context, bool internal) QueryExecutorPtr queryExecute(Context & context, bool internal) { + auto & dag_context = *context.getDAGContext(); + const auto & logger = dag_context.log; + LOG_INFO(logger, fmt::format("gjt debug queryExecute {} {} {} {}", context.isExecutorTest(), + context.isInterpreterTest(), context.getSettingsRef().enable_planner, + context.getSettingsRef().enable_pipeline)); // now only support pipeline model in executor/interpreter test. if ((context.isExecutorTest() || context.isInterpreterTest()) && context.getSettingsRef().enable_planner && context.getSettingsRef().enable_pipeline) { + LOG_INFO(logger, "gjt debug queryExecute 1"); if (auto res = executeAsPipeline(context, internal); res) return std::move(*res); } + LOG_INFO(logger, "gjt debug queryExecute 2"); return std::make_unique(executeAsBlockIO(context, internal)); } } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_generated_column.cpp b/dbms/src/Flash/tests/gtest_generated_column.cpp new file mode 100644 index 00000000000..a928843c24f --- /dev/null +++ b/dbms/src/Flash/tests/gtest_generated_column.cpp @@ -0,0 +1,58 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed 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 +#include +#include + +namespace DB +{ +namespace tests +{ + +class GeneratedColumnTestRunner : public DB::tests::ExecutorTest +{ +public: + void initializeContext() override + { + ExecutorTest::initializeContext(); + + context.addMockTable({"test_db", "test_table"}, + {{"col_1", TiDB::TP::TypeString}, {"col_2", TiDB::TP::TypeString}}, + {toNullableVec("col_1", {"banana"}), + toNullableVec("col_2", {"apple"})}); + } +}; + +TEST_F(GeneratedColumnTestRunner, BasicTest) +try +{ + for (size_t i = 0; i < 100; ++i) + { + auto request = context + .scan("test_db", "test_table") + .exchangeSender(tipb::PassThrough) + .build(context); + + // insertGeneratedColumnToTableScanDAGRequest(i, request); + + // Add generated column to table_scan tipb. + executeAndAssertColumnsEqual(request, + {toNullableVec({"banana"}), + toNullableVec({"apple"})}); + } +} +CATCH +} // namespace tests +} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index e129c5587a5..8b1501dc0c9 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB { @@ -32,7 +33,7 @@ class InterpreterExecuteTest : public DB::tests::InterpreterTestUtils // 1. manually delete the gtest_interpreter.out // 2. call setRecord() // 3. ./gtests_dbms --gtest_filter=InterpreterExecuteTest.* - // setRecord(); + setRecord(); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); @@ -394,5 +395,21 @@ try } } CATCH + +TEST_F(InterpreterExecuteTest, GeneratedColumn) +try +{ + { + // col_1, col_2, gen_col. + auto request = context + .scan("test_db", "test_table") + .exchangeSender(tipb::PassThrough) + .build(context); + const auto & tbl_info = context.mockStorage()->getTableInfo("test_table"); + insertGeneratedColumnToTableScanDAGRequest(tbl_info.columns.size(), request); + runAndAssert(request, 1); + } +} +CATCH } // namespace tests } // namespace DB diff --git a/dbms/src/Storages/Transaction/TiDB.h b/dbms/src/Storages/Transaction/TiDB.h index c41a2f5157e..5062e38079d 100644 --- a/dbms/src/Storages/Transaction/TiDB.h +++ b/dbms/src/Storages/Transaction/TiDB.h @@ -117,7 +117,8 @@ enum TP M(NoDefaultValue, (1 << 12)) \ M(OnUpdateNow, (1 << 13)) \ M(PartKey, (1 << 14)) \ - M(Num, (1 << 15)) + M(Num, (1 << 15)) \ + M(GeneratedColumn, (1 << 23)) enum ColumnFlag { diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index fb5592c4083..14faa13ae5a 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -23,6 +24,7 @@ #include #include +#include namespace DB::tests { @@ -280,4 +282,51 @@ void ExecutorTest::dagRequestEqual(const String & expected_string, const std::sh ASSERT_EQ(Poco::trim(expected_string), Poco::trim(ExecutorSerializer().serialize(actual.get()))); } +void insertGeneratedColumnToTableScanDAGRequest(UInt64 gen_col_num, std::shared_ptr dag_req) +{ + TiDB::ColumnInfo tidb_column_info; + tidb_column_info.tp = TiDB::TP::TypeLong; + tidb_column_info.flag = TiDB::ColumnFlag::ColumnFlagGeneratedColumn; + + tipb::Executor * exec = dag_req->mutable_root_executor(); + tipb::TableScan * tsc = nullptr; + while (tsc == nullptr) + { + switch (exec->tp()) + { + case tipb::ExecType::TypeExchangeSender: + { + exec = exec->mutable_exchange_sender()->mutable_child(); + break; + } + case tipb::ExecType::TypeTableScan: + { + tsc = exec->mutable_tbl_scan(); + break; + } + default: + { + throw Exception("insertGeneratedColumnToTableScanDAGRequest not implement for now"); + } + } + } + + auto ori_columns = tsc->columns(); + tsc->clear_columns(); + int ori_column_index = 0; + for (size_t i = 0; i < ori_columns.size() + gen_col_num; ++i) + { + tipb::ColumnInfo * tipb_column_info = tsc->add_columns(); + if (std::rand() % 2 == 0 || ori_column_index >= ori_columns.size()) + { + String gen_column_name = fmt::format("gen_col_{}", i); + tidb_column_info.name = gen_column_name; + mock::TableScanBinder::setTipbColumnInfo(tipb_column_info, DAGColumnInfo{gen_column_name, tidb_column_info}, 100 + i); + } + else + { + *tipb_column_info = ori_columns[ori_column_index++]; + } + } +} } // namespace DB::tests diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index 85caf2358a4..abc537fe5f1 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -162,4 +162,6 @@ using ColumnWithInt64 = std::vector; using ColumnWithUInt64 = std::vector; using ColumnWithFloat64 = std::vector; using ColumnWithString = std::vector; + +void insertGeneratedColumnToTableScanDAGRequest(UInt64 gen_col_index, std::shared_ptr dag_req); } // namespace DB::tests From b58875ee446d870ce97cf0be1211ee9bf387d821 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 10 Feb 2023 11:55:11 +0800 Subject: [PATCH 02/23] clear gtest change Signed-off-by: guo-shaoge --- .../Debug/MockExecutor/TableScanBinder.cpp | 37 +----------- dbms/src/Debug/MockExecutor/TableScanBinder.h | 1 - dbms/src/Debug/MockStorage.cpp | 2 +- dbms/src/Debug/MockStorage.h | 2 +- dbms/src/Flash/Planner/PhysicalPlan.cpp | 1 - .../Planner/Plans/PhysicalMockTableScan.cpp | 3 - dbms/src/Flash/executeQuery.cpp | 7 --- .../Flash/tests/gtest_generated_column.cpp | 58 ------------------- dbms/src/Flash/tests/gtest_interpreter.cpp | 19 +----- dbms/src/TestUtils/ExecutorTestUtils.cpp | 49 ---------------- dbms/src/TestUtils/ExecutorTestUtils.h | 2 - 11 files changed, 4 insertions(+), 177 deletions(-) delete mode 100644 dbms/src/Flash/tests/gtest_generated_column.cpp diff --git a/dbms/src/Debug/MockExecutor/TableScanBinder.cpp b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp index 556bac5bf90..e5319054bdc 100644 --- a/dbms/src/Debug/MockExecutor/TableScanBinder.cpp +++ b/dbms/src/Debug/MockExecutor/TableScanBinder.cpp @@ -17,8 +17,6 @@ #include #include -#include - namespace DB::mock { bool TableScanBinder::toTiPBExecutor(tipb::Executor * tipb_executor, int32_t, const MPPInfo &, const Context &) @@ -54,25 +52,12 @@ TableID TableScanBinder::getTableId() const } void TableScanBinder::setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info) const -{ - auto names = splitQualifiedName(dag_column_info.first); - Int64 column_id = 0; - if (names.column_name == MutableSupport::tidb_pk_column_name) - column_id = -1; - else if (dag_column_info.second.hasGeneratedColumnFlag()) - column_id = dag_column_info.second.id; - else - column_id = table_info.getColumnID(names.column_name); - setTipbColumnInfo(ci, dag_column_info, column_id); -} - -void TableScanBinder::setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info, Int64 column_id) { auto names = splitQualifiedName(dag_column_info.first); if (names.column_name == MutableSupport::tidb_pk_column_name) ci->set_column_id(-1); else - ci->set_column_id(column_id); + ci->set_column_id(table_info.getColumnID(names.column_name)); ci->set_tp(dag_column_info.second.tp); ci->set_flag(dag_column_info.second.flag); ci->set_columnlen(dag_column_info.second.flen); @@ -135,26 +120,6 @@ ExecutorBinderPtr compileTableScan(size_t & executor_index, TableInfo & table_in ci.setNotNullFlag(); ts_output.emplace_back(std::make_pair(MutableSupport::tidb_pk_column_name, std::move(ci))); } - // if (random_gennerated_column) - int gen_col_num = 100; - auto ori_ts_output = ts_output; - size_t ori_output_index = 0; - for (size_t i = 0; i < ori_ts_output.size() + gen_col_num; ++i) - { - if (std::rand() % 2 == 0 || ori_output_index >= ori_ts_output.size()) - { - TiDB::ColumnInfo ci; - ci.tp = TiDB::TP::TypeLong; - ci.flag = TiDB::ColumnFlag::ColumnFlagGeneratedColumn; - String gen_column_name = fmt::format("gen_col_{}", i); - ci.name = gen_column_name; - ts_output.emplace_back(std::make_pair(db + "." + table_name + "." + gen_column_name, std::move(ci))); - } - else - { - ts_output.emplace_back(ori_ts_output[ori_output_index]); - } - } return std::make_shared(executor_index, ts_output, table_info); } diff --git a/dbms/src/Debug/MockExecutor/TableScanBinder.h b/dbms/src/Debug/MockExecutor/TableScanBinder.h index d40c85c398f..6eaeae7a035 100644 --- a/dbms/src/Debug/MockExecutor/TableScanBinder.h +++ b/dbms/src/Debug/MockExecutor/TableScanBinder.h @@ -38,7 +38,6 @@ class TableScanBinder : public ExecutorBinder TableID getTableId() const; - static void setTipbColumnInfo(tipb::ColumnInfo * ci, const DAGColumnInfo & dag_column_info, Int64 column_id); private: TableInfo table_info; /// used by column pruner diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index 149882be636..79be39bf0df 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -427,7 +427,7 @@ void MockStorage::addTableInfo(const String & name, const MockColumnInfoVec & co table_infos[name] = table_info; } -TableInfo & MockStorage::getTableInfo(const String & name) +TableInfo MockStorage::getTableInfo(const String & name) { return table_infos[name]; } diff --git a/dbms/src/Debug/MockStorage.h b/dbms/src/Debug/MockStorage.h index 3c7b941db4c..e36858472d5 100644 --- a/dbms/src/Debug/MockStorage.h +++ b/dbms/src/Debug/MockStorage.h @@ -105,7 +105,7 @@ class MockStorage /// for MPP Tasks, it will split data by partition num, then each MPP service will have a subset of mock data. ColumnsWithTypeAndName getColumnsForMPPTableScan(const TiDBTableScan & table_scan, Int64 partition_id, Int64 partition_num); - TableInfo & getTableInfo(const String & name); + TableInfo getTableInfo(const String & name); TableInfo getTableInfoForDeltaMerge(const String & name); size_t getTableScanConcurrencyHint(const TiDBTableScan & table_scan); diff --git a/dbms/src/Flash/Planner/PhysicalPlan.cpp b/dbms/src/Flash/Planner/PhysicalPlan.cpp index ced4db70a2d..8aabeef3885 100644 --- a/dbms/src/Flash/Planner/PhysicalPlan.cpp +++ b/dbms/src/Flash/Planner/PhysicalPlan.cpp @@ -96,7 +96,6 @@ void PhysicalPlan::build(const tipb::DAGRequest * dag_request) void PhysicalPlan::buildTableScan(const String & executor_id, const tipb::Executor * executor) { - LOG_INFO(log, "gjt debug in PhysicalPlan::buildTableScan"); TiDBTableScan table_scan(executor, executor_id, dagContext()); if (unlikely(context.isTest())) pushBack(PhysicalMockTableScan::build(context, executor_id, log, table_scan)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index cd5d4a71932..4a1143686f5 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -90,7 +90,6 @@ PhysicalPlanNodePtr PhysicalMockTableScan::build( const LoggerPtr & log, const TiDBTableScan & table_scan) { - LOG_INFO(log, "gjt debug in PhysicalMockTableScan::build("); assert(context.isTest()); auto [schema, mock_streams] = mockSchemaAndStreams(context, executor_id, log, table_scan); @@ -106,14 +105,12 @@ PhysicalPlanNodePtr PhysicalMockTableScan::build( void PhysicalMockTableScan::buildBlockInputStreamImpl(DAGPipeline & pipeline, Context & /*context*/, size_t /*max_streams*/) { - LOG_INFO(log, "gjt debug in PhysicalMockTableScan 2"); assert(pipeline.streams.empty()); pipeline.streams.insert(pipeline.streams.end(), mock_streams.begin(), mock_streams.end()); } void PhysicalMockTableScan::buildPipelineExec(PipelineExecGroupBuilder & group_builder, Context & /*context*/, size_t /*concurrency*/) { - LOG_INFO(log, "gjt debug in PhysicalMockTableScan 3"); group_builder.init(mock_streams.size()); size_t i = 0; group_builder.transform([&](auto & builder) { diff --git a/dbms/src/Flash/executeQuery.cpp b/dbms/src/Flash/executeQuery.cpp index 817230273ff..dba7f904082 100644 --- a/dbms/src/Flash/executeQuery.cpp +++ b/dbms/src/Flash/executeQuery.cpp @@ -153,21 +153,14 @@ BlockIO executeAsBlockIO(Context & context, bool internal) QueryExecutorPtr queryExecute(Context & context, bool internal) { - auto & dag_context = *context.getDAGContext(); - const auto & logger = dag_context.log; - LOG_INFO(logger, fmt::format("gjt debug queryExecute {} {} {} {}", context.isExecutorTest(), - context.isInterpreterTest(), context.getSettingsRef().enable_planner, - context.getSettingsRef().enable_pipeline)); // now only support pipeline model in executor/interpreter test. if ((context.isExecutorTest() || context.isInterpreterTest()) && context.getSettingsRef().enable_planner && context.getSettingsRef().enable_pipeline) { - LOG_INFO(logger, "gjt debug queryExecute 1"); if (auto res = executeAsPipeline(context, internal); res) return std::move(*res); } - LOG_INFO(logger, "gjt debug queryExecute 2"); return std::make_unique(executeAsBlockIO(context, internal)); } } // namespace DB diff --git a/dbms/src/Flash/tests/gtest_generated_column.cpp b/dbms/src/Flash/tests/gtest_generated_column.cpp deleted file mode 100644 index a928843c24f..00000000000 --- a/dbms/src/Flash/tests/gtest_generated_column.cpp +++ /dev/null @@ -1,58 +0,0 @@ -// Copyright 2023 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -namespace DB -{ -namespace tests -{ - -class GeneratedColumnTestRunner : public DB::tests::ExecutorTest -{ -public: - void initializeContext() override - { - ExecutorTest::initializeContext(); - - context.addMockTable({"test_db", "test_table"}, - {{"col_1", TiDB::TP::TypeString}, {"col_2", TiDB::TP::TypeString}}, - {toNullableVec("col_1", {"banana"}), - toNullableVec("col_2", {"apple"})}); - } -}; - -TEST_F(GeneratedColumnTestRunner, BasicTest) -try -{ - for (size_t i = 0; i < 100; ++i) - { - auto request = context - .scan("test_db", "test_table") - .exchangeSender(tipb::PassThrough) - .build(context); - - // insertGeneratedColumnToTableScanDAGRequest(i, request); - - // Add generated column to table_scan tipb. - executeAndAssertColumnsEqual(request, - {toNullableVec({"banana"}), - toNullableVec({"apple"})}); - } -} -CATCH -} // namespace tests -} // namespace DB diff --git a/dbms/src/Flash/tests/gtest_interpreter.cpp b/dbms/src/Flash/tests/gtest_interpreter.cpp index 8b1501dc0c9..e129c5587a5 100644 --- a/dbms/src/Flash/tests/gtest_interpreter.cpp +++ b/dbms/src/Flash/tests/gtest_interpreter.cpp @@ -14,7 +14,6 @@ #include #include -#include namespace DB { @@ -33,7 +32,7 @@ class InterpreterExecuteTest : public DB::tests::InterpreterTestUtils // 1. manually delete the gtest_interpreter.out // 2. call setRecord() // 3. ./gtests_dbms --gtest_filter=InterpreterExecuteTest.* - setRecord(); + // setRecord(); context.addMockTable({"test_db", "test_table"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}}); context.addMockTable({"test_db", "test_table_1"}, {{"s1", TiDB::TP::TypeString}, {"s2", TiDB::TP::TypeString}, {"s3", TiDB::TP::TypeString}}); @@ -395,21 +394,5 @@ try } } CATCH - -TEST_F(InterpreterExecuteTest, GeneratedColumn) -try -{ - { - // col_1, col_2, gen_col. - auto request = context - .scan("test_db", "test_table") - .exchangeSender(tipb::PassThrough) - .build(context); - const auto & tbl_info = context.mockStorage()->getTableInfo("test_table"); - insertGeneratedColumnToTableScanDAGRequest(tbl_info.columns.size(), request); - runAndAssert(request, 1); - } -} -CATCH } // namespace tests } // namespace DB diff --git a/dbms/src/TestUtils/ExecutorTestUtils.cpp b/dbms/src/TestUtils/ExecutorTestUtils.cpp index 14faa13ae5a..fb5592c4083 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.cpp +++ b/dbms/src/TestUtils/ExecutorTestUtils.cpp @@ -15,7 +15,6 @@ #include #include #include -#include #include #include #include @@ -24,7 +23,6 @@ #include #include -#include namespace DB::tests { @@ -282,51 +280,4 @@ void ExecutorTest::dagRequestEqual(const String & expected_string, const std::sh ASSERT_EQ(Poco::trim(expected_string), Poco::trim(ExecutorSerializer().serialize(actual.get()))); } -void insertGeneratedColumnToTableScanDAGRequest(UInt64 gen_col_num, std::shared_ptr dag_req) -{ - TiDB::ColumnInfo tidb_column_info; - tidb_column_info.tp = TiDB::TP::TypeLong; - tidb_column_info.flag = TiDB::ColumnFlag::ColumnFlagGeneratedColumn; - - tipb::Executor * exec = dag_req->mutable_root_executor(); - tipb::TableScan * tsc = nullptr; - while (tsc == nullptr) - { - switch (exec->tp()) - { - case tipb::ExecType::TypeExchangeSender: - { - exec = exec->mutable_exchange_sender()->mutable_child(); - break; - } - case tipb::ExecType::TypeTableScan: - { - tsc = exec->mutable_tbl_scan(); - break; - } - default: - { - throw Exception("insertGeneratedColumnToTableScanDAGRequest not implement for now"); - } - } - } - - auto ori_columns = tsc->columns(); - tsc->clear_columns(); - int ori_column_index = 0; - for (size_t i = 0; i < ori_columns.size() + gen_col_num; ++i) - { - tipb::ColumnInfo * tipb_column_info = tsc->add_columns(); - if (std::rand() % 2 == 0 || ori_column_index >= ori_columns.size()) - { - String gen_column_name = fmt::format("gen_col_{}", i); - tidb_column_info.name = gen_column_name; - mock::TableScanBinder::setTipbColumnInfo(tipb_column_info, DAGColumnInfo{gen_column_name, tidb_column_info}, 100 + i); - } - else - { - *tipb_column_info = ori_columns[ori_column_index++]; - } - } -} } // namespace DB::tests diff --git a/dbms/src/TestUtils/ExecutorTestUtils.h b/dbms/src/TestUtils/ExecutorTestUtils.h index abc537fe5f1..85caf2358a4 100644 --- a/dbms/src/TestUtils/ExecutorTestUtils.h +++ b/dbms/src/TestUtils/ExecutorTestUtils.h @@ -162,6 +162,4 @@ using ColumnWithInt64 = std::vector; using ColumnWithUInt64 = std::vector; using ColumnWithFloat64 = std::vector; using ColumnWithString = std::vector; - -void insertGeneratedColumnToTableScanDAGRequest(UInt64 gen_col_index, std::shared_ptr dag_req); } // namespace DB::tests From fddbabe5e5ef163997826baa458e91e3426e1dd1 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 10 Feb 2023 11:56:01 +0800 Subject: [PATCH 03/23] fix log Signed-off-by: guo-shaoge --- dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp | 2 -- 1 file changed, 2 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index bbc19365455..a7b26d62756 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -945,13 +945,11 @@ std::tuple> DAGStorageIn if (ci.hasGeneratedColumnFlag()) { - LOG_WARNING(log, "got generated column"); auto data_type = getDataTypeByColumnInfoForComputingLayer(ci); generated_column_infos.push_back(std::make_pair(i, data_type)); source_columns_tmp.emplace_back(NameAndTypePair{GeneratedColumnPlaceholderBlockInputStream::getColumnName(i), data_type}); continue; } - LOG_WARNING(log, "no got generated column"); // Column ID -1 return the handle column String name; if (cid == TiDBPkColumnID) From 93a4c11374cfe4033460c197b8baa113ece013a0 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 10 Feb 2023 12:20:56 +0800 Subject: [PATCH 04/23] add case Signed-off-by: guo-shaoge --- .../GeneratedColumnPlaceholderInputStream.h | 2 +- .../fullstack-test/expr/generated_index.test | 147 ++++++++++++++++++ 2 files changed, 148 insertions(+), 1 deletion(-) create mode 100644 tests/fullstack-test/expr/generated_index.test diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h index 5e61f77d6b1..ee147734dbb 100644 --- a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h @@ -15,9 +15,9 @@ #pragma once #include -#include #include #include +#include #include #include #include diff --git a/tests/fullstack-test/expr/generated_index.test b/tests/fullstack-test/expr/generated_index.test new file mode 100644 index 00000000000..f09f2c3fcde --- /dev/null +++ b/tests/fullstack-test/expr/generated_index.test @@ -0,0 +1,147 @@ +# Copyright 2023 PingCAP, Ltd. +# +# Licensed 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. + +mysql> drop table if exists test.t; +mysql> create table test.t(c1 varchar(100), c2 varchar(100)); +mysql> insert into test.t values('ABC', 'DEF'); +mysql> alter table test.t set tiflash replica 1; +func> wait_table test t +mysql> alter table test.t add index idx2((lower(c2))); + +mysql> select /*+ nth_plan(1) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(2) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(3) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(4) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(5) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(6) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(7) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(8) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(9) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ +mysql> select /*+ nth_plan(10) */ * from test.t where lower(test.t.c2) = 'def'; ++------+------+ +| c1 | c2 | ++------+------+ +| ABC | DEF | ++------+------+ + +mysql> drop table if exists test.t; +mysql> create table test.t(id int, value int); +mysql> alter table test.t set tiflash replica 1; +func> wait_table test t +mysql> create unique index uk on test.t((tidb_shard(id)), id); +mysql> select /*+ nth_paln(1) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(2) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(3) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(4) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(5) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(6) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(7) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(8) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(9) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ +mysql> select /*+ nth_paln(10) */ max(value) from test.t; ++------------+ +| max(value) | ++------------+ +| NULL | ++------------+ From a6daf46370a5baf620a8cbd7062930909553fb54 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Wed, 8 Feb 2023 18:11:38 +0800 Subject: [PATCH 05/23] least/greatest function for string (#6662) close pingcap/tiflash#6660 --- dbms/src/Flash/Coprocessor/DAGUtils.cpp | 6 +- dbms/src/Functions/FunctionsComparison.h | 2 +- dbms/src/Functions/FunctionsConversion.h | 18 +- dbms/src/Functions/LeastGreatest.h | 464 ++++++++++++++++- dbms/src/Functions/greatest.cpp | 11 +- dbms/src/Functions/least.cpp | 11 +- .../Functions/tests/gtest_least_greatest.cpp | 472 +++++++++++++++++- tests/fullstack-test/expr/least_greatest.test | 58 ++- 8 files changed, 1014 insertions(+), 28 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index d57aa58333f..a7462671c77 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -162,13 +162,13 @@ const std::unordered_map scalar_func_map({ {tipb::ScalarFuncSig::GreatestInt, "tidbGreatest"}, {tipb::ScalarFuncSig::GreatestReal, "tidbGreatest"}, - {tipb::ScalarFuncSig::GreatestString, "greatest"}, + {tipb::ScalarFuncSig::GreatestString, "tidbGreatestString"}, {tipb::ScalarFuncSig::GreatestDecimal, "greatest"}, {tipb::ScalarFuncSig::GreatestTime, "greatest"}, {tipb::ScalarFuncSig::LeastInt, "tidbLeast"}, {tipb::ScalarFuncSig::LeastReal, "tidbLeast"}, - {tipb::ScalarFuncSig::LeastString, "least"}, + {tipb::ScalarFuncSig::LeastString, "tidbLeastString"}, {tipb::ScalarFuncSig::LeastDecimal, "least"}, {tipb::ScalarFuncSig::LeastTime, "least"}, diff --git a/dbms/src/Functions/FunctionsComparison.h b/dbms/src/Functions/FunctionsComparison.h index fda8a113ada..416f3d5fffc 100644 --- a/dbms/src/Functions/FunctionsComparison.h +++ b/dbms/src/Functions/FunctionsComparison.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. diff --git a/dbms/src/Functions/FunctionsConversion.h b/dbms/src/Functions/FunctionsConversion.h index e8333ceeeea..82df7472792 100644 --- a/dbms/src/Functions/FunctionsConversion.h +++ b/dbms/src/Functions/FunctionsConversion.h @@ -205,7 +205,7 @@ struct ConvertToDecimalImpl } else { - if (const ColumnVector * col_from + if (const auto * col_from = checkAndGetColumn>(block.getByPosition(arguments[0]).column.get())) { auto col_to = ColumnDecimal::create(0, scale); @@ -245,7 +245,7 @@ struct ConvertToDecimalImpl const IDataType & data_type_to = *block.getByPosition(result).type; - if (const ColumnString * col_from_string = checkAndGetColumn(&col_from)) + if (const auto * col_from_string = checkAndGetColumn(&col_from)) { auto res = data_type_to.createColumn(); @@ -758,8 +758,8 @@ struct ConvertThroughParsing } const IColumn * col_from = block.getByPosition(arguments[0]).column.get(); - const ColumnString * col_from_string = checkAndGetColumn(col_from); - const ColumnFixedString * col_from_fixed_string = checkAndGetColumn(col_from); + const auto * col_from_string = checkAndGetColumn(col_from); + const auto * col_from_fixed_string = checkAndGetColumn(col_from); if (std::is_same_v && !col_from_string) throw Exception("Illegal column " + col_from->getName() @@ -880,7 +880,7 @@ struct ConvertImplGenericFromString const IDataType & data_type_to = *block.getByPosition(result).type; - if (const ColumnString * col_from_string = checkAndGetColumn(&col_from)) + if (const auto * col_from_string = checkAndGetColumn(&col_from)) { auto res = data_type_to.createColumn(); @@ -947,7 +947,7 @@ struct ConvertImpl { static void execute(Block & block, const ColumnNumbers & arguments, size_t result) { - if (const ColumnFixedString * col_from = checkAndGetColumn(block.getByPosition(arguments[0]).column.get())) + if (const auto * col_from = checkAndGetColumn(block.getByPosition(arguments[0]).column.get())) { auto col_to = ColumnString::create(); @@ -1754,7 +1754,7 @@ class FunctionDateFormat : public IFunction class FunctionGetFormat : public IFunction { private: - static String get_format(const StringRef & time_type, const StringRef & location) + static String getFormat(const StringRef & time_type, const StringRef & location) { if (time_type == "DATE") { @@ -1850,7 +1850,7 @@ class FunctionGetFormat : public IFunction for (size_t i = 0; i < size; ++i) { const auto & location = location_col->getDataAt(i); - const auto & result = get_format(StringRef(time_type), location); + const auto & result = getFormat(StringRef(time_type), location); write_buffer.write(result.c_str(), result.size()); writeChar(0, write_buffer); offsets_to[i] = write_buffer.count(); @@ -2615,7 +2615,7 @@ class FunctionCast final : public IFunctionBase const size_t result) { const auto & array_arg = block.getByPosition(arguments.front()); - if (const ColumnArray * col_array = checkAndGetColumn(array_arg.column.get())) + if (const auto * col_array = checkAndGetColumn(array_arg.column.get())) { /// create block for converting nested column containing original and result columns Block nested_block{ diff --git a/dbms/src/Functions/LeastGreatest.h b/dbms/src/Functions/LeastGreatest.h index 55e76b66524..d0b12354767 100644 --- a/dbms/src/Functions/LeastGreatest.h +++ b/dbms/src/Functions/LeastGreatest.h @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -16,6 +16,7 @@ #include #include +#include #include #include #include @@ -29,12 +30,13 @@ #include #include #include +#include #include #include #include #include -#include +#include #include namespace DB @@ -120,5 +122,463 @@ class FunctionVectorizedLeastGreatest : public IFunction private: const Context & context; }; +template +struct LeastGreatestStringImpl +{ + static void mergeImpl( + const TiDB::TiDBCollatorPtr & collator, + size_t a_size, + size_t b_size, + const unsigned char * a_data, + const unsigned char * b_data, + ColumnString::Chars_t & c_data, + ColumnString::Offsets & c_offsets, + size_t i) + { + int res = 0; + auto pre_offset = StringUtil::offsetAt(c_offsets, i); + if constexpr (use_collator) + res = collator->compare(reinterpret_cast(&a_data[0]), a_size, reinterpret_cast(&b_data[0]), b_size); + else + res = mem_utils::CompareStrView({reinterpret_cast(&a_data[0]), a_size}, {reinterpret_cast(&b_data[0]), b_size}); + + if constexpr (least) + { + if (res < 0) + { + memcpy(&c_data[pre_offset], &a_data[0], a_size); + c_offsets[i] = pre_offset + a_size + 1; + } + else if (res == 0) + { + size_t size = std::min(a_size, b_size); + memcpy(&c_data[pre_offset], &b_data[0], size); + c_offsets[i] = pre_offset + size + 1; + } + else + { + memcpy(&c_data[pre_offset], &b_data[0], b_size); + c_offsets[i] = pre_offset + b_size + 1; + } + } + else + { + if (res < 0) + { + memcpy(&c_data[pre_offset], &b_data[0], b_size); + c_offsets[i] = pre_offset + b_size + 1; + } + else if (res == 0) + { + size_t size = std::max(a_size, b_size); + if (a_size > b_size) + { + memcpy(&c_data[pre_offset], &a_data[0], size); + c_offsets[i] = pre_offset + size + 1; + } + else + { + memcpy(&c_data[pre_offset], &b_data[0], size); + c_offsets[i] = pre_offset + size + 1; + } + } + else + { + memcpy(&c_data[pre_offset], &a_data[0], a_size); + c_offsets[i] = pre_offset + a_size + 1; + } + } + } + + static void processImpl( + const TiDB::TiDBCollatorPtr & collator, + size_t a_size, + size_t b_size, + const unsigned char * a_data, + const unsigned char * b_data, + std::vector & res_ref, + size_t i) + { + int res = 0; + if constexpr (use_collator) + res = collator->compare(reinterpret_cast(&a_data[0]), a_size, reinterpret_cast(&b_data[0]), b_size); + else + res = mem_utils::CompareStrView({reinterpret_cast(&a_data[0]), a_size}, {reinterpret_cast(&b_data[0]), b_size}); + + if constexpr (least) + { + if (res < 0) + { + res_ref[i] = StringRef(&a_data[0], a_size); + } + else if (res == 0) + { + size_t size = std::min(a_size, b_size); + res_ref[i] = StringRef(&b_data[0], size); + } + else + { + res_ref[i] = StringRef(&b_data[0], b_size); + } + } + else + { + if (res < 0) + { + res_ref[i] = StringRef(&b_data[0], b_size); + } + else if (res == 0) + { + if (a_size > b_size) + res_ref[i] = StringRef(&a_data[0], a_size); + else + res_ref[i] = StringRef(&b_data[0], b_size); + } + else + { + res_ref[i] = StringRef(&a_data[0], a_size); + } + } + } + + // StringRef_string + static void process( + const TiDB::TiDBCollatorPtr & collator, + std::vector & res_ref, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets, + size_t i) + { + size_t a_size = res_ref[i].size; + size_t b_size = StringUtil::sizeAt(b_offsets, i) - 1; + const auto * a_data = reinterpret_cast(res_ref[i].data); + processImpl(collator, a_size, b_size, a_data, &b_data[b_offsets[i - 1]], res_ref, i); + } + + // StringRef_constant + static void process( + const TiDB::TiDBCollatorPtr & collator, + std::vector & res_ref, + StringRef & b, + ColumnString::Chars_t & c_data, + ColumnString::Offsets & c_offsets, + size_t i) + { + size_t a_size = res_ref[i].size; + ColumnString::Offset b_size = b.size; + const auto * a_data = reinterpret_cast(res_ref[i].data); + const auto * b_data = reinterpret_cast(b.data); + mergeImpl(collator, a_size, b_size, &a_data[0], &b_data[0], c_data, c_offsets, i); + } + + // string_string + static void process( + const TiDB::TiDBCollatorPtr & collator, + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets, + std::vector & res_ref, + size_t i) + { + size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1; + size_t b_size = StringUtil::sizeAt(b_offsets, i) - 1; + processImpl(collator, a_size, b_size, &a_data[a_offsets[i - 1]], &b_data[b_offsets[i - 1]], res_ref, i); + } + + // string_constant + static void process( + const TiDB::TiDBCollatorPtr & collator, + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const StringRef & b, + std::vector & res_ref, + size_t i) + { + const auto * b_data = reinterpret_cast(b.data); + ColumnString::Offset b_size = b.size; + size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1; + processImpl(collator, a_size, b_size, &a_data[a_offsets[i - 1]], &b_data[0], res_ref, i); + } + + // string_constant + static void process( + const TiDB::TiDBCollatorPtr & collator, + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const StringRef & b, + ColumnString::Chars_t & c_data, + ColumnString::Offsets & c_offsets, + size_t i) + { + const auto * b_data = reinterpret_cast(b.data); + ColumnString::Offset b_size = b.size; + size_t a_size = StringUtil::sizeAt(a_offsets, i) - 1; + mergeImpl(collator, a_size, b_size, &a_data[a_offsets[i - 1]], &b_data[0], c_data, c_offsets, i); + } + + // constant_constant + static void process( + const TiDB::TiDBCollatorPtr & collator, + StringRef & a, + const StringRef & b) + { + int res = 0; + if constexpr (use_collator) + res = collator->compare(reinterpret_cast(a.data), a.size, reinterpret_cast(b.data), b.size); + else + res = a.compare(b); + + if constexpr (least) + { + if (res > 0 || (res == 0 && a.size > b.size)) + a = b; + } + else + { + if (res < 0 || (res == 0 && a.size < b.size)) + a = b; + } + } +}; + +template +struct StringOperationImpl +{ + static void NO_INLINE stringVectorStringVector( + const TiDB::TiDBCollatorPtr & collator, + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets, + std::vector & res_ref) + { + size_t size = a_offsets.size(); + for (size_t i = 0; i < size; ++i) + LeastGreatestStringImpl::process(collator, a_data, a_offsets, b_data, b_offsets, res_ref, i); + } + + static void NO_INLINE stringRefVectorStringVector( + const TiDB::TiDBCollatorPtr & collator, + std::vector & res_ref, + const ColumnString::Chars_t & b_data, + const ColumnString::Offsets & b_offsets) + { + size_t size = b_offsets.size(); + for (size_t i = 0; i < size; ++i) + LeastGreatestStringImpl::process(collator, res_ref, b_data, b_offsets, i); + } + + static void NO_INLINE stringRefVectorConstant( + const TiDB::TiDBCollatorPtr & collator, + std::vector & res_ref, + StringRef & b, + ColumnString::Chars_t & c_data, + ColumnString::Offsets & c_offsets) + { + size_t size = res_ref.size(); + size_t res_ref_size = 0; + for (auto & ref : res_ref) + res_ref_size += ref.size; + c_data.resize(std::max(res_ref_size, b.size * size)); + c_offsets.resize(size); + for (size_t i = 0; i < size; ++i) + LeastGreatestStringImpl::process(collator, res_ref, b, c_data, c_offsets, i); + } + + static void NO_INLINE stringVectorConstant( + const TiDB::TiDBCollatorPtr & collator, + const ColumnString::Chars_t & a_data, + const ColumnString::Offsets & a_offsets, + const StringRef & b, + ColumnString::Chars_t & c_data, + ColumnString::Offsets & c_offsets) + { + size_t size = a_offsets.size(); + c_data.resize(std::max(a_data.size(), b.size * size)); + c_offsets.resize(size); + for (size_t i = 0; i < size; ++i) + LeastGreatestStringImpl::process(collator, a_data, a_offsets, b, c_data, c_offsets, i); + } + + static void constantConstant( + const TiDB::TiDBCollatorPtr & collator, + StringRef & a, + StringRef & b) + { + LeastGreatestStringImpl::process(collator, a, b); + } +}; + +template +class FunctionLeastGreatestString : public IFunction +{ +public: + static constexpr auto name = least ? "tidbLeastString" : "tidbGreatestString"; + explicit FunctionLeastGreatestString() = default; + + static FunctionPtr create(const Context & context [[maybe_unused]]) + { + return std::make_shared(); + } + + String getName() const override { return name; } + bool isVariadic() const override { return true; } + + void setCollator(const TiDB::TiDBCollatorPtr & collator_) override { collator = collator_; } + + size_t getNumberOfArguments() const override { return 0; } + bool useDefaultImplementationForNulls() const override { return true; } + + DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override + { + if (arguments.size() < 2) + throw Exception( + fmt::format("Number of arguments for function {} doesn't match: passed {}, should be at least 2.", getName(), arguments.size()), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + + for (const auto & argument : arguments) + { + if (!argument->isString()) + { + throw Exception( + fmt::format("argument type not string"), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + } + } + return std::make_shared(); + } + + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) const override + { + size_t num_arguments = arguments.size(); + if (num_arguments < 2) + { + throw Exception( + fmt::format("Number of arguments for function {} doesn't match: passed {}, should be at least 2.", getName(), arguments.size()), + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + } + if (collator) + executeInternal(block, arguments, result); + else + executeInternal(block, arguments, result); + } + + template + void executeInternal(Block & block, const ColumnNumbers & arguments, size_t result) const + { + size_t num_arguments = arguments.size(); + + using impl = StringOperationImpl; + + std::vector const_columns; + std::vector string_columns; + for (size_t i = 0; i < num_arguments; ++i) + { + const auto * c = block.getByPosition(arguments[i]).column.get(); + const auto * c_string = checkAndGetColumn(c); + const ColumnConst * c_const = checkAndGetColumnConstStringOrFixedString(c); + if (c_const) + const_columns.emplace_back(c_const); + if (c_string) + string_columns.emplace_back(c_string); + } + + // 1. calculate result column for const columns + StringRef const_res; + if (!const_columns.empty()) + { + const_res = const_columns[0]->getDataAt(0); + for (size_t i = 1; i < const_columns.size(); ++i) + { + StringRef b = const_columns[i]->getDataAt(0); + impl::constantConstant(collator, const_res, b); + } + + if (string_columns.empty()) // fill the result column + { + block.getByPosition(result).column + = block + .getByPosition(result) + .type->createColumnConst(const_columns[0]->size(), Field(const_res.toString())); + return; + } + } + + // 2. calculate result column for string columns + auto string_columns_size = string_columns.size(); + if (string_columns_size == 1) + { + // 3A. merge result columns of const columns and result_col + ColumnString * result_col = nullptr; + result_col = const_cast(string_columns[0]); + auto col_str = ColumnString::create(); + impl::stringVectorConstant( + collator, + result_col->getChars(), + result_col->getOffsets(), + const_res, + col_str->getChars(), + col_str->getOffsets()); + block.getByPosition(result).column = std::move(col_str); + return; + } + else if (string_columns_size >= 2) + { + std::vector result_string_refs; + result_string_refs.resize(string_columns[0]->size()); + for (size_t i = 1; i < string_columns_size; ++i) + { + const DB::ColumnString * c0_string; + const auto * c1_string = string_columns[i]; + if (i == 1) + { + c0_string = checkAndGetColumn(string_columns[0]); + + impl::stringVectorStringVector( + collator, + c0_string->getChars(), + c0_string->getOffsets(), + c1_string->getChars(), + c1_string->getOffsets(), + result_string_refs); + } + else + { + impl::stringRefVectorStringVector( + collator, + result_string_refs, + c1_string->getChars(), + c1_string->getOffsets()); + } + } + if (const_columns.empty()) // no const columns, use string columns result + { + // materialize string columns result + auto res_column = ColumnString::create(); + for (auto & ref : result_string_refs) + res_column->insertData(ref.data, ref.size); + block.getByPosition(result).column = std::move(res_column); + return; + } + else + { + // 3B. merge result columns of const columns and vector columns + auto col_str = ColumnString::create(); + impl::stringRefVectorConstant( + collator, + result_string_refs, + const_res, + col_str->getChars(), + col_str->getOffsets()); + block.getByPosition(result).column = std::move(col_str); + return; + } + } + } + +private: + TiDB::TiDBCollatorPtr collator{}; +}; } // namespace DB diff --git a/dbms/src/Functions/greatest.cpp b/dbms/src/Functions/greatest.cpp index 7efabef8470..35538cf05ce 100644 --- a/dbms/src/Functions/greatest.cpp +++ b/dbms/src/Functions/greatest.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -26,8 +26,8 @@ struct BinaryGreatestBaseImpl template static Result apply(A a, B b) { - const Result tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) - const Result tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) return accurate::greaterOp(tmp_a, tmp_b) ? tmp_a : tmp_b; } template @@ -46,8 +46,8 @@ struct BinaryGreatestBaseImpl template static Result apply(A a, B b) { - const Result tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) - const Result tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) return tmp_a > tmp_b ? tmp_a : tmp_b; } template @@ -71,6 +71,7 @@ using FunctionTiDBGreatest = FunctionVectorizedLeastGreatest(); + factory.registerFunction>(); } } // namespace DB diff --git a/dbms/src/Functions/least.cpp b/dbms/src/Functions/least.cpp index 7508ca81320..52785632256 100644 --- a/dbms/src/Functions/least.cpp +++ b/dbms/src/Functions/least.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -30,8 +30,8 @@ struct BinaryLeastBaseImpl static Result apply(A a, B b) { /** gcc 4.9.2 successfully vectorizes a loop from this function. */ - const Result tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) - const Result tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) return accurate::lessOp(tmp_a, tmp_b) ? tmp_a : tmp_b; } template @@ -50,8 +50,8 @@ struct BinaryLeastBaseImpl template static Result apply(A a, B b) { - const Result tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) - const Result tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_a = static_cast(a); // NOLINT(bugprone-signed-char-misuse) + const auto tmp_b = static_cast(b); // NOLINT(bugprone-signed-char-misuse) return tmp_a < tmp_b ? tmp_a : tmp_b; } template @@ -75,6 +75,7 @@ using FunctionTiDBLeast = FunctionVectorizedLeastGreatest(); + factory.registerFunction>(); } } // namespace DB diff --git a/dbms/src/Functions/tests/gtest_least_greatest.cpp b/dbms/src/Functions/tests/gtest_least_greatest.cpp index cbf7552fdc1..339eb291cb8 100644 --- a/dbms/src/Functions/tests/gtest_least_greatest.cpp +++ b/dbms/src/Functions/tests/gtest_least_greatest.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -193,6 +193,476 @@ try } CATCH +TEST_F(LeastGreatestTest, leastString) +try +{ + const String & func_name = "tidbLeastString"; + const auto * utf8mb4_general_ci_collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI); + + /// without collator + // vector vector + ASSERT_COLUMN_EQ(createColumn>({"11"}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"11"}), + createColumn>({"111"}), + createColumn>({"111111"})})); + + ASSERT_COLUMN_EQ(createColumn>({""}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"11"}), + createColumn>({"111"}), + createColumn>({""})})); + + ASSERT_COLUMN_EQ(createColumn>({"11", "2"}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", "2"}), + createColumn>({"111", "22"})})); + // vector constant + ASSERT_COLUMN_EQ( + createColumn>({"11", "22222", "22", "11111111"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createConstColumn>(4, "22222")})); + + ASSERT_COLUMN_EQ( + createColumn>({"11", "211", "211", "11111111"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createConstColumn>(4, "22222"), + createConstColumn>(4, "211")})); + + ASSERT_COLUMN_EQ( + createColumn>({"11", "03", "11", "11111111"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createColumn>({"33", "03", "11", "2333"}), + createConstColumn>(4, "22222")})); + + // constant vector + ASSERT_COLUMN_EQ( + createColumn>({"11", "22222", "22", "11111111"}), + executeFunction( + func_name, + {createConstColumn>(4, "22222"), + createColumn>({"11", "33", "22", "11111111"})})); + + // constant constant + ASSERT_COLUMN_EQ(createConstColumn(4, "11"), + executeFunction( + func_name, + {createConstColumn>(4, "1111"), + createConstColumn>(4, "11")})); + + ASSERT_COLUMN_EQ(createConstColumn(4, "11"), + executeFunction( + func_name, + {createConstColumn>(4, "11"), + createConstColumn>(4, "1111")})); + + + ASSERT_COLUMN_EQ(createConstColumn(4, "11"), + executeFunction( + func_name, + {createConstColumn>(4, "11"), + createConstColumn>(4, "111"), + createConstColumn>(4, "1111")})); + + /// with collator + // vector vector + ASSERT_COLUMN_EQ(createColumn>({"11"}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"11"}), + createColumn>({"111"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"111"}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"2222"}), + createColumn>({"111"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"a", "b"}), + executeFunction( + func_name, + {createColumn>({"a", "b"}), + createColumn>({"b", "c"}), + createColumn>({"c", "d"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"a", "b", "asdhkas-\\"}), + executeFunction( + func_name, + {createColumn>({"a", "b", "asdhkas-\\"}), + createColumn>({"b", "c", "sadhhdask"}), + createColumn>({"c", "d", "sahdjkdsahk"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"11", "22"}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", "22"}), + createColumn>({"111", "222"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"1", "2"}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", "22"}), + createColumn>({"111", "222"}), + createColumn>({"1", "2"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"11", {}}), + executeFunction( + func_name, + {createColumn>({"1111", {}}), + createColumn>({"11", "22"}), + createColumn>({"111", "222"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn({"11", "2"}), + executeFunction( + func_name, + {createColumn({"1111", "2"}), + createColumn({"11", "22"}), + createColumn({"111", "222"})}, + utf8mb4_general_ci_collator)); + + + ASSERT_COLUMN_EQ(createColumn>({{}, {}}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", {}}), + createColumn>({{}, "222"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({{}, {}}), + executeFunction( + func_name, + {createColumn>({{}, {}}), + createColumn>({{}, {}}), + createColumn>({{}, {}})}, + utf8mb4_general_ci_collator)); + + // vector constant + ASSERT_COLUMN_EQ( + createColumn>({"11", "22222", "22", "11111111"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createConstColumn>(4, "22222")}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ( + createColumn>({"11", "22222", "11111", "1111"}), + executeFunction( + func_name, + {createColumn>({"11", "44", "333", "22222"}), + createColumn>({"11", "367", "121", "234"}), + createColumn>({"1111", "33", "11111", "9999"}), + createColumn>({"111", "33", "7777", "1111"}), + createColumn>({"11", "66", "6767", "78878"}), + createConstColumn>(4, "22222"), + createConstColumn>(4, "33333"), + createConstColumn>(4, "44444")}, + utf8mb4_general_ci_collator)); + + // constant vector + ASSERT_COLUMN_EQ( + createColumn>({"11", "22222", "22", "11111111"}), + executeFunction( + func_name, + {createConstColumn>(4, "22222"), + createColumn>({"11", "33", "22", "11111111"})}, + utf8mb4_general_ci_collator)); + + // constant constant + ASSERT_COLUMN_EQ( + createConstColumn(4, "11111"), + executeFunction( + func_name, + {createConstColumn>(4, "11111"), + createConstColumn>(4, "22222")}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ( + createConstColumn>(4, {}), + executeFunction( + func_name, + {createConstColumn>(4, {}), + createConstColumn>(4, "22222")}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ( + createConstColumn(4, "111"), + executeFunction( + func_name, + { + createConstColumn>(4, "1111"), + createConstColumn>(4, "22222"), + createConstColumn>(4, "111"), + }, + utf8mb4_general_ci_collator)); + + const auto * bin_col = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::BINARY); + ASSERT_COLUMN_EQ(createColumn>({{}, {}}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", {}}), + createColumn>({{}, "222"})}, + bin_col)); +} +CATCH + +TEST_F(LeastGreatestTest, greatestString) +try +{ + const String & func_name = "tidbGreatestString"; + + const auto * utf8mb4_general_ci_collator = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::UTF8MB4_GENERAL_CI); + + /// without collator + // vector vector + ASSERT_COLUMN_EQ(createColumn>({"111111"}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"11"}), + createColumn>({"111"}), + createColumn>({"111111"})})); + + ASSERT_COLUMN_EQ(createColumn>({"1111"}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"11"}), + createColumn>({"111"}), + createColumn>({""})})); + + ASSERT_COLUMN_EQ(createColumn>({"1111", "2222"}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", "2"}), + createColumn>({"111", "22"})})); + // vector constant + ASSERT_COLUMN_EQ( + createColumn>({"22222", "33", "22222", "22222"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createConstColumn>(4, "22222")})); + + ASSERT_COLUMN_EQ( + createColumn>({"31111", "33", "31111", "31111"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createConstColumn>(4, "22222"), + createConstColumn>(4, "31111")})); + + ASSERT_COLUMN_EQ( + createColumn>({"33", "33", "22222", "2333"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createColumn>({"33", "33", "11", "2333"}), + createConstColumn>(4, "22222")})); + + // constant vector + ASSERT_COLUMN_EQ( + createColumn>({"22222", "33", "22222", "22222"}), + executeFunction( + func_name, + {createConstColumn>(4, "22222"), + createColumn>({"11", "33", "22", "11111111"})})); + + // constant constant + ASSERT_COLUMN_EQ(createConstColumn(4, "1111"), + executeFunction( + func_name, + {createConstColumn>(4, "1111"), + createConstColumn>(4, "11")})); + + + ASSERT_COLUMN_EQ(createConstColumn(4, "1111"), + executeFunction( + func_name, + {createConstColumn>(4, "11"), + createConstColumn>(4, "1111")})); + + /// with collator + // vector vector + ASSERT_COLUMN_EQ(createColumn>({"1111"}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"11"}), + createColumn>({"111"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"2222"}), + executeFunction( + func_name, + {createColumn>({"1111"}), + createColumn>({"2222"}), + createColumn>({"111"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"c", "d"}), + executeFunction( + func_name, + {createColumn>({"a", "b"}), + createColumn>({"b", "c"}), + createColumn>({"c", "d"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"c", "d", "sahdjkdsahk"}), + executeFunction( + func_name, + {createColumn>({"a", "b", "asdhkas-\\"}), + createColumn>({"b", "c", "sadhhdask"}), + createColumn>({"c", "d", "sahdjkdsahk"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"1111", "2222"}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", "22"}), + createColumn>({"111", "222"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"1111", "2222"}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", "22"}), + createColumn>({"111", "222"}), + createColumn>({"1", "2"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({"1111", {}}), + executeFunction( + func_name, + {createColumn>({"1111", {}}), + createColumn>({"11", "22"}), + createColumn>({"111", "222"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn({"1111", "222"}), + executeFunction( + func_name, + {createColumn({"1111", "2"}), + createColumn({"11", "22"}), + createColumn({"111", "222"})}, + utf8mb4_general_ci_collator)); + + + ASSERT_COLUMN_EQ(createColumn>({{}, {}}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", {}}), + createColumn>({{}, "222"})}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ(createColumn>({{}, {}}), + executeFunction( + func_name, + {createColumn>({{}, {}}), + createColumn>({{}, {}}), + createColumn>({{}, {}})}, + utf8mb4_general_ci_collator)); + + // vector constant + ASSERT_COLUMN_EQ( + createColumn>({"22222", "33", "22222", "22222"}), + executeFunction( + func_name, + {createColumn>({"11", "33", "22", "11111111"}), + createConstColumn>(4, "22222")}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ( + createColumn>({"44444", "66", "7777", "9999"}), + executeFunction( + func_name, + {createColumn>({"11", "44", "333", "22222"}), + createColumn>({"11", "367", "121", "234"}), + createColumn>({"1111", "33", "11111", "9999"}), + createColumn>({"111", "33", "7777", "1111"}), + createColumn>({"11", "66", "6767", "78878"}), + createConstColumn>(4, "22222"), + createConstColumn>(4, "33333"), + createConstColumn>(4, "44444")}, + utf8mb4_general_ci_collator)); + + // constant vector + ASSERT_COLUMN_EQ( + createColumn>({"22222", "33", "22222", "22222"}), + executeFunction( + func_name, + {createConstColumn>(4, "22222"), + createColumn>({"11", "33", "22", "11111111"})}, + utf8mb4_general_ci_collator)); + + // constant constant + ASSERT_COLUMN_EQ( + createConstColumn(4, "22222"), + executeFunction( + func_name, + {createConstColumn>(4, "11111"), + createConstColumn>(4, "22222")}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ( + createConstColumn>(4, {}), + executeFunction( + func_name, + {createConstColumn>(4, {}), + createConstColumn>(4, "22222")}, + utf8mb4_general_ci_collator)); + + ASSERT_COLUMN_EQ( + createConstColumn(4, "33"), + executeFunction( + func_name, + { + createConstColumn>(4, "1111"), + createConstColumn>(4, "22222"), + createConstColumn>(4, "33"), + }, + utf8mb4_general_ci_collator)); + + const auto * bin_col = TiDB::ITiDBCollator::getCollator(TiDB::ITiDBCollator::BINARY); + ASSERT_COLUMN_EQ(createColumn>({{}, {}}), + executeFunction( + func_name, + {createColumn>({"1111", "2222"}), + createColumn>({"11", {}}), + createColumn>({{}, "222"})}, + bin_col)); +} +CATCH + TEST_F(LeastGreatestTest, testGreatest) try { diff --git a/tests/fullstack-test/expr/least_greatest.test b/tests/fullstack-test/expr/least_greatest.test index 38b2ba6bc3d..bb3ecad4483 100644 --- a/tests/fullstack-test/expr/least_greatest.test +++ b/tests/fullstack-test/expr/least_greatest.test @@ -1,4 +1,4 @@ -# Copyright 2022 PingCAP, Ltd. +# Copyright 2023 PingCAP, Ltd. # # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. @@ -24,8 +24,12 @@ mysql> drop table if exists test.t3 mysql> create table test.t3 (id1 int, id2 int) mysql> alter table test.t3 set tiflash replica 1 mysql> insert into test.t3 values (NULL, NULL), (NULL, NULL), (NULL, NULL) +mysql> drop table if exists test.t4 +mysql> create table test.t4 (a varchar(20), b varchar(20)) +mysql> alter table test.t4 set tiflash replica 1 +mysql> insert into test.t4 values ("111", "422"), ("11", "333"), ("1", "343") -func> wait_table test t1 t2 t3 +func> wait_table test t1 t2 t3 t4 # parse error mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select least() from test.t1 @@ -143,6 +147,27 @@ mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; sele | NULL | +-----------------+ +# least String +mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select least(a, b) from test.t4; ++-------------+ +| least(a, b) | ++-------------+ +| 111 | +| 11 | +| 1 | ++-------------+ +mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select least(a, b, '11', '1') from test.t4 +least(a, b, '11', '1') +1 +1 +1 + +mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select least(a, b, "11", "1", null) from test.t4 +least(a, b, 11, 1, null) +NULL +NULL +NULL + # greatest tests mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select id, greatest(id, id1, id2, id3, id4, id5, id6, id7, id8) from test.t1 order by 1 +------+------------------------------------------------------+ @@ -284,6 +309,35 @@ mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; sele +--------------------+ # greatest(decimal, int) not support for now. + +# greatest String +mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select greatest(a, b) from test.t4 ++----------------+ +| greatest(a, b) | ++----------------+ +| 422 | +| 333 | +| 343 | ++----------------+ + +mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select greatest(a, b, "342") from test.t4 +greatest(a, b, 342) +422 +342 +343 + +mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select greatest(a, b, "342", "344") from test.t4 +greatest(a, b, 342, 344) +422 +344 +344 + +mysql> set @@tidb_isolation_read_engines='tiflash'; set tidb_enforce_mpp=1; select greatest(a, b, "342", "344", null) from test.t4 +greatest(a, b, 342, 344, null) +NULL +NULL +NULL + # mysql> drop table if exists test.t1; # mysql> create table test.t1(c1 decimal(10, 1), c2 int); # mysql> insert into test.t1 values(1, 11); From 5b033a1e5ba95af7245228b4e1436001ad0ab927 Mon Sep 17 00:00:00 2001 From: yanweiqi <592838129@qq.com> Date: Wed, 8 Feb 2023 18:51:59 +0800 Subject: [PATCH 06/23] fix tls ipv6 metric (#6736) close pingcap/tiflash#6734 --- dbms/src/Server/MetricsPrometheus.cpp | 33 +++++++++++++++++++-------- 1 file changed, 24 insertions(+), 9 deletions(-) diff --git a/dbms/src/Server/MetricsPrometheus.cpp b/dbms/src/Server/MetricsPrometheus.cpp index 09a8542c9ee..61a49548266 100644 --- a/dbms/src/Server/MetricsPrometheus.cpp +++ b/dbms/src/Server/MetricsPrometheus.cpp @@ -1,4 +1,4 @@ -// Copyright 2022 PingCAP, Ltd. +// Copyright 2023 PingCAP, Ltd. // // Licensed under the Apache License, Version 2.0 (the "License"); // you may not use this file except in compliance with the License. @@ -104,7 +104,7 @@ class MetricHandlerFactory : public Poco::Net::HTTPRequestHandlerFactory std::shared_ptr getHTTPServer( Context & global_context, const std::weak_ptr & collectable, - const String & metrics_port) + const String & address) { auto security_config = global_context.getSecurityConfig(); auto [ca_path, cert_path, key_path] = security_config->getPaths(); @@ -126,8 +126,7 @@ std::shared_ptr getHTTPServer( Poco::Net::SecureServerSocket socket(context); Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - - Poco::Net::SocketAddress addr("0.0.0.0", std::stoi(metrics_port)); + Poco::Net::SocketAddress addr = Poco::Net::SocketAddress(address); socket.bind(addr, true); socket.listen(); auto server = std::make_shared(new MetricHandlerFactory(collectable), socket, http_params); @@ -137,6 +136,17 @@ std::shared_ptr getHTTPServer( constexpr Int64 MILLISECOND = 1000; constexpr Int64 INIT_DELAY = 5; +namespace +{ +inline bool isIPv6(const String & input_address) +{ + if (input_address.empty()) + return false; + char str[INET6_ADDRSTRLEN]; + return inet_pton(AF_INET6, input_address.c_str(), str) == 1; +} +} // namespace + MetricsPrometheus::MetricsPrometheus( Context & context, const AsynchronousMetrics & async_metrics_) @@ -202,18 +212,23 @@ MetricsPrometheus::MetricsPrometheus( if (conf.hasOption(status_metrics_port) || !conf.hasOption(status_metrics_addr)) { auto metrics_port = conf.getString(status_metrics_port, DB::toString(DEFAULT_METRICS_PORT)); - + auto listen_host = conf.getString("listen_host", "0.0.0.0"); + String addr; + if (isIPv6(listen_host)) + addr = "[" + listen_host + "]:" + metrics_port; + else + addr = listen_host + ":" + metrics_port; if (context.getSecurityConfig()->hasTlsConfig()) { - server = getHTTPServer(context, tiflash_metrics.registry, metrics_port); + server = getHTTPServer(context, tiflash_metrics.registry, addr); server->start(); - LOG_INFO(log, "Enable prometheus secure pull mode; Metrics Port = {}", metrics_port); + LOG_INFO(log, "Enable prometheus secure pull mode; Listen Host = {}, Metrics Port = {}", listen_host, metrics_port); } else { - exposer = std::make_shared(metrics_port); + exposer = std::make_shared(addr); exposer->RegisterCollectable(tiflash_metrics.registry); - LOG_INFO(log, "Enable prometheus pull mode; Metrics Port = {}", metrics_port); + LOG_INFO(log, "Enable prometheus pull mode; Listen Host = {}, Metrics Port = {}", listen_host, metrics_port); } } else From 6a1bee45b99713d54cc468028b2fee36b1ffdde9 Mon Sep 17 00:00:00 2001 From: xufei Date: Wed, 8 Feb 2023 20:35:59 +0800 Subject: [PATCH 07/23] Refine spiller (#6780) ref pingcap/tiflash#6528 --- dbms/src/Core/SpillHandler.cpp | 13 +- dbms/src/Core/Spiller.cpp | 29 ++- dbms/src/Core/Spiller.h | 21 ++- dbms/src/Core/tests/gtest_spiller.cpp | 173 ++++++------------ .../DataStreams/SpilledFilesInputStream.cpp | 15 +- .../src/DataStreams/SpilledFilesInputStream.h | 19 +- 6 files changed, 125 insertions(+), 145 deletions(-) diff --git a/dbms/src/Core/SpillHandler.cpp b/dbms/src/Core/SpillHandler.cpp index 23a68d252ae..fc778b19ff2 100644 --- a/dbms/src/Core/SpillHandler.cpp +++ b/dbms/src/Core/SpillHandler.cpp @@ -64,21 +64,21 @@ void SpillHandler::spillBlocks(const Blocks & blocks) { Stopwatch watch; RUNTIME_CHECK_MSG(spiller->spill_finished == false, "{}: spill after the spiller is finished.", spiller->config.spill_id); - LOG_INFO(spiller->logger, "Spilling {} blocks data into temporary file {}", blocks.size(), current_spill_file_name); - size_t spilled_data_size = 0; + auto block_size = blocks.size(); + LOG_INFO(spiller->logger, "Spilling {} blocks data into temporary file {}", block_size, current_spill_file_name); + size_t total_rows = 0; if (unlikely(writer == nullptr)) { writer = std::make_unique(spiller->config.file_provider, current_spill_file_name, blocks[0].cloneEmpty(), spiller->spill_version); } for (const auto & block : blocks) { - auto block_bytes_size = block.bytes(); + total_rows += block.rows(); writer->write(block); - spilled_data_size += block_bytes_size; } double cost = watch.elapsedSeconds(); time_cost += cost; - LOG_INFO(spiller->logger, "Finish Spilling data into temporary file {}, spilled data size: {}, time cost: {:.3f} sec.", current_spill_file_name, spilled_data_size, cost); + LOG_INFO(spiller->logger, "Spilled {} rows from {} blocks into temporary file, time cost: {:.3f} sec.", total_rows, block_size, cost); RUNTIME_CHECK_MSG(current_spilled_file_index >= 0, "{}: spill after the spill handler is finished.", spiller->config.spill_id); RUNTIME_CHECK_MSG(spiller->spill_finished == false, "{}: spill after the spiller is finished.", spiller->config.spill_id); return; @@ -104,7 +104,7 @@ void SpillHandler::finish() SpillDetails details{0, 0, 0}; for (Int64 i = 0; i <= current_spilled_file_index; i++) details.merge(spilled_files[i]->getSpillDetails()); - return fmt::format("Spill {} rows in {:.3f} sec," + return fmt::format("Commit spilled data, details: spill {} rows in {:.3f} sec," " {:.3f} MiB uncompressed, {:.3f} MiB compressed, {:.3f} uncompressed bytes per row, {:.3f} compressed bytes per row, " "compression rate: {:.3f} ({:.3f} rows/sec., {:.3f} MiB/sec. uncompressed, {:.3f} MiB/sec. compressed)", details.rows, @@ -125,6 +125,7 @@ void SpillHandler::finish() spilled_files.clear(); spiller->has_spilled_data = true; current_spilled_file_index = -1; + RUNTIME_CHECK_MSG(spiller->spill_finished == false, "{}: spill after the spiller is finished.", spiller->config.spill_id); } } diff --git a/dbms/src/Core/Spiller.cpp b/dbms/src/Core/Spiller.cpp index 6497ae2e980..1d791e0dfff 100644 --- a/dbms/src/Core/Spiller.cpp +++ b/dbms/src/Core/Spiller.cpp @@ -47,13 +47,14 @@ SpilledFile::~SpilledFile() } } -Spiller::Spiller(const SpillConfig & config_, bool is_input_sorted_, size_t partition_num_, const Block & input_schema_, const LoggerPtr & logger_, Int64 spill_version_) +Spiller::Spiller(const SpillConfig & config_, bool is_input_sorted_, size_t partition_num_, const Block & input_schema_, const LoggerPtr & logger_, Int64 spill_version_, bool release_spilled_file_on_restore_) : config(config_) , is_input_sorted(is_input_sorted_) , partition_num(partition_num_) , input_schema(input_schema_) , logger(logger_) , spill_version(spill_version_) + , release_spilled_file_on_restore(release_spilled_file_on_restore_) { for (size_t i = 0; i < partition_num; ++i) spilled_files.push_back(std::make_unique()); @@ -117,32 +118,42 @@ BlockInputStreams Spiller::restoreBlocks(size_t partition_id, size_t max_stream_ BlockInputStreams ret; if (is_input_sorted) { - for (const auto & file : spilled_files[partition_id]->spilled_files) + for (auto & file : spilled_files[partition_id]->spilled_files) { RUNTIME_CHECK_MSG(file->exists(), "Spill file {} does not exists", file->path()); details.merge(file->getSpillDetails()); - std::vector files{file->path()}; - ret.push_back(std::make_shared(files, input_schema, config.file_provider, spill_version)); + std::vector file_infos; + file_infos.emplace_back(file->path()); + if (release_spilled_file_on_restore) + file_infos.back().file = std::move(file); + ret.push_back(std::make_shared(std::move(file_infos), input_schema, config.file_provider, spill_version)); } } else { size_t return_stream_num = std::min(max_stream_size, spilled_files[partition_id]->spilled_files.size()); - std::vector> files(return_stream_num); + std::vector> file_infos(return_stream_num); // todo balance based on SpilledRows for (size_t i = 0; i < spilled_files[partition_id]->spilled_files.size(); ++i) { - const auto & file = spilled_files[partition_id]->spilled_files[i]; + auto & file = spilled_files[partition_id]->spilled_files[i]; RUNTIME_CHECK_MSG(file->exists(), "Spill file {} does not exists", file->path()); details.merge(file->getSpillDetails()); - files[i % return_stream_num].push_back(file->path()); + file_infos[i % return_stream_num].push_back(file->path()); + if (release_spilled_file_on_restore) + file_infos[i % return_stream_num].back().file = std::move(file); } for (size_t i = 0; i < return_stream_num; ++i) { - if (likely(!files[i].empty())) - ret.push_back(std::make_shared(files[i], input_schema, config.file_provider, spill_version)); + if (likely(!file_infos[i].empty())) + ret.push_back(std::make_shared(std::move(file_infos[i]), input_schema, config.file_provider, spill_version)); } } + if (release_spilled_file_on_restore) + { + /// clear the spilled_files so we can safely assume that the element in spilled_files is always not nullptr + spilled_files[partition_id]->spilled_files.clear(); + } LOG_DEBUG(logger, "Will restore {} rows from file of size {:.3f} MiB compressed, {:.3f} MiB uncompressed.", details.rows, (details.data_bytes_compressed / 1048576.0), (details.data_bytes_uncompressed / 1048576.0)); if (ret.empty()) ret.push_back(std::make_shared(input_schema)); diff --git a/dbms/src/Core/Spiller.h b/dbms/src/Core/Spiller.h index f1d2afc1ab4..8fd01b8fed6 100644 --- a/dbms/src/Core/Spiller.h +++ b/dbms/src/Core/Spiller.h @@ -67,7 +67,7 @@ struct SpilledFiles class Spiller { public: - Spiller(const SpillConfig & config, bool is_input_sorted, size_t partition_num, const Block & input_schema, const LoggerPtr & logger, Int64 spill_version = 1); + Spiller(const SpillConfig & config, bool is_input_sorted, size_t partition_num, const Block & input_schema, const LoggerPtr & logger, Int64 spill_version = 1, bool release_spilled_file_on_restore = true); void spillBlocks(const Blocks & blocks, size_t partition_id); /// spill blocks by reading from BlockInputStream, this is more memory friendly compared to spillBlocks void spillBlocksUsingBlockInputStream(IBlockInputStream & block_in, size_t partition_id, const std::function & is_cancelled); @@ -75,7 +75,9 @@ class Spiller BlockInputStreams restoreBlocks(size_t partition_id, size_t max_stream_size = 0); size_t spilledRows(size_t partition_id); void finishSpill() { spill_finished = true; }; - bool hasSpilledData() { return has_spilled_data; }; + bool hasSpilledData() const { return has_spilled_data; }; + /// only for test now + bool releaseSpilledFileOnRestore() const { return release_spilled_file_on_restore; } private: friend class SpillHandler; @@ -83,16 +85,21 @@ class Spiller SpillHandler createSpillHandler(size_t partition_id); const SpillConfig config; - bool is_input_sorted; - size_t partition_num; + const bool is_input_sorted; + const size_t partition_num; /// todo remove input_schema if spiller does not rely on BlockInputStream - Block input_schema; - LoggerPtr logger; + const Block input_schema; + const LoggerPtr logger; std::atomic spill_finished{false}; std::atomic has_spilled_data{false}; static std::atomic tmp_file_index; std::vector> spilled_files; - Int64 spill_version = 1; + const Int64 spill_version = 1; + /// If release_spilled_file_on_restore is true, the spilled file will be released once all the data in the spilled + /// file is read, otherwise, the spilled file will be released when destruct the spiller. Currently, all the spilled + /// file can be released on restore since it is only read once, but in the future if SharedScan(shared cte) need spill, + /// the data may be restored multiple times and release_spilled_file_on_restore need to be set to false. + const bool release_spilled_file_on_restore; }; } // namespace DB diff --git a/dbms/src/Core/tests/gtest_spiller.cpp b/dbms/src/Core/tests/gtest_spiller.cpp index 532bbd7c7e0..18ac0cd0545 100644 --- a/dbms/src/Core/tests/gtest_spiller.cpp +++ b/dbms/src/Core/tests/gtest_spiller.cpp @@ -85,6 +85,26 @@ class SpillerTest : public testing::Test } return ret; } + static void verifyRestoreBlocks(Spiller & spiller, size_t restore_partition_id, size_t restore_max_stream_size, size_t expected_stream_size, const Blocks & expected_blocks) + { + auto block_streams = spiller.restoreBlocks(restore_partition_id, restore_max_stream_size); + if (expected_stream_size > 0) + { + GTEST_ASSERT_EQ(block_streams.size(), expected_stream_size); + } + Blocks restored_blocks; + for (auto & block_stream : block_streams) + { + for (Block block = block_stream->read(); block; block = block_stream->read()) + restored_blocks.push_back(block); + } + GTEST_ASSERT_EQ(expected_blocks.size(), restored_blocks.size()); + for (size_t i = 0; i < expected_blocks.size(); ++i) + { + blockEqual(expected_blocks[i], restored_blocks[i]); + } + } + static String spill_dir; Block spiller_test_header; std::shared_ptr spill_config_ptr; @@ -194,22 +214,8 @@ try for (size_t partition_id = 0; partition_id < partition_num; ++partition_id) { size_t max_restore_streams = 2 + partition_id * 10; - auto restore_block_streams = spiller.restoreBlocks(partition_id, max_restore_streams); size_t expected_streams = std::min(max_restore_streams, spill_num); - GTEST_ASSERT_EQ(restore_block_streams.size(), expected_streams); - Blocks all_restored_blocks; - for (const auto & block_stream : restore_block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - { - all_restored_blocks.push_back(block); - } - } - GTEST_ASSERT_EQ(all_restored_blocks.size(), all_blocks[partition_id].size()); - for (size_t i = 0; i < all_restored_blocks.size(); ++i) - { - blockEqual(all_blocks[partition_id][i], all_restored_blocks[i]); - } + verifyRestoreBlocks(spiller, partition_id, max_restore_streams, expected_streams, all_blocks[partition_id]); } } CATCH @@ -244,22 +250,40 @@ try for (size_t partition_id = 0; partition_id < partition_num; ++partition_id) { size_t max_restore_streams = 2 + partition_id * 10; - auto restore_block_streams = spiller->restoreBlocks(partition_id, max_restore_streams); size_t expected_streams = std::min(max_restore_streams, spill_num); - GTEST_ASSERT_EQ(restore_block_streams.size(), expected_streams); - Blocks all_restored_blocks; - for (const auto & block_stream : restore_block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - { - all_restored_blocks.push_back(block); - } - } - GTEST_ASSERT_EQ(all_restored_blocks.size(), all_blocks[partition_id].size()); - for (size_t i = 0; i < all_restored_blocks.size(); ++i) - { - blockEqual(all_blocks[partition_id][i], all_restored_blocks[i]); - } + verifyRestoreBlocks(*spiller, partition_id, max_restore_streams, expected_streams, all_blocks[partition_id]); + } + } +} +CATCH + +TEST_F(SpillerTest, ReleaseFileOnRestore) +try +{ + std::vector> spillers; + spillers.push_back(std::make_unique(*spill_config_ptr, false, 1, spiller_test_header, logger, 1, false)); + auto new_spill_path = fmt::format("{}{}_{}", spill_config_ptr->spill_dir, "release_file_on_restore_test", rand()); + SpillConfig new_spill_config(new_spill_path, spill_config_ptr->spill_id, spill_config_ptr->max_spilled_size_per_spill, spill_config_ptr->file_provider); + Poco::File new_spiller_dir(new_spill_config.spill_dir); + /// remove spiller dir if exists + if (new_spiller_dir.exists()) + new_spiller_dir.remove(true); + new_spiller_dir.createDirectories(); + spillers.push_back(std::make_unique(new_spill_config, false, 1, spiller_test_header, logger)); + + Blocks blocks = generateBlocks(50); + for (auto & spiller : spillers) + { + spiller->spillBlocks(blocks, 0); + spiller->finishSpill(); + verifyRestoreBlocks(*spiller, 0, 0, 0, blocks); + if (!spiller->releaseSpilledFileOnRestore()) + verifyRestoreBlocks(*spiller, 0, 0, 0, blocks); + else + { + std::vector files; + new_spiller_dir.list(files); + GTEST_ASSERT_EQ(files.size(), 0); } } } @@ -285,23 +309,8 @@ try for (size_t partition_id = 0; partition_id < partition_num; ++partition_id) { size_t max_restore_streams = 2 + partition_id * 10; - auto restore_block_streams = spiller.restoreBlocks(partition_id, max_restore_streams); - /// for sorted spill, the restored stream num is always equal to the spill time size_t expected_streams = spill_num; - GTEST_ASSERT_EQ(restore_block_streams.size(), expected_streams); - Blocks all_restored_blocks; - for (const auto & block_stream : restore_block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - { - all_restored_blocks.push_back(block); - } - } - GTEST_ASSERT_EQ(all_restored_blocks.size(), all_blocks[partition_id].size()); - for (size_t i = 0; i < all_restored_blocks.size(); ++i) - { - blockEqual(all_blocks[partition_id][i], all_restored_blocks[i]); - } + verifyRestoreBlocks(spiller, partition_id, max_restore_streams, expected_streams, all_blocks[partition_id]); } } CATCH @@ -336,23 +345,9 @@ try for (size_t partition_id = 0; partition_id < partition_num; ++partition_id) { size_t max_restore_streams = 2 + partition_id * 10; - auto restore_block_streams = spiller->restoreBlocks(partition_id, max_restore_streams); /// for sorted spill, the restored stream num is always equal to the spill time size_t expected_streams = spill_num; - GTEST_ASSERT_EQ(restore_block_streams.size(), expected_streams); - Blocks all_restored_blocks; - for (const auto & block_stream : restore_block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - { - all_restored_blocks.push_back(block); - } - } - GTEST_ASSERT_EQ(all_restored_blocks.size(), all_blocks[partition_id].size()); - for (size_t i = 0; i < all_restored_blocks.size(); ++i) - { - blockEqual(all_blocks[partition_id][i], all_restored_blocks[i]); - } + verifyRestoreBlocks(*spiller, partition_id, max_restore_streams, expected_streams, all_blocks[partition_id]); } } } @@ -449,19 +444,7 @@ try ret.emplace_back(data); spiller.spillBlocks(ret, 0); spiller.finishSpill(); - auto block_streams = spiller.restoreBlocks(0, 2); - GTEST_ASSERT_EQ(block_streams.size(), 1); - Blocks restored_blocks; - for (auto & block_stream : block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - restored_blocks.push_back(block); - } - GTEST_ASSERT_EQ(ret.size(), restored_blocks.size()); - for (size_t i = 0; i < ret.size(); ++i) - { - blockEqual(ret[i], restored_blocks[i]); - } + verifyRestoreBlocks(spiller, 0, 2, 1, ret); } CATCH @@ -497,19 +480,7 @@ try ret.emplace_back(data); spiller.spillBlocks(ret, 0); spiller.finishSpill(); - auto block_streams = spiller.restoreBlocks(0, 2); - GTEST_ASSERT_EQ(block_streams.size(), 1); - Blocks restored_blocks; - for (auto & block_stream : block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - restored_blocks.push_back(block); - } - GTEST_ASSERT_EQ(ret.size(), restored_blocks.size()); - for (size_t i = 0; i < ret.size(); ++i) - { - blockEqual(ret[i], restored_blocks[i]); - } + verifyRestoreBlocks(spiller, 0, 2, 1, ret); } CATCH @@ -543,19 +514,7 @@ try ret.emplace_back(data); spiller.spillBlocks(ret, 0); spiller.finishSpill(); - auto block_streams = spiller.restoreBlocks(0, 2); - GTEST_ASSERT_EQ(block_streams.size(), 1); - Blocks restored_blocks; - for (auto & block_stream : block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - restored_blocks.push_back(block); - } - GTEST_ASSERT_EQ(ret.size(), restored_blocks.size()); - for (size_t i = 0; i < ret.size(); ++i) - { - blockEqual(ret[i], restored_blocks[i]); - } + verifyRestoreBlocks(spiller, 0, 2, 1, ret); } CATCH @@ -585,19 +544,7 @@ try ret.emplace_back(data); spiller.spillBlocks(ret, 0); spiller.finishSpill(); - auto block_streams = spiller.restoreBlocks(0, 2); - GTEST_ASSERT_EQ(block_streams.size(), 1); - Blocks restored_blocks; - for (auto & block_stream : block_streams) - { - for (Block block = block_stream->read(); block; block = block_stream->read()) - restored_blocks.push_back(block); - } - GTEST_ASSERT_EQ(ret.size(), restored_blocks.size()); - for (size_t i = 0; i < ret.size(); ++i) - { - blockEqual(ret[i], restored_blocks[i]); - } + verifyRestoreBlocks(spiller, 0, 2, 1, ret); } CATCH diff --git a/dbms/src/DataStreams/SpilledFilesInputStream.cpp b/dbms/src/DataStreams/SpilledFilesInputStream.cpp index dbace6f1192..b580382b3c4 100644 --- a/dbms/src/DataStreams/SpilledFilesInputStream.cpp +++ b/dbms/src/DataStreams/SpilledFilesInputStream.cpp @@ -16,15 +16,15 @@ namespace DB { -SpilledFilesInputStream::SpilledFilesInputStream(const std::vector & spilled_files_, const Block & header_, const FileProviderPtr & file_provider_, Int64 max_supported_spill_version_) - : spilled_files(spilled_files_) +SpilledFilesInputStream::SpilledFilesInputStream(std::vector && spilled_file_infos_, const Block & header_, const FileProviderPtr & file_provider_, Int64 max_supported_spill_version_) + : spilled_file_infos(std::move(spilled_file_infos_)) , header(header_) , file_provider(file_provider_) , max_supported_spill_version(max_supported_spill_version_) { - RUNTIME_CHECK_MSG(!spilled_files.empty(), "Spilled files must not be empty"); + RUNTIME_CHECK_MSG(!spilled_file_infos.empty(), "Spilled files must not be empty"); current_reading_file_index = 0; - current_file_stream = std::make_unique(spilled_files[0], header, file_provider, max_supported_spill_version); + current_file_stream = std::make_unique(std::move(spilled_file_infos[0]), header, file_provider, max_supported_spill_version); } Block SpilledFilesInputStream::readImpl() @@ -35,9 +35,12 @@ Block SpilledFilesInputStream::readImpl() if (ret) return ret; - for (++current_reading_file_index; current_reading_file_index < spilled_files.size(); ++current_reading_file_index) + for (++current_reading_file_index; current_reading_file_index < spilled_file_infos.size(); ++current_reading_file_index) { - current_file_stream = std::make_unique(spilled_files[current_reading_file_index], header, file_provider, max_supported_spill_version); + current_file_stream = std::make_unique(std::move(spilled_file_infos[current_reading_file_index]), + header, + file_provider, + max_supported_spill_version); ret = current_file_stream->block_in->read(); if (ret) return ret; diff --git a/dbms/src/DataStreams/SpilledFilesInputStream.h b/dbms/src/DataStreams/SpilledFilesInputStream.h index 0b8dbd54613..18cf4bd7205 100644 --- a/dbms/src/DataStreams/SpilledFilesInputStream.h +++ b/dbms/src/DataStreams/SpilledFilesInputStream.h @@ -23,10 +23,19 @@ namespace DB { +struct SpilledFileInfo +{ + String path; + std::unique_ptr file; + SpilledFileInfo(const String path_) + : path(path_) + {} +}; + class SpilledFilesInputStream : public IProfilingBlockInputStream { public: - SpilledFilesInputStream(const std::vector & spilled_files, const Block & header, const FileProviderPtr & file_provider, Int64 max_supported_spill_version); + SpilledFilesInputStream(std::vector && spilled_file_infos, const Block & header, const FileProviderPtr & file_provider, Int64 max_supported_spill_version); Block getHeader() const override; String getName() const override; @@ -36,12 +45,14 @@ class SpilledFilesInputStream : public IProfilingBlockInputStream private: struct SpilledFileStream { + SpilledFileInfo spilled_file_info; ReadBufferFromFileProvider file_in; CompressedReadBuffer<> compressed_in; BlockInputStreamPtr block_in; - SpilledFileStream(const std::string & path, const Block & header, const FileProviderPtr & file_provider, Int64 max_supported_spill_version) - : file_in(file_provider, path, EncryptionPath(path, "")) + SpilledFileStream(SpilledFileInfo && spilled_file_info_, const Block & header, const FileProviderPtr & file_provider, Int64 max_supported_spill_version) + : spilled_file_info(std::move(spilled_file_info_)) + , file_in(file_provider, spilled_file_info.path, EncryptionPath(spilled_file_info.path, "")) , compressed_in(file_in) { Int64 file_spill_version = 0; @@ -54,7 +65,7 @@ class SpilledFilesInputStream : public IProfilingBlockInputStream } }; - std::vector spilled_files; + std::vector spilled_file_infos; size_t current_reading_file_index; Block header; FileProviderPtr file_provider; From 9f8724e66d485b36180f4df558878cf4c424cf22 Mon Sep 17 00:00:00 2001 From: lidezhu <47731263+lidezhu@users.noreply.github.com> Date: Wed, 8 Feb 2023 23:09:58 +0800 Subject: [PATCH 08/23] remove legacy code in RegionPersister (#6781) ref pingcap/tiflash#6728 --- dbms/src/Common/FailPoint.cpp | 2 - dbms/src/Interpreters/Context.cpp | 4 +- dbms/src/Interpreters/Context.h | 1 - dbms/src/Server/DTTool/DTTool.h | 1 - dbms/src/Server/RaftConfigParser.cpp | 6 - dbms/src/Server/RaftConfigParser.h | 4 - dbms/src/Server/Server.cpp | 1 - .../V3/tests/gtest_page_storage_mix_mode.cpp | 2 +- dbms/src/Storages/PathPool.cpp | 4 +- dbms/src/Storages/PathPool.h | 8 +- .../Storages/Transaction/RegionPersister.cpp | 172 +++++------------- .../Storages/Transaction/RegionPersister.h | 2 - .../tests/gtest_region_persister.cpp | 123 +------------ dbms/src/TestUtils/TiFlashTestEnv.cpp | 3 +- 14 files changed, 48 insertions(+), 285 deletions(-) diff --git a/dbms/src/Common/FailPoint.cpp b/dbms/src/Common/FailPoint.cpp index 9a79fd334d5..af0323a1d67 100644 --- a/dbms/src/Common/FailPoint.cpp +++ b/dbms/src/Common/FailPoint.cpp @@ -43,8 +43,6 @@ std::unordered_map> FailPointHelper::f M(region_exception_after_read_from_storage_all_error) \ M(exception_before_dmfile_remove_encryption) \ M(exception_before_dmfile_remove_from_disk) \ - M(force_enable_region_persister_compatible_mode) \ - M(force_disable_region_persister_compatible_mode) \ M(force_triggle_background_merge_delta) \ M(force_triggle_foreground_flush) \ M(exception_before_mpp_register_non_root_mpp_task) \ diff --git a/dbms/src/Interpreters/Context.cpp b/dbms/src/Interpreters/Context.cpp index 1503fc9c088..6487cd33921 100644 --- a/dbms/src/Interpreters/Context.cpp +++ b/dbms/src/Interpreters/Context.cpp @@ -554,7 +554,6 @@ void Context::setPathPool( const Strings & main_data_paths, const Strings & latest_data_paths, const Strings & kvstore_paths, - bool enable_raft_compatible_mode, PathCapacityMetricsPtr global_capacity_, FileProviderPtr file_provider_) { @@ -564,8 +563,7 @@ void Context::setPathPool( latest_data_paths, kvstore_paths, global_capacity_, - file_provider_, - enable_raft_compatible_mode); + file_provider_); } void Context::setConfig(const ConfigurationPtr & config) diff --git a/dbms/src/Interpreters/Context.h b/dbms/src/Interpreters/Context.h index 08a678de2e4..9ff6aa2500c 100644 --- a/dbms/src/Interpreters/Context.h +++ b/dbms/src/Interpreters/Context.h @@ -203,7 +203,6 @@ class Context void setPathPool(const Strings & main_data_paths, const Strings & latest_data_paths, const Strings & kvstore_paths, - bool enable_raft_compatible_mode, PathCapacityMetricsPtr global_capacity_, FileProviderPtr file_provider); diff --git a/dbms/src/Server/DTTool/DTTool.h b/dbms/src/Server/DTTool/DTTool.h index 71e0c109ace..4909726ae65 100644 --- a/dbms/src/Server/DTTool/DTTool.h +++ b/dbms/src/Server/DTTool/DTTool.h @@ -108,7 +108,6 @@ class ImitativeEnv /*main_data_paths*/ {path}, /*latest_data_paths*/ {path}, /*kvstore_paths*/ Strings{}, - /*enable_raft_compatible_mode*/ true, global_context->getPathCapacity(), global_context->getFileProvider()); TiFlashRaftConfig raft_config; diff --git a/dbms/src/Server/RaftConfigParser.cpp b/dbms/src/Server/RaftConfigParser.cpp index 4ebade882fc..c53a0c44a30 100644 --- a/dbms/src/Server/RaftConfigParser.cpp +++ b/dbms/src/Server/RaftConfigParser.cpp @@ -82,12 +82,6 @@ TiFlashRaftConfig TiFlashRaftConfig::parseSettings(Poco::Util::AbstractConfigura res.engine = DEFAULT_ENGINE; } - // just for test - if (config.has("raft.enable_compatible_mode")) - { - res.enable_compatible_mode = config.getBool("raft.enable_compatible_mode"); - } - LOG_INFO(log, "Default storage engine [type={}]", static_cast(res.engine)); return res; diff --git a/dbms/src/Server/RaftConfigParser.h b/dbms/src/Server/RaftConfigParser.h index 34900af7e66..58575f096ca 100644 --- a/dbms/src/Server/RaftConfigParser.h +++ b/dbms/src/Server/RaftConfigParser.h @@ -39,10 +39,6 @@ struct TiFlashRaftConfig // Actually it is "flash.service_addr" std::string flash_server_addr; - // Use PageStorage V1 for kvstore or not. - // TODO: remove this config - bool enable_compatible_mode = true; - bool for_unit_test = false; static constexpr TiDB::StorageEngine DEFAULT_ENGINE = TiDB::StorageEngine::DT; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 4c06f53ccdf..50306f4b453 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -967,7 +967,6 @@ int Server::main(const std::vector & /*args*/) storage_config.main_data_paths, // storage_config.latest_data_paths, // storage_config.kvstore_data_path, // - raft_config.enable_compatible_mode, // global_context->getPathCapacity(), global_context->getFileProvider()); diff --git a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp index 2e638986938..6ea5297a600 100644 --- a/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp +++ b/dbms/src/Storages/Page/V3/tests/gtest_page_storage_mix_mode.cpp @@ -43,7 +43,7 @@ class PageStorageMixedTest : public DB::base::TiFlashStorageTestBasic auto & global_context = TiFlashTestEnv::getGlobalContext(); - storage_path_pool_v3 = std::make_unique(Strings{path}, Strings{path}, Strings{}, std::make_shared(0, paths, caps, Strings{}, caps), global_context.getFileProvider(), true); + storage_path_pool_v3 = std::make_unique(Strings{path}, Strings{path}, Strings{}, std::make_shared(0, paths, caps, Strings{}, caps), global_context.getFileProvider()); global_context.setPageStorageRunMode(PageStorageRunMode::MIX_MODE); } diff --git a/dbms/src/Storages/PathPool.cpp b/dbms/src/Storages/PathPool.cpp index d5121c16e91..71c9a7b7a8b 100644 --- a/dbms/src/Storages/PathPool.cpp +++ b/dbms/src/Storages/PathPool.cpp @@ -58,12 +58,10 @@ PathPool::PathPool( const Strings & latest_data_paths_, const Strings & kvstore_paths_, // PathCapacityMetricsPtr global_capacity_, - FileProviderPtr file_provider_, - bool enable_raft_compatible_mode_) + FileProviderPtr file_provider_) : main_data_paths(main_data_paths_) , latest_data_paths(latest_data_paths_) , kvstore_paths(kvstore_paths_) - , enable_raft_compatible_mode(enable_raft_compatible_mode_) , global_capacity(global_capacity_) , file_provider(file_provider_) , log(Logger::get("PathPool")) diff --git a/dbms/src/Storages/PathPool.h b/dbms/src/Storages/PathPool.h index 599d0ad4715..2e13b2d53f0 100644 --- a/dbms/src/Storages/PathPool.h +++ b/dbms/src/Storages/PathPool.h @@ -56,15 +56,11 @@ class PathPool const Strings & latest_data_paths, const Strings & kvstore_paths, PathCapacityMetricsPtr global_capacity_, - FileProviderPtr file_provider_, - bool enable_raft_compatible_mode_ = false); + FileProviderPtr file_provider_); // Constructor to create PathPool for one Storage StoragePathPool withTable(const String & database_, const String & table_, bool path_need_database_name_) const; - // TODO: remove this outdated code - bool isRaftCompatibleModeEnabled() const { return enable_raft_compatible_mode; } - // Generate a delegator for managing the paths of `RegionPersister`. // Those paths are generated from `kvstore_paths`. // User should keep the pointer to track the PageFileID -> path index mapping. @@ -133,8 +129,6 @@ class PathPool Strings kvstore_paths; Strings global_page_paths; - bool enable_raft_compatible_mode; - PathCapacityMetricsPtr global_capacity; FileProviderPtr file_provider; diff --git a/dbms/src/Storages/Transaction/RegionPersister.cpp b/dbms/src/Storages/Transaction/RegionPersister.cpp index cf19962312e..74ec3709fbf 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.cpp +++ b/dbms/src/Storages/Transaction/RegionPersister.cpp @@ -41,26 +41,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } // namespace ErrorCodes -namespace FailPoints -{ -extern const char force_enable_region_persister_compatible_mode[]; -extern const char force_disable_region_persister_compatible_mode[]; -} // namespace FailPoints - void RegionPersister::drop(RegionID region_id, const RegionTaskLock &) { - if (page_writer) - { - DB::WriteBatch wb_v2{ns_id}; - wb_v2.delPage(region_id); - page_writer->write(std::move(wb_v2), global_context.getWriteLimiter()); - } - else - { - PS::V1::WriteBatch wb_v1; - wb_v1.delPage(region_id); - stable_page_storage->write(std::move(wb_v1)); - } + DB::WriteBatch wb_v2{ns_id}; + wb_v2.delPage(region_id); + page_writer->write(std::move(wb_v2), global_context.getWriteLimiter()); } void RegionPersister::computeRegionWriteBuffer(const Region & region, RegionCacheWriteElement & region_write_buffer) @@ -108,18 +93,9 @@ void RegionPersister::doPersist(RegionCacheWriteElement & region_write_buffer, c std::lock_guard lock(mutex); - if (page_reader) - { - auto entry = page_reader->getPageEntry(region_id); - if (entry.isValid() && entry.tag > applied_index) - return; - } - else - { - auto entry = stable_page_storage->getEntry(region_id, nullptr); - if (entry.isValid() && entry.tag > applied_index) - return; - } + auto entry = page_reader->getPageEntry(region_id); + if (entry.isValid() && entry.tag > applied_index) + return; if (region.isPendingRemove()) { @@ -128,18 +104,9 @@ void RegionPersister::doPersist(RegionCacheWriteElement & region_write_buffer, c } auto read_buf = buffer.tryGetReadBuffer(); - if (page_writer) - { - DB::WriteBatch wb{ns_id}; - wb.putPage(region_id, applied_index, read_buf, region_size); - page_writer->write(std::move(wb), global_context.getWriteLimiter()); - } - else - { - PS::V1::WriteBatch wb; - wb.putPage(region_id, applied_index, read_buf, region_size); - stable_page_storage->write(std::move(wb)); - } + DB::WriteBatch wb{ns_id}; + wb.putPage(region_id, applied_index, read_buf, region_size); + page_writer->write(std::move(wb), global_context.getWriteLimiter()); } RegionPersister::RegionPersister(Context & global_context_, const RegionManager & region_manager_) @@ -150,32 +117,9 @@ RegionPersister::RegionPersister(Context & global_context_, const RegionManager PageStorageConfig RegionPersister::getPageStorageSettings() const { - if (!page_writer) - { - throw Exception("Not support for PS v1", ErrorCodes::LOGICAL_ERROR); - } - return page_writer->getSettings(); } -PS::V1::PageStorage::Config getV1PSConfig(const PageStorageConfig & config) -{ - PS::V1::PageStorage::Config c; - c.sync_on_write = config.sync_on_write; - c.file_roll_size = config.file_roll_size; - c.file_max_size = config.file_max_size; - c.file_small_size = config.file_max_size; - - c.merge_hint_low_used_rate = config.gc_max_valid_rate; - c.merge_hint_low_used_file_total_size = config.gc_min_bytes; - c.merge_hint_low_used_file_num = config.gc_min_files; - c.gc_compact_legacy_min_num = config.gc_min_legacy_num; - - c.version_set_config.compact_hint_delta_deletions = config.version_set_config.compact_hint_delta_deletions; - c.version_set_config.compact_hint_delta_entries = config.version_set_config.compact_hint_delta_entries; - return c; -} - void RegionPersister::forceTransformKVStoreV2toV3() { assert(page_reader != nullptr); @@ -229,36 +173,23 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH { // If there is no PageFile with basic version binary format, use version 2 of PageStorage. auto detect_binary_version = DB::PS::V2::PageStorage::getMaxDataVersion(provider, delegator); - bool use_v1_format = path_pool.isRaftCompatibleModeEnabled() && (detect_binary_version == PageFormat::V1); - - fiu_do_on(FailPoints::force_enable_region_persister_compatible_mode, { use_v1_format = true; }); - fiu_do_on(FailPoints::force_disable_region_persister_compatible_mode, { use_v1_format = false; }); - - if (!use_v1_format) - { - mergeConfigFromSettings(global_context.getSettingsRef(), config); - config.num_write_slots = 4; // extend write slots to 4 at least - - auto page_storage_v2 = std::make_shared( - "RegionPersister", - delegator, - config, - provider, - global_context.getPSBackgroundPool()); - page_storage_v2->restore(); - page_writer = std::make_shared(global_run_mode, page_storage_v2, /*storage_v3_*/ nullptr); - page_reader = std::make_shared(global_run_mode, ns_id, page_storage_v2, /*storage_v3_*/ nullptr, /*readlimiter*/ global_context.getReadLimiter()); - } - else + if (detect_binary_version == PageFormat::V1) { - LOG_INFO(log, "RegionPersister running in v1 mode"); - auto c = getV1PSConfig(config); - stable_page_storage = std::make_unique( - "RegionPersister", - delegator->defaultPath(), - c, - provider); + LOG_WARNING(log, "Detect V1 format data, and we will read it using V2 format code."); } + + mergeConfigFromSettings(global_context.getSettingsRef(), config); + config.num_write_slots = 4; // extend write slots to 4 at least + + auto page_storage_v2 = std::make_shared( + "RegionPersister", + delegator, + config, + provider, + global_context.getPSBackgroundPool()); + page_storage_v2->restore(); + page_writer = std::make_shared(global_run_mode, page_storage_v2, /*storage_v3_*/ nullptr); + page_reader = std::make_shared(global_run_mode, ns_id, page_storage_v2, /*storage_v3_*/ nullptr, /*readlimiter*/ global_context.getReadLimiter()); break; } case PageStorageRunMode::ONLY_V3: @@ -339,51 +270,32 @@ RegionMap RegionPersister::restore(PathPool & path_pool, const TiFlashRaftProxyH } RegionMap regions; - if (page_reader) - { - auto acceptor = [&](const DB::Page & page) { - // We will traverse the pages in V3 before traverse the pages in V2 When we used MIX MODE - // If we found the page_id has been restored, just skip it. - if (const auto it = regions.find(page.page_id); it != regions.end()) - { - LOG_INFO(log, "Already exist [page_id={}], skip it.", page.page_id); - return; - } + auto acceptor = [&](const DB::Page & page) { + // We will traverse the pages in V3 before traverse the pages in V2 When we used MIX MODE + // If we found the page_id has been restored, just skip it. + if (const auto it = regions.find(page.page_id); it != regions.end()) + { + LOG_INFO(log, "Already exist [page_id={}], skip it.", page.page_id); + return; + } - ReadBufferFromMemory buf(page.data.begin(), page.data.size()); - auto region = Region::deserialize(buf, proxy_helper); - if (page.page_id != region->id()) - throw Exception("region id and page id not match!", ErrorCodes::LOGICAL_ERROR); + ReadBufferFromMemory buf(page.data.begin(), page.data.size()); + auto region = Region::deserialize(buf, proxy_helper); + if (page.page_id != region->id()) + throw Exception("region id and page id not match!", ErrorCodes::LOGICAL_ERROR); - regions.emplace(page.page_id, region); - }; - page_reader->traverse(acceptor); - } - else - { - auto acceptor = [&](const PS::V1::Page & page) { - ReadBufferFromMemory buf(page.data.begin(), page.data.size()); - auto region = Region::deserialize(buf, proxy_helper); - if (page.page_id != region->id()) - throw Exception("region id and page id not match!", ErrorCodes::LOGICAL_ERROR); - regions.emplace(page.page_id, region); - }; - stable_page_storage->traverse(acceptor, nullptr); - } + regions.emplace(page.page_id, region); + }; + page_reader->traverse(acceptor); return regions; } bool RegionPersister::gc() { - if (page_writer) - { - PageStorageConfig config = getConfigFromSettings(global_context.getSettingsRef()); - page_writer->reloadSettings(config); - return page_writer->gc(false, nullptr, nullptr); - } - else - return stable_page_storage->gc(); + PageStorageConfig config = getConfigFromSettings(global_context.getSettingsRef()); + page_writer->reloadSettings(config); + return page_writer->gc(false, nullptr, nullptr); } FileUsageStatistics RegionPersister::getFileUsageStatistics() const diff --git a/dbms/src/Storages/Transaction/RegionPersister.h b/dbms/src/Storages/Transaction/RegionPersister.h index ea5682487cb..2a6a6ec4eef 100644 --- a/dbms/src/Storages/Transaction/RegionPersister.h +++ b/dbms/src/Storages/Transaction/RegionPersister.h @@ -78,8 +78,6 @@ class RegionPersister final : private boost::noncopyable PageWriterPtr page_writer; PageReaderPtr page_reader; - std::shared_ptr stable_page_storage; - NamespaceId ns_id = KVSTORE_NAMESPACE_ID; const RegionManager & region_manager; std::mutex mutex; diff --git a/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp b/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp index 8a5e9e70621..5d62778a1f4 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_region_persister.cpp @@ -31,11 +31,6 @@ namespace DB { -namespace FailPoints -{ -extern const char force_enable_region_persister_compatible_mode[]; -extern const char force_disable_region_persister_compatible_mode[]; -} // namespace FailPoints namespace tests { @@ -226,8 +221,7 @@ class RegionPersisterTest : public ::testing::Test main_data_paths, /*kvstore_paths=*/Strings{}, path_capacity, - provider, - /*enable_raft_compatible_mode_=*/true); + provider); } protected: @@ -305,120 +299,5 @@ try } CATCH -TEST_F(RegionPersisterTest, persisterPSVersionUpgrade) -try -{ - auto & global_ctx = TiFlashTestEnv::getGlobalContext(); - auto saved_storage_run_mode = global_ctx.getPageStorageRunMode(); - global_ctx.setPageStorageRunMode(PageStorageRunMode::ONLY_V2); - // Force to run in ps v1 mode for the default region persister - SCOPE_EXIT({ - FailPointHelper::disableFailPoint(FailPoints::force_enable_region_persister_compatible_mode); - global_ctx.setPageStorageRunMode(saved_storage_run_mode); - }); - - size_t region_num = 500; - RegionMap regions; - TableID table_id = 100; - - PageStorageConfig config; - config.file_roll_size = 16 * 1024; - RegionManager region_manager; - DB::Timestamp tso = 0; - { - RegionPersister persister(global_ctx, region_manager); - // Force to run in ps v1 mode - FailPointHelper::enableFailPoint(FailPoints::force_enable_region_persister_compatible_mode); - persister.restore(*mocked_path_pool, nullptr, config); - ASSERT_EQ(persister.page_writer, nullptr); - ASSERT_EQ(persister.page_reader, nullptr); - ASSERT_NE(persister.stable_page_storage, nullptr); // ps v1 - - for (size_t i = 0; i < region_num; ++i) - { - auto region = std::make_shared(createRegionMeta(i, table_id)); - TiKVKey key = RecordKVFormat::genKey(table_id, i, tso++); - region->insert("default", TiKVKey::copyFrom(key), TiKVValue("value1")); - region->insert("write", TiKVKey::copyFrom(key), RecordKVFormat::encodeWriteCfValue('P', 0)); - region->insert("lock", TiKVKey::copyFrom(key), RecordKVFormat::encodeLockCfValue('P', "", 0, 0)); - - persister.persist(*region); - - regions.emplace(region->id(), region); - } - LOG_DEBUG(&Poco::Logger::get("fff"), "v1 write done"); - } - - { - RegionPersister persister(global_ctx, region_manager); - // restore normally, should run in ps v1 mode. - RegionMap new_regions = persister.restore(*mocked_path_pool, nullptr, config); - ASSERT_EQ(persister.page_writer, nullptr); - ASSERT_EQ(persister.page_reader, nullptr); - ASSERT_NE(persister.stable_page_storage, nullptr); // ps v1 - // Try to read - for (size_t i = 0; i < region_num; ++i) - { - auto new_iter = new_regions.find(i); - ASSERT_NE(new_iter, new_regions.end()); - auto old_region = regions[i]; - auto new_region = new_regions[i]; - ASSERT_EQ(*new_region, *old_region); - } - } - - size_t region_num_under_nromal_mode = 200; - { - RegionPersister persister(global_ctx, region_manager); - // Force to run in ps v2 mode - FailPointHelper::enableFailPoint(FailPoints::force_disable_region_persister_compatible_mode); - RegionMap new_regions = persister.restore(*mocked_path_pool, nullptr, config); - ASSERT_NE(persister.page_writer, nullptr); - ASSERT_NE(persister.page_reader, nullptr); - ASSERT_EQ(persister.stable_page_storage, nullptr); - // Try to read - for (size_t i = 0; i < region_num; ++i) - { - auto new_iter = new_regions.find(i); - ASSERT_NE(new_iter, new_regions.end()); - auto old_region = regions[i]; - auto new_region = new_regions[i]; - ASSERT_EQ(*new_region, *old_region); - } - // Try to write more regions under ps v2 mode - for (size_t i = region_num; i < region_num + region_num_under_nromal_mode; ++i) - { - auto region = std::make_shared(createRegionMeta(i, table_id)); - TiKVKey key = RecordKVFormat::genKey(table_id, i, tso++); - region->insert("default", TiKVKey::copyFrom(key), TiKVValue("value1")); - region->insert("write", TiKVKey::copyFrom(key), RecordKVFormat::encodeWriteCfValue('P', 0)); - region->insert("lock", TiKVKey::copyFrom(key), RecordKVFormat::encodeLockCfValue('P', "", 0, 0)); - - persister.persist(*region); - - regions.emplace(region->id(), region); - } - } - - { - RegionPersister persister(global_ctx, region_manager); - // Restore normally, should run in ps v2 mode. - RegionMap new_regions = persister.restore(*mocked_path_pool, nullptr, config); - ASSERT_NE(persister.page_writer, nullptr); - ASSERT_NE(persister.page_reader, nullptr); - ASSERT_EQ(persister.stable_page_storage, nullptr); - // Try to read - for (size_t i = 0; i < region_num + region_num_under_nromal_mode; ++i) - { - auto new_iter = new_regions.find(i); - ASSERT_NE(new_iter, new_regions.end()) << " region:" << i; - auto old_region = regions[i]; - auto new_region = new_regions[i]; - ASSERT_EQ(*new_region, *old_region) << " region:" << i; - } - } -} -CATCH - } // namespace tests } // namespace DB diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index 8ae2aa4c35a..6c96c5177fa 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -119,7 +119,6 @@ void TiFlashTestEnv::addGlobalContext(Strings testdata_path, PageStorageRunMode paths.first, paths.second, Strings{}, - /*enable_raft_compatible_mode=*/true, global_context->getPathCapacity(), global_context->getFileProvider()); @@ -157,7 +156,7 @@ Context TiFlashTestEnv::getContext(const DB::Settings & settings, Strings testda testdata_path.push_back(root_path); context.setPath(root_path); auto paths = getPathPool(testdata_path); - context.setPathPool(paths.first, paths.second, Strings{}, true, context.getPathCapacity(), context.getFileProvider()); + context.setPathPool(paths.first, paths.second, Strings{}, context.getPathCapacity(), context.getFileProvider()); global_contexts[0]->initializeGlobalStoragePoolIfNeed(context.getPathPool()); context.getSettingsRef() = settings; return context; From 3e8ce033d121e92616f16c25a914c5b7d308d458 Mon Sep 17 00:00:00 2001 From: Flowyi Date: Wed, 8 Feb 2023 23:53:58 +0800 Subject: [PATCH 09/23] fix RegionKVStoreTest (#6768) close pingcap/tiflash#6767 --- dbms/src/Debug/MockRaftStoreProxy.cpp | 2 +- dbms/src/Debug/MockStorage.cpp | 9 ++++++--- dbms/src/Debug/MockStorage.h | 4 ++-- dbms/src/Flash/tests/gtest_mock_storage.cpp | 4 ++-- .../Transaction/tests/gtest_new_kvstore.cpp | 8 ++++++++ .../Storages/Transaction/tests/kvstore_helper.h | 8 ++++++-- dbms/src/TestUtils/FunctionTestUtils.cpp | 4 ++-- dbms/src/TestUtils/TiFlashTestEnv.cpp | 14 ++++++++++++++ dbms/src/TestUtils/TiFlashTestEnv.h | 2 ++ 9 files changed, 43 insertions(+), 12 deletions(-) diff --git a/dbms/src/Debug/MockRaftStoreProxy.cpp b/dbms/src/Debug/MockRaftStoreProxy.cpp index 17c52d9029e..024ae1e506e 100644 --- a/dbms/src/Debug/MockRaftStoreProxy.cpp +++ b/dbms/src/Debug/MockRaftStoreProxy.cpp @@ -657,7 +657,7 @@ TableID MockRaftStoreProxy::bootstrap_table( columns.ordinary = NamesAndTypesList({NameAndTypePair{"a", data_type_factory.get("Int64")}}); auto tso = tmt.getPDClient()->getTS(); MockTiDB::instance().newDataBase("d"); - UInt64 table_id = MockTiDB::instance().newTable("d", "t", columns, tso, "", "dt"); + UInt64 table_id = MockTiDB::instance().newTable("d", "t" + toString(random()), columns, tso, "", "dt"); auto schema_syncer = tmt.getSchemaSyncer(); schema_syncer->syncSchemas(ctx); diff --git a/dbms/src/Debug/MockStorage.cpp b/dbms/src/Debug/MockStorage.cpp index 79be39bf0df..788c5c9c9c2 100644 --- a/dbms/src/Debug/MockStorage.cpp +++ b/dbms/src/Debug/MockStorage.cpp @@ -91,14 +91,16 @@ MockColumnInfoVec MockStorage::getTableSchema(const String & name) } /// for delta merge -void MockStorage::addTableSchemaForDeltaMerge(const String & name, const MockColumnInfoVec & columnInfos) +Int64 MockStorage::addTableSchemaForDeltaMerge(const String & name, const MockColumnInfoVec & columnInfos) { - name_to_id_map_for_delta_merge[name] = MockTableIdGenerator::instance().nextTableId(); + auto table_id = MockTableIdGenerator::instance().nextTableId(); + name_to_id_map_for_delta_merge[name] = table_id; table_schema_for_delta_merge[getTableIdForDeltaMerge(name)] = columnInfos; addTableInfoForDeltaMerge(name, columnInfos); + return table_id; } -void MockStorage::addTableDataForDeltaMerge(Context & context, const String & name, ColumnsWithTypeAndName & columns) +Int64 MockStorage::addTableDataForDeltaMerge(Context & context, const String & name, ColumnsWithTypeAndName & columns) { auto table_id = getTableIdForDeltaMerge(name); addNamesAndTypesForDeltaMerge(table_id, columns); @@ -136,6 +138,7 @@ void MockStorage::addTableDataForDeltaMerge(Context & context, const String & na output->write(insert_block); output->writeSuffix(); } + return table_id; } BlockInputStreamPtr MockStorage::getStreamFromDeltaMerge(Context & context, Int64 table_id, const FilterConditions * filter_conditions) diff --git a/dbms/src/Debug/MockStorage.h b/dbms/src/Debug/MockStorage.h index e36858472d5..196a4cbad1c 100644 --- a/dbms/src/Debug/MockStorage.h +++ b/dbms/src/Debug/MockStorage.h @@ -72,9 +72,9 @@ class MockStorage bool tableExists(Int64 table_id); /// for storage delta merge table scan - void addTableSchemaForDeltaMerge(const String & name, const MockColumnInfoVec & columnInfos); + Int64 addTableSchemaForDeltaMerge(const String & name, const MockColumnInfoVec & columnInfos); - void addTableDataForDeltaMerge(Context & context, const String & name, ColumnsWithTypeAndName & columns); + Int64 addTableDataForDeltaMerge(Context & context, const String & name, ColumnsWithTypeAndName & columns); MockColumnInfoVec getTableSchemaForDeltaMerge(const String & name); diff --git a/dbms/src/Flash/tests/gtest_mock_storage.cpp b/dbms/src/Flash/tests/gtest_mock_storage.cpp index 1d38b2aefd1..6c5886ff430 100644 --- a/dbms/src/Flash/tests/gtest_mock_storage.cpp +++ b/dbms/src/Flash/tests/gtest_mock_storage.cpp @@ -40,9 +40,9 @@ TEST_F(MockStorageTestRunner, DeltaMergeStorageBasic) try { ColumnsWithTypeAndName columns{toVec("col0", col0), toNullableVec("col1", col1)}; - mock_storage.addTableSchemaForDeltaMerge("test", {{"col0", TiDB::TP::TypeLongLong}, {"col1", TiDB::TP::TypeString}}); + auto table_id = mock_storage.addTableSchemaForDeltaMerge("test", {{"col0", TiDB::TP::TypeLongLong}, {"col1", TiDB::TP::TypeString}}); mock_storage.addTableDataForDeltaMerge(context.context, "test", columns); - auto in = mock_storage.getStreamFromDeltaMerge(context.context, 1); + auto in = mock_storage.getStreamFromDeltaMerge(context.context, table_id); ASSERT_INPUTSTREAM_BLOCK_UR( in, diff --git a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp index 36361db53bf..f93d3c30143 100644 --- a/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp +++ b/dbms/src/Storages/Transaction/tests/gtest_new_kvstore.cpp @@ -19,6 +19,7 @@ namespace DB namespace tests { TEST_F(RegionKVStoreTest, KVStoreFailRecovery) +try { auto ctx = TiFlashTestEnv::getGlobalContext(); { @@ -153,8 +154,10 @@ TEST_F(RegionKVStoreTest, KVStoreFailRecovery) } } } +CATCH TEST_F(RegionKVStoreTest, KVStoreInvalidWrites) +try { auto ctx = TiFlashTestEnv::getGlobalContext(); { @@ -188,8 +191,10 @@ TEST_F(RegionKVStoreTest, KVStoreInvalidWrites) } } } +CATCH TEST_F(RegionKVStoreTest, KVStoreAdminCommands) +try { auto ctx = TiFlashTestEnv::getGlobalContext(); { @@ -244,8 +249,10 @@ TEST_F(RegionKVStoreTest, KVStoreAdminCommands) ASSERT_EQ(kvs.handleAdminRaftCmd(raft_cmdpb::AdminRequest{request}, std::move(response), 1999, 22, 6, ctx.getTMTContext()), EngineStoreApplyRes::NotFound); } } +CATCH TEST_F(RegionKVStoreTest, KVStoreSnapshot) +try { auto ctx = TiFlashTestEnv::getGlobalContext(); { @@ -403,6 +410,7 @@ TEST_F(RegionKVStoreTest, KVStoreSnapshot) } } } +CATCH } // namespace tests } // namespace DB \ No newline at end of file diff --git a/dbms/src/Storages/Transaction/tests/kvstore_helper.h b/dbms/src/Storages/Transaction/tests/kvstore_helper.h index cb179cb11c3..1d040feae65 100644 --- a/dbms/src/Storages/Transaction/tests/kvstore_helper.h +++ b/dbms/src/Storages/Transaction/tests/kvstore_helper.h @@ -113,6 +113,9 @@ class RegionKVStoreTest : public ::testing::Test } void initStorages() { + bool v = false; + if (!has_init.compare_exchange_strong(v, true)) + return; try { registerStorages(); @@ -123,9 +126,9 @@ class RegionKVStoreTest : public ::testing::Test } String path = TiFlashTestEnv::getContext().getPath(); auto p = path + "/metadata/"; - TiFlashTestEnv::tryRemovePath(p, /*recreate=*/true); + TiFlashTestEnv::tryCreatePath(p); p = path + "/data/"; - TiFlashTestEnv::tryRemovePath(p, /*recreate=*/true); + TiFlashTestEnv::tryCreatePath(p); } protected: @@ -150,6 +153,7 @@ class RegionKVStoreTest : public ::testing::Test return std::make_unique(main_data_paths, main_data_paths, Strings{}, path_capacity, provider); } + std::atomic_bool has_init{false}; std::string test_path; std::unique_ptr path_pool; diff --git a/dbms/src/TestUtils/FunctionTestUtils.cpp b/dbms/src/TestUtils/FunctionTestUtils.cpp index f242f12b6d1..a679bbd9885 100644 --- a/dbms/src/TestUtils/FunctionTestUtils.cpp +++ b/dbms/src/TestUtils/FunctionTestUtils.cpp @@ -189,7 +189,7 @@ std::multiset columnsToRowSet(const ColumnsWithTypeAndName & cols) r.resize(cols_size, true); } - for (auto const & [col_id, col] : ext::enumerate(cols)) + for (auto && [col_id, col] : ext::enumerate(cols)) { for (size_t i = 0, size = col.column->size(); i < size; ++i) { @@ -300,7 +300,7 @@ ColumnsWithTypeAndName toColumnsWithUniqueName(const ColumnsWithTypeAndName & co ColumnsWithTypeAndName toColumnsReordered(const ColumnsWithTypeAndName & columns, const ColumnNumbers & new_offsets) { ColumnsWithTypeAndName columns_reordered(columns.size()); - for (const auto & [i, offset] : ext::enumerate(new_offsets)) + for (auto && [i, offset] : ext::enumerate(new_offsets)) { columns_reordered[offset] = columns[i]; } diff --git a/dbms/src/TestUtils/TiFlashTestEnv.cpp b/dbms/src/TestUtils/TiFlashTestEnv.cpp index 6c96c5177fa..80000fde2c1 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.cpp +++ b/dbms/src/TestUtils/TiFlashTestEnv.cpp @@ -48,6 +48,20 @@ String TiFlashTestEnv::getTemporaryPath(const std::string_view test_case, bool g return poco_path.toString(); } +void TiFlashTestEnv::tryCreatePath(const std::string & path) +{ + try + { + Poco::File p(path); + if (!p.exists()) + p.createDirectories(); + } + catch (...) + { + tryLogCurrentException("gtest", fmt::format("while removing dir `{}`", path)); + } +} + void TiFlashTestEnv::tryRemovePath(const std::string & path, bool recreate) { try diff --git a/dbms/src/TestUtils/TiFlashTestEnv.h b/dbms/src/TestUtils/TiFlashTestEnv.h index 5cb9662c5db..1de5c3b9467 100644 --- a/dbms/src/TestUtils/TiFlashTestEnv.h +++ b/dbms/src/TestUtils/TiFlashTestEnv.h @@ -34,6 +34,8 @@ class TiFlashTestEnv public: static String getTemporaryPath(const std::string_view test_case = "", bool get_abs = true); + static void tryCreatePath(const std::string & path); + static void tryRemovePath(const std::string & path, bool recreate = false); static std::pair getPathPool(const Strings & testdata_path = {}) From 8981dfa16362a67ffd5797dbfe721a9b7876c235 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 9 Feb 2023 00:33:58 +0800 Subject: [PATCH 10/23] Collect request unit for tiflash compute (#6649) close pingcap/tiflash#6678 --- dbms/src/Common/ComputeLabelHolder.cpp | 58 +++ dbms/src/Common/ComputeLabelHolder.h | 46 ++ dbms/src/Common/TiFlashMetrics.h | 445 +++++++++--------- .../CreatingSetsBlockInputStream.cpp | 20 + .../CreatingSetsBlockInputStream.h | 6 +- dbms/src/DataStreams/IBlockInputStream.cpp | 10 +- dbms/src/DataStreams/IBlockInputStream.h | 49 +- .../IProfilingBlockInputStream.cpp | 163 ++++--- .../DataStreams/IProfilingBlockInputStream.h | 4 +- .../ParallelAggregatingBlockInputStream.cpp | 11 + .../ParallelAggregatingBlockInputStream.h | 1 + .../DataStreams/SharedQueryBlockInputStream.h | 14 + dbms/src/DataStreams/UnionBlockInputStream.h | 14 + .../Flash/Coprocessor/DAGBlockOutputStream.h | 8 +- dbms/src/Flash/Coprocessor/DAGDriver.cpp | 18 +- dbms/src/Flash/Coprocessor/DAGDriver.h | 3 +- .../src/Flash/Executor/DataStreamExecutor.cpp | 13 + dbms/src/Flash/Executor/DataStreamExecutor.h | 16 +- dbms/src/Flash/Executor/PipelineExecutor.cpp | 6 + dbms/src/Flash/Executor/PipelineExecutor.h | 2 + dbms/src/Flash/Executor/QueryExecutor.h | 3 + dbms/src/Flash/Executor/tests/gtest_to_ru.cpp | 46 ++ dbms/src/Flash/Executor/toRU.cpp | 45 ++ dbms/src/Flash/Executor/toRU.h | 24 + dbms/src/Flash/Mpp/MPPTask.cpp | 3 + dbms/src/Server/MetricsPrometheus.cpp | 54 ++- dbms/src/Server/MetricsPrometheus.h | 4 +- dbms/src/Server/Server.cpp | 4 + 28 files changed, 748 insertions(+), 342 deletions(-) create mode 100644 dbms/src/Common/ComputeLabelHolder.cpp create mode 100644 dbms/src/Common/ComputeLabelHolder.h create mode 100644 dbms/src/Flash/Executor/tests/gtest_to_ru.cpp create mode 100644 dbms/src/Flash/Executor/toRU.cpp create mode 100644 dbms/src/Flash/Executor/toRU.h diff --git a/dbms/src/Common/ComputeLabelHolder.cpp b/dbms/src/Common/ComputeLabelHolder.cpp new file mode 100644 index 00000000000..cc3d973c966 --- /dev/null +++ b/dbms/src/Common/ComputeLabelHolder.cpp @@ -0,0 +1,58 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed 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 +#include +#include + +namespace DB +{ +namespace +{ +constexpr auto cluster_id_key = "cluster.cluster_id"; + +auto microsecondsUTC() +{ + return std::chrono::duration_cast(std::chrono::system_clock::now().time_since_epoch()).count(); +} +} // namespace + +void ComputeLabelHolder::init(const Poco::Util::LayeredConfiguration & conf) +{ + std::lock_guard lock(mu); + RUNTIME_ASSERT(!label_got, log, "Can't init after label got"); + cluster_id = conf.getString(cluster_id_key, "unknown"); + auto service_addr = conf.getString("flash.service_addr", "unknown"); + std::replace(service_addr.begin(), service_addr.end(), ':', '_'); + std::replace(service_addr.begin(), service_addr.end(), '.', '_'); + process_id = fmt::format("compute_{}_{}", service_addr, microsecondsUTC()); +} + +std::pair ComputeLabelHolder::getClusterIdLabel() +{ + std::lock_guard lock(mu); + label_got = true; + LOG_INFO(log, "get cluster id: {}", cluster_id); + return {"cluster_id", cluster_id}; +} + +std::pair ComputeLabelHolder::getProcessIdLabel() +{ + std::lock_guard lock(mu); + label_got = true; + LOG_INFO(log, "get process id: {}", process_id); + return {"process_id", process_id}; +} + +} // namespace DB diff --git a/dbms/src/Common/ComputeLabelHolder.h b/dbms/src/Common/ComputeLabelHolder.h new file mode 100644 index 00000000000..213de4aba7d --- /dev/null +++ b/dbms/src/Common/ComputeLabelHolder.h @@ -0,0 +1,46 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed 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 +#include +#include + +namespace DB +{ +// Holds the labels for tiflash compute node metrics. +class ComputeLabelHolder : public ext::Singleton +{ +public: + void init(const Poco::Util::LayeredConfiguration & conf); + + std::pair getClusterIdLabel(); + std::pair getProcessIdLabel(); + +private: + std::mutex mu; + bool label_got = false; + // the id of tiflash compute cluster + std::string cluster_id{"unknown"}; + // the id of tiflash compute process, used to distinguish between processes that have been started multiple times. + // Format: `compute_${service_addr}_${start_time_second_utc}` + std::string process_id{"unknown"}; + + LoggerPtr log = Logger::get(); +}; +} // namespace DB diff --git a/dbms/src/Common/TiFlashMetrics.h b/dbms/src/Common/TiFlashMetrics.h index 4273e0c137c..caf2f1cf300 100644 --- a/dbms/src/Common/TiFlashMetrics.h +++ b/dbms/src/Common/TiFlashMetrics.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -42,50 +43,50 @@ namespace DB /// 3. Add metrics of new subsystems at tail. /// 4. Keep it proper formatted using clang-format. // clang-format off -#define APPLY_FOR_METRICS(M, F) \ - M(tiflash_coprocessor_request_count, "Total number of request", Counter, F(type_cop, {"type", "cop"}), \ - F(type_cop_executing, {"type", "cop_executing"}), F(type_batch, {"type", "batch"}), \ - F(type_batch_executing, {"type", "batch_executing"}), F(type_dispatch_mpp_task, {"type", "dispatch_mpp_task"}), \ - F(type_mpp_establish_conn, {"type", "mpp_establish_conn"}), F(type_cancel_mpp_task, {"type", "cancel_mpp_task"}), \ - F(type_run_mpp_task, {"type", "run_mpp_task"}), F(type_remote_read, {"type", "remote_read"}), \ - F(type_remote_read_constructed, {"type", "remote_read_constructed"}), F(type_remote_read_sent, {"type", "remote_read_sent"})) \ - M(tiflash_coprocessor_handling_request_count, "Number of handling request", Gauge, F(type_cop, {"type", "cop"}), \ - F(type_cop_executing, {"type", "cop_executing"}), F(type_batch, {"type", "batch"}), \ - F(type_batch_executing, {"type", "batch_executing"}), F(type_dispatch_mpp_task, {"type", "dispatch_mpp_task"}), \ - F(type_mpp_establish_conn, {"type", "mpp_establish_conn"}), F(type_cancel_mpp_task, {"type", "cancel_mpp_task"}), \ - F(type_run_mpp_task, {"type", "run_mpp_task"}), F(type_remote_read, {"type", "remote_read"}), \ - F(type_remote_read_executing, {"type", "remote_read_executing"})) \ - M(tiflash_coprocessor_executor_count, "Total number of each executor", Counter, F(type_ts, {"type", "table_scan"}), \ - F(type_sel, {"type", "selection"}), F(type_agg, {"type", "aggregation"}), F(type_topn, {"type", "top_n"}), \ - F(type_limit, {"type", "limit"}), F(type_join, {"type", "join"}), F(type_exchange_sender, {"type", "exchange_sender"}), \ - F(type_exchange_receiver, {"type", "exchange_receiver"}), F(type_projection, {"type", "projection"}), \ - F(type_partition_ts, {"type", "partition_table_scan"}), \ - F(type_window, {"type", "window"}), F(type_window_sort, {"type", "window_sort"})) \ - M(tiflash_coprocessor_request_duration_seconds, "Bucketed histogram of request duration", Histogram, \ - F(type_cop, {{"type", "cop"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_batch, {{"type", "batch"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_mpp_establish_conn, {{"type", "mpp_establish_conn"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_coprocessor_request_memory_usage, "Bucketed histogram of request memory usage", Histogram, \ - F(type_cop, {{"type", "cop"}}, ExpBuckets{1024 * 1024, 2, 16}), \ - F(type_batch, {{"type", "batch"}}, ExpBuckets{1024 * 1024, 2, 20}), \ - F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{1024 * 1024, 2, 20})) \ - M(tiflash_coprocessor_request_error, "Total number of request error", Counter, F(reason_meet_lock, {"reason", "meet_lock"}), \ - F(reason_region_not_found, {"reason", "region_not_found"}), F(reason_epoch_not_match, {"reason", "epoch_not_match"}), \ - F(reason_kv_client_error, {"reason", "kv_client_error"}), F(reason_internal_error, {"reason", "internal_error"}), \ - F(reason_other_error, {"reason", "other_error"})) \ - M(tiflash_coprocessor_request_handle_seconds, "Bucketed histogram of request handle duration", Histogram, \ - F(type_cop, {{"type", "cop"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_batch, {{"type", "batch"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_coprocessor_response_bytes, "Total bytes of response body", Counter, \ - F(type_cop, {{"type", "cop"}}), \ - F(type_batch_cop, {{"type", "batch_cop"}}), \ - F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}), \ - F(type_mpp_establish_conn, {{"type", "mpp_tunnel"}}), \ - F(type_mpp_establish_conn_local, {{"type", "mpp_tunnel_local"}}), \ - F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}})) \ +#define APPLY_FOR_METRICS(M, F) \ + M(tiflash_coprocessor_request_count, "Total number of request", Counter, F(type_cop, {"type", "cop"}), \ + F(type_cop_executing, {"type", "cop_executing"}), F(type_batch, {"type", "batch"}), \ + F(type_batch_executing, {"type", "batch_executing"}), F(type_dispatch_mpp_task, {"type", "dispatch_mpp_task"}), \ + F(type_mpp_establish_conn, {"type", "mpp_establish_conn"}), F(type_cancel_mpp_task, {"type", "cancel_mpp_task"}), \ + F(type_run_mpp_task, {"type", "run_mpp_task"}), F(type_remote_read, {"type", "remote_read"}), \ + F(type_remote_read_constructed, {"type", "remote_read_constructed"}), F(type_remote_read_sent, {"type", "remote_read_sent"})) \ + M(tiflash_coprocessor_handling_request_count, "Number of handling request", Gauge, F(type_cop, {"type", "cop"}), \ + F(type_cop_executing, {"type", "cop_executing"}), F(type_batch, {"type", "batch"}), \ + F(type_batch_executing, {"type", "batch_executing"}), F(type_dispatch_mpp_task, {"type", "dispatch_mpp_task"}), \ + F(type_mpp_establish_conn, {"type", "mpp_establish_conn"}), F(type_cancel_mpp_task, {"type", "cancel_mpp_task"}), \ + F(type_run_mpp_task, {"type", "run_mpp_task"}), F(type_remote_read, {"type", "remote_read"}), \ + F(type_remote_read_executing, {"type", "remote_read_executing"})) \ + M(tiflash_coprocessor_executor_count, "Total number of each executor", Counter, F(type_ts, {"type", "table_scan"}), \ + F(type_sel, {"type", "selection"}), F(type_agg, {"type", "aggregation"}), F(type_topn, {"type", "top_n"}), \ + F(type_limit, {"type", "limit"}), F(type_join, {"type", "join"}), F(type_exchange_sender, {"type", "exchange_sender"}), \ + F(type_exchange_receiver, {"type", "exchange_receiver"}), F(type_projection, {"type", "projection"}), \ + F(type_partition_ts, {"type", "partition_table_scan"}), \ + F(type_window, {"type", "window"}), F(type_window_sort, {"type", "window_sort"})) \ + M(tiflash_coprocessor_request_duration_seconds, "Bucketed histogram of request duration", Histogram, \ + F(type_cop, {{"type", "cop"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_batch, {{"type", "batch"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_mpp_establish_conn, {{"type", "mpp_establish_conn"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_coprocessor_request_memory_usage, "Bucketed histogram of request memory usage", Histogram, \ + F(type_cop, {{"type", "cop"}}, ExpBuckets{1024 * 1024, 2, 16}), \ + F(type_batch, {{"type", "batch"}}, ExpBuckets{1024 * 1024, 2, 20}), \ + F(type_run_mpp_task, {{"type", "run_mpp_task"}}, ExpBuckets{1024 * 1024, 2, 20})) \ + M(tiflash_coprocessor_request_error, "Total number of request error", Counter, F(reason_meet_lock, {"reason", "meet_lock"}), \ + F(reason_region_not_found, {"reason", "region_not_found"}), F(reason_epoch_not_match, {"reason", "epoch_not_match"}), \ + F(reason_kv_client_error, {"reason", "kv_client_error"}), F(reason_internal_error, {"reason", "internal_error"}), \ + F(reason_other_error, {"reason", "other_error"})) \ + M(tiflash_coprocessor_request_handle_seconds, "Bucketed histogram of request handle duration", Histogram, \ + F(type_cop, {{"type", "cop"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_batch, {{"type", "batch"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_coprocessor_response_bytes, "Total bytes of response body", Counter, \ + F(type_cop, {{"type", "cop"}}), \ + F(type_batch_cop, {{"type", "batch_cop"}}), \ + F(type_dispatch_mpp_task, {{"type", "dispatch_mpp_task"}}), \ + F(type_mpp_establish_conn, {{"type", "mpp_tunnel"}}), \ + F(type_mpp_establish_conn_local, {{"type", "mpp_tunnel_local"}}), \ + F(type_cancel_mpp_task, {{"type", "cancel_mpp_task"}})) \ M(tiflash_exchange_data_bytes, "Total bytes sent by exchange operators", Counter, \ F(type_hash_original, {"type", "hash_original"}), /*the original data size by hash exchange*/ \ F(type_hash_none_compression_remote, {"type", "hash_none_compression_remote"}), /*the remote exchange data size by hash partition with no compression*/\ @@ -96,182 +97,186 @@ namespace DB F(type_broadcast_passthrough_none_compression_local, {"type", "broadcast_passthrough_none_compression_local"}), /*the local exchange data size by broadcast/passthough with no compression*/ \ F(type_broadcast_passthrough_none_compression_remote, {"type", "broadcast_passthrough_none_compression_remote"}), /*the remote exchange data size by broadcast/passthough with no compression*/ \ ) \ - M(tiflash_schema_version, "Current version of tiflash cached schema", Gauge) \ - M(tiflash_schema_applying, "Whether the schema is applying or not (holding lock)", Gauge) \ - M(tiflash_schema_apply_count, "Total number of each kinds of apply", Counter, F(type_diff, {"type", "diff"}), \ - F(type_full, {"type", "full"}), F(type_failed, {"type", "failed"})) \ - M(tiflash_schema_trigger_count, "Total number of each kinds of schema sync trigger", Counter, /**/ \ - F(type_timer, {"type", "timer"}), F(type_raft_decode, {"type", "raft_decode"}), F(type_cop_read, {"type", "cop_read"})) \ - M(tiflash_schema_internal_ddl_count, "Total number of each kinds of internal ddl operations", Counter, \ - F(type_create_table, {"type", "create_table"}), F(type_create_db, {"type", "create_db"}), \ - F(type_drop_table, {"type", "drop_table"}), F(type_drop_db, {"type", "drop_db"}), F(type_rename_table, {"type", "rename_table"}), \ - F(type_add_column, {"type", "add_column"}), F(type_drop_column, {"type", "drop_column"}), \ - F(type_alter_column_tp, {"type", "alter_column_type"}), F(type_rename_column, {"type", "rename_column"}), \ - F(type_exchange_partition, {"type", "exchange_partition"})) \ - M(tiflash_schema_apply_duration_seconds, "Bucketed histogram of ddl apply duration", Histogram, \ - F(type_ddl_apply_duration, {{"req", "ddl_apply_duration"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ - M(tiflash_stale_read_count, "Total number of stale read", Counter) \ - M(tiflash_raft_read_index_duration_seconds, "Bucketed histogram of raft read index duration", Histogram, \ - F(type_raft_read_index_duration, {{"type", "tmt_raft_read_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_raft_wait_index_duration_seconds, "Bucketed histogram of raft wait index duration", Histogram, \ - F(type_raft_wait_index_duration, {{"type", "tmt_raft_wait_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_syncing_data_freshness, "The freshness of tiflash data with tikv data", Histogram, \ - F(type_syncing_data_freshness, {{"type", "data_freshness"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_storage_read_tasks_count, "Total number of storage engine read tasks", Counter) \ - M(tiflash_storage_command_count, "Total number of storage's command, such as delete range / shutdown /startup", Counter, \ - F(type_delete_range, {"type", "delete_range"}), F(type_ingest, {"type", "ingest"})) \ - M(tiflash_storage_subtask_count, "Total number of storage's sub task", Counter, \ - F(type_delta_merge_bg, {"type", "delta_merge_bg"}), \ - F(type_delta_merge_bg_gc, {"type", "delta_merge_bg_gc"}), \ - F(type_delta_merge_fg, {"type", "delta_merge_fg"}), \ - F(type_delta_merge_manual, {"type", "delta_merge_manual"}), \ - F(type_delta_compact, {"type", "delta_compact"}), \ - F(type_delta_flush, {"type", "delta_flush"}), \ - F(type_seg_split_bg, {"type", "seg_split_bg"}), \ - F(type_seg_split_fg, {"type", "seg_split_fg"}), \ - F(type_seg_split_ingest, {"type", "seg_split_ingest"}), \ - F(type_seg_merge_bg_gc, {"type", "seg_merge_bg_gc"}), \ - F(type_place_index_update, {"type", "place_index_update"})) \ - M(tiflash_storage_subtask_duration_seconds, "Bucketed histogram of storage's sub task duration", Histogram, \ - F(type_delta_merge_bg, {{"type", "delta_merge_bg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delta_merge_bg_gc, {{"type", "delta_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delta_merge_fg, {{"type", "delta_merge_fg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delta_merge_manual, {{"type", "delta_merge_manual"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delta_compact, {{"type", "delta_compact"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delta_flush, {{"type", "delta_flush"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_seg_split_bg, {{"type", "seg_split_bg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_seg_split_fg, {{"type", "seg_split_fg"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_seg_split_ingest, {{"type", "seg_split_ingest"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_seg_merge_bg_gc, {{"type", "seg_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ - F(type_write, {"type", "write"}), /**/ \ - F(type_ingest, {"type", "ingest"}), /**/ \ - F(type_delta_merge, {"type", "delta_merge"}), /**/ \ - F(type_split, {"type", "split"}), /**/ \ - F(type_merge, {"type", "merge"})) /**/ \ - M(tiflash_storage_throughput_rows, "Calculate the throughput of tasks of storage in rows", Gauge, /**/ \ - F(type_write, {"type", "write"}), /**/ \ - F(type_ingest, {"type", "ingest"}), /**/ \ - F(type_delta_merge, {"type", "delta_merge"}), /**/ \ - F(type_split, {"type", "split"}), /**/ \ - F(type_merge, {"type", "merge"})) /**/ \ - M(tiflash_storage_write_stall_duration_seconds, "The write stall duration of storage, in seconds", Histogram, \ - F(type_write, {{"type", "write"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delta_merge_by_write, {{"type", "delta_merge_by_write"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_delta_merge_by_delete_range, {{"type", "delta_merge_by_delete_range"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_flush, {{"type", "flush"}}, ExpBuckets{0.001, 2, 20}), \ - F(type_split, {{"type", "split"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_storage_page_gc_count, "Total number of page's gc execution.", Counter, \ - F(type_v2, {"type", "v2"}), \ - F(type_v2_low, {"type", "v2_low"}), \ - F(type_v3, {"type", "v3"}), \ - F(type_v3_mvcc_dumped, {"type", "v3_mvcc_dumped"}), \ - F(type_v3_bs_full_gc, {"type", "v3_bs_full_gc"})) \ - M(tiflash_storage_page_gc_duration_seconds, "Bucketed histogram of page's gc task duration", Histogram, \ - F(type_v2, {{"type", "v2"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_v2_data_compact, {{"type", "v2_data_compact"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_v2_ver_compact, {{"type", "v2_ver_compact"}}, ExpBuckets{0.0005, 2, 20}), \ - /* Below are metrics for PageStorage V3 */ \ - F(type_compact_wal, {{"type", "compact_wal"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_compact_directory, {{"type", "compact_directory"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_compact_spacemap, {{"type", "compact_spacemap"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_fullgc_rewrite, {{"type", "fullgc_rewrite"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_fullgc_commit, {{"type", "fullgc_commit"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_clean_external, {{"type", "clean_external"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_v3, {{"type", "v3"}}, ExpBuckets{0.0005, 2, 20})) \ - M(tiflash_storage_page_write_batch_size, "The size of each write batch in bytes", Histogram, \ - F(type_v3, {{"type", "v3"}}, ExpBuckets{4 * 1024, 4, 10})) \ - M(tiflash_storage_page_write_duration_seconds, "The duration of each write batch", Histogram, \ - F(type_total, {{"type", "total"}}, ExpBuckets{0.0001, 2, 20}), \ - /* the bucket range for apply in memory is 50us ~ 120s */ \ - F(type_choose_stat, {{"type", "choose_stat"}}, ExpBuckets{0.00005, 1.8, 26}), \ - F(type_search_pos, {{"type", "search_pos"}}, ExpBuckets{0.00005, 1.8, 26}), \ - F(type_blob_write, {{"type", "blob_write"}}, ExpBuckets{0.00005, 1.8, 26}), \ - F(type_latch, {{"type", "latch"}}, ExpBuckets{0.00005, 1.8, 26}), \ - F(type_wal, {{"type", "wal"}}, ExpBuckets{0.00005, 1.8, 26}), \ - F(type_commit, {{"type", "commit"}}, ExpBuckets{0.00005, 1.8, 26})) \ - M(tiflash_storage_logical_throughput_bytes, "The logical throughput of read tasks of storage in bytes", Histogram, \ - F(type_read, {{"type", "read"}}, EqualWidthBuckets{1 * 1024 * 1024, 60, 50 * 1024 * 1024})) \ - M(tiflash_storage_io_limiter, "Storage I/O limiter metrics", Counter, F(type_fg_read_req_bytes, {"type", "fg_read_req_bytes"}), \ - F(type_fg_read_alloc_bytes, {"type", "fg_read_alloc_bytes"}), F(type_bg_read_req_bytes, {"type", "bg_read_req_bytes"}), \ - F(type_bg_read_alloc_bytes, {"type", "bg_read_alloc_bytes"}), F(type_fg_write_req_bytes, {"type", "fg_write_req_bytes"}), \ - F(type_fg_write_alloc_bytes, {"type", "fg_write_alloc_bytes"}), F(type_bg_write_req_bytes, {"type", "bg_write_req_bytes"}), \ - F(type_bg_write_alloc_bytes, {"type", "bg_write_alloc_bytes"})) \ - M(tiflash_storage_rough_set_filter_rate, "Bucketed histogram of rough set filter rate", Histogram, \ - F(type_dtfile_pack, {{"type", "dtfile_pack"}}, EqualWidthBuckets{0, 6, 20})) \ - M(tiflash_raft_command_duration_seconds, "Bucketed histogram of some raft command: apply snapshot", \ - Histogram, /* these command usually cost servel seconds, increase the start bucket to 50ms */ \ - F(type_ingest_sst, {{"type", "ingest_sst"}}, ExpBuckets{0.05, 2, 10}), \ - F(type_apply_snapshot_predecode, {{"type", "snapshot_predecode"}}, ExpBuckets{0.05, 2, 10}), \ - F(type_apply_snapshot_predecode_sst2dt, {{"type", "snapshot_predecode_sst2dt"}}, ExpBuckets{0.05, 2, 10}), \ - F(type_apply_snapshot_flush, {{"type", "snapshot_flush"}}, ExpBuckets{0.05, 2, 10})) \ - M(tiflash_raft_process_keys, "Total number of keys processed in some types of Raft commands", Counter, \ - F(type_apply_snapshot, {"type", "apply_snapshot"}), F(type_ingest_sst, {"type", "ingest_sst"})) \ - M(tiflash_raft_apply_write_command_duration_seconds, "Bucketed histogram of applying write command Raft logs", Histogram, \ - F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_admin, {{"type", "admin"}}, ExpBuckets{0.0005, 2, 20}), \ - F(type_flush_region, {{"type", "flush_region"}}, ExpBuckets{0.0005, 2, 20})) \ - M(tiflash_raft_upstream_latency, "The latency that tikv sends raft log to tiflash.", Histogram, \ - F(type_write, {{"type", "write"}}, ExpBuckets{0.001, 2, 30})) \ - M(tiflash_raft_write_data_to_storage_duration_seconds, "Bucketed histogram of writting region into storage layer", Histogram, \ - F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ - /* required by DBaaS */ \ - M(tiflash_server_info, "Indicate the tiflash server info, and the value is the start timestamp (s).", Gauge, \ - F(start_time, {"version", TiFlashBuildInfo::getReleaseVersion()}, {"hash", TiFlashBuildInfo::getGitHash()})) \ - M(tiflash_object_count, "Number of objects", Gauge, \ - F(type_count_of_establish_calldata, {"type", "count_of_establish_calldata"}), \ - F(type_count_of_mpptunnel, {"type", "count_of_mpptunnel"})) \ - M(tiflash_thread_count, "Number of threads", Gauge, \ - F(type_max_threads_of_thdpool, {"type", "thread_pool_total_max"}), \ - F(type_active_threads_of_thdpool, {"type", "thread_pool_active"}), \ - F(type_max_active_threads_of_thdpool, {"type", "thread_pool_active_max"}), \ - F(type_total_threads_of_thdpool, {"type", "thread_pool_total"}), \ - F(type_max_threads_of_raw, {"type", "total_max"}), \ - F(type_total_threads_of_raw, {"type", "total"}), \ - F(type_threads_of_client_cq_pool, {"type", "rpc_client_cq_pool"}), \ - F(type_threads_of_receiver_read_loop, {"type", "rpc_receiver_read_loop"}), \ - F(type_threads_of_receiver_reactor, {"type", "rpc_receiver_reactor"}), \ - F(type_max_threads_of_establish_mpp, {"type", "rpc_establish_mpp_max"}), \ - F(type_active_threads_of_establish_mpp, {"type", "rpc_establish_mpp"}), \ - F(type_max_threads_of_dispatch_mpp, {"type", "rpc_dispatch_mpp_max"}), \ - F(type_active_threads_of_dispatch_mpp, {"type", "rpc_dispatch_mpp"}), \ - F(type_active_rpc_async_worker, {"type", "rpc_async_worker_active"}), \ - F(type_total_rpc_async_worker, {"type", "rpc_async_worker_total"})) \ - M(tiflash_task_scheduler, "Min-tso task scheduler", Gauge, \ - F(type_min_tso, {"type", "min_tso"}), \ - F(type_waiting_queries_count, {"type", "waiting_queries_count"}), \ - F(type_active_queries_count, {"type", "active_queries_count"}), \ - F(type_waiting_tasks_count, {"type", "waiting_tasks_count"}), \ - F(type_active_tasks_count, {"type", "active_tasks_count"}), \ - F(type_estimated_thread_usage, {"type", "estimated_thread_usage"}), \ - F(type_thread_soft_limit, {"type", "thread_soft_limit"}), \ - F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ - F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) \ - M(tiflash_task_scheduler_waiting_duration_seconds, "Bucketed histogram of task waiting for scheduling duration", Histogram, \ - F(type_task_scheduler_waiting_duration, {{"type", "task_waiting_duration"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_storage_read_thread_counter, "The counter of storage read thread", Counter, \ - F(type_sche_no_pool, {"type", "sche_no_pool"}), \ - F(type_sche_no_slot, {"type", "sche_no_slot"}), \ - F(type_sche_no_segment, {"type", "sche_no_segment"}), \ - F(type_sche_from_cache, {"type", "sche_from_cache"}), \ - F(type_sche_new_task, {"type", "sche_new_task"}), \ - F(type_add_cache_succ, {"type", "add_cache_succ"}), \ - F(type_add_cache_stale, {"type", "add_cache_stale"}), \ - F(type_get_cache_miss, {"type", "get_cache_miss"}), \ - F(type_get_cache_part, {"type", "get_cache_part"}), \ - F(type_get_cache_hit, {"type", "get_cache_hit"}), \ - F(type_get_cache_copy, {"type", "get_cache_copy"})) \ - M(tiflash_storage_read_thread_gauge, "The gauge of storage read thread", Gauge, \ - F(type_merged_task, {"type", "merged_task"})) \ - M(tiflash_storage_read_thread_seconds, "Bucketed histogram of read thread", Histogram, \ - F(type_merged_task, {{"type", "merged_task"}}, ExpBuckets{0.001, 2, 20})) \ - M(tiflash_mpp_task_manager, "The gauge of mpp task manager", Gauge, \ - F(type_mpp_query_count, {"type", "mpp_query_count"})) \ - M(tiflash_exchange_queueing_data_bytes, "Total bytes of data contained in the queue", Gauge, \ - F(type_send, {{"type", "send_queue"}}), \ - F(type_receive, {{"type", "recv_queue"}})) \ + M(tiflash_schema_version, "Current version of tiflash cached schema", Gauge) \ + M(tiflash_schema_applying, "Whether the schema is applying or not (holding lock)", Gauge) \ + M(tiflash_schema_apply_count, "Total number of each kinds of apply", Counter, F(type_diff, {"type", "diff"}), \ + F(type_full, {"type", "full"}), F(type_failed, {"type", "failed"})) \ + M(tiflash_schema_trigger_count, "Total number of each kinds of schema sync trigger", Counter, /**/ \ + F(type_timer, {"type", "timer"}), F(type_raft_decode, {"type", "raft_decode"}), F(type_cop_read, {"type", "cop_read"})) \ + M(tiflash_schema_internal_ddl_count, "Total number of each kinds of internal ddl operations", Counter, \ + F(type_create_table, {"type", "create_table"}), F(type_create_db, {"type", "create_db"}), \ + F(type_drop_table, {"type", "drop_table"}), F(type_drop_db, {"type", "drop_db"}), F(type_rename_table, {"type", "rename_table"}), \ + F(type_add_column, {"type", "add_column"}), F(type_drop_column, {"type", "drop_column"}), \ + F(type_alter_column_tp, {"type", "alter_column_type"}), F(type_rename_column, {"type", "rename_column"}), \ + F(type_exchange_partition, {"type", "exchange_partition"})) \ + M(tiflash_schema_apply_duration_seconds, "Bucketed histogram of ddl apply duration", Histogram, \ + F(type_ddl_apply_duration, {{"req", "ddl_apply_duration"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_raft_read_index_count, "Total number of raft read index", Counter) \ + M(tiflash_stale_read_count, "Total number of stale read", Counter) \ + M(tiflash_raft_read_index_duration_seconds, "Bucketed histogram of raft read index duration", Histogram, \ + F(type_raft_read_index_duration, {{"type", "tmt_raft_read_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_raft_wait_index_duration_seconds, "Bucketed histogram of raft wait index duration", Histogram, \ + F(type_raft_wait_index_duration, {{"type", "tmt_raft_wait_index_duration"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_syncing_data_freshness, "The freshness of tiflash data with tikv data", Histogram, \ + F(type_syncing_data_freshness, {{"type", "data_freshness"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_storage_read_tasks_count, "Total number of storage engine read tasks", Counter) \ + M(tiflash_storage_command_count, "Total number of storage's command, such as delete range / shutdown /startup", Counter, \ + F(type_delete_range, {"type", "delete_range"}), F(type_ingest, {"type", "ingest"})) \ + M(tiflash_storage_subtask_count, "Total number of storage's sub task", Counter, \ + F(type_delta_merge_bg, {"type", "delta_merge_bg"}), \ + F(type_delta_merge_bg_gc, {"type", "delta_merge_bg_gc"}), \ + F(type_delta_merge_fg, {"type", "delta_merge_fg"}), \ + F(type_delta_merge_manual, {"type", "delta_merge_manual"}), \ + F(type_delta_compact, {"type", "delta_compact"}), \ + F(type_delta_flush, {"type", "delta_flush"}), \ + F(type_seg_split_bg, {"type", "seg_split_bg"}), \ + F(type_seg_split_fg, {"type", "seg_split_fg"}), \ + F(type_seg_split_ingest, {"type", "seg_split_ingest"}), \ + F(type_seg_merge_bg_gc, {"type", "seg_merge_bg_gc"}), \ + F(type_place_index_update, {"type", "place_index_update"})) \ + M(tiflash_storage_subtask_duration_seconds, "Bucketed histogram of storage's sub task duration", Histogram, \ + F(type_delta_merge_bg, {{"type", "delta_merge_bg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_bg_gc, {{"type", "delta_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_fg, {{"type", "delta_merge_fg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_manual, {{"type", "delta_merge_manual"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_compact, {{"type", "delta_compact"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_flush, {{"type", "delta_flush"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_seg_split_bg, {{"type", "seg_split_bg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_seg_split_fg, {{"type", "seg_split_fg"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_seg_split_ingest, {{"type", "seg_split_ingest"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_seg_merge_bg_gc, {{"type", "seg_merge_bg_gc"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_place_index_update, {{"type", "place_index_update"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_storage_throughput_bytes, "Calculate the throughput of tasks of storage in bytes", Gauge, /**/ \ + F(type_write, {"type", "write"}), /**/ \ + F(type_ingest, {"type", "ingest"}), /**/ \ + F(type_delta_merge, {"type", "delta_merge"}), /**/ \ + F(type_split, {"type", "split"}), /**/ \ + F(type_merge, {"type", "merge"})) /**/ \ + M(tiflash_storage_throughput_rows, "Calculate the throughput of tasks of storage in rows", Gauge, /**/ \ + F(type_write, {"type", "write"}), /**/ \ + F(type_ingest, {"type", "ingest"}), /**/ \ + F(type_delta_merge, {"type", "delta_merge"}), /**/ \ + F(type_split, {"type", "split"}), /**/ \ + F(type_merge, {"type", "merge"})) /**/ \ + M(tiflash_storage_write_stall_duration_seconds, "The write stall duration of storage, in seconds", Histogram, \ + F(type_write, {{"type", "write"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_by_write, {{"type", "delta_merge_by_write"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_delta_merge_by_delete_range, {{"type", "delta_merge_by_delete_range"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_flush, {{"type", "flush"}}, ExpBuckets{0.001, 2, 20}), \ + F(type_split, {{"type", "split"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_storage_page_gc_count, "Total number of page's gc execution.", Counter, \ + F(type_v2, {"type", "v2"}), \ + F(type_v2_low, {"type", "v2_low"}), \ + F(type_v3, {"type", "v3"}), \ + F(type_v3_mvcc_dumped, {"type", "v3_mvcc_dumped"}), \ + F(type_v3_bs_full_gc, {"type", "v3_bs_full_gc"})) \ + M(tiflash_storage_page_gc_duration_seconds, "Bucketed histogram of page's gc task duration", Histogram, \ + F(type_v2, {{"type", "v2"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_v2_data_compact, {{"type", "v2_data_compact"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_v2_ver_compact, {{"type", "v2_ver_compact"}}, ExpBuckets{0.0005, 2, 20}), \ + /* Below are metrics for PageStorage V3 */ \ + F(type_compact_wal, {{"type", "compact_wal"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_compact_directory, {{"type", "compact_directory"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_compact_spacemap, {{"type", "compact_spacemap"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_fullgc_rewrite, {{"type", "fullgc_rewrite"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_fullgc_commit, {{"type", "fullgc_commit"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_clean_external, {{"type", "clean_external"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_v3, {{"type", "v3"}}, ExpBuckets{0.0005, 2, 20})) \ + M(tiflash_storage_page_write_batch_size, "The size of each write batch in bytes", Histogram, \ + F(type_v3, {{"type", "v3"}}, ExpBuckets{4 * 1024, 4, 10})) \ + M(tiflash_storage_page_write_duration_seconds, "The duration of each write batch", Histogram, \ + F(type_total, {{"type", "total"}}, ExpBuckets{0.0001, 2, 20}), \ + /* the bucket range for apply in memory is 50us ~ 120s */ \ + F(type_choose_stat, {{"type", "choose_stat"}}, ExpBuckets{0.00005, 1.8, 26}), \ + F(type_search_pos, {{"type", "search_pos"}}, ExpBuckets{0.00005, 1.8, 26}), \ + F(type_blob_write, {{"type", "blob_write"}}, ExpBuckets{0.00005, 1.8, 26}), \ + F(type_latch, {{"type", "latch"}}, ExpBuckets{0.00005, 1.8, 26}), \ + F(type_wal, {{"type", "wal"}}, ExpBuckets{0.00005, 1.8, 26}), \ + F(type_commit, {{"type", "commit"}}, ExpBuckets{0.00005, 1.8, 26})) \ + M(tiflash_storage_logical_throughput_bytes, "The logical throughput of read tasks of storage in bytes", Histogram, \ + F(type_read, {{"type", "read"}}, EqualWidthBuckets{1 * 1024 * 1024, 60, 50 * 1024 * 1024})) \ + M(tiflash_storage_io_limiter, "Storage I/O limiter metrics", Counter, F(type_fg_read_req_bytes, {"type", "fg_read_req_bytes"}), \ + F(type_fg_read_alloc_bytes, {"type", "fg_read_alloc_bytes"}), F(type_bg_read_req_bytes, {"type", "bg_read_req_bytes"}), \ + F(type_bg_read_alloc_bytes, {"type", "bg_read_alloc_bytes"}), F(type_fg_write_req_bytes, {"type", "fg_write_req_bytes"}), \ + F(type_fg_write_alloc_bytes, {"type", "fg_write_alloc_bytes"}), F(type_bg_write_req_bytes, {"type", "bg_write_req_bytes"}), \ + F(type_bg_write_alloc_bytes, {"type", "bg_write_alloc_bytes"})) \ + M(tiflash_storage_rough_set_filter_rate, "Bucketed histogram of rough set filter rate", Histogram, \ + F(type_dtfile_pack, {{"type", "dtfile_pack"}}, EqualWidthBuckets{0, 6, 20})) \ + M(tiflash_raft_command_duration_seconds, "Bucketed histogram of some raft command: apply snapshot", \ + Histogram, /* these command usually cost servel seconds, increase the start bucket to 50ms */ \ + F(type_ingest_sst, {{"type", "ingest_sst"}}, ExpBuckets{0.05, 2, 10}), \ + F(type_apply_snapshot_predecode, {{"type", "snapshot_predecode"}}, ExpBuckets{0.05, 2, 10}), \ + F(type_apply_snapshot_predecode_sst2dt, {{"type", "snapshot_predecode_sst2dt"}}, ExpBuckets{0.05, 2, 10}), \ + F(type_apply_snapshot_flush, {{"type", "snapshot_flush"}}, ExpBuckets{0.05, 2, 10})) \ + M(tiflash_raft_process_keys, "Total number of keys processed in some types of Raft commands", Counter, \ + F(type_apply_snapshot, {"type", "apply_snapshot"}), F(type_ingest_sst, {"type", "ingest_sst"})) \ + M(tiflash_raft_apply_write_command_duration_seconds, "Bucketed histogram of applying write command Raft logs", Histogram, \ + F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_admin, {{"type", "admin"}}, ExpBuckets{0.0005, 2, 20}), \ + F(type_flush_region, {{"type", "flush_region"}}, ExpBuckets{0.0005, 2, 20})) \ + M(tiflash_raft_upstream_latency, "The latency that tikv sends raft log to tiflash.", Histogram, \ + F(type_write, {{"type", "write"}}, ExpBuckets{0.001, 2, 30})) \ + M(tiflash_raft_write_data_to_storage_duration_seconds, "Bucketed histogram of writting region into storage layer", Histogram, \ + F(type_decode, {{"type", "decode"}}, ExpBuckets{0.0005, 2, 20}), F(type_write, {{"type", "write"}}, ExpBuckets{0.0005, 2, 20})) \ + /* required by DBaaS */ \ + M(tiflash_server_info, "Indicate the tiflash server info, and the value is the start timestamp (s).", Gauge, \ + F(start_time, {"version", TiFlashBuildInfo::getReleaseVersion()}, {"hash", TiFlashBuildInfo::getGitHash()})) \ + M(tiflash_object_count, "Number of objects", Gauge, \ + F(type_count_of_establish_calldata, {"type", "count_of_establish_calldata"}), \ + F(type_count_of_mpptunnel, {"type", "count_of_mpptunnel"})) \ + M(tiflash_thread_count, "Number of threads", Gauge, \ + F(type_max_threads_of_thdpool, {"type", "thread_pool_total_max"}), \ + F(type_active_threads_of_thdpool, {"type", "thread_pool_active"}), \ + F(type_max_active_threads_of_thdpool, {"type", "thread_pool_active_max"}), \ + F(type_total_threads_of_thdpool, {"type", "thread_pool_total"}), \ + F(type_max_threads_of_raw, {"type", "total_max"}), \ + F(type_total_threads_of_raw, {"type", "total"}), \ + F(type_threads_of_client_cq_pool, {"type", "rpc_client_cq_pool"}), \ + F(type_threads_of_receiver_read_loop, {"type", "rpc_receiver_read_loop"}), \ + F(type_threads_of_receiver_reactor, {"type", "rpc_receiver_reactor"}), \ + F(type_max_threads_of_establish_mpp, {"type", "rpc_establish_mpp_max"}), \ + F(type_active_threads_of_establish_mpp, {"type", "rpc_establish_mpp"}), \ + F(type_max_threads_of_dispatch_mpp, {"type", "rpc_dispatch_mpp_max"}), \ + F(type_active_threads_of_dispatch_mpp, {"type", "rpc_dispatch_mpp"}), \ + F(type_active_rpc_async_worker, {"type", "rpc_async_worker_active"}), \ + F(type_total_rpc_async_worker, {"type", "rpc_async_worker_total"})) \ + M(tiflash_task_scheduler, "Min-tso task scheduler", Gauge, \ + F(type_min_tso, {"type", "min_tso"}), \ + F(type_waiting_queries_count, {"type", "waiting_queries_count"}), \ + F(type_active_queries_count, {"type", "active_queries_count"}), \ + F(type_waiting_tasks_count, {"type", "waiting_tasks_count"}), \ + F(type_active_tasks_count, {"type", "active_tasks_count"}), \ + F(type_estimated_thread_usage, {"type", "estimated_thread_usage"}), \ + F(type_thread_soft_limit, {"type", "thread_soft_limit"}), \ + F(type_thread_hard_limit, {"type", "thread_hard_limit"}), \ + F(type_hard_limit_exceeded_count, {"type", "hard_limit_exceeded_count"})) \ + M(tiflash_task_scheduler_waiting_duration_seconds, "Bucketed histogram of task waiting for scheduling duration", Histogram, \ + F(type_task_scheduler_waiting_duration, {{"type", "task_waiting_duration"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_storage_read_thread_counter, "The counter of storage read thread", Counter, \ + F(type_sche_no_pool, {"type", "sche_no_pool"}), \ + F(type_sche_no_slot, {"type", "sche_no_slot"}), \ + F(type_sche_no_segment, {"type", "sche_no_segment"}), \ + F(type_sche_from_cache, {"type", "sche_from_cache"}), \ + F(type_sche_new_task, {"type", "sche_new_task"}), \ + F(type_add_cache_succ, {"type", "add_cache_succ"}), \ + F(type_add_cache_stale, {"type", "add_cache_stale"}), \ + F(type_get_cache_miss, {"type", "get_cache_miss"}), \ + F(type_get_cache_part, {"type", "get_cache_part"}), \ + F(type_get_cache_hit, {"type", "get_cache_hit"}), \ + F(type_get_cache_copy, {"type", "get_cache_copy"})) \ + M(tiflash_storage_read_thread_gauge, "The gauge of storage read thread", Gauge, \ + F(type_merged_task, {"type", "merged_task"})) \ + M(tiflash_storage_read_thread_seconds, "Bucketed histogram of read thread", Histogram, \ + F(type_merged_task, {{"type", "merged_task"}}, ExpBuckets{0.001, 2, 20})) \ + M(tiflash_mpp_task_manager, "The gauge of mpp task manager", Gauge, \ + F(type_mpp_query_count, {"type", "mpp_query_count"})) \ + M(tiflash_exchange_queueing_data_bytes, "Total bytes of data contained in the queue", Gauge, \ + F(type_send, {{"type", "send_queue"}}), \ + F(type_receive, {{"type", "recv_queue"}})) \ + M(tiflash_compute_request_unit, "Request Unit used by tiflash compute", Counter, \ + F(type_mpp, {{"type", "mpp"}, ComputeLabelHolder::instance().getClusterIdLabel(), ComputeLabelHolder::instance().getProcessIdLabel()}), \ + F(type_cop, {{"type", "cop"}, ComputeLabelHolder::instance().getClusterIdLabel(), ComputeLabelHolder::instance().getProcessIdLabel()}), \ + F(type_batch, {{"type", "batch"}, ComputeLabelHolder::instance().getClusterIdLabel(), ComputeLabelHolder::instance().getProcessIdLabel()})) \ // clang-format on /// Buckets with boundaries [start * base^0, start * base^1, ..., start * base^(size-1)] diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 35409b0825e..6b4618baef7 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -292,4 +292,24 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) } } +uint64_t CreatingSetsBlockInputStream::collectCPUTimeNsImpl(bool is_thread_runner) +{ + // `CreatingSetsBlockInputStream` does not count its own execute time, + // whether `CreatingSetsBlockInputStream` is `thread-runner` or not, + // because `CreatingSetsBlockInputStream` basically does not use cpu, only `condition_cv.wait`. + uint64_t cpu_time_ns = 0; + std::shared_lock lock(children_mutex); + if (!children.empty()) + { + // Each of `CreatingSetsBlockInputStream`'s children is a thread-runner. + size_t i = 0; + for (; i < children.size() - 1; ++i) + cpu_time_ns += children[i]->collectCPUTimeNs(true); + // The last child is running on the same thread as `CreatingSetsBlockInputStream`. + // Since we don't count `CreatingSetsBlockInputStream`'s execute time, we try to collect the last child's cpu time here. + cpu_time_ns += children[i]->collectCPUTimeNs(is_thread_runner); + } + return cpu_time_ns; +} + } // namespace DB diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h index b8e2ee6fe87..fd4235116b2 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.h +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.h @@ -62,11 +62,11 @@ class CreatingSetsBlockInputStream : public IProfilingBlockInputStream virtual void collectNewThreadCount(int & cnt) override { - if (!collected) + if (!thread_cnt_collected) { int cnt_s1 = 0; int cnt_s2 = 0; - collected = true; + thread_cnt_collected = true; collectNewThreadCountOfThisLevel(cnt_s1); for (int i = 0; i < static_cast(children.size()) - 1; ++i) { @@ -83,6 +83,8 @@ class CreatingSetsBlockInputStream : public IProfilingBlockInputStream Block readImpl() override; void readPrefixImpl() override; + uint64_t collectCPUTimeNsImpl(bool is_thread_runner) override; + private: void init(const BlockInputStreamPtr & input); diff --git a/dbms/src/DataStreams/IBlockInputStream.cpp b/dbms/src/DataStreams/IBlockInputStream.cpp index a05fbf83c96..178320bf656 100644 --- a/dbms/src/DataStreams/IBlockInputStream.cpp +++ b/dbms/src/DataStreams/IBlockInputStream.cpp @@ -97,7 +97,7 @@ void IBlockInputStream::dumpTree(FmtBuffer & buffer, size_t indent, size_t multi for (const auto & child : children) ++multipliers[child->getTreeID()]; - for (auto & child : children) + for (const auto & child : children) { String id = child->getTreeID(); size_t & subtree_multiplier = multipliers[id]; @@ -109,4 +109,12 @@ void IBlockInputStream::dumpTree(FmtBuffer & buffer, size_t indent, size_t multi } } +uint64_t IBlockInputStream::collectCPUTimeNs(bool is_thread_runner) +{ + if (cpu_time_ns_collected) + return 0; + + cpu_time_ns_collected = true; + return collectCPUTimeNsImpl(is_thread_runner); +} } // namespace DB diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index a5e05f80f97..451fea38858 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -108,6 +108,23 @@ class IBlockInputStream : private boost::noncopyable return cnt; } + /** Estimate the cpu time nanoseconds used by block input stream dag. + * In this method, streams are divided into two categories: + * - thread-runner: Called directly by a thread + * - non-thread-runner: Called by a thread-runner or non-thread-runner + * Here we should count the execution time of each thread-runner. + * Note: Because of more threads than vcore, and blocking relationships between streams, + * the result may not be 100% identical to the actual cpu time nanoseconds. + */ + uint64_t estimateCPUTimeNs() + { + resetCPUTimeCompute(); + // The first stream of stream dag is thread-runner. + return collectCPUTimeNs(/*is_thread_runner=*/true); + } + + uint64_t collectCPUTimeNs(bool is_thread_runner); + virtual ~IBlockInputStream() = default; /** To output the data stream transformation tree (query execution plan). @@ -150,9 +167,9 @@ class IBlockInputStream : private boost::noncopyable virtual void collectNewThreadCount(int & cnt) { - if (!collected) + if (!thread_cnt_collected) { - collected = true; + thread_cnt_collected = true; collectNewThreadCountOfThisLevel(cnt); for (auto & child : children) { @@ -164,11 +181,16 @@ class IBlockInputStream : private boost::noncopyable virtual void collectNewThreadCountOfThisLevel(int &) {} - virtual void resetNewThreadCountCompute() + virtual void appendInfo(FmtBuffer & /*buffer*/) const {}; + +protected: + virtual uint64_t collectCPUTimeNsImpl(bool /*is_thread_runner*/) { return 0; } + + void resetNewThreadCountCompute() { - if (collected) + if (thread_cnt_collected) { - collected = false; + thread_cnt_collected = false; for (auto & child : children) { if (child) @@ -177,12 +199,25 @@ class IBlockInputStream : private boost::noncopyable } } - virtual void appendInfo(FmtBuffer & /*buffer*/) const {}; + void resetCPUTimeCompute() + { + if (cpu_time_ns_collected) + { + cpu_time_ns_collected = false; + for (auto & child : children) + { + if (child) + child->resetCPUTimeCompute(); + } + } + } protected: BlockInputStreams children; mutable std::shared_mutex children_mutex; - bool collected = false; // a flag to avoid duplicated collecting, since some InputStream is shared by multiple inputStreams + // flags to avoid duplicated collecting, since some InputStream is shared by multiple inputStreams + bool thread_cnt_collected = false; + bool cpu_time_ns_collected = false; private: TableLockHolders table_locks; diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 94a8af1f181..94c2295a720 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -12,9 +12,9 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include -#include #include +#include +#include namespace DB @@ -22,14 +22,14 @@ namespace DB namespace ErrorCodes { - extern const int TOO_MANY_ROWS; - extern const int TOO_MANY_BYTES; - extern const int TOO_MANY_ROWS_OR_BYTES; - extern const int TIMEOUT_EXCEEDED; - extern const int TOO_SLOW; - extern const int LOGICAL_ERROR; - extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; -} +extern const int TOO_MANY_ROWS; +extern const int TOO_MANY_BYTES; +extern const int TOO_MANY_ROWS_OR_BYTES; +extern const int TIMEOUT_EXCEEDED; +extern const int TOO_SLOW; +extern const int LOGICAL_ERROR; +extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; +} // namespace ErrorCodes IProfilingBlockInputStream::IProfilingBlockInputStream() @@ -120,8 +120,7 @@ void IProfilingBlockInputStream::readPrefix() auto start_time = info.total_stopwatch.elapsed(); readPrefixImpl(); - forEachChild([&] (IBlockInputStream & child) - { + forEachChild([&](IBlockInputStream & child) { child.readPrefix(); return false; }); @@ -132,8 +131,7 @@ void IProfilingBlockInputStream::readPrefix() void IProfilingBlockInputStream::readSuffix() { auto start_time = info.total_stopwatch.elapsed(); - forEachChild([&] (IBlockInputStream & child) - { + forEachChild([&](IBlockInputStream & child) { child.readSuffix(); return false; }); @@ -213,24 +211,24 @@ static bool handleOverflowMode(OverflowMode mode, const String & message, int co { switch (mode) { - case OverflowMode::THROW: - throw Exception(message, code); - case OverflowMode::BREAK: - return false; - default: - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + case OverflowMode::THROW: + throw Exception(message, code); + case OverflowMode::BREAK: + return false; + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); } }; -bool IProfilingBlockInputStream::checkTimeLimit() +bool IProfilingBlockInputStream::checkTimeLimit() const { if (limits.max_execution_time != 0 && info.total_stopwatch.elapsed() > static_cast(limits.max_execution_time.totalMicroseconds()) * 1000) return handleOverflowMode(limits.timeout_overflow_mode, - "Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds()) - + " seconds, maximum: " + toString(limits.max_execution_time.totalMicroseconds() / 1000000.0), - ErrorCodes::TIMEOUT_EXCEEDED); + "Timeout exceeded: elapsed " + toString(info.total_stopwatch.elapsedSeconds()) + + " seconds, maximum: " + toString(limits.max_execution_time.totalMicroseconds() / 1000000.0), + ErrorCodes::TIMEOUT_EXCEEDED); return true; } @@ -240,24 +238,24 @@ void IProfilingBlockInputStream::checkQuota(Block & block) { switch (limits.mode) { - case LIMITS_TOTAL: - /// Checked in `progress` method. - break; + case LIMITS_TOTAL: + /// Checked in `progress` method. + break; - case LIMITS_CURRENT: - { - time_t current_time = time(nullptr); - double total_elapsed = info.total_stopwatch.elapsedSeconds(); + case LIMITS_CURRENT: + { + time_t current_time = time(nullptr); + double total_elapsed = info.total_stopwatch.elapsedSeconds(); - quota->checkAndAddResultRowsBytes(current_time, block.rows(), block.bytes()); - quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0)); + quota->checkAndAddResultRowsBytes(current_time, block.rows(), block.bytes()); + quota->checkAndAddExecutionTime(current_time, Poco::Timespan((total_elapsed - prev_elapsed) * 1000000.0)); - prev_elapsed = total_elapsed; - break; - } + prev_elapsed = total_elapsed; + break; + } - default: - throw Exception("Logical error: unknown limits mode.", ErrorCodes::LOGICAL_ERROR); + default: + throw Exception("Logical error: unknown limits mode.", ErrorCodes::LOGICAL_ERROR); } } @@ -287,33 +285,33 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) { switch (limits.size_limits.overflow_mode) { - case OverflowMode::THROW: - { - if (limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows) - throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate) - + " rows read (or to read), maximum: " + toString(limits.size_limits.max_rows), - ErrorCodes::TOO_MANY_ROWS); - else - throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(progress.bytes) - + " bytes read, maximum: " + toString(limits.size_limits.max_bytes), - ErrorCodes::TOO_MANY_BYTES); - break; - } + case OverflowMode::THROW: + { + if (limits.size_limits.max_rows && total_rows_estimate > limits.size_limits.max_rows) + throw Exception("Limit for rows to read exceeded: " + toString(total_rows_estimate) + + " rows read (or to read), maximum: " + toString(limits.size_limits.max_rows), + ErrorCodes::TOO_MANY_ROWS); + else + throw Exception("Limit for (uncompressed) bytes to read exceeded: " + toString(progress.bytes) + + " bytes read, maximum: " + toString(limits.size_limits.max_bytes), + ErrorCodes::TOO_MANY_BYTES); + break; + } - case OverflowMode::BREAK: + case OverflowMode::BREAK: + { + /// For `break`, we will stop only if so many rows were actually read, and not just supposed to be read. + if ((limits.size_limits.max_rows && progress.rows > limits.size_limits.max_rows) + || (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes)) { - /// For `break`, we will stop only if so many rows were actually read, and not just supposed to be read. - if ((limits.size_limits.max_rows && progress.rows > limits.size_limits.max_rows) - || (limits.size_limits.max_bytes && progress.bytes > limits.size_limits.max_bytes)) - { - cancel(false); - } - - break; + cancel(false); } - default: - throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); + break; + } + + default: + throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); } } @@ -327,8 +325,8 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) { if (limits.min_execution_speed && progress.rows / total_elapsed < limits.min_execution_speed) throw Exception("Query is executing too slow: " + toString(progress.rows / total_elapsed) - + " rows/sec., minimum: " + toString(limits.min_execution_speed), - ErrorCodes::TOO_SLOW); + + " rows/sec., minimum: " + toString(limits.min_execution_speed), + ErrorCodes::TOO_SLOW); size_t total_rows = progress.total_rows; @@ -339,9 +337,9 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds()) throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)" - + " is too long. Maximum: " + toString(limits.max_execution_time.totalSeconds()) - + ". Estimated rows to process: " + toString(total_rows), - ErrorCodes::TOO_SLOW); + + " is too long. Maximum: " + toString(limits.max_execution_time.totalSeconds()) + + ". Estimated rows to process: " + toString(total_rows), + ErrorCodes::TOO_SLOW); } } } @@ -363,8 +361,7 @@ void IProfilingBlockInputStream::cancel(bool kill) if (!is_cancelled.compare_exchange_strong(old_val, true, std::memory_order_seq_cst, std::memory_order_relaxed)) return; - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { + forEachProfilingChild([&](IProfilingBlockInputStream & child) { child.cancel(kill); return false; }); @@ -390,8 +387,7 @@ void IProfilingBlockInputStream::setProgressCallback(const ProgressCallback & ca { progress_callback = callback; - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { + forEachProfilingChild([&](IProfilingBlockInputStream & child) { child.setProgressCallback(callback); return false; }); @@ -402,8 +398,7 @@ void IProfilingBlockInputStream::setProcessListElement(ProcessListElement * elem { process_list_elem = elem; - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { + forEachProfilingChild([&](IProfilingBlockInputStream & child) { child.setProcessListElement(elem); return false; }); @@ -416,12 +411,9 @@ Block IProfilingBlockInputStream::getTotals() return totals; Block res; - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { + forEachProfilingChild([&](IProfilingBlockInputStream & child) { res = child.getTotals(); - if (res) - return true; - return false; + return static_cast(res); }); return res; } @@ -432,14 +424,21 @@ Block IProfilingBlockInputStream::getExtremes() return extremes; Block res; - forEachProfilingChild([&] (IProfilingBlockInputStream & child) - { + forEachProfilingChild([&](IProfilingBlockInputStream & child) { res = child.getExtremes(); - if (res) - return true; - return false; + return static_cast(res); }); return res; } +uint64_t IProfilingBlockInputStream::collectCPUTimeNsImpl(bool is_thread_runner) +{ + uint64_t cpu_time_ns = is_thread_runner ? info.execution_time : 0; + forEachChild([&](IBlockInputStream & child) { + cpu_time_ns += child.collectCPUTimeNs(false); + return false; + }); + return cpu_time_ns; } + +} // namespace DB diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h index 2b78ced4df0..ae326c2a08f 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.h @@ -210,6 +210,8 @@ class IProfilingBlockInputStream : public IBlockInputStream children.push_back(child); } + uint64_t collectCPUTimeNsImpl(bool is_thread_runner) override; + private: bool enabled_extremes = false; @@ -242,7 +244,7 @@ class IProfilingBlockInputStream : public IBlockInputStream /** Check limits and quotas. * But only those that can be checked within each separate stream. */ - bool checkTimeLimit(); + bool checkTimeLimit() const; void checkQuota(Block & block); diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index 549524d60a5..86ce5724597 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -235,4 +235,15 @@ void ParallelAggregatingBlockInputStream::appendInfo(FmtBuffer & buffer) const buffer.fmtAppend(", max_threads: {}, final: {}", max_threads, final ? "true" : "false"); } +uint64_t ParallelAggregatingBlockInputStream::collectCPUTimeNsImpl(bool is_thread_runner) +{ + uint64_t cpu_time_ns = impl ? impl->collectCPUTimeNs(is_thread_runner) : 0; + // Each of ParallelAggregatingBlockInputStream's children is a thread-runner. + forEachChild([&](IBlockInputStream & child) { + cpu_time_ns += child.collectCPUTimeNs(true); + return false; + }); + return cpu_time_ns; +} + } // namespace DB diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index c68e36b5972..a23b7f7a516 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -60,6 +60,7 @@ class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream Block readImpl() override; void appendInfo(FmtBuffer & buffer) const override; + uint64_t collectCPUTimeNsImpl(bool is_thread_runner) override; private: const LoggerPtr log; diff --git a/dbms/src/DataStreams/SharedQueryBlockInputStream.h b/dbms/src/DataStreams/SharedQueryBlockInputStream.h index ef89df00f17..265e7d37551 100644 --- a/dbms/src/DataStreams/SharedQueryBlockInputStream.h +++ b/dbms/src/DataStreams/SharedQueryBlockInputStream.h @@ -183,6 +183,20 @@ class SharedQueryBlockInputStream : public IProfilingBlockInputStream throw Exception(exception_msg); } + uint64_t collectCPUTimeNsImpl(bool /*is_thread_runner*/) override + { + // `SharedQueryBlockInputStream` does not count its own execute time, + // whether `SharedQueryBlockInputStream` is `thread-runner` or not, + // because `SharedQueryBlockInputStream` basically does not use cpu, only `condition_cv.wait`. + uint64_t cpu_time_ns = 0; + forEachChild([&](IBlockInputStream & child) { + // Each of SharedQueryBlockInputStream's children is a thread-runner. + cpu_time_ns += child.collectCPUTimeNs(true); + return false; + }); + return cpu_time_ns; + } + private: MPMCQueue queue; diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index e185c68cc42..edcdd761e1f 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -255,6 +255,20 @@ class UnionBlockInputStream final : public IProfilingBlockInputStream children[i]->readSuffix(); } + uint64_t collectCPUTimeNsImpl(bool /*is_thread_runner*/) override + { + // `UnionBlockInputStream` does not count its own execute time, + // whether `UnionBlockInputStream` is `thread-runner` or not, + // because `UnionBlockInputStream` basically does not use cpu, only `condition_cv.wait`. + uint64_t cpu_time_ns = 0; + forEachChild([&](IBlockInputStream & child) { + // Each of `UnionBlockInputStream`'s children is a thread-runner. + cpu_time_ns += child.collectCPUTimeNs(true); + return false; + }); + return cpu_time_ns; + } + private: BlockExtraInfo doGetBlockExtraInfo() const { diff --git a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h index dd65af1a779..ed83480be79 100644 --- a/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h +++ b/dbms/src/Flash/Coprocessor/DAGBlockOutputStream.h @@ -35,10 +35,10 @@ class DAGBlockOutputStream : public IBlockOutputStream public: DAGBlockOutputStream(Block && header_, std::unique_ptr response_writer_); - Block getHeader() const { return header; } - void write(const Block & block); - void writePrefix(); - void writeSuffix(); + Block getHeader() const override { return header; } + void write(const Block & block) override; + void writePrefix() override; + void writeSuffix() override; private: Block header; diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.cpp b/dbms/src/Flash/Coprocessor/DAGDriver.cpp index 2b7ae2d6d1c..eccb3944ed1 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.cpp +++ b/dbms/src/Flash/Coprocessor/DAGDriver.cpp @@ -24,6 +24,7 @@ #include #include #include +#include #include #include #include @@ -56,7 +57,7 @@ DAGDriver::DAGDriver( , dag_response(dag_response_) , writer(nullptr) , internal(internal_) - , log(&Poco::Logger::get("DAGDriver")) + , log(Logger::get("DAGDriver")) { context.setSetting("read_tso", start_ts); if (schema_ver) @@ -76,7 +77,7 @@ DAGDriver::DAGDriver( , dag_response(nullptr) , writer(writer_) , internal(internal_) - , log(&Poco::Logger::get("DAGDriver")) + , log(Logger::get("DAGDriver")) { context.setSetting("read_tso", start_ts); if (schema_ver) @@ -92,6 +93,7 @@ try auto start_time = Clock::now(); DAGContext & dag_context = *context.getDAGContext(); + // TODO use query executor for cop/batch cop. BlockIO streams = executeAsBlockIO(context, internal); if (!streams.in || streams.out) // Only query is allowed, so streams.in must not be null and streams.out must be null @@ -150,6 +152,18 @@ try } } + auto ru = toRU(streams.in->estimateCPUTimeNs()); + if constexpr (!batch) + { + LOG_INFO(log, "cop finish with request unit: {}", ru); + GET_METRIC(tiflash_compute_request_unit, type_cop).Increment(ru); + } + else + { + LOG_INFO(log, "batch cop finish with request unit: {}", ru); + GET_METRIC(tiflash_compute_request_unit, type_batch).Increment(ru); + } + if (auto throughput = dag_context.getTableScanThroughput(); throughput.first) GET_METRIC(tiflash_storage_logical_throughput_bytes).Observe(throughput.second); diff --git a/dbms/src/Flash/Coprocessor/DAGDriver.h b/dbms/src/Flash/Coprocessor/DAGDriver.h index 59a475a9028..03aa1932662 100644 --- a/dbms/src/Flash/Coprocessor/DAGDriver.h +++ b/dbms/src/Flash/Coprocessor/DAGDriver.h @@ -14,6 +14,7 @@ #pragma once +#include #include #include #include @@ -72,6 +73,6 @@ class DAGDriver bool internal; - Poco::Logger * log; + LoggerPtr log; }; } // namespace DB diff --git a/dbms/src/Flash/Executor/DataStreamExecutor.cpp b/dbms/src/Flash/Executor/DataStreamExecutor.cpp index 00069ff6e9d..ddbee01e780 100644 --- a/dbms/src/Flash/Executor/DataStreamExecutor.cpp +++ b/dbms/src/Flash/Executor/DataStreamExecutor.cpp @@ -13,11 +13,19 @@ // limitations under the License. #include +#include #include #include namespace DB { +DataStreamExecutor::DataStreamExecutor(const BlockIO & block_io) + : QueryExecutor(block_io.process_list_entry) + , data_stream(block_io.in) +{ + assert(data_stream); +} + ExecutionResult DataStreamExecutor::execute(ResultHandler result_handler) { try @@ -59,4 +67,9 @@ int DataStreamExecutor::estimateNewThreadCount() { return data_stream->estimateNewThreadCount(); } + +RU DataStreamExecutor::collectRequestUnit() +{ + return toRU(data_stream->estimateCPUTimeNs()); +} } // namespace DB diff --git a/dbms/src/Flash/Executor/DataStreamExecutor.h b/dbms/src/Flash/Executor/DataStreamExecutor.h index fdcc8431d1a..dd7bbbd554c 100644 --- a/dbms/src/Flash/Executor/DataStreamExecutor.h +++ b/dbms/src/Flash/Executor/DataStreamExecutor.h @@ -14,21 +14,19 @@ #pragma once -#include -#include #include namespace DB { +class IBlockInputStream; +using BlockInputStreamPtr = std::shared_ptr; + +struct BlockIO; + class DataStreamExecutor : public QueryExecutor { public: - explicit DataStreamExecutor(const BlockIO & block_io) - : QueryExecutor(block_io.process_list_entry) - , data_stream(block_io.in) - { - assert(data_stream); - } + explicit DataStreamExecutor(const BlockIO & block_io); String toString() const override; @@ -36,6 +34,8 @@ class DataStreamExecutor : public QueryExecutor int estimateNewThreadCount() override; + RU collectRequestUnit() override; + protected: ExecutionResult execute(ResultHandler result_handler) override; diff --git a/dbms/src/Flash/Executor/PipelineExecutor.cpp b/dbms/src/Flash/Executor/PipelineExecutor.cpp index 5fda4fb5605..deaf64bbc37 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.cpp +++ b/dbms/src/Flash/Executor/PipelineExecutor.cpp @@ -80,4 +80,10 @@ int PipelineExecutor::estimateNewThreadCount() { return 0; } + +RU PipelineExecutor::collectRequestUnit() +{ + // TODO support collectRequestUnit + return 0; +} } // namespace DB diff --git a/dbms/src/Flash/Executor/PipelineExecutor.h b/dbms/src/Flash/Executor/PipelineExecutor.h index 5fff4fdd1c3..ad7db76f9df 100644 --- a/dbms/src/Flash/Executor/PipelineExecutor.h +++ b/dbms/src/Flash/Executor/PipelineExecutor.h @@ -59,6 +59,8 @@ class PipelineExecutor : public QueryExecutor int estimateNewThreadCount() override; + RU collectRequestUnit() override; + protected: ExecutionResult execute(ResultHandler result_handler) override; diff --git a/dbms/src/Flash/Executor/QueryExecutor.h b/dbms/src/Flash/Executor/QueryExecutor.h index c1295a04298..74477b76c31 100644 --- a/dbms/src/Flash/Executor/QueryExecutor.h +++ b/dbms/src/Flash/Executor/QueryExecutor.h @@ -16,6 +16,7 @@ #include #include +#include #include #include @@ -43,6 +44,8 @@ class QueryExecutor virtual int estimateNewThreadCount() = 0; + virtual RU collectRequestUnit() = 0; + protected: virtual ExecutionResult execute(ResultHandler) = 0; diff --git a/dbms/src/Flash/Executor/tests/gtest_to_ru.cpp b/dbms/src/Flash/Executor/tests/gtest_to_ru.cpp new file mode 100644 index 00000000000..cadf3af80f0 --- /dev/null +++ b/dbms/src/Flash/Executor/tests/gtest_to_ru.cpp @@ -0,0 +1,46 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed 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 +#include + +namespace DB::tests +{ +class TestToRU : public ::testing::Test +{ +}; + +TEST_F(TestToRU, base) +{ + ASSERT_EQ(0, toRU(0)); + + auto base_ru = toRU(1); + ASSERT_TRUE(base_ru > 0); + + for (size_t i = 1; i < 10; ++i) + { + auto ru = toRU(i); + ASSERT_TRUE(ru >= base_ru); + base_ru = ru; + } + + constexpr auto ten_ms = 10'000'000; + for (size_t i = 1; i < 20; ++i) + { + auto ru = toRU(i * ten_ms); + ASSERT_TRUE(ru > base_ru); + base_ru = ru; + } +} +} // namespace DB::tests diff --git a/dbms/src/Flash/Executor/toRU.cpp b/dbms/src/Flash/Executor/toRU.cpp new file mode 100644 index 00000000000..672a6643174 --- /dev/null +++ b/dbms/src/Flash/Executor/toRU.cpp @@ -0,0 +1,45 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed 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 +#include + +namespace DB +{ +namespace +{ +// Convert cpu time nanoseconds to cpu time millisecond, and round up. +UInt64 toCPUTimeMillisecond(UInt64 cpu_time_ns) +{ + if (unlikely(cpu_time_ns == 0)) + return 0; + + double cpu_time_millisecond = static_cast(cpu_time_ns) / 1'000'000L; + auto ceil_cpu_time_millisecond = ceil(cpu_time_millisecond); + return ceil_cpu_time_millisecond; +} +} // namespace + +// 1 ru = 3 millisecond cpu time +RU toRU(UInt64 cpu_time_ns) +{ + if (unlikely(cpu_time_ns == 0)) + return 0; + + auto cpu_time_millisecond = toCPUTimeMillisecond(cpu_time_ns); + auto ru = static_cast(cpu_time_millisecond) / 3; + auto ceil_ru = ceil(ru); + return ceil_ru; +} +} // namespace DB diff --git a/dbms/src/Flash/Executor/toRU.h b/dbms/src/Flash/Executor/toRU.h new file mode 100644 index 00000000000..9fb760886a1 --- /dev/null +++ b/dbms/src/Flash/Executor/toRU.h @@ -0,0 +1,24 @@ +// Copyright 2022 PingCAP, Ltd. +// +// Licensed 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 + +namespace DB +{ +using RU = UInt64; +// Convert cpu time nanoseconds to Request Unit. +RU toRU(UInt64 cpu_time_ns); +} // namespace DB diff --git a/dbms/src/Flash/Mpp/MPPTask.cpp b/dbms/src/Flash/Mpp/MPPTask.cpp index 9086d63b2af..7efc855bbe4 100644 --- a/dbms/src/Flash/Mpp/MPPTask.cpp +++ b/dbms/src/Flash/Mpp/MPPTask.cpp @@ -412,6 +412,9 @@ void MPPTask::runImpl() { err_msg = result.err_msg; } + auto ru = query_executor_holder->collectRequestUnit(); + LOG_INFO(log, "mpp finish with request unit: {}", ru); + GET_METRIC(tiflash_compute_request_unit, type_mpp).Increment(ru); const auto & return_statistics = mpp_task_statistics.collectRuntimeStatistics(); LOG_DEBUG( diff --git a/dbms/src/Server/MetricsPrometheus.cpp b/dbms/src/Server/MetricsPrometheus.cpp index 61a49548266..2a1c1bae42d 100644 --- a/dbms/src/Server/MetricsPrometheus.cpp +++ b/dbms/src/Server/MetricsPrometheus.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -29,6 +30,7 @@ #include #include #include +#include #include #include #include @@ -38,6 +40,41 @@ namespace DB { +namespace +{ +std::string getHostName() +{ + char hostname[1024]; + if (::gethostname(hostname, sizeof(hostname))) + { + return {}; + } + return hostname; +} + +std::string getInstanceValue(const Poco::Util::AbstractConfiguration & conf) +{ + if (conf.has("flash.service_addr")) + { + auto service_addr = conf.getString("flash.service_addr"); + if (service_addr.empty()) + return getHostName(); + // "0.0.0.0", "127.x.x.x", "locallhost", "0:0:0:0:0:0:0:0", "0:0:0:0:0:0:0:1", "::", "::1", ":${port}" + static const std::vector blacklist{"0.0.0.0", "127.", "locallhost", "0:0:0:0:0:0:0", ":"}; + for (const auto & prefix : blacklist) + { + if (startsWith(service_addr, prefix)) + return getHostName(); + } + return service_addr; + } + else + { + return getHostName(); + } +} +} // namespace + class MetricHandler : public Poco::Net::HTTPRequestHandler { public: @@ -152,7 +189,7 @@ MetricsPrometheus::MetricsPrometheus( const AsynchronousMetrics & async_metrics_) : timer("Prometheus") , async_metrics(async_metrics_) - , log(&Poco::Logger::get("Prometheus")) + , log(Logger::get("Prometheus")) { auto & tiflash_metrics = TiFlashMetrics::instance(); auto & conf = context.getConfigRef(); @@ -191,19 +228,12 @@ MetricsPrometheus::MetricsPrometheus( auto host = metrics_addr.substr(0, pos); auto port = metrics_addr.substr(pos + 1, metrics_addr.size()); - auto service_addr = conf.getString("flash.service_addr"); - std::string job_name = service_addr; - std::replace(job_name.begin(), job_name.end(), ':', '_'); - std::replace(job_name.begin(), job_name.end(), '.', '_'); - job_name = "tiflash_" + job_name; - - char hostname[1024]; - ::gethostname(hostname, sizeof(hostname)); - - gateway = std::make_shared(host, port, job_name, prometheus::Gateway::GetInstanceLabel(hostname)); + const String & job_name = "tiflash"; + const auto & labels = prometheus::Gateway::GetInstanceLabel(getInstanceValue(conf)); + gateway = std::make_shared(host, port, job_name, labels); gateway->RegisterCollectable(tiflash_metrics.registry); - LOG_INFO(log, "Enable prometheus push mode; interval ={}; addr = {}", metrics_interval, metrics_addr); + LOG_INFO(log, "Enable prometheus push mode; interval = {}; addr = {}", metrics_interval, metrics_addr); } } diff --git a/dbms/src/Server/MetricsPrometheus.h b/dbms/src/Server/MetricsPrometheus.h index f6650173031..4e45c9c699f 100644 --- a/dbms/src/Server/MetricsPrometheus.h +++ b/dbms/src/Server/MetricsPrometheus.h @@ -14,9 +14,9 @@ #pragma once +#include #include #include -#include #include #include @@ -46,7 +46,7 @@ class MetricsPrometheus Timer timer; const AsynchronousMetrics & async_metrics; - Poco::Logger * log; + LoggerPtr log; int metrics_interval; std::shared_ptr gateway; diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 50306f4b453..87f1b55d29f 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -15,6 +15,7 @@ #include #include #include +#include #include #include #include @@ -1076,6 +1077,9 @@ int Server::main(const std::vector & /*args*/) if (config().has("macros")) global_context->setMacros(std::make_unique(config(), "macros")); + /// Initialize the labels of tiflash compute node. + ComputeLabelHolder::instance().init(config()); + /// Init TiFlash metrics. global_context->initializeTiFlashMetrics(); From c6493d432dd740191c2a55f7a9420b38bd7a5f07 Mon Sep 17 00:00:00 2001 From: JaySon Date: Thu, 9 Feb 2023 12:33:59 +0800 Subject: [PATCH 11/23] Remove useless IDAsPathUpgrader (#6782) ref pingcap/tiflash#6233 --- dbms/src/CMakeLists.txt | 1 - dbms/src/Common/CPUAffinityManager.cpp | 4 +- dbms/src/Common/CPUAffinityManager.h | 8 +- dbms/src/Common/Config/ConfigProcessor.h | 1 + dbms/src/Common/FileChecker.h | 2 + dbms/src/Common/LRUCache.h | 2 +- dbms/src/Common/MyTime.cpp | 2 +- dbms/src/Common/getNumberOfCPUCores.cpp | 3 +- dbms/src/Flash/Mpp/MPPHandler.h | 6 +- dbms/src/Interpreters/CMakeLists.txt | 25 - dbms/src/Interpreters/IDAsPathUpgrader.cpp | 929 ------------------ dbms/src/Interpreters/IDAsPathUpgrader.h | 202 ---- dbms/src/Interpreters/sortBlock.h | 2 +- dbms/src/Interpreters/tests/CMakeLists.txt | 55 -- dbms/src/Interpreters/tests/aggregate.cpp | 117 --- dbms/src/Interpreters/tests/create_query.cpp | 115 --- .../tests/gtest_id_as_path_upgrader.cpp | 210 ---- dbms/src/Interpreters/tests/hash_map.cpp | 319 ------ dbms/src/Interpreters/tests/hash_map3.cpp | 105 -- .../Interpreters/tests/hash_map_string.cpp | 505 ---------- .../Interpreters/tests/hash_map_string_2.cpp | 686 ------------- .../Interpreters/tests/hash_map_string_3.cpp | 525 ---------- .../tests/hash_map_string_small.cpp | 206 ---- .../tests/logical_expressions_optimizer.cpp | 310 ------ dbms/src/Interpreters/tests/select_query.cpp | 69 -- .../Interpreters/tests/two_level_hash_map.cpp | 149 --- dbms/src/Interpreters/tests/users.cpp | 299 ------ dbms/src/Server/Server.cpp | 28 +- .../Storages/DeltaMerge/DeltaIndexManager.h | 4 +- dbms/src/Storages/DeltaMerge/File/DMFile.cpp | 21 +- dbms/src/Storages/DeltaMerge/File/DMFile.h | 5 +- .../Storages/DeltaMerge/ReadThread/CPU.cpp | 5 +- dbms/src/Storages/DeltaMerge/ReadThread/CPU.h | 12 +- .../DeltaMerge/ReadThread/MergedTask.h | 10 +- .../ReadThread/SegmentReadTaskScheduler.cpp | 2 +- .../ReadThread/SegmentReadTaskScheduler.h | 2 +- .../DeltaMerge/ReadThread/SegmentReader.cpp | 10 +- .../DeltaMerge/ReadThread/SegmentReader.h | 11 +- dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp | 6 +- dbms/src/Storages/IStorage.h | 2 +- dbms/src/Storages/MarkCache.h | 2 +- dbms/src/Storages/Page/V2/PageFile.cpp | 4 +- dbms/src/Storages/Page/V3/BlobStore.cpp | 4 +- dbms/src/Storages/Page/V3/PageDirectory.cpp | 3 +- .../Page/V3/Universal/UniversalWriteBatch.h | 21 +- dbms/src/Storages/Page/WriteBatch.h | 2 +- libs/libdaemon/src/BaseDaemon.cpp | 7 +- tests/_env.sh | 2 +- .../misc/timestamp_rough_set_filter.test | 3 + tests/delta-merge-test/run.sh | 6 +- tests/docker/_env.sh | 2 +- .../test%2Ddb/test%2Dtbl/stable/.gitignore | 0 .../data/test%2Ddb/test-tbl/data/.gitignore | 0 .../data/test%2Ddb/test-tbl/log/.gitignore | 0 .../data/test%2Ddb/test-tbl/meta/.gitignore | 0 .../data/test/test%2Dtbl/stable/.gitignore | 0 .../data0/data/test/test-tbl/data/.gitignore | 0 .../data0/data/test/test-tbl/log/.gitignore | 0 .../data0/data/test/test-tbl/meta/.gitignore | 0 .../data0/metadata/test%2Ddb.sql | 16 - .../data0/metadata/test%2Ddb/test%2Dtbl.sql | 20 - .../flash-1136-v3.1.0/data0/metadata/test.sql | 16 - .../data0/metadata/test/test%2Dtbl.sql | 20 - .../test%2Ddb/test%2Dtbl/stable/.gitignore | 0 .../data/test%2Ddb/test-tbl/data/.gitignore | 0 .../data/test%2Ddb/test-tbl/log/.gitignore | 0 .../data/test%2Ddb/test-tbl/meta/.gitignore | 0 .../data/test-db/test-tbl/stable/.gitignore | 0 .../data0/data/test/test-tbl/data/.gitignore | 0 .../data0/data/test/test-tbl/log/.gitignore | 0 .../data0/data/test/test-tbl/meta/.gitignore | 0 .../data/test/test-tbl/stable/.gitignore | 0 .../flash-1136/data0/metadata/test%2Ddb.sql | 16 - .../data0/metadata/test%2Ddb/test%2Dtbl.sql | 20 - .../flash-1136/data0/metadata/test.sql | 16 - .../data0/metadata/test/test%2Dtbl.sql | 20 - .../data/test-db/test-tbl/stable/.gitignore | 0 .../data/test/#hello-world/data/.gitignore | 0 .../data/test/#hello-world/log/.gitignore | 0 .../data/test/#hello-world/meta/.gitignore | 0 .../data/test/#hello-world/stable/.gitignore | 0 .../issue-941/data0/metadata/test.sql | 16 - .../data0/metadata/test/%23hello%2Dworld.sql | 20 - .../data/test/#hello-world/stable/.gitignore | 0 .../db/data/test/abc/data/.gitignore | 0 .../db/data/test/abc/log/.gitignore | 0 .../db/data/test/abc/meta/.gitignore | 0 .../db/data/test/abc/stable/.gitignore | 0 .../db/data/test/emp_bak/data/.gitignore | 0 .../db/data/test/emp_bak/log/.gitignore | 0 .../db/data/test/emp_bak/meta/.gitignore | 0 .../db/data/test/emp_bak/stable/.gitignore | 0 .../db/data/test/emp_bak_49/data/.gitignore | 0 .../db/data/test/emp_bak_49/log/.gitignore | 0 .../db/data/test/emp_bak_49/meta/.gitignore | 0 .../db/data/test/emp_bak_49/stable/.gitignore | 0 .../db/data/test/emp_bak_50/data/.gitignore | 0 .../db/data/test/emp_bak_50/log/.gitignore | 0 .../db/data/test/emp_bak_50/meta/.gitignore | 0 .../db/data/test/emp_bak_50/stable/.gitignore | 0 .../db/data/test/emp_bak_51/data/.gitignore | 0 .../db/data/test/emp_bak_51/log/.gitignore | 0 .../db/data/test/emp_bak_51/meta/.gitignore | 0 .../db/data/test/emp_bak_51/stable/.gitignore | 0 .../db/data/test/emp_bak_52/data/.gitignore | 0 .../db/data/test/emp_bak_52/log/.gitignore | 0 .../db/data/test/emp_bak_52/meta/.gitignore | 0 .../db/data/test/emp_bak_52/stable/.gitignore | 0 .../testdata/oncall-1651/db/metadata/test.sql | 16 - .../oncall-1651/db/metadata/test/abc.sql | 20 - .../oncall-1651/db/metadata/test/emp_bak.sql | 23 - .../db/metadata/test/emp_bak_49.sql | 23 - .../db/metadata/test/emp_bak_50.sql | 23 - .../db/metadata/test/emp_bak_51.sql | 23 - .../db/metadata/test/emp_bak_52.sql | 23 - 115 files changed, 96 insertions(+), 5266 deletions(-) delete mode 100644 dbms/src/Interpreters/CMakeLists.txt delete mode 100644 dbms/src/Interpreters/IDAsPathUpgrader.cpp delete mode 100644 dbms/src/Interpreters/IDAsPathUpgrader.h delete mode 100644 dbms/src/Interpreters/tests/CMakeLists.txt delete mode 100644 dbms/src/Interpreters/tests/aggregate.cpp delete mode 100644 dbms/src/Interpreters/tests/create_query.cpp delete mode 100644 dbms/src/Interpreters/tests/gtest_id_as_path_upgrader.cpp delete mode 100644 dbms/src/Interpreters/tests/hash_map.cpp delete mode 100644 dbms/src/Interpreters/tests/hash_map3.cpp delete mode 100644 dbms/src/Interpreters/tests/hash_map_string.cpp delete mode 100644 dbms/src/Interpreters/tests/hash_map_string_2.cpp delete mode 100644 dbms/src/Interpreters/tests/hash_map_string_3.cpp delete mode 100644 dbms/src/Interpreters/tests/hash_map_string_small.cpp delete mode 100644 dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp delete mode 100644 dbms/src/Interpreters/tests/select_query.cpp delete mode 100644 dbms/src/Interpreters/tests/two_level_hash_map.cpp delete mode 100644 dbms/src/Interpreters/tests/users.cpp delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test%2Dtbl/stable/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/data/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/log/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/meta/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test/test%2Dtbl/stable/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/data/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/log/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/meta/.gitignore delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb.sql delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb/test%2Dtbl.sql delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/metadata/test.sql delete mode 100644 tests/testdata/flash-1136-v3.1.0/data0/metadata/test/test%2Dtbl.sql delete mode 100644 tests/testdata/flash-1136-v3.1.0/data1/data/test%2Ddb/test%2Dtbl/stable/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/data/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/log/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/meta/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test-db/test-tbl/stable/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test/test-tbl/data/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test/test-tbl/log/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test/test-tbl/meta/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/data/test/test-tbl/stable/.gitignore delete mode 100644 tests/testdata/flash-1136/data0/metadata/test%2Ddb.sql delete mode 100644 tests/testdata/flash-1136/data0/metadata/test%2Ddb/test%2Dtbl.sql delete mode 100644 tests/testdata/flash-1136/data0/metadata/test.sql delete mode 100644 tests/testdata/flash-1136/data0/metadata/test/test%2Dtbl.sql delete mode 100644 tests/testdata/flash-1136/data1/data/test-db/test-tbl/stable/.gitignore delete mode 100644 tests/testdata/issue-941/data0/data/test/#hello-world/data/.gitignore delete mode 100644 tests/testdata/issue-941/data0/data/test/#hello-world/log/.gitignore delete mode 100644 tests/testdata/issue-941/data0/data/test/#hello-world/meta/.gitignore delete mode 100644 tests/testdata/issue-941/data0/data/test/#hello-world/stable/.gitignore delete mode 100644 tests/testdata/issue-941/data0/metadata/test.sql delete mode 100644 tests/testdata/issue-941/data0/metadata/test/%23hello%2Dworld.sql delete mode 100644 tests/testdata/issue-941/data1/data/test/#hello-world/stable/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/abc/data/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/abc/log/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/abc/meta/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/abc/stable/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak/data/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak/log/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak/meta/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak/stable/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_49/data/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_49/log/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_49/meta/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_49/stable/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_50/data/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_50/log/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_50/meta/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_50/stable/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_51/data/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_51/log/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_51/meta/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_51/stable/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_52/data/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_52/log/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_52/meta/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/data/test/emp_bak_52/stable/.gitignore delete mode 100644 tests/testdata/oncall-1651/db/metadata/test.sql delete mode 100644 tests/testdata/oncall-1651/db/metadata/test/abc.sql delete mode 100644 tests/testdata/oncall-1651/db/metadata/test/emp_bak.sql delete mode 100644 tests/testdata/oncall-1651/db/metadata/test/emp_bak_49.sql delete mode 100644 tests/testdata/oncall-1651/db/metadata/test/emp_bak_50.sql delete mode 100644 tests/testdata/oncall-1651/db/metadata/test/emp_bak_51.sql delete mode 100644 tests/testdata/oncall-1651/db/metadata/test/emp_bak_52.sql diff --git a/dbms/src/CMakeLists.txt b/dbms/src/CMakeLists.txt index 058c15d7cc5..9c8fe816ba0 100644 --- a/dbms/src/CMakeLists.txt +++ b/dbms/src/CMakeLists.txt @@ -23,7 +23,6 @@ add_subdirectory (Storages) add_subdirectory (Parsers) add_subdirectory (IO) add_subdirectory (Functions) -add_subdirectory (Interpreters) add_subdirectory (AggregateFunctions) add_subdirectory (Server) add_subdirectory (Client) diff --git a/dbms/src/Common/CPUAffinityManager.cpp b/dbms/src/Common/CPUAffinityManager.cpp index 44676903321..ca326e23fd7 100644 --- a/dbms/src/Common/CPUAffinityManager.cpp +++ b/dbms/src/Common/CPUAffinityManager.cpp @@ -24,9 +24,9 @@ #include #include +#include #include #include -#include #include #include #include @@ -75,7 +75,7 @@ CPUAffinityManager & CPUAffinityManager::getInstance() CPUAffinityManager::CPUAffinityManager() : query_cpu_percent(0) , cpu_cores(0) - , log(&Poco::Logger::get("CPUAffinityManager")) + , log(Logger::get()) {} #ifdef __linux__ diff --git a/dbms/src/Common/CPUAffinityManager.h b/dbms/src/Common/CPUAffinityManager.h index 239604f6213..8cac799a90c 100644 --- a/dbms/src/Common/CPUAffinityManager.h +++ b/dbms/src/Common/CPUAffinityManager.h @@ -24,7 +24,6 @@ namespace Poco { -class Logger; namespace Util { class LayeredConfiguration; @@ -33,10 +32,13 @@ class LayeredConfiguration; namespace DB { +class Logger; +using LoggerPtr = std::shared_ptr; namespace tests { class CPUAffinityManagerTest_CPUAffinityManager_Test; -} +} // namespace tests + struct CPUAffinityConfig { CPUAffinityConfig() @@ -132,7 +134,7 @@ class CPUAffinityManager MAYBE_UNUSED_MEMBER int cpu_cores; std::vector query_threads; - Poco::Logger * log; + LoggerPtr log; CPUAffinityManager(); // Disable copy and move diff --git a/dbms/src/Common/Config/ConfigProcessor.h b/dbms/src/Common/Config/ConfigProcessor.h index db5e4fca9bb..7618f58e915 100644 --- a/dbms/src/Common/Config/ConfigProcessor.h +++ b/dbms/src/Common/Config/ConfigProcessor.h @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include diff --git a/dbms/src/Common/FileChecker.h b/dbms/src/Common/FileChecker.h index c0f606a3a54..15c0ccccb25 100644 --- a/dbms/src/Common/FileChecker.h +++ b/dbms/src/Common/FileChecker.h @@ -15,8 +15,10 @@ #pragma once #include +#include #include +#include #include diff --git a/dbms/src/Common/LRUCache.h b/dbms/src/Common/LRUCache.h index d2a817d2f51..cea64994c24 100644 --- a/dbms/src/Common/LRUCache.h +++ b/dbms/src/Common/LRUCache.h @@ -315,7 +315,7 @@ class LRUCache { // If queue.insert() throws exception, cells and queue will be in inconsistent. cells.erase(it); - tryLogCurrentException(Logger::get("LRUCache"), "queue.insert throw exception"); + tryLogCurrentException(Logger::get(), "queue.insert throw exception"); throw; } } diff --git a/dbms/src/Common/MyTime.cpp b/dbms/src/Common/MyTime.cpp index be3bcc19c4a..e01d27b6581 100644 --- a/dbms/src/Common/MyTime.cpp +++ b/dbms/src/Common/MyTime.cpp @@ -2647,7 +2647,7 @@ std::optional MyDateTimeParser::parseAsPackedUInt(const StringRef & str_ if (!f(ctx, my_time)) { #ifndef NDEBUG - LOG_TRACE(&Poco::Logger::get("MyDateTimeParser"), + LOG_TRACE(Logger::get(), "parse error, [str={}] [format={}] [parse_pos={}]", ctx.view.toString(), format, diff --git a/dbms/src/Common/getNumberOfCPUCores.cpp b/dbms/src/Common/getNumberOfCPUCores.cpp index fd615d6a471..2d955368c63 100644 --- a/dbms/src/Common/getNumberOfCPUCores.cpp +++ b/dbms/src/Common/getNumberOfCPUCores.cpp @@ -62,9 +62,8 @@ void computeAndSetNumberOfPhysicalCPUCores(UInt16 number_of_logical_cpu_cores_, auto hardware_logical_cpu_cores = std::thread::hardware_concurrency(); UInt16 physical_cpu_cores = number_of_logical_cpu_cores_ / (hardware_logical_cpu_cores / number_of_hardware_physical_cores); CPUCores::number_of_physical_cpu_cores = physical_cpu_cores > 0 ? physical_cpu_cores : 1; - auto log = DB::Logger::get("CPUCores"); LOG_INFO( - log, + DB::Logger::get(), "logical cpu cores: {}, hardware logical cpu cores: {}, hardware physical cpu cores: {}, physical cpu cores: {}, number_of_physical_cpu_cores: {}", number_of_logical_cpu_cores_, hardware_logical_cpu_cores, diff --git a/dbms/src/Flash/Mpp/MPPHandler.h b/dbms/src/Flash/Mpp/MPPHandler.h index 8041d02d444..79e548546f6 100644 --- a/dbms/src/Flash/Mpp/MPPHandler.h +++ b/dbms/src/Flash/Mpp/MPPHandler.h @@ -24,12 +24,12 @@ class MPPHandler { const mpp::DispatchTaskRequest & task_request; - Poco::Logger * log; + LoggerPtr log; public: - MPPHandler(const mpp::DispatchTaskRequest & task_request_) + explicit MPPHandler(const mpp::DispatchTaskRequest & task_request_) : task_request(task_request_) - , log(&Poco::Logger::get("MPPHandler")) + , log(Logger::get()) {} grpc::Status execute(const ContextPtr & context, mpp::DispatchTaskResponse * response); void handleError(const MPPTaskPtr & task, String error); diff --git a/dbms/src/Interpreters/CMakeLists.txt b/dbms/src/Interpreters/CMakeLists.txt deleted file mode 100644 index f6f7fb6c660..00000000000 --- a/dbms/src/Interpreters/CMakeLists.txt +++ /dev/null @@ -1,25 +0,0 @@ -# Copyright 2022 PingCAP, Ltd. -# -# Licensed 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. - - -list(GET Poco_INCLUDE_DIRS 0 Poco_Foundation_INCLUDE_DIR) -list(GET Poco_INCLUDE_DIRS 1 Poco_Util_INCLUDE_DIR) - -if (NOT DOUBLE_CONVERSION_INCLUDE_DIR) - get_target_property(DOUBLE_CONVERSION_INCLUDE_DIR ${DOUBLE_CONVERSION_LIBRARIES} INTERFACE_INCLUDE_DIRECTORIES) -endif () - -if (ENABLE_TESTS) - add_subdirectory (tests EXCLUDE_FROM_ALL) -endif () diff --git a/dbms/src/Interpreters/IDAsPathUpgrader.cpp b/dbms/src/Interpreters/IDAsPathUpgrader.cpp deleted file mode 100644 index e253dfdb66d..00000000000 --- a/dbms/src/Interpreters/IDAsPathUpgrader.cpp +++ /dev/null @@ -1,929 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ -extern const int BAD_ARGUMENTS; -extern const int LOGICAL_ERROR; -extern const int FILE_DOESNT_EXIST; -extern const int SYNTAX_ERROR; -} // namespace ErrorCodes - -static constexpr auto SYSTEM_DATABASE = "system"; - -namespace -{ -std::shared_ptr getDatabaseEngine(const FileProviderPtr & file_provider, const String & filename) -{ - String query; - if (Poco::File(filename).exists()) - { - ReadBufferFromFileProvider in(file_provider, filename, EncryptionPath(filename, ""), 1024); - readStringUntilEOF(query, in); - } - else - { - // only directory exists, "default" database, return "Ordinary" engine by default. - return std::static_pointer_cast(makeASTFunction("Ordinary")); - } - - ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, query.data(), query.data() + query.size(), "in file " + filename, 0); - ASTCreateQuery & ast_create_query = typeid_cast(*ast); - auto * storage = ast_create_query.storage; - if (storage == nullptr || storage->engine == nullptr || storage->engine->name.empty()) - { - throw Exception("Can not get database engine for file: " + filename, ErrorCodes::LOGICAL_ERROR); - } - - return std::static_pointer_cast(storage->engine->clone()); -} - -// Get from `table_metadata_file` -std::pair getTableInfo(const FileProviderPtr & file_provider, const String & table_metadata_file) -{ - String definition; - if (Poco::File(table_metadata_file).exists()) - { - ReadBufferFromFileProvider in(file_provider, table_metadata_file, EncryptionPath(table_metadata_file, ""), 1024); - readStringUntilEOF(definition, in); - } - else - { - throw Exception("Can not open table schema file: " + table_metadata_file, ErrorCodes::LOGICAL_ERROR); - } - - ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, definition.data(), definition.data() + definition.size(), "in file " + table_metadata_file, 0); - ASTCreateQuery & ast_create_query = typeid_cast(*ast); - auto * storage = ast_create_query.storage; - if (storage == nullptr || storage->engine == nullptr || storage->engine->name.empty()) - { - throw Exception("Can not get table engine for file: " + table_metadata_file, ErrorCodes::LOGICAL_ERROR); - } - - TiDB::TableInfo info; - ASTFunction * engine = storage->engine; - const auto * args = typeid_cast(engine->arguments.get()); - if (args == nullptr) - throw Exception("Can not cast table engine arguments", ErrorCodes::BAD_ARGUMENTS); - - const ASTLiteral * table_info_ast = nullptr; - if (engine->name == MutableSupport::delta_tree_storage_name) - { - if (args->children.size() >= 2) - { - table_info_ast = typeid_cast(args->children[1].get()); - } - } - else if (engine->name == MutableSupport::txn_storage_name) - { - if (args->children.size() >= 3) - { - table_info_ast = typeid_cast(args->children[2].get()); - } - } - else - { - throw Exception("Unknown storage engine: " + engine->name, ErrorCodes::LOGICAL_ERROR); - } - - if (table_info_ast && table_info_ast->value.getType() == Field::Types::String) - { - const auto table_info_json = safeGet(table_info_ast->value); - if (!table_info_json.empty()) - { - info.deserialize(table_info_json); - return {ast_create_query.table, info}; - } - } - - throw Exception("Can not get TableInfo for file: " + table_metadata_file, ErrorCodes::BAD_ARGUMENTS); -} - -void renamePath(const String & old_path, const String & new_path, Poco::Logger * log, bool must_success) -{ - if (auto file = Poco::File{old_path}; file.exists()) - { - file.renameTo(new_path); - } - else - { - std::string err_msg = fmt::format(R"(Path "{}" is missing.)", old_path); - if (must_success) - throw Exception(err_msg); - else - LOG_WARNING(log, err_msg); - } -} - -void writeTableDefinitionToFile( - const FileProviderPtr & file_provider, - const String & table_meta_path, - const ASTPtr & query, - bool fsync_metadata) -{ - String table_meta_tmp_path = table_meta_path + ".tmp"; - { - String statement = getTableDefinitionFromCreateQuery(query); - - /// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown. - WriteBufferFromFileProvider out(file_provider, table_meta_tmp_path, EncryptionPath(table_meta_tmp_path, ""), true, nullptr, statement.size(), O_WRONLY | O_CREAT | O_EXCL); - writeString(statement, out); - out.next(); - if (fsync_metadata) - out.sync(); - out.close(); - } - file_provider->renameFile( - table_meta_tmp_path, - EncryptionPath(table_meta_tmp_path, ""), - table_meta_path, - EncryptionPath(table_meta_path, ""), - true); -} - -void writeDatabaseDefinitionToFile( - const FileProviderPtr & file_provider, - const String & database_meta_path, - const ASTPtr & query, - bool fsync_metadata) -{ - String db_meta_tmp_path = database_meta_path + ".tmp"; - { - String statement = getDatabaseDefinitionFromCreateQuery(query); - - /// Exclusive flags guarantees, that table is not created right now in another thread. Otherwise, exception will be thrown. - WriteBufferFromFileProvider out(file_provider, db_meta_tmp_path, EncryptionPath(db_meta_tmp_path, ""), true, nullptr, statement.size(), O_WRONLY | O_CREAT | O_EXCL); - writeString(statement, out); - out.next(); - if (fsync_metadata) - out.sync(); - out.close(); - } - file_provider->renameFile( - db_meta_tmp_path, - EncryptionPath(db_meta_tmp_path, ""), - database_meta_path, - EncryptionPath(database_meta_path, ""), - true); -} - -ASTPtr parseCreateDatabaseAST(const String & statement) -{ - ParserCreateQuery parser; - const char * pos = statement.data(); - std::string error_msg; - auto ast = tryParseQuery(parser, - pos, - pos + statement.size(), - error_msg, - /*hilite=*/false, - String("in ") + __PRETTY_FUNCTION__, - /*allow_multi_statements=*/false, - 0); - if (!ast) - throw Exception(error_msg, ErrorCodes::SYNTAX_ERROR); - return ast; -} - -// By default, only remove directory if it is empy -void tryRemoveDirectory(const String & directory, Poco::Logger * log, bool recursive = false) -{ - if (auto dir = Poco::File(directory); dir.exists() && dir.isDirectory()) - { - try - { - dir.remove(/*recursive=*/recursive); - } - catch (Poco::DirectoryNotEmptyException &) - { - // just ignore and keep that directory if it is not empty - LOG_WARNING(log, "Can not remove directory: {}, it is not empty", directory); - } - } -} - -// This function will tidy up path and compare if them are the same one. -// For example "/tmp/data/a.sql" is equal to "/tmp//data//a.sql" -inline bool isSamePath(const String & lhs, const String & rhs) -{ - return Poco::Path{lhs}.absolute().toString() == Poco::Path{rhs}.absolute().toString(); -} - -} // namespace - - -// ================================================ -// TableDiskInfo -// ================================================ - -String IDAsPathUpgrader::TableDiskInfo::name() const -{ - // Name in table_info may not be updated, use the name in `ATTACH TABLE ...`. - // The name in table_info will be updated in later schema sync. - return old_name; -} -String IDAsPathUpgrader::TableDiskInfo::newName() const -{ - return mapper->mapTableName(*tidb_table_info); -} -const TiDB::TableInfo & IDAsPathUpgrader::TableDiskInfo::getInfo() const -{ - return *tidb_table_info; -} - -// "metadata/${db_name}/${tbl_name}.sql" -String IDAsPathUpgrader::TableDiskInfo::getMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const -{ - return db.getMetaDirectory(root_path) + escapeForFileName(name()) + ".sql"; -} -// "data/${db_name}/${tbl_name}/" -String IDAsPathUpgrader::TableDiskInfo::getDataDirectory( - const String & root_path, - const DatabaseDiskInfo & db, - bool escape_db, - bool escape_tbl) const -{ - String res = db.getDataDirectory(root_path, escape_db); - if (escape_tbl) - res += escapeForFileName(name()); - else - res += name(); - return res + "/"; -} -// "extra_data/${db_name}/${tbl_name}/" -String IDAsPathUpgrader::TableDiskInfo::getExtraDirectory( - const String & root_path, - const DatabaseDiskInfo & db, - bool escape_db, - bool escape_tbl) const -{ - String res = db.getExtraDirectory(root_path, escape_db); - if (escape_tbl) - res += escapeForFileName(name()); - else - res += name(); - return res + "/"; -} - -// "metadata/db_${db_id}/t_${id}.sql" -String IDAsPathUpgrader::TableDiskInfo::getNewMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const -{ - return db.getNewMetaDirectory(root_path) + escapeForFileName(newName()) + ".sql"; -} -// "data/t_${id}/" -String IDAsPathUpgrader::TableDiskInfo::getNewDataDirectory(const String & root_path, const DatabaseDiskInfo & db) const -{ - return db.getNewDataDirectory(root_path) + escapeForFileName(newName()) + "/"; -} -// "extra_data/t_${id}" -String IDAsPathUpgrader::TableDiskInfo::getNewExtraDirectory(const String & root_path, const DatabaseDiskInfo & db) const -{ - return db.getNewExtraDirectory(root_path) + escapeForFileName(newName()) + "/"; -} - -// ================================================ -// DatabaseDiskInfo -// ================================================ - -void IDAsPathUpgrader::DatabaseDiskInfo::setDBInfo(TiDB::DBInfoPtr info_) -{ - tidb_db_info = info_; -} - -const TiDB::DBInfo & IDAsPathUpgrader::DatabaseDiskInfo::getInfo() const -{ - if (!hasValidTiDBInfo()) - throw Exception("Try to get database info of not inited database: " + name); - return *tidb_db_info; -} - -String IDAsPathUpgrader::DatabaseDiskInfo::newName() const -{ - return mapper->mapDatabaseName(getInfo()); -} - -String IDAsPathUpgrader::DatabaseDiskInfo::getTiDBSerializeInfo() const -{ - if (!hasValidTiDBInfo()) - throw Exception("Try to serialize database info of not inited database: " + name); - return tidb_db_info->serialize(); -} - -// "metadata/${db_name}.sql" -String IDAsPathUpgrader::DatabaseDiskInfo::doGetMetaFilePath(const String & root_path, bool tmp) const -{ - String meta_dir = doGetMetaDirectory(root_path, tmp); - return (endsWith(meta_dir, "/") ? meta_dir.substr(0, meta_dir.size() - 1) : meta_dir) + ".sql"; -} -// "metadata/${db_name}/" -String IDAsPathUpgrader::DatabaseDiskInfo::doGetMetaDirectory(const String & root_path, bool tmp) const -{ - return root_path + (endsWith(root_path, "/") ? "" : "/") + "metadata/" + escapeForFileName(name + (tmp ? TMP_SUFFIX : "")) + "/"; -} -// "data/${db_name}/" -String IDAsPathUpgrader::DatabaseDiskInfo::doGetDataDirectory(const String & root_path, bool escape, bool tmp) const -{ - // Old data path don't do escape for path - if (escape) - return root_path + (endsWith(root_path, "/") ? "" : "/") + "data/" + escapeForFileName(name + (tmp ? TMP_SUFFIX : "")) + "/"; - else - { - // Old extra data path (in PathPool) don't escape for path. - return root_path + (endsWith(root_path, "/") ? "" : "/") + "data/" + name + (tmp ? TMP_SUFFIX : "") + "/"; - } -} -// "extra_data/${db_name}/" -String IDAsPathUpgrader::DatabaseDiskInfo::doGetExtraDirectory(const String & extra_root, bool escape, bool tmp) const -{ - if (escape) - return extra_root + (endsWith(extra_root, "/") ? "" : "/") + escapeForFileName(name + (tmp ? TMP_SUFFIX : "")) + "/"; - else - { - // Old extra data path (in PathPool) don't escape for path. - return extra_root + (endsWith(extra_root, "/") ? "" : "/") + name + (tmp ? TMP_SUFFIX : "") + "/"; - } -} - -// "metadata/db_${id}.sql" -String IDAsPathUpgrader::DatabaseDiskInfo::getNewMetaFilePath(const String & root_path) const -{ - String meta_dir = getNewMetaDirectory(root_path); - return (endsWith(meta_dir, "/") ? meta_dir.substr(0, meta_dir.size() - 1) : meta_dir) + ".sql"; -} -// "metadata/db_${id}/" -String IDAsPathUpgrader::DatabaseDiskInfo::getNewMetaDirectory(const String & root_path) const -{ - return root_path + (endsWith(root_path, "/") ? "" : "/") + "/metadata/" + escapeForFileName(newName()) + "/"; -} -// "data/" -String IDAsPathUpgrader::DatabaseDiskInfo::getNewDataDirectory(const String & root_path) -{ - return root_path + "/data/"; -} -// "extra_data/" -String IDAsPathUpgrader::DatabaseDiskInfo::getNewExtraDirectory(const String & extra_root) -{ - return extra_root + "/"; -} - - -void IDAsPathUpgrader::DatabaseDiskInfo::renameToTmpDirectories(const Context & ctx, Poco::Logger * log) -{ - if (moved_to_tmp) - return; - - auto root_path = ctx.getPath(); - // Rename database meta file if exist - renamePath(doGetMetaFilePath(root_path, false), doGetMetaFilePath(root_path, true), log, false); - // Rename database meta dir - renamePath(doGetMetaDirectory(root_path, false), doGetMetaDirectory(root_path, true), log, true); - - // Rename database data dir - renamePath( // - doGetDataDirectory(root_path, /*escape*/ true, /*tmp*/ false), - doGetDataDirectory(root_path, /*escape*/ true, /*tmp*/ true), - log, - true); - - // Rename database data dir for multi-paths - auto root_pool = ctx.getPathPool(); - for (const auto & extra_path : root_pool.listPaths()) - renamePath( // - doGetExtraDirectory(extra_path, /*escape*/ true, /*tmp*/ false), // - doGetExtraDirectory(extra_path, /*escape*/ true, /*tmp*/ true), - log, - false); - - moved_to_tmp = true; -} - - -// ================================================ -// IDAsPathUpgrader -// ================================================ - -IDAsPathUpgrader::IDAsPathUpgrader(Context & global_ctx_, bool is_mock_, std::unordered_set reserved_databases_) - : global_context(global_ctx_) - , root_path{global_context.getPath()} - , is_mock(is_mock_) - , mapper(is_mock ? std::make_shared() // - : std::make_shared()) - , reserved_databases{std::move(reserved_databases_)} - , log{&Poco::Logger::get("IDAsPathUpgrader")} -{} - -bool IDAsPathUpgrader::needUpgrade() -{ - const auto metadata_path = global_context.getPath() + "/metadata"; - - // For old version, we have database directories and its `.sql` file - Poco::DirectoryIterator dir_end; - for (Poco::DirectoryIterator it(metadata_path); it != dir_end; ++it) - { - if (!it->isDirectory()) - continue; - - /// For '.svn', '.gitignore' directory and similar. - if (it.name().at(0) == '.') - continue; - - if (it.name() == SYSTEM_DATABASE) - continue; - - String db_name = unescapeForFileName(it.name()); - databases.emplace(db_name, DatabaseDiskInfo{db_name, mapper}); - } - - bool has_old_db_engine = false; - for (auto && [db_name, db_info] : databases) - { - (void)db_name; - const String database_metadata_file = db_info.getMetaFilePath(root_path); - auto engine = getDatabaseEngine(global_context.getFileProvider(), database_metadata_file); - db_info.engine = engine->name; - if (db_info.engine != "TiFlash") - { - has_old_db_engine = true; - LOG_INFO(log, "Find old style of database engine, doing upgrade [path={}] [engine={}]", database_metadata_file, db_info.engine); - } - } - - return has_old_db_engine; -} - -std::vector IDAsPathUpgrader::fetchInfosFromTiDB() const -{ - // Fetch DBs info from TiDB/TiKV - // Note: Not get table info from TiDB, just rename according to TableID in persisted TableInfo - for (size_t i = 0; i < 60; i++) // retry for 3 mins - { - try - { - auto schema_syncer = global_context.getTMTContext().getSchemaSyncer(); - return schema_syncer->fetchAllDBs(); - } - catch (Poco::Exception & e) - { - const int wait_seconds = 3; - LOG_ERROR( - log, - "Upgrade failed because fetch schema error: {}\nWe will sleep for {} seconds and try again.", - e.displayText(), - wait_seconds); - ::sleep(wait_seconds); - } - } - throw Exception("Upgrade failed because fetch schema error."); -} - -static void dropAbsentDatabase( - Context & context, - const String & db_name, - const IDAsPathUpgrader::DatabaseDiskInfo & db_info, - Poco::Logger * log) -{ - if (db_info.hasValidTiDBInfo()) - throw Exception("Invalid call for dropAbsentDatabase for database " + db_name + " with info: " + db_info.getTiDBSerializeInfo()); - - /// tryRemoveDirectory with recursive=true to clean up - - const auto root_path = context.getPath(); - // Remove old metadata dir - const String old_meta_dir = db_info.getMetaDirectory(root_path); - tryRemoveDirectory(old_meta_dir, log, true); - // Remove old metadata file - const String old_meta_file = db_info.getMetaFilePath(root_path); - if (auto file = Poco::File(old_meta_file); file.exists()) - file.remove(); - else - LOG_WARNING(log, "Can not remove database meta file: {}", old_meta_file); - // Remove old data dir - const String old_data_dir = db_info.getDataDirectory(root_path); - tryRemoveDirectory(old_data_dir, log, true); - // not escaped dir created by old PathPool - const String old_data_dir_not_escaped = db_info.getDataDirectory(root_path, false); - tryRemoveDirectory(old_data_dir_not_escaped, log, true); - - const auto & data_extra_paths = context.getPathPool(); - for (const auto & extra_root_path : data_extra_paths.listPaths()) - { - tryRemoveDirectory(db_info.getExtraDirectory(extra_root_path), log, true); - tryRemoveDirectory(db_info.getExtraDirectory(extra_root_path, false), log, true); - } -} - -void IDAsPathUpgrader::linkDatabaseTableInfos(const std::vector & all_databases) -{ - for (const auto & db : all_databases) - { - if (auto iter = databases.find(db->name); iter != databases.end()) - { - iter->second.setDBInfo(db); - } - } - - // list all table in old style. - for (auto iter = databases.begin(); iter != databases.end(); /*empty*/) - { - const auto & db_name = iter->first; - auto & db_info = iter->second; - if (!db_info.hasValidTiDBInfo()) - { - // If we can't find it in TiDB, maybe it already dropped. - if (reserved_databases.count(db_name) > 0) - { - // For mock test or develop environment, we may reserve some database - // for convenience. Keep them as what they are. Print warnings and - // ignore it in later upgrade. - LOG_WARNING(log, "Database {} is reserved, ignored in upgrade.", db_name); - } - else - { - // If we keep them as "Ordinary", when user actually create database with - // same name, next time TiFlash restart and will try to do "upgrade" on - // those legacy data, and it will mess everything up. - // Drop them. - dropAbsentDatabase(global_context, db_name, db_info, log); - } - iter = databases.erase(iter); - continue; - } - - if (db_info.engine == "TiFlash") - { - ++iter; - continue; - } - - const String db_meta_dir = db_info.getMetaDirectory(root_path); - std::vector file_names = DatabaseLoading::listSQLFilenames(db_meta_dir, log); - for (const auto & table_filename : file_names) - { - String table_meta_file = db_meta_dir + "/" + table_filename; - // Name in table_info may not be updated, use the name in `ATTACH TABLE ...`. - auto [old_name, table_info] = getTableInfo(global_context.getFileProvider(), table_meta_file); - db_info.tables.emplace_back( // - TableDiskInfo{old_name, std::make_shared(table_info), mapper}); - } - ++iter; - } -} - -void IDAsPathUpgrader::fixNotEscapedDirectories() -{ - for (const auto & [db_name, db_info] : databases) - { - const auto db_name_escaped = escapeForFileName(db_name); - - // database's meta file, meta dir (created by old DatabaseOrdinary) is escaped. - // only need to create data path - if (db_name != db_name_escaped) - { - LOG_INFO(log, "database `{}` fixing name escape to `{}`", db_name, db_name_escaped); - // Create directory for escaped database - auto escaped_db_data_dir = db_info.getDataDirectory(root_path, /*escape=*/true); - if (Poco::File dir(escaped_db_data_dir); !dir.exists()) - dir.createDirectory(); - - const auto & data_extra_paths = global_context.getPathPool(); - for (const auto & extra_root_path : data_extra_paths.listPaths()) - { - auto escaped_extra_dir = db_info.getExtraDirectory(extra_root_path, /*escape=*/true); - if (Poco::File dir(escaped_extra_dir); !dir.exists()) - dir.createDirectory(); - } - } - - /// Fix not escaped name for table - for (const auto & table : db_info.tables) - { - const auto table_name_escaped = escapeForFileName(table.name()); - if (db_name_escaped == db_name && table_name_escaped == table.name()) - continue; - - LOG_INFO( - log, - "table `{}`.`{}` fixing name escape to `{}`.`{}`", - db_name, - table.name(), - db_name_escaped, - table_name_escaped); - // Table's metadata don't need to fix. - - // Fix data path. It was create by DatabaseOrdinary and StorageDeltaMerge, - // database name is escaped but table name not. - auto not_escaped_path = table.getDataDirectory(root_path, db_info, /*escape_db*/ true, /*escape_tbl*/ false); - auto escaped_path = table.getDataDirectory(root_path, db_info, /*escape_db*/ true, /*escape_tbl*/ true); - if (auto file = Poco::File{not_escaped_path}; file.exists()) - { - if (auto escaped_dir = Poco::File{escaped_path}; !escaped_dir.exists()) - escaped_dir.createDirectory(); - renamePath(not_escaped_path + "/meta", escaped_path + "/meta", log, true); - renamePath(not_escaped_path + "/data", escaped_path + "/data", log, true); - renamePath(not_escaped_path + "/log", escaped_path + "/log", log, true); - // For the cases that database's name did not need to be escaped but table's name did. - renamePath(not_escaped_path + "/stable", escaped_path + "/stable", log, false); - tryRemoveDirectory(not_escaped_path, log); - } - auto db_tbl_not_escaped_path = not_escaped_path; - if (db_name != db_name_escaped) - { - // For the cases that database's name need to be escaped. - // Stable dir was created by old PathPool, database name and table name were not escaped. - db_tbl_not_escaped_path = table.getDataDirectory(root_path, db_info, false, false); - auto not_escaped_stable = db_tbl_not_escaped_path + "/stable"; - auto escaped_stable = table.getDataDirectory(root_path, db_info, true, true) + "/stable"; - if (auto file = Poco::File{not_escaped_stable}; file.exists()) - renamePath(not_escaped_stable, escaped_stable, log, true); - } - - // Fix extra path. - const auto & data_extra_paths = global_context.getPathPool(); - for (const auto & extra_root_path : data_extra_paths.listPaths()) - { - // It was created by old PathPool, both database name and table name are not escaped. - auto not_escaped_extra_path = table.getExtraDirectory(extra_root_path, db_info, /*escape_db*/ false, /*escape_tbl*/ false); - if (isSamePath(not_escaped_extra_path, db_tbl_not_escaped_path)) - continue; - auto escaped_extra_path = table.getExtraDirectory(extra_root_path, db_info, /*escape_db*/ true, /*escape_tbl*/ true); - renamePath(not_escaped_extra_path, escaped_extra_path, log, false); - } - LOG_INFO( - log, - "table `{}`.`{}` fixing name escape to `{}`.`{}` done.", - db_name, - table.name(), - db_name_escaped, - table_name_escaped); - } - - if (db_name != db_name_escaped) - { - // clean not escaped database dir created by old PathPool - const String not_escaped_data_dir = db_info.getDataDirectory(root_path, /*escape*/ false); - tryRemoveDirectory(not_escaped_data_dir, log, true); - const auto & data_extra_paths = global_context.getPathPool(); - for (const auto & extra_root_path : data_extra_paths.listPaths()) - { - auto not_escaped_extra_data_dir = db_info.getExtraDirectory(extra_root_path, /*escape*/ false); - if (isSamePath(not_escaped_data_dir, not_escaped_extra_data_dir)) - continue; - tryRemoveDirectory(not_escaped_extra_data_dir, log); - } - } - LOG_INFO(log, "database `{}` fixing name escape to `{}` done.", db_name, db_name_escaped); - } -} - -void IDAsPathUpgrader::resolveConflictDirectories() -{ - std::unordered_set conflict_databases; - for (const auto & [db_name, db_info] : databases) - { - // In theory, user can create database naming "t_xx" and there is cyclic renaming between table and database. - // First detect if there is any database may have cyclic rename with table. - for (const auto & table : db_info.tables) - { - const auto new_tbl_name = table.newName(); - if (auto iter = databases.find(new_tbl_name); iter != databases.end()) - { - conflict_databases.insert(iter->first); - LOG_INFO( - log, - "Detect cyclic renaming between table `{}`.`{}`(new name:{}) and database `{}`", - db_name, - table.name(), - new_tbl_name, - iter->first); - } - } - - // In theory, user can create two database naming "db_xx" and there is cyclic renaming. - // We need to break that cyclic. - const auto new_database_name = db_info.newName(); - if (auto iter = databases.find(new_database_name); iter != databases.end()) - { - conflict_databases.insert(iter->first); - LOG_INFO( - log, - "Detect cyclic renaming between database `{}`(new name:{}) and database `{}`", - db_name, - new_database_name, - iter->first); - } - } - LOG_INFO(log, "Detect {} cyclic renaming", conflict_databases.size()); - for (const auto & db_name : conflict_databases) - { - auto iter = databases.find(db_name); - auto & db_info = iter->second; - LOG_INFO(log, "Move {} to tmp directories..", db_name); - db_info.renameToTmpDirectories(global_context, log); - } -} - -void IDAsPathUpgrader::doRename() -{ - for (const auto & [db_name, db_info] : databases) - { - renameDatabase(db_name, db_info); - } -} - -void IDAsPathUpgrader::renameDatabase(const String & db_name, const DatabaseDiskInfo & db_info) -{ - const auto mapped_db_name = db_info.newName(); - - { - // Create directory for target database - auto new_db_meta_dir = db_info.getNewMetaDirectory(root_path); - Poco::File(new_db_meta_dir).createDirectory(); - } - - // Rename all tables of this database - for (const auto & table : db_info.tables) - { - renameTable(db_name, db_info, mapped_db_name, table); - } - - // Then rename database - LOG_INFO(log, "database `{}` to `{}` renaming", db_name, mapped_db_name); - { - // Recreate metadata file for database - const String new_meta_file = db_info.getNewMetaFilePath(root_path); - const String statement = "ATTACH DATABASE `" + mapped_db_name + "` ENGINE=TiFlash('" + db_info.getTiDBSerializeInfo() + "', 1)\n"; - auto ast = parseCreateDatabaseAST(statement); - const auto & settings = global_context.getSettingsRef(); - writeDatabaseDefinitionToFile(global_context.getFileProvider(), new_meta_file, ast, settings.fsync_metadata); - } - - { - // Remove old metadata dir - const String old_meta_dir = db_info.getMetaDirectory(root_path); - tryRemoveDirectory(old_meta_dir, log); - // Remove old metadata file - const String old_meta_file = db_info.getMetaFilePath(root_path); - if (auto file = Poco::File(old_meta_file); file.exists()) - file.remove(); - else - LOG_WARNING(log, "Can not remove database meta file: {}", old_meta_file); - // Remove old data dir - const String old_data_dir = db_info.getDataDirectory(root_path); - tryRemoveDirectory(old_data_dir, log); - const auto & data_extra_paths = global_context.getPathPool(); - for (const auto & extra_root_path : data_extra_paths.listPaths()) - { - tryRemoveDirectory(db_info.getExtraDirectory(extra_root_path), log); - } - } - LOG_INFO(log, "database `{}` to `{}` rename done.", db_name, mapped_db_name); -} - -void IDAsPathUpgrader::renameTable( - const String & db_name, - const DatabaseDiskInfo & db_info, - const String & mapped_db_name, - const TableDiskInfo & table) -{ - const auto mapped_table_name = table.newName(); - LOG_INFO( - log, - "table `{}`.`{}` to `{}`.`{}` renaming", - db_name, - table.name(), - mapped_db_name, - mapped_table_name); - - String old_tbl_data_path; - { - // Former data path use ${path}/data/${database}/${table}/ as data path. - // Rename it to ${path}/data/${mapped_table_name}. - old_tbl_data_path = table.getDataDirectory(root_path, db_info); - renamePath(old_tbl_data_path, table.getNewDataDirectory(root_path, db_info), log, true); - } - - { - // Rename data path for multi disk - auto data_extra_paths = global_context.getPathPool(); - for (const auto & extra_root_path : data_extra_paths.listPaths()) - { - auto old_tbl_extra_data_path = table.getExtraDirectory(extra_root_path, db_info); - if (isSamePath(old_tbl_extra_data_path, old_tbl_data_path)) - continue; - renamePath(old_tbl_extra_data_path, table.getNewExtraDirectory(extra_root_path, db_info), log, false); - } - } - - // Recreate metadata file - { - auto old_tbl_meta_file = table.getMetaFilePath(root_path, db_info); - auto ast = DatabaseLoading::getQueryFromMetadata(global_context, old_tbl_meta_file, /*throw_on_error=*/true); - if (!ast) - throw Exception("There is no metadata file for table " + table.name() + ", expected file: " + old_tbl_meta_file, - ErrorCodes::FILE_DOESNT_EXIST); - - ASTCreateQuery & ast_create_query = typeid_cast(*ast); - ast_create_query.table = mapped_table_name; - ASTStorage * storage_ast = ast_create_query.storage; - TiDB::TableInfo table_info = table.getInfo(); // get a copy - if (table_info.is_partition_table) - { - LOG_INFO( - log, - "partition table `{}`.`{}` to `{}`.`{}` update table info", - db_name, - table.name(), - mapped_db_name, - mapped_table_name); - // Old partition name is "${table_name}_${physical_id}" while new name is "t_${physical_id}" - // If it is a partition table, we need to update TiDB::TableInfo::name - do - { - if (!storage_ast || !storage_ast->engine) - break; - auto * args = typeid_cast(storage_ast->engine->arguments.get()); - if (!args) - break; - - table_info.name = mapper->mapPartitionName(table_info); - std::shared_ptr literal = std::make_shared(Field(table_info.serialize())); - if (args->children.size() == 1) - args->children.emplace_back(literal); - else if (args->children.size() >= 2) - args->children.at(1) = literal; - } while (false); - } - - const String new_tbl_meta_file = table.getNewMetaFilePath(root_path, db_info); - const auto & settings = global_context.getSettingsRef(); - writeTableDefinitionToFile(global_context.getFileProvider(), new_tbl_meta_file, ast, settings.fsync_metadata); - - // Remove old metadata file - if (auto file = Poco::File(old_tbl_meta_file); file.exists()) - file.remove(); - } - - LOG_INFO( - log, - "table `{}`.`{}` to `{}`.`{}` rename done.", - db_name, - table.name(), - mapped_db_name, - mapped_table_name); -} - -void IDAsPathUpgrader::doUpgrade() -{ - auto all_databases = fetchInfosFromTiDB(); - linkDatabaseTableInfos(all_databases); - fixNotEscapedDirectories(); - // Check if destination db / tbl file exists and resolve conflict - resolveConflictDirectories(); - // Rename - doRename(); -} - -} // namespace DB diff --git a/dbms/src/Interpreters/IDAsPathUpgrader.h b/dbms/src/Interpreters/IDAsPathUpgrader.h deleted file mode 100644 index 38dc37536aa..00000000000 --- a/dbms/src/Interpreters/IDAsPathUpgrader.h +++ /dev/null @@ -1,202 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -namespace Poco -{ -class Logger; -} - -namespace TiDB -{ -struct TableInfo; -using TableInfoPtr = std::shared_ptr; - -struct DBInfo; -using DBInfoPtr = std::shared_ptr; -} // namespace TiDB - -namespace DB -{ -class Context; -class PathPool; -struct SchemaNameMapper; - -class IDAsPathUpgrader -{ -public: - struct DatabaseDiskInfo; - - struct TableDiskInfo - { - public: - String name() const; - String newName() const; - - TableDiskInfo(String old_name_, TiDB::TableInfoPtr info_, std::shared_ptr mapper_) - : old_name(old_name_) - , tidb_table_info(std::move(info_)) - , mapper(std::move(mapper_)) - {} - - private: - String old_name; - TiDB::TableInfoPtr tidb_table_info; - std::shared_ptr mapper; - - public: - // "metadata/${db_name}/${tbl_name}.sql" - String getMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const; - // "data/${db_name}/${tbl_name}/" - String getDataDirectory(const String & root_path, const DatabaseDiskInfo & db, bool escape_db = true, bool escape_tbl = true) const; - // "extra_data/${db_name}/${tbl_name}/" - String getExtraDirectory( - const String & root_path, - const DatabaseDiskInfo & db, - bool escape_db = true, - bool escape_tbl = true) const; - - // "metadata/db_${db_id}/t_${id}.sql" - String getNewMetaFilePath(const String & root_path, const DatabaseDiskInfo & db) const; - // "data/t_${id}/" - String getNewDataDirectory(const String & root_path, const DatabaseDiskInfo & db) const; - // "extra_data/t_${id}" - String getNewExtraDirectory(const String & root_path, const DatabaseDiskInfo & db) const; - - const TiDB::TableInfo & getInfo() const; - }; - - struct DatabaseDiskInfo - { - public: - static constexpr auto TMP_SUFFIX = "_flash_upgrade"; - - String engine; - std::vector tables; - - private: - String name; - std::shared_ptr mapper; - bool moved_to_tmp = false; - TiDB::DBInfoPtr tidb_db_info = nullptr; - - const TiDB::DBInfo & getInfo() const; - - public: - DatabaseDiskInfo(String name_, std::shared_ptr mapper_) - : name(std::move(name_)) - , mapper(std::move(mapper_)) - {} - - void setDBInfo(TiDB::DBInfoPtr info_); - - bool hasValidTiDBInfo() const { return tidb_db_info != nullptr; } - - String newName() const; - - String getTiDBSerializeInfo() const; - - // "metadata/${db_name}.sql" - String getMetaFilePath(const String & root_path) const { return doGetMetaFilePath(root_path, moved_to_tmp); } - // "metadata/${db_name}/" - String getMetaDirectory(const String & root_path) const { return doGetMetaDirectory(root_path, moved_to_tmp); } - // "data/${db_name}/" - String getDataDirectory(const String & root_path, bool escape = true) const - { - return doGetDataDirectory(root_path, escape, moved_to_tmp); - } - // "extra_data/${db_name}/". db_name is not escaped. - String getExtraDirectory(const String & extra_root, bool escape = true) const - { - return doGetExtraDirectory(extra_root, escape, moved_to_tmp); - } - - void renameToTmpDirectories(const Context & ctx, Poco::Logger * log); - - // "metadata/db_${id}.sql" - String getNewMetaFilePath(const String & root_path) const; - // "metadata/db_${id}/" - String getNewMetaDirectory(const String & root_path) const; - // "data/" - static String getNewDataDirectory(const String & root_path); - // "extra_data/" - static String getNewExtraDirectory(const String & extra_root); - - private: - // "metadata/${db_name}.sql" - String doGetMetaFilePath(const String & root_path, bool tmp) const; - // "metadata/${db_name}/" - String doGetMetaDirectory(const String & root_path, bool tmp) const; - // "data/${db_name}/" - String doGetDataDirectory(const String & root_path, bool escape, bool tmp) const; - // "extra_data/${db_name}/" - String doGetExtraDirectory(const String & extra_root, bool escape, bool tmp) const; - }; - -public: - /// Upgrader - // If some database can not find in TiDB, they will be dropped - // if theirs name is not in reserved_databases - IDAsPathUpgrader(Context & global_ctx_, bool is_mock_, std::unordered_set reserved_databases_); - - bool needUpgrade(); - - void doUpgrade(); - -private: - std::vector fetchInfosFromTiDB() const; - - void linkDatabaseTableInfos(const std::vector & all_databases); - - // Some path created by old PathPool, its database / table name is not escaped, - // normalized those names first. - void fixNotEscapedDirectories(); - - void resolveConflictDirectories(); - - void doRename(); - - void renameDatabase(const String & db_name, const DatabaseDiskInfo & db_info); - - void renameTable( - const String & db_name, - const DatabaseDiskInfo & db_info, - const String & mapped_db_name, - const TableDiskInfo & table_info); - -private: - Context & global_context; - - const String root_path; - - std::map databases; - - const bool is_mock = false; - - std::shared_ptr mapper; - - const std::unordered_set reserved_databases; - - Poco::Logger * log; -}; - -} // namespace DB diff --git a/dbms/src/Interpreters/sortBlock.h b/dbms/src/Interpreters/sortBlock.h index 80098066c72..a768920dab0 100644 --- a/dbms/src/Interpreters/sortBlock.h +++ b/dbms/src/Interpreters/sortBlock.h @@ -24,7 +24,7 @@ namespace DB void sortBlock(Block & block, const SortDescription & description, size_t limit = 0); -/** Used only in StorageMergeTree to sort the data with INSERT. +/** Used only in StorageDeltaMerge to sort the data with INSERT. * Sorting is stable. This is important for keeping the order of rows in the CollapsingMergeTree engine * - because based on the order of rows it is determined whether to delete or leave groups of rows when collapsing. * Collations are not supported. Partial sorting is not supported. diff --git a/dbms/src/Interpreters/tests/CMakeLists.txt b/dbms/src/Interpreters/tests/CMakeLists.txt deleted file mode 100644 index f310837b3f0..00000000000 --- a/dbms/src/Interpreters/tests/CMakeLists.txt +++ /dev/null @@ -1,55 +0,0 @@ -# Copyright 2022 PingCAP, Ltd. -# -# Licensed 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. - -add_executable (create_query create_query.cpp) -target_link_libraries (create_query dbms) - -add_executable (select_query select_query.cpp) -target_link_libraries (select_query clickhouse_storages_system dbms) - -add_executable (aggregate aggregate.cpp) -target_link_libraries (aggregate dbms) - -add_executable (hash_map hash_map.cpp) -target_include_directories (hash_map BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map dbms) - -add_executable (hash_map3 hash_map3.cpp) -target_link_libraries (hash_map3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) - -add_executable (hash_map_string hash_map_string.cpp) -target_include_directories (hash_map_string BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map_string dbms) - -add_executable (hash_map_string_2 hash_map_string_2.cpp) -target_link_libraries (hash_map_string_2 dbms) - -add_executable (hash_map_string_3 hash_map_string_3.cpp) -target_link_libraries (hash_map_string_3 dbms ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES}) -target_include_directories (hash_map_string_3 BEFORE PRIVATE ${TiFlash_SOURCE_DIR}/contrib/libfarmhash) -target_include_directories (hash_map_string_3 BEFORE PRIVATE ${TiFlash_SOURCE_DIR}/contrib/libmetrohash/src) - -add_executable (hash_map_string_small hash_map_string_small.cpp) -target_include_directories (hash_map_string_small BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (hash_map_string_small dbms) - -add_executable (two_level_hash_map two_level_hash_map.cpp) -target_include_directories (two_level_hash_map BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) -target_link_libraries (two_level_hash_map dbms) - -add_executable (logical_expressions_optimizer logical_expressions_optimizer.cpp) -target_link_libraries (logical_expressions_optimizer dbms) - -add_executable (users users.cpp) -target_link_libraries (users dbms ${Boost_FILESYSTEM_LIBRARY} dbms) diff --git a/dbms/src/Interpreters/tests/aggregate.cpp b/dbms/src/Interpreters/tests/aggregate.cpp deleted file mode 100644 index 27491f4afbb..00000000000 --- a/dbms/src/Interpreters/tests/aggregate.cpp +++ /dev/null @@ -1,117 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -int main(int argc, char ** argv) -{ - using namespace DB; - - try - { - size_t n = argc == 2 ? atoi(argv[1]) : 10; - - Block block; - - { - ColumnWithTypeAndName column; - column.name = "x"; - column.type = std::make_shared(); - auto col = ColumnInt16::create(); - auto & vec_x = col->getData(); - - vec_x.resize(n); - for (size_t i = 0; i < n; ++i) - vec_x[i] = i % 9; - - column.column = std::move(col); - block.insert(column); - } - - const char * strings[] = {"abc", "def", "abcd", "defg", "ac"}; - - { - ColumnWithTypeAndName column; - column.name = "s1"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 5])); - - column.column = std::move(col); - block.insert(column); - } - - { - ColumnWithTypeAndName column; - column.name = "s2"; - column.type = std::make_shared(); - auto col = ColumnString::create(); - - for (size_t i = 0; i < n; ++i) - col->insert(std::string(strings[i % 3])); - - column.column = std::move(col); - block.insert(column); - } - - BlockInputStreamPtr stream = std::make_shared(block); - AggregatedDataVariants aggregated_data_variants; - - AggregateFunctionFactory factory; - - AggregateDescriptions aggregate_descriptions(1); - - DataTypes empty_list_of_types; - aggregate_descriptions[0].function = factory.get("count", empty_list_of_types); - - Aggregator::Params params(stream->getHeader(), {0, 1}, aggregate_descriptions, false); - - Aggregator aggregator(params); - - { - Stopwatch stopwatch; - stopwatch.start(); - - KeyManagerPtr key_manager = std::make_shared(false); - FileProviderPtr file_provider = std::make_shared(key_manager, false); - aggregator.execute(stream, aggregated_data_variants, file_provider); - - stopwatch.stop(); - std::cout << std::fixed << std::setprecision(2) - << "Elapsed " << stopwatch.elapsedSeconds() << " sec." - << ", " << n / stopwatch.elapsedSeconds() << " rows/sec." - << std::endl; - } - } - catch (const Exception & e) - { - std::cerr << e.displayText() << std::endl; - } - - return 0; -} \ No newline at end of file diff --git a/dbms/src/Interpreters/tests/create_query.cpp b/dbms/src/Interpreters/tests/create_query.cpp deleted file mode 100644 index 03f787e1547..00000000000 --- a/dbms/src/Interpreters/tests/create_query.cpp +++ /dev/null @@ -1,115 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include - -#include -#include -#include - -#include - -#include -#include - - -using namespace DB; - -int main(int, char **) -try -{ - std::string input = "CREATE TABLE IF NOT EXISTS hits (\n" - "WatchID UInt64,\n" - "JavaEnable UInt8,\n" - "Title String,\n" - "EventTime DateTime,\n" - "CounterID UInt32,\n" - "ClientIP UInt32,\n" - "RegionID UInt32,\n" - "UniqID UInt64,\n" - "CounterClass UInt8,\n" - "OS UInt8,\n" - "UserAgent UInt8,\n" - "URL String,\n" - "Referer String,\n" - "ResolutionWidth UInt16,\n" - "ResolutionHeight UInt16,\n" - "ResolutionDepth UInt8,\n" - "FlashMajor UInt8,\n" - "FlashMinor UInt8,\n" - "FlashMinor2 String,\n" - "NetMajor UInt8,\n" - "NetMinor UInt8,\n" - "UserAgentMajor UInt16,\n" - "UserAgentMinor FixedString(2),\n" - "CookieEnable UInt8,\n" - "JavascriptEnable UInt8,\n" - "IsMobile UInt8,\n" - "MobilePhone UInt8,\n" - "MobilePhoneModel String,\n" - "Params String,\n" - "IPNetworkID UInt32,\n" - "TraficSourceID Int8,\n" - "SearchEngineID UInt16,\n" - "SearchPhrase String,\n" - "AdvEngineID UInt8,\n" - "IsArtifical UInt8,\n" - "WindowClientWidth UInt16,\n" - "WindowClientHeight UInt16,\n" - "ClientTimeZone Int16,\n" - "ClientEventTime DateTime,\n" - "SilverlightVersion1 UInt8,\n" - "SilverlightVersion2 UInt8,\n" - "SilverlightVersion3 UInt32,\n" - "SilverlightVersion4 UInt16,\n" - "PageCharset String,\n" - "CodeVersion UInt32,\n" - "IsLink UInt8,\n" - "IsDownload UInt8,\n" - "IsNotBounce UInt8,\n" - "FUniqID UInt64,\n" - "OriginalURL String,\n" - "HID UInt32,\n" - "IsOldCounter UInt8,\n" - "IsEvent UInt8,\n" - "IsParameter UInt8,\n" - "DontCountHits UInt8,\n" - "WithHash UInt8\n" - ") ENGINE = Log"; - - ParserCreateQuery parser; - ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "", 0); - - Context context = Context::createGlobal(); - - context.setPath("./"); - auto database = std::make_shared("test", "./metadata/test/", context); - context.addDatabase("test", database); - database->loadTables(context, nullptr, false); - context.setCurrentDatabase("test"); - - InterpreterCreateQuery interpreter(ast, context); - interpreter.execute(); - - return 0; -} -catch (const Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl - << std::endl - << "Stack trace:" << std::endl - << e.getStackTrace().toString(); - return 1; -} diff --git a/dbms/src/Interpreters/tests/gtest_id_as_path_upgrader.cpp b/dbms/src/Interpreters/tests/gtest_id_as_path_upgrader.cpp deleted file mode 100644 index 1ff1d8febb7..00000000000 --- a/dbms/src/Interpreters/tests/gtest_id_as_path_upgrader.cpp +++ /dev/null @@ -1,210 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include -#include -#include -#include -#include -#include - -namespace DB::tests -{ - -class IDAsPathUpgrader_test : public ::testing::Test -{ -public: - void SetUp() override - { - try - { - registerStorages(); - } - catch (DB::Exception &) - { - // Maybe another test has already registed, ignore exception here. - } - } - - void TearDown() override - { - auto ctx = TiFlashTestEnv::getContext(); - auto databases = ctx.getDatabases(); - for (auto database : databases) - { - database.second->shutdown(); - ctx.detachDatabase(database.first); - } - } -}; - -TEST_F(IDAsPathUpgrader_test, ONCALL_1651) -try -{ - CHECK_TESTS_WITH_DATA_ENABLED; - - // prepare a "test" database for upgrader - MockTiDB::instance().newDataBase("test"); // id == 2 - - // Generated by running these SQL on cluster version v3.1.0 - // > create table test.aaa(pk int); - // > rename table test.aaa TO test.abc; - // > CREATE TABLE test.employees(id INT NOT NULL AUTO_INCREMENT PRIMARY KEY, - // fname VARCHAR(25) NOT NULL, - // lname VARCHAR(25) NOT NULL, - // store_id INT NOT NULL, - // department_id INT NOT NULL) - // PARTITION BY RANGE(id)(PARTITION p0 VALUES LESS THAN(5), - // PARTITION p1 VALUES LESS THAN(10), - // PARTITION p2 VALUES LESS THAN(15), - // PARTITION p3 VALUES LESS THAN MAXVALUE); - // > RENAME TABLE test.employees TO test.emp; - // > RENAME TABLE test.emp TO test.emp_bak; - const auto test_path = TiFlashTestEnv::findTestDataPath("oncall-1651"); - auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path); - - IDAsPathUpgrader upgrader(ctx, false, {}); - ASSERT_TRUE(upgrader.needUpgrade()); - upgrader.doUpgrade(); - - { - // After upgrade, next time we don't need it. - IDAsPathUpgrader checker_after_upgrade(ctx, false, {}); - ASSERT_FALSE(checker_after_upgrade.needUpgrade()); - } - - // load metadata should not throw any exception - loadMetadata(ctx); - - ASSERT_TRUE(ctx.isDatabaseExist("db_2")); // "test" - auto & storages = ctx.getTMTContext().getStorages(); - ASSERT_NE(storages.get(45), nullptr); // `test`.`abc` - ASSERT_NE(storages.get(48), nullptr); // `test`.`emp_bak` - ASSERT_NE(storages.get(49), nullptr); - ASSERT_NE(storages.get(50), nullptr); - ASSERT_NE(storages.get(51), nullptr); - ASSERT_NE(storages.get(52), nullptr); -} -CATCH - -TEST_F(IDAsPathUpgrader_test, FLASH_1136_4_0_0_rc) -try -{ - CHECK_TESTS_WITH_DATA_ENABLED; - - // prepare a "test" database for upgrader - MockTiDB::instance().newDataBase("test"); // id == 2 - MockTiDB::instance().newDataBase("test-db"); // id == 3 - - // Generated by running these SQL on cluster version v4.0.0-rc - // > create database `test-db`; - // > create table `test-tbl` (pk int); - // > cerate table `test`.`test-tbl` (pk int); - const auto test_path = TiFlashTestEnv::findTestDataPath("flash-1136"); - auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path); - - IDAsPathUpgrader upgrader(ctx, false, {}); - ASSERT_TRUE(upgrader.needUpgrade()); - upgrader.doUpgrade(); - - { - // After upgrade, next time we don't need it. - IDAsPathUpgrader checker_after_upgrade(ctx, false, {}); - ASSERT_FALSE(checker_after_upgrade.needUpgrade()); - } - - // load metadata should not throw any exception - loadMetadata(ctx); - - ASSERT_TRUE(ctx.isDatabaseExist("db_2")); - ASSERT_TRUE(ctx.isDatabaseExist("db_3")); - auto & storages = ctx.getTMTContext().getStorages(); - ASSERT_NE(storages.get(66), nullptr); // `test-db`.`test-tbl` - ASSERT_NE(storages.get(666), nullptr); // `test`.`test-tbl` -} -CATCH - -TEST_F(IDAsPathUpgrader_test, FLASH_1136_3_1_0) -try -{ - CHECK_TESTS_WITH_DATA_ENABLED; - - // prepare a "test" database for upgrader - MockTiDB::instance().newDataBase("test"); // id == 2 - MockTiDB::instance().newDataBase("test-db"); // id == 3 - - // Generated by running these SQL on cluster version v4.0.0-rc - // > create database `test-db`; - // > create table `test-tbl` (pk int); - // > cerate table `test`.`test-tbl` (pk int); - const auto test_path = TiFlashTestEnv::findTestDataPath("flash-1136-v3.1.0"); - auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path); - - IDAsPathUpgrader upgrader(ctx, false, {}); - ASSERT_TRUE(upgrader.needUpgrade()); - upgrader.doUpgrade(); - - { - // After upgrade, next time we don't need it. - IDAsPathUpgrader checker_after_upgrade(ctx, false, {}); - ASSERT_FALSE(checker_after_upgrade.needUpgrade()); - } - - // load metadata should not throw any exception - loadMetadata(ctx); - - ASSERT_TRUE(ctx.isDatabaseExist("db_2")); - ASSERT_TRUE(ctx.isDatabaseExist("db_3")); - auto & storages = ctx.getTMTContext().getStorages(); - ASSERT_NE(storages.get(66), nullptr); // `test-db`.`test-tbl` - ASSERT_NE(storages.get(666), nullptr); // `test`.`test-tbl` -} -CATCH - -TEST_F(IDAsPathUpgrader_test, ISSUE_941) -try -{ - CHECK_TESTS_WITH_DATA_ENABLED; - - // prepare a "test" database for upgrader - MockTiDB::instance().newDataBase("test"); // id == 2 - - // Generated by running these SQL on cluster version v4.0.0-rc - // > create database `test`; - // > create table `#hello-world` (pk int); - const auto test_path = TiFlashTestEnv::findTestDataPath("issue-941"); - auto ctx = TiFlashTestEnv::getContext(DB::Settings(), test_path); - - IDAsPathUpgrader upgrader(ctx, false, {}); - ASSERT_TRUE(upgrader.needUpgrade()); - upgrader.doUpgrade(); - - { - // After upgrade, next time we don't need it. - IDAsPathUpgrader checker_after_upgrade(ctx, false, {}); - ASSERT_FALSE(checker_after_upgrade.needUpgrade()); - } - - // load metadata should not throw any exception - loadMetadata(ctx); - - ASSERT_TRUE(ctx.isDatabaseExist("db_2")); - auto & storages = ctx.getTMTContext().getStorages(); - ASSERT_NE(storages.get(45), nullptr); // `test`.`#hello-world` -} -CATCH - -} // namespace DB::tests diff --git a/dbms/src/Interpreters/tests/hash_map.cpp b/dbms/src/Interpreters/tests/hash_map.cpp deleted file mode 100644 index d75f578367f..00000000000 --- a/dbms/src/Interpreters/tests/hash_map.cpp +++ /dev/null @@ -1,319 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -#include - -#include -#include - -#include -/* -#define DBMS_HASH_MAP_COUNT_COLLISIONS -*/ -#include -#include -#include -#include -#include -#include -#include -#include - - -/** The test checks the speed of hash tables, simulating their use for aggregation. - * The first argument specifies the number of elements to be inserted. - * The second argument can be a number from 1 to 4 - the number of the data structure being tested. - * This is important, because if you run all the tests one by one, the results will be incorrect. - * (Due to the peculiarities of the work of the allocator, the first test takes advantage.) - * - * Depending on USE_AUTO_ARRAY, one of the structures is selected as the value. - * USE_AUTO_ARRAY = 0 - uses std::vector (hard-copy structure, sizeof = 24 bytes). - * USE_AUTO_ARRAY = 1 - uses AutoArray (a structure specially designed for such cases, sizeof = 8 bytes). - * - * That is, the test also allows you to compare AutoArray and std::vector. - * - * If USE_AUTO_ARRAY = 0, then HashMap confidently overtakes all. - * If USE_AUTO_ARRAY = 1, then HashMap is slightly less serious (20%) ahead of google::dense_hash_map. - * - * When using HashMap, AutoArray has a rather serious (40%) advantage over std::vector. - * And when using other hash tables, AutoArray even more seriously overtakes std::vector - * (up to three and a half times in the case of std::unordered_map and google::sparse_hash_map). - * - * HashMap, unlike google::dense_hash_map, much more depends on the quality of the hash function. - * - * PS. Measure everything yourself, otherwise I'm almost confused. - * - * PPS. Now the aggregation does not use an array of aggregate functions as values. - * States of aggregate functions were separated from the interface to manipulate them, and put in the pool. - * But in this test, there was something similar to the old scenario of using hash tables in the aggregation. - */ - -#define USE_AUTO_ARRAY 0 - - -struct AlternativeHash -{ - size_t operator() (UInt64 x) const - { - x ^= x >> 23; - x *= 0x2127599bf4325c37ULL; - x ^= x >> 47; - - return x; - } -}; - - -#if defined(__x86_64__) - -struct CRC32Hash_ -{ - size_t operator() (UInt64 x) const - { - UInt64 crc = -1ULL; - asm("crc32q %[x], %[crc]\n" : [crc] "+r" (crc) : [x] "rm" (x)); - return crc; - } -}; - -#endif - - -int main(int argc, char ** argv) -{ - using namespace DB; - - using Key = UInt64; - -#if USE_AUTO_ARRAY - using Value = AutoArray; -#else - using Value = std::vector; -#endif - - size_t n = argc < 2 ? 10000000 : atoi(argv[1]); - //size_t m = atoi(argv[2]); - - AggregateFunctionFactory factory; - DataTypes data_types_empty; - DataTypes data_types_uint64; - data_types_uint64.push_back(std::make_shared()); - - std::vector data(n); - Value value; - - AggregateFunctionPtr func_count = factory.get("count", data_types_empty); - AggregateFunctionPtr func_avg = factory.get("avg", data_types_uint64); - AggregateFunctionPtr func_uniq = factory.get("uniq", data_types_uint64); - - #define INIT \ - { \ - value.resize(3); \ - \ - value[0] = func_count.get();\ - value[1] = func_avg.get(); \ - value[2] = func_uniq.get(); \ - } - - INIT; - -#ifndef USE_AUTO_ARRAY - #undef INIT - #define INIT -#endif - - Row row(1); - row[0] = UInt64(0); - - std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl; - - { - Stopwatch watch; - /* for (size_t i = 0; i < n; ++i) - data[i] = rand() % m; - - for (size_t i = 0; i < n; i += 10) - data[i] = 0;*/ - - ReadBufferFromFile in1("UniqID.bin"); - CompressedReadBuffer in2(in1); - - in2.readStrict(reinterpret_cast(&data[0]), sizeof(data[0]) * n); - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "Vector. Size: " << n - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (argc < 3 || atoi(argv[2]) == 1) - { - Stopwatch watch; - - HashMap map; - HashMap::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - { - new(&it->second) Value(std::move(value)); - INIT; - } - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - - if (argc < 3 || atoi(argv[2]) == 2) - { - Stopwatch watch; - - using Map = HashMap; - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - { - new(&it->second) Value(std::move(value)); - INIT; - } - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap, AlternativeHash. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - -#if defined(__x86_64__) - if (argc < 3 || atoi(argv[2]) == 3) - { - Stopwatch watch; - - using Map = HashMap; - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - { - new(&it->second) Value(std::move(value)); - INIT; - } - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap, CRC32Hash. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } -#endif - - if (argc < 3 || atoi(argv[2]) == 4) - { - Stopwatch watch; - - std::unordered_map> map; - std::unordered_map>::iterator it; - for (size_t i = 0; i < n; ++i) - { - it = map.insert(std::make_pair(data[i], std::move(value))).first; - INIT; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "std::unordered_map. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (argc < 3 || atoi(argv[2]) == 5) - { - Stopwatch watch; - - google::dense_hash_map> map; - google::dense_hash_map>::iterator it; - map.set_empty_key(-1ULL); - for (size_t i = 0; i < n; ++i) - { - it = map.insert(std::make_pair(data[i], std::move(value))).first; - INIT; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "google::dense_hash_map. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (argc < 3 || atoi(argv[2]) == 6) - { - Stopwatch watch; - - google::sparse_hash_map> map; - google::sparse_hash_map>::iterator it; - for (size_t i = 0; i < n; ++i) - { - map.insert(std::make_pair(data[i], std::move(value))); - INIT; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "google::sparse_hash_map. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - return 0; -} diff --git a/dbms/src/Interpreters/tests/hash_map3.cpp b/dbms/src/Interpreters/tests/hash_map3.cpp deleted file mode 100644 index daadb8301f8..00000000000 --- a/dbms/src/Interpreters/tests/hash_map3.cpp +++ /dev/null @@ -1,105 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 - -#define DBMS_HASH_MAP_DEBUG_RESIZES -#define DBMS_HASH_MAP_COUNT_COLLISIONS - - -#include - -#include - -#include - -#include -#include - -#include - -#include - -#include - - -template -< - typename Key, - typename Mapped, - typename Hash = DefaultHash, - typename Grower = HashTableGrower<>, - typename Allocator = HashTableAllocator -> -class HashMapWithDump : public HashMap -{ -public: - void dump() const - { - for (size_t i = 0; i < this->grower.bufSize(); ++i) - { - if (this->buf[i].isZero(*this)) - std::cerr << "[ ]"; - else - std::cerr << '[' << this->buf[i].getValue().first.data << ", " << this->buf[i].getValue().second << ']'; - } - std::cerr << std::endl; - } -}; - - -struct SimpleHash -{ - size_t operator() (UInt64 x) const { return x; } - size_t operator() (StringRef x) const { return DB::parse(x.data); } -}; - -struct Grower : public HashTableGrower<2> -{ - void increaseSize() - { - ++size_degree; - } -}; - -int main(int, char **) -{ - using Map = HashMapWithDump< - StringRef, - UInt64, - SimpleHash, - Grower, - HashTableAllocatorWithStackMemory<4 * 24>>; - - Map map; - - map.dump(); - std::cerr << "size: " << map.size() << std::endl; - map[StringRef("1", 1)] = 1; - map.dump(); - std::cerr << "size: " << map.size() << std::endl; - map[StringRef("9", 1)] = 1; - map.dump(); - std::cerr << "size: " << map.size() << std::endl; - std::cerr << "Collisions: " << map.getCollisions() << std::endl; - map[StringRef("3", 1)] = 2; - map.dump(); - std::cerr << "size: " << map.size() << std::endl; - std::cerr << "Collisions: " << map.getCollisions() << std::endl; - - for (auto x : map) - std::cerr << x.first.toString() << " -> " << x.second << std::endl; - - return 0; -} diff --git a/dbms/src/Interpreters/tests/hash_map_string.cpp b/dbms/src/Interpreters/tests/hash_map_string.cpp deleted file mode 100644 index af7717112a6..00000000000 --- a/dbms/src/Interpreters/tests/hash_map_string.cpp +++ /dev/null @@ -1,505 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -#include - -#include -#include - -#include - -//#define DBMS_HASH_MAP_COUNT_COLLISIONS -#define DBMS_HASH_MAP_DEBUG_RESIZES - -#include -#include -#include -#include -#include -#include -#include - - -struct CompactStringRef -{ - union - { - const char * data_mixed = nullptr; - struct - { - char dummy[6]; - UInt16 size; - }; - }; - - CompactStringRef(const char * data_, size_t size_) - { - data_mixed = data_; - size = size_; - } - - CompactStringRef(const unsigned char * data_, size_t size_) : CompactStringRef(reinterpret_cast(data_), size_) {} - explicit CompactStringRef(const std::string & s) : CompactStringRef(s.data(), s.size()) {} - CompactStringRef() {} - - const char * data() const { return reinterpret_cast(reinterpret_cast(data_mixed) & 0x0000FFFFFFFFFFFFULL); } - - std::string toString() const { return std::string(data(), size); } -}; - -inline bool operator==(CompactStringRef lhs, CompactStringRef rhs) -{ - if (lhs.size != rhs.size) - return false; - - const char * lhs_data = lhs.data(); - const char * rhs_data = rhs.data(); - for (size_t pos = lhs.size - 1; pos < lhs.size; --pos) - if (lhs_data[pos] != rhs_data[pos]) - return false; - - return true; -} - -namespace ZeroTraits -{ - template <> - inline bool check(CompactStringRef x) { return nullptr == x.data_mixed; } - - template <> - inline void set(CompactStringRef & x) { x.data_mixed = nullptr; } -}; - -template <> -struct DefaultHash -{ - size_t operator() (CompactStringRef x) const - { - return CityHash_v1_0_2::CityHash64(x.data(), x.size); - } -}; - - -#define mix(h) ({ \ - (h) ^= (h) >> 23; \ - (h) *= 0x2127599bf4325c37ULL; \ - (h) ^= (h) >> 47; }) - -struct FastHash64 -{ - size_t operator() (CompactStringRef x) const - { - const char * buf = x.data(); - size_t len = x.size; - - const UInt64 m = 0x880355f21e6d1965ULL; - const UInt64 *pos = reinterpret_cast(buf); - const UInt64 *end = pos + (len / 8); - const unsigned char *pos2; - UInt64 h = len * m; - UInt64 v; - - while (pos != end) - { - v = *pos++; - h ^= mix(v); - h *= m; - } - - pos2 = reinterpret_cast(pos); - v = 0; - - switch (len & 7) - { - case 7: v ^= static_cast(pos2[6]) << 48; [[fallthrough]]; - case 6: v ^= static_cast(pos2[5]) << 40; [[fallthrough]]; - case 5: v ^= static_cast(pos2[4]) << 32; [[fallthrough]]; - case 4: v ^= static_cast(pos2[3]) << 24; [[fallthrough]]; - case 3: v ^= static_cast(pos2[2]) << 16; [[fallthrough]]; - case 2: v ^= static_cast(pos2[1]) << 8; [[fallthrough]]; - case 1: v ^= static_cast(pos2[0]); - h ^= mix(v); - h *= m; - } - - return mix(h); - } -}; - - -struct CrapWow -{ - size_t operator() (CompactStringRef x) const - { -#if __x86_64__ - const char * key = x.data(); - size_t len = x.size; - size_t seed = 0; - - const UInt64 m = 0x95b47aa3355ba1a1, n = 0x8a970be7488fda55; - - UInt64 hash; - // 3 = m, 4 = n - // r12 = h, r13 = k, ecx = seed, r12 = key - asm( - "leaq (%%rcx,%4), %%r13\n" - "movq %%rdx, %%r14\n" - "movq %%rcx, %%r15\n" - "movq %%rcx, %%r12\n" - "addq %%rax, %%r13\n" - "andq $0xfffffffffffffff0, %%rcx\n" - "jz QW%=\n" - "addq %%rcx, %%r14\n\n" - "negq %%rcx\n" - "XW%=:\n" - "movq %4, %%rax\n" - "mulq (%%r14,%%rcx)\n" - "xorq %%rax, %%r12\n" - "xorq %%rdx, %%r13\n" - "movq %3, %%rax\n" - "mulq 8(%%r14,%%rcx)\n" - "xorq %%rdx, %%r12\n" - "xorq %%rax, %%r13\n" - "addq $16, %%rcx\n" - "jnz XW%=\n" - "QW%=:\n" - "movq %%r15, %%rcx\n" - "andq $8, %%r15\n" - "jz B%=\n" - "movq %4, %%rax\n" - "mulq (%%r14)\n" - "addq $8, %%r14\n" - "xorq %%rax, %%r12\n" - "xorq %%rdx, %%r13\n" - "B%=:\n" - "andq $7, %%rcx\n" - "jz F%=\n" - "movq $1, %%rdx\n" - "shlq $3, %%rcx\n" - "movq %3, %%rax\n" - "shlq %%cl, %%rdx\n" - "addq $-1, %%rdx\n" - "andq (%%r14), %%rdx\n" - "mulq %%rdx\n" - "xorq %%rdx, %%r12\n" - "xorq %%rax, %%r13\n" - "F%=:\n" - "leaq (%%r13,%4), %%rax\n" - "xorq %%r12, %%rax\n" - "mulq %4\n" - "xorq %%rdx, %%rax\n" - "xorq %%r12, %%rax\n" - "xorq %%r13, %%rax\n" - : "=a"(hash), "=c"(key), "=d"(key) - : "r"(m), "r"(n), "a"(seed), "c"(len), "d"(key) - : "%r12", "%r13", "%r14", "%r15", "cc" - ); - return hash; -#else - return 0; -#endif - } -}; - - -struct SimpleHash -{ - size_t operator() (CompactStringRef x) const - { - const char * pos = x.data(); - size_t size = x.size; - - const char * end = pos + size; - - size_t res = 0; - - if (size == 0) - return 0; - - if (size < 8) - { - memcpy(reinterpret_cast(&res), pos, size); - return intHash64(res); - } - - while (pos + 8 < end) - { - UInt64 word = *reinterpret_cast(pos); - res = intHash64(word ^ res); - - pos += 8; - } - - UInt64 word = *reinterpret_cast(end - 8); - res = intHash64(word ^ res); - - return res; - } -}; - - -using Key = CompactStringRef; -using Value = UInt64; - - -struct Grower : public HashTableGrower<> -{ - /// The state of this structure is enough to get the buffer size of the hash table. - - /// Defines the initial size of the hash table. - static const size_t initial_size_degree = 16; - Grower() { size_degree = initial_size_degree; } - - size_t max_fill = (1ULL << initial_size_degree) * 0.9; - - /// The size of the hash table in the cells. - size_t bufSize() const { return 1ULL << size_degree; } - - size_t maxFill() const { return max_fill /*1 << (size_degree - 1)*/; } - size_t mask() const { return bufSize() - 1; } - - /// From the hash value, get the cell number in the hash table. - size_t place(size_t x) const { return x & mask(); } - - /// The next cell in the collision resolution chain. - size_t next(size_t pos) const { ++pos; return pos & mask(); } - - /// Whether the hash table is sufficiently full. You need to increase the size of the hash table, or remove something unnecessary from it. - bool overflow(size_t elems) const { return elems > maxFill(); } - - /// Increase the size of the hash table. - void increaseSize() - { - size_degree += size_degree >= 23 ? 1 : 2; - max_fill = (1ULL << size_degree) * 0.9; - } - - /// Set the buffer size by the number of elements in the hash table. Used when deserializing a hash table. - void set(size_t /*num_elems*/) - { - throw Poco::Exception(__PRETTY_FUNCTION__); - } -}; - - -int main(int argc, char ** argv) -{ - if (argc < 3) - { - std::cerr << "Usage: program n m\n"; - return 1; - } - - size_t n = atoi(argv[1]); - size_t m = atoi(argv[2]); - - DB::Arena pool; - std::vector data(n); - - std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl; - - { - Stopwatch watch; - DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO); - DB::CompressedReadBuffer in2(in1); - - std::string tmp; - for (size_t i = 0; i < n && !in2.eof(); ++i) - { - DB::readStringBinary(tmp, in2); - data[i] = Key(pool.insert(tmp.data(), tmp.size()), tmp.size()); - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "Vector. Size: " << n - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (!m || m == 1) - { - Stopwatch watch; - - //using Map = HashMap; - - /// Saving the hash accelerates the resize by about 2 times, and the overall performance by 6-8%. - using Map = HashMapWithSavedHash, Grower>; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (CityHash64). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - - if (!m || m == 2) - { - Stopwatch watch; - - using Map = HashMapWithSavedHash; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (FastHash64). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - - if (!m || m == 3) - { - Stopwatch watch; - - using Map = HashMapWithSavedHash; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (CrapWow). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - - if (!m || m == 4) - { - Stopwatch watch; - - using Map = HashMapWithSavedHash; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (SimpleHash). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - - if (!m || m == 5) - { - Stopwatch watch; - - std::unordered_map> map; - for (size_t i = 0; i < n; ++i) - ++map[data[i]]; - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "std::unordered_map. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (!m || m == 6) - { - Stopwatch watch; - - google::dense_hash_map> map; - map.set_empty_key(Key("\0", 1)); - for (size_t i = 0; i < n; ++i) - ++map[data[i]]; - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "google::dense_hash_map. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (!m || m == 7) - { - Stopwatch watch; - - google::sparse_hash_map> map; - for (size_t i = 0; i < n; ++i) - ++map[data[i]]; - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "google::sparse_hash_map. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - return 0; -} diff --git a/dbms/src/Interpreters/tests/hash_map_string_2.cpp b/dbms/src/Interpreters/tests/hash_map_string_2.cpp deleted file mode 100644 index e073084aa8c..00000000000 --- a/dbms/src/Interpreters/tests/hash_map_string_2.cpp +++ /dev/null @@ -1,686 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -#include - -//#define DBMS_HASH_MAP_COUNT_COLLISIONS -#define DBMS_HASH_MAP_DEBUG_RESIZES - -#include -#include -#include -#include -#include -#include -#include - -#if __SSE4_1__ - #include -#endif - - -/** Do this: -for file in MobilePhoneModel PageCharset Params URLDomain UTMSource Referer URL Title; do - for size in 30000 100000 300000 1000000 5000000; do - echo - BEST_METHOD=0 - BEST_RESULT=0 - for method in {1..12}; do - echo -ne $file $size $method ''; - TOTAL_ELEMS=0 - for i in {0..1000}; do - TOTAL_ELEMS=$(( $TOTAL_ELEMS + $size )) - if [[ $TOTAL_ELEMS -gt 25000000 ]]; then break; fi - ./hash_map_string_2 $size $method < ${file}.bin 2>&1 | - grep HashMap | grep -oE '[0-9\.]+ elem'; - done | awk -W interactive '{ if ($1 > x) { x = $1 }; printf(".") } END { print x }' | tee /tmp/hash_map_string_2_res; - CUR_RESULT=$(cat /tmp/hash_map_string_2_res | tr -d '.') - if [[ $CUR_RESULT -gt $BEST_RESULT ]]; then - BEST_METHOD=$method - BEST_RESULT=$CUR_RESULT - fi; - done; - echo Best: $BEST_METHOD - $BEST_RESULT - done; -done -*/ - - -#define DefineStringRef(STRUCT) \ -\ -struct STRUCT : public StringRef {}; \ -\ -namespace ZeroTraits \ -{ \ - template <> \ - inline bool check(STRUCT x) { return 0 == x.size; } \ - \ - template <> \ - inline void set(STRUCT & x) { x.size = 0; } \ -}; \ - \ -template <> \ -struct DefaultHash \ -{ \ - size_t operator() (STRUCT x) const \ - { \ - return CityHash_v1_0_2::CityHash64(x.data, x.size); \ - } \ -}; - -DefineStringRef(StringRef_Compare1_Ptrs) -DefineStringRef(StringRef_Compare1_Index) -DefineStringRef(StringRef_CompareMemcmp) -DefineStringRef(StringRef_Compare8_1_byUInt64) -DefineStringRef(StringRef_Compare16_1_byMemcmp) -DefineStringRef(StringRef_Compare16_1_byUInt64_logicAnd) -DefineStringRef(StringRef_Compare16_1_byUInt64_bitAnd) - -#if __SSE4_1__ -DefineStringRef(StringRef_Compare16_1_byIntSSE) -DefineStringRef(StringRef_Compare16_1_byFloatSSE) -DefineStringRef(StringRef_Compare16_1_bySSE4) -DefineStringRef(StringRef_Compare16_1_bySSE4_wide) -DefineStringRef(StringRef_Compare16_1_bySSE_wide) -#endif - -DefineStringRef(StringRef_CompareAlwaysTrue) -DefineStringRef(StringRef_CompareAlmostAlwaysTrue) - - -inline bool operator==(StringRef_Compare1_Ptrs lhs, StringRef_Compare1_Ptrs rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - const char * pos1 = lhs.data; - const char * pos2 = rhs.data; - - const char * end1 = pos1 + lhs.size; - - while (pos1 < end1) - { - if (*pos1 != *pos2) - return false; - - ++pos1; - ++pos2; - } - - return true; -} - -inline bool operator==(StringRef_Compare1_Index lhs, StringRef_Compare1_Index rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - for (size_t i = 0; i < lhs.size; ++i) - if (lhs.data[i] != rhs.data[i]) - return false; - - return true; -} - -inline bool operator==(StringRef_CompareMemcmp lhs, StringRef_CompareMemcmp rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - return 0 == memcmp(lhs.data, rhs.data, lhs.size); -} - - -inline bool operator==(StringRef_Compare8_1_byUInt64 lhs, StringRef_Compare8_1_byUInt64 rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - const char * p1 = lhs.data; - const char * p2 = rhs.data; - size_t size = lhs.size; - - const char * p1_end = p1 + size; - const char * p1_end_8 = p1 + size / 8 * 8; - - while (p1 < p1_end_8) - { - if (reinterpret_cast(p1)[0] != reinterpret_cast(p2)[0]) - return false; - - p1 += 8; - p2 += 8; - } - - while (p1 < p1_end) - { - if (*p1 != *p2) - return false; - - ++p1; - ++p2; - } - - return true; -} - - - -inline bool compare_byMemcmp(const char * p1, const char * p2) -{ - return 0 == memcmp(p1, p2, 16); -} - -inline bool compare_byUInt64_logicAnd(const char * p1, const char * p2) -{ - return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0] - && reinterpret_cast(p1)[1] == reinterpret_cast(p2)[1]; -} - -inline bool compare_byUInt64_bitAnd(const char * p1, const char * p2) -{ - return (reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]) - & (reinterpret_cast(p1)[1] == reinterpret_cast(p2)[1]); -} - -#if __SSE4_1__ - -inline bool compare_byIntSSE(const char * p1, const char * p2) -{ - return 0xFFFF == _mm_movemask_epi8(_mm_cmpeq_epi8( - _mm_loadu_si128(reinterpret_cast(p1)), - _mm_loadu_si128(reinterpret_cast(p2)))); -} - -inline bool compare_byFloatSSE(const char * p1, const char * p2) -{ - return !_mm_movemask_ps(_mm_cmpneq_ps( /// Looks like incorrect while comparing subnormal floats. - _mm_loadu_ps(reinterpret_cast(p1)), - _mm_loadu_ps(reinterpret_cast(p2)))); -} - -#endif - - -template -inline bool memequal(const char * p1, const char * p2, size_t size) -{ -// const char * p1_end = p1 + size; - const char * p1_end_16 = p1 + size / 16 * 16; - - while (p1 < p1_end_16) - { - if (!compare(p1, p2)) - return false; - - p1 += 16; - p2 += 16; - } - -/* while (p1 < p1_end) - { - if (*p1 != *p2) - return false; - - ++p1; - ++p2; - }*/ - - switch (size % 16) - { - case 15: if (p1[14] != p2[14]) return false; [[fallthrough]]; - case 14: if (p1[13] != p2[13]) return false; [[fallthrough]]; - case 13: if (p1[12] != p2[12]) return false; [[fallthrough]]; - case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; - case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; - case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; - l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; - } - - return true; -} - - -#if __SSE4_1__ - -inline bool memequal_sse41(const char * p1, const char * p2, size_t size) -{ -// const char * p1_end = p1 + size; - const char * p1_end_16 = p1 + size / 16 * 16; - - __m128i zero16 = _mm_setzero_si128(); - - while (p1 < p1_end_16) - { - if (!_mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(reinterpret_cast(p1)), - _mm_loadu_si128(reinterpret_cast(p2))))) - return false; - - p1 += 16; - p2 += 16; - } - -/* while (p1 < p1_end) - { - if (*p1 != *p2) - return false; - - ++p1; - ++p2; - }*/ - - switch (size % 16) - { - case 15: if (p1[14] != p2[14]) return false; [[fallthrough]]; - case 14: if (p1[13] != p2[13]) return false; [[fallthrough]]; - case 13: if (p1[12] != p2[12]) return false; [[fallthrough]]; - case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; - case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; - case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; - l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; - } - - return true; -} - - -inline bool memequal_sse41_wide(const char * p1, const char * p2, size_t size) -{ - __m128i zero16 = _mm_setzero_si128(); -// const char * p1_end = p1 + size; - - while (size >= 64) - { - if (_mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[0]), - _mm_loadu_si128(&reinterpret_cast(p2)[0]))) - && _mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[1]), - _mm_loadu_si128(&reinterpret_cast(p2)[1]))) - && _mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[2]), - _mm_loadu_si128(&reinterpret_cast(p2)[2]))) - && _mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[3]), - _mm_loadu_si128(&reinterpret_cast(p2)[3])))) - { - p1 += 64; - p2 += 64; - size -= 64; - } - else - return false; - } - - switch ((size % 64) / 16) - { - case 3: - if (!_mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[2]), - _mm_loadu_si128(&reinterpret_cast(p2)[2])))) - return false; - [[fallthrough]]; - case 2: - if (!_mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[1]), - _mm_loadu_si128(&reinterpret_cast(p2)[1])))) - return false; - [[fallthrough]]; - case 1: - if (!_mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[0]), - _mm_loadu_si128(&reinterpret_cast(p2)[0])))) - return false; - } - - p1 += (size % 64) / 16 * 16; - p2 += (size % 64) / 16 * 16; - -/* - - if (size >= 32) - { - if (_mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[0]), - _mm_loadu_si128(&reinterpret_cast(p2)[0]))) - & _mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[1]), - _mm_loadu_si128(&reinterpret_cast(p2)[1])))) - { - p1 += 32; - p2 += 32; - size -= 32; - } - else - return false; - } - - if (size >= 16) - { - if (_mm_testc_si128( - zero16, - _mm_xor_si128( - _mm_loadu_si128(&reinterpret_cast(p1)[0]), - _mm_loadu_si128(&reinterpret_cast(p2)[0])))) - { - p1 += 16; - p2 += 16; - size -= 16; - } - else - return false; - }*/ - - switch (size % 16) - { - case 15: if (p1[14] != p2[14]) return false; [[fallthrough]]; - case 14: if (p1[13] != p2[13]) return false; [[fallthrough]]; - case 13: if (p1[12] != p2[12]) return false; [[fallthrough]]; - case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; - case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; - case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; - l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; - } - - return true; -} - - -inline bool memequal_sse_wide(const char * p1, const char * p2, size_t size) -{ - while (size >= 64) - { - if ( compare_byIntSSE(p1, p2) - && compare_byIntSSE(p1 + 16, p2 + 16) - && compare_byIntSSE(p1 + 32, p2 + 32) - && compare_byIntSSE(p1 + 48, p2 + 48)) - { - p1 += 64; - p2 += 64; - size -= 64; - } - else - return false; - } - - switch ((size % 64) / 16) - { - case 3: if (!compare_byIntSSE(p1 + 32, p2 + 32)) return false; [[fallthrough]]; - case 2: if (!compare_byIntSSE(p1 + 16, p2 + 16)) return false; [[fallthrough]]; - case 1: if (!compare_byIntSSE(p1 , p2 )) return false; - } - - p1 += (size % 64) / 16 * 16; - p2 += (size % 64) / 16 * 16; - - switch (size % 16) - { - case 15: if (p1[14] != p2[14]) return false; [[fallthrough]]; - case 14: if (p1[13] != p2[13]) return false; [[fallthrough]]; - case 13: if (p1[12] != p2[12]) return false; [[fallthrough]]; - case 12: if (reinterpret_cast(p1)[2] == reinterpret_cast(p2)[2]) goto l8; else return false; - case 11: if (p1[10] != p2[10]) return false; [[fallthrough]]; - case 10: if (p1[9] != p2[9]) return false; [[fallthrough]]; - case 9: if (p1[8] != p2[8]) return false; - l8: [[fallthrough]]; - case 8: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 7: if (p1[6] != p2[6]) return false; [[fallthrough]]; - case 6: if (p1[5] != p2[5]) return false; [[fallthrough]]; - case 5: if (p1[4] != p2[4]) return false; [[fallthrough]]; - case 4: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 3: if (p1[2] != p2[2]) return false; [[fallthrough]]; - case 2: return reinterpret_cast(p1)[0] == reinterpret_cast(p2)[0]; - case 1: if (p1[0] != p2[0]) return false; [[fallthrough]]; - case 0: break; - } - - return true; -} - -#endif - - -#define Op(METHOD) \ -inline bool operator==(StringRef_Compare16_1_ ## METHOD lhs, StringRef_Compare16_1_ ## METHOD rhs) \ -{ \ - if (lhs.size != rhs.size) \ - return false; \ -\ - if (lhs.size == 0) \ - return true; \ -\ - return memequal(lhs.data, rhs.data, lhs.size); \ -} - -Op(byMemcmp) -Op(byUInt64_logicAnd) -Op(byUInt64_bitAnd) - -#if __SSE4_1__ - -Op(byIntSSE) -Op(byFloatSSE) - - -inline bool operator==(StringRef_Compare16_1_bySSE4 lhs, StringRef_Compare16_1_bySSE4 rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - return memequal_sse41(lhs.data, rhs.data, lhs.size); -} - -inline bool operator==(StringRef_Compare16_1_bySSE4_wide lhs, StringRef_Compare16_1_bySSE4_wide rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - return memequal_sse41_wide(lhs.data, rhs.data, lhs.size); -} - -inline bool operator==(StringRef_Compare16_1_bySSE_wide lhs, StringRef_Compare16_1_bySSE_wide rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - return memequal_sse_wide(lhs.data, rhs.data, lhs.size); -} - -#endif - - -inline bool operator==(StringRef_CompareAlwaysTrue, StringRef_CompareAlwaysTrue) -{ - return true; -} - -inline bool operator==(StringRef_CompareAlmostAlwaysTrue lhs, StringRef_CompareAlmostAlwaysTrue rhs) -{ - if (lhs.size != rhs.size) - return false; - - return true; -} - - - -using Value = UInt64; - - -template -void NO_INLINE bench(const std::vector & data, const char * name) -{ - Stopwatch watch; - - using Map = HashMapWithSavedHash>; - - Map map; - typename Map::iterator it; - bool inserted; - - for (size_t i = 0, size = data.size(); i < size; ++i) - { - map.emplace(static_cast(data[i]), it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (" << name << "). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; -} - - -int main(int argc, char ** argv) -{ - if (argc < 3) - { - std::cerr << "Usage: program n m\n"; - return 1; - } - - size_t n = atoi(argv[1]); - size_t m = atoi(argv[2]); - - DB::Arena pool; - std::vector data(n); - - std::cerr << "sizeof(Key) = " << sizeof(StringRef) << ", sizeof(Value) = " << sizeof(Value) << std::endl; - - { - Stopwatch watch; - DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO); - DB::CompressedReadBuffer in2(in1); - - std::string tmp; - for (size_t i = 0; i < n && !in2.eof(); ++i) - { - DB::readStringBinary(tmp, in2); - data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size()); - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "Vector. Size: " << n - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (!m || m == 1) bench (data, "StringRef_Compare1_Ptrs"); - if (!m || m == 2) bench (data, "StringRef_Compare1_Index"); - if (!m || m == 3) bench (data, "StringRef_CompareMemcmp"); - if (!m || m == 4) bench (data, "StringRef_Compare8_1_byUInt64"); - if (!m || m == 5) bench (data, "StringRef_Compare16_1_byMemcmp"); - if (!m || m == 6) bench(data, "StringRef_Compare16_1_byUInt64_logicAnd"); - if (!m || m == 7) bench (data, "StringRef_Compare16_1_byUInt64_bitAnd"); -#if __SSE4_1__ - if (!m || m == 8) bench (data, "StringRef_Compare16_1_byIntSSE"); - if (!m || m == 9) bench (data, "StringRef_Compare16_1_byFloatSSE"); - if (!m || m == 10) bench (data, "StringRef_Compare16_1_bySSE4"); - if (!m || m == 11) bench (data, "StringRef_Compare16_1_bySSE4_wide"); - if (!m || m == 12) bench (data, "StringRef_Compare16_1_bySSE_wide"); -#endif - if (!m || m == 100) bench (data, "StringRef_CompareAlwaysTrue"); - if (!m || m == 101) bench (data, "StringRef_CompareAlmostAlwaysTrue"); - - /// 10 > 8, 9 - /// 1, 2, 5 - bad - - - return 0; -} diff --git a/dbms/src/Interpreters/tests/hash_map_string_3.cpp b/dbms/src/Interpreters/tests/hash_map_string_3.cpp deleted file mode 100644 index 1d6be35a5b1..00000000000 --- a/dbms/src/Interpreters/tests/hash_map_string_3.cpp +++ /dev/null @@ -1,525 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -#include - -#include -#include - -#define DBMS_HASH_MAP_COUNT_COLLISIONS -#define DBMS_HASH_MAP_DEBUG_RESIZES - -#include -#include -#include -#include -#include -#include -#include - -#if __SSE4_1__ - #include -#endif - - -/** Do this: -for file in MobilePhoneModel PageCharset Params URLDomain UTMSource Referer URL Title; do - for size in 30000 100000 300000 1000000 5000000; do - echo - BEST_METHOD=0 - BEST_RESULT=0 - for method in {1..11}; do - echo -ne $file $size $method ''; - TOTAL_ELEMS=0 - for i in {0..1000}; do - TOTAL_ELEMS=$(( $TOTAL_ELEMS + $size )) - if [[ $TOTAL_ELEMS -gt 25000000 ]]; then break; fi - ./hash_map_string_3 $size $method < ${file}.bin 2>&1 | - grep HashMap | grep -oE '[0-9\.]+ elem'; - done | awk -W interactive '{ if ($1 > x) { x = $1 }; printf(".") } END { print x }' | tee /tmp/hash_map_string_3_res; - CUR_RESULT=$(cat /tmp/hash_map_string_3_res | tr -d '.') - if [[ $CUR_RESULT -gt $BEST_RESULT ]]; then - BEST_METHOD=$method - BEST_RESULT=$CUR_RESULT - fi; - done; - echo Best: $BEST_METHOD - $BEST_RESULT - done; -done -*/ - - -#define DefineStringRef(STRUCT) \ -\ -struct STRUCT : public StringRef {}; \ -\ -namespace ZeroTraits \ -{ \ - template <> \ - inline bool check(STRUCT x) { return nullptr == x.data; } \ - \ - template <> \ - inline void set(STRUCT & x) { x.data = nullptr; } \ -}; \ - \ -template <> \ -struct DefaultHash \ -{ \ - size_t operator() (STRUCT x) const \ - { \ - return CityHash_v1_0_2::CityHash64(x.data, x.size); \ - } \ -}; - - -DefineStringRef(StringRef_CompareMemcmp) -DefineStringRef(StringRef_CompareAlwaysTrue) - - -inline bool operator==(StringRef_CompareMemcmp lhs, StringRef_CompareMemcmp rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - - return 0 == memcmp(lhs.data, rhs.data, lhs.size); -} - -inline bool operator==(StringRef_CompareAlwaysTrue, StringRef_CompareAlwaysTrue) -{ - return true; -} - - -#define mix(h) ({ \ - (h) ^= (h) >> 23; \ - (h) *= 0x2127599bf4325c37ULL; \ - (h) ^= (h) >> 47; }) - -struct FastHash64 -{ - size_t operator() (StringRef x) const - { - const char * buf = x.data; - size_t len = x.size; - - const uint64_t m = 0x880355f21e6d1965ULL; - const uint64_t *pos = reinterpret_cast(buf); - const uint64_t *end = pos + (len / 8); - const unsigned char *pos2; - uint64_t h = len * m; - uint64_t v; - - while (pos != end) - { - v = *pos++; - h ^= mix(v); - h *= m; - } - - pos2 = reinterpret_cast(pos); - v = 0; - - switch (len & 7) - { - case 7: v ^= static_cast(pos2[6]) << 48; [[fallthrough]]; - case 6: v ^= static_cast(pos2[5]) << 40; [[fallthrough]]; - case 5: v ^= static_cast(pos2[4]) << 32; [[fallthrough]]; - case 4: v ^= static_cast(pos2[3]) << 24; [[fallthrough]]; - case 3: v ^= static_cast(pos2[2]) << 16; [[fallthrough]]; - case 2: v ^= static_cast(pos2[1]) << 8; [[fallthrough]]; - case 1: v ^= static_cast(pos2[0]); - h ^= mix(v); - h *= m; - } - - return mix(h); - } -}; - - -struct FNV1a -{ - size_t operator() (StringRef x) const - { - size_t res = 0xcbf29ce484222325ULL; - - const char * pos = x.data; - const char * end = x.data + x.size; - - for (; pos < end; ++pos) - { - res *= 1099511628211ULL; - res ^= *pos; - } - - return res; - } -}; - - -#if __SSE4_1__ - -struct CrapWow -{ - size_t operator() (StringRef x) const - { - const char * key = x.data; - size_t len = x.size; - size_t seed = 0; - - const UInt64 m = 0x95b47aa3355ba1a1, n = 0x8a970be7488fda55; - UInt64 hash; - // 3 = m, 4 = n - // r12 = h, r13 = k, ecx = seed, r12 = key - asm( - "leaq (%%rcx,%4), %%r13\n" - "movq %%rdx, %%r14\n" - "movq %%rcx, %%r15\n" - "movq %%rcx, %%r12\n" - "addq %%rax, %%r13\n" - "andq $0xfffffffffffffff0, %%rcx\n" - "jz QW%=\n" - "addq %%rcx, %%r14\n\n" - "negq %%rcx\n" - "XW%=:\n" - "movq %4, %%rax\n" - "mulq (%%r14,%%rcx)\n" - "xorq %%rax, %%r12\n" - "xorq %%rdx, %%r13\n" - "movq %3, %%rax\n" - "mulq 8(%%r14,%%rcx)\n" - "xorq %%rdx, %%r12\n" - "xorq %%rax, %%r13\n" - "addq $16, %%rcx\n" - "jnz XW%=\n" - "QW%=:\n" - "movq %%r15, %%rcx\n" - "andq $8, %%r15\n" - "jz B%=\n" - "movq %4, %%rax\n" - "mulq (%%r14)\n" - "addq $8, %%r14\n" - "xorq %%rax, %%r12\n" - "xorq %%rdx, %%r13\n" - "B%=:\n" - "andq $7, %%rcx\n" - "jz F%=\n" - "movq $1, %%rdx\n" - "shlq $3, %%rcx\n" - "movq %3, %%rax\n" - "shlq %%cl, %%rdx\n" - "addq $-1, %%rdx\n" - "andq (%%r14), %%rdx\n" - "mulq %%rdx\n" - "xorq %%rdx, %%r12\n" - "xorq %%rax, %%r13\n" - "F%=:\n" - "leaq (%%r13,%4), %%rax\n" - "xorq %%r12, %%rax\n" - "mulq %4\n" - "xorq %%rdx, %%rax\n" - "xorq %%r12, %%rax\n" - "xorq %%r13, %%rax\n" - : "=a"(hash), "=c"(key), "=d"(key) - : "r"(m), "r"(n), "a"(seed), "c"(len), "d"(key) - : "%r12", "%r13", "%r14", "%r15", "cc" - ); - return hash; - } -}; - -#endif - - -struct SimpleHash -{ - size_t operator() (StringRef x) const - { - const char * pos = x.data; - size_t size = x.size; - - const char * end = pos + size; - - size_t res = 0; - - if (size == 0) - return 0; - - if (size < 8) - { -#if __SSE4_1__ - return hashLessThan8(x.data, x.size); -#endif - } - - while (pos + 8 < end) - { - uint64_t word = *reinterpret_cast(pos); - res = intHash64(word ^ res); - - pos += 8; - } - - uint64_t word = *reinterpret_cast(end - 8); - res = intHash64(word ^ res); - - return res; - } -}; - - -struct VerySimpleHash -{ - size_t operator() (StringRef x) const - { - const char * pos = x.data; - size_t size = x.size; - - const char * end = pos + size; - - size_t res = 0; - - if (size == 0) - return 0; - - if (size < 8) - { -#if __SSE4_1__ - return hashLessThan8(x.data, x.size); -#endif - } - - while (pos + 8 < end) - { - res ^= reinterpret_cast(pos)[0]; - res ^= res >> 33; - res *= 0xff51afd7ed558ccdULL; - - pos += 8; - } - - res ^= *reinterpret_cast(end - 8); - res ^= res >> 33; - res *= 0xc4ceb9fe1a85ec53ULL; - res ^= res >> 33; - - return res; - } -}; - - -struct FarmHash64 -{ - size_t operator() (StringRef x) const - { - return farmhash::Hash64(x.data, x.size); - } -}; - - -template -struct MetroHash64 -{ - size_t operator() (StringRef x) const - { - union - { - uint64_t u64; - std::uint8_t u8[sizeof(u64)]; - }; - - metrohash64(reinterpret_cast(x.data), x.size, 0, u8); - - return u64; - } -}; - - -#if __SSE4_1__ - -/*struct CRC32Hash -{ - size_t operator() (StringRef x) const - { - const char * pos = x.data; - size_t size = x.size; - - if (size == 0) - return 0; - - if (size < 8) - { - return hashLessThan8(x.data, x.size); - } - - const char * end = pos + size; - size_t res = -1ULL; - - do - { - uint64_t word = *reinterpret_cast(pos); - res = _mm_crc32_u64(res, word); - - pos += 8; - } while (pos + 8 < end); - - uint64_t word = *reinterpret_cast(end - 8); - res = _mm_crc32_u64(res, word); - - return res; - } -};*/ - - -struct CRC32ILPHash -{ - size_t operator() (StringRef x) const - { - const char * pos = x.data; - size_t size = x.size; - - if (size == 0) - return 0; - - if (size < 16) - { - return hashLessThan16(x.data, x.size); - } - - const char * end = pos + size; - const char * end_16 = pos + size / 16 * 16; - size_t res0 = -1ULL; - size_t res1 = -1ULL; - - do - { - uint64_t word0 = reinterpret_cast(pos)[0]; - uint64_t word1 = reinterpret_cast(pos)[1]; - res0 = _mm_crc32_u64(res0, word0); - res1 = _mm_crc32_u64(res1, word1); - - pos += 16; - } while (pos < end_16); - - uint64_t word0 = *reinterpret_cast(end - 8); - uint64_t word1 = *reinterpret_cast(end - 16); - - /* return HashLen16(Rotate(word0 - word1, 43) + Rotate(res0, 30) + res1, - word0 + Rotate(word1 ^ k3, 20) - res0 + size);*/ - - res0 = _mm_crc32_u64(res0, word0); - res1 = _mm_crc32_u64(res1, word1); - - return hashLen16(res0, res1); - } -}; - -#endif - - -using Value = uint64_t; - - -template -void NO_INLINE bench(const std::vector & data, const char * name) -{ - Stopwatch watch; - - using Map = HashMapWithSavedHash; - - Map map; - typename Map::iterator it; - bool inserted; - - for (size_t i = 0, size = data.size(); i < size; ++i) - { - map.emplace(static_cast(data[i]), it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (" << name << "). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << data.size() / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; -} - - -int main(int argc, char ** argv) -{ - if (argc < 3) - { - std::cerr << "Usage: program n m\n"; - return 1; - } - - size_t n = atoi(argv[1]); - size_t m = atoi(argv[2]); - - DB::Arena pool; - std::vector data(n); - - std::cerr << "sizeof(Key) = " << sizeof(StringRef) << ", sizeof(Value) = " << sizeof(Value) << std::endl; - - { - Stopwatch watch; - DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO); - DB::CompressedReadBuffer in2(in1); - - std::string tmp; - for (size_t i = 0; i < n && !in2.eof(); ++i) - { - DB::readStringBinary(tmp, in2); - data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size()); - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "Vector. Size: " << n - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (!m || m == 1) bench(data, "StringRef_CityHash64"); - if (!m || m == 2) bench (data, "StringRef_FastHash64"); - if (!m || m == 3) bench (data, "StringRef_SimpleHash"); - if (!m || m == 4) bench (data, "StringRef_FNV1a"); - -#if __SSE4_1__ - if (!m || m == 5) bench (data, "StringRef_CrapWow"); - if (!m || m == 6) bench (data, "StringRef_CRC32Hash"); - if (!m || m == 7) bench (data, "StringRef_CRC32ILPHash"); -#endif - - if (!m || m == 8) bench (data, "StringRef_VerySimpleHash"); - if (!m || m == 9) bench (data, "StringRef_FarmHash64"); - if (!m || m == 10) bench>(data, "StringRef_MetroHash64_1"); - if (!m || m == 11) bench>(data, "StringRef_MetroHash64_2"); - - return 0; -} diff --git a/dbms/src/Interpreters/tests/hash_map_string_small.cpp b/dbms/src/Interpreters/tests/hash_map_string_small.cpp deleted file mode 100644 index 7c7059d6381..00000000000 --- a/dbms/src/Interpreters/tests/hash_map_string_small.cpp +++ /dev/null @@ -1,206 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -#include - -#include -#include - -#include - -//#define DBMS_HASH_MAP_COUNT_COLLISIONS -#define DBMS_HASH_MAP_DEBUG_RESIZES - -#include -#include -#include -#include -#include -#include -#include - - -struct SmallStringRef -{ - UInt32 size; - - union - { - const char * data_big; - char data_small[12]; - }; - - bool isSmall() const { return size <= 12; } - - const char * data() const - { - return isSmall() ? data_small : data_big; - } - - SmallStringRef(const char * data_, size_t size_) - { - size = size_; - - if (isSmall()) - memcpy(data_small, data_, size_); - else - data_big = data_; - } - - SmallStringRef(const unsigned char * data_, size_t size_) : SmallStringRef(reinterpret_cast(data_), size_) {} - explicit SmallStringRef(const std::string & s) : SmallStringRef(s.data(), s.size()) {} - SmallStringRef() {} - - std::string toString() const { return std::string(data(), size); } -}; - - -inline bool operator==(SmallStringRef lhs, SmallStringRef rhs) -{ - if (lhs.size != rhs.size) - return false; - - if (lhs.size == 0) - return true; - -#if __SSE2__ - return memequalSSE2Wide(lhs.data(), rhs.data(), lhs.size); -#else - return false; -#endif -} - - -namespace ZeroTraits -{ - template <> - inline bool check(SmallStringRef x) { return x.size == 0; } - - template <> - inline void set(SmallStringRef & x) { x.size = 0; } -}; - -template <> -struct DefaultHash -{ - size_t operator() (SmallStringRef x) const - { - return DefaultHash()(StringRef(x.data(), x.size)); - } -}; - - -using Value = UInt64; - - -int main(int argc, char ** argv) -{ - if (argc < 3) - { - std::cerr << "Usage: program n m\n"; - return 1; - } - - size_t n = atoi(argv[1]); - size_t m = atoi(argv[2]); - - DB::Arena pool; - std::vector data(n); - - std::cerr << "sizeof(Key) = " << sizeof(SmallStringRef) << ", sizeof(Value) = " << sizeof(Value) << std::endl; - - { - Stopwatch watch; - DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO); - DB::CompressedReadBuffer in2(in1); - - std::string tmp; - for (size_t i = 0; i < n && !in2.eof(); ++i) - { - DB::readStringBinary(tmp, in2); - data[i] = StringRef(pool.insert(tmp.data(), tmp.size()), tmp.size()); - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "Vector. Size: " << n - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - if (!m || m == 1) - { - Stopwatch watch; - - using Map = HashMapWithSavedHash; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (StringRef). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - - if (!m || m == 2) - { - Stopwatch watch; - - using Map = HashMapWithSavedHash; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(SmallStringRef(data[i].data, data[i].size), it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap (SmallStringRef). Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" -#ifdef DBMS_HASH_MAP_COUNT_COLLISIONS - << ", collisions: " << map.getCollisions() -#endif - << std::endl; - } - - return 0; -} diff --git a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp b/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp deleted file mode 100644 index 6da69abade0..00000000000 --- a/dbms/src/Interpreters/tests/logical_expressions_optimizer.cpp +++ /dev/null @@ -1,310 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include -#include -#include -#include -#include - -#include -#include -#include -#include - -namespace -{ - -struct TestEntry -{ - std::string input; - std::string expected_output; - UInt64 limit; -}; - -using TestEntries = std::vector; -using TestResult = std::pair; - -void run(); -void performTests(const TestEntries & entries); -TestResult check(const TestEntry & entry); -bool parse(DB::ASTPtr & ast, const std::string & query); -bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs); -void reorder(DB::IAST * ast); - - -void run() -{ - /// NOTE: Queries are not always realistic, but we are only interested in the syntax. - TestEntries entries = - { - { - "SELECT 1", - "SELECT 1", - 3 - }, - - // WHERE - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - 4 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')", - 3 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report WHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report WHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)", - 2 - }, - - { - "SELECT value FROM report WHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)", - "SELECT value FROM report WHERE ((value + 1) IN (1000, 3000)) OR ((2 * value) IN (2000, 4000))", - 2 - }, - - { - "SELECT name, value FROM report WHERE ((name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')) AND ((value = 1000) OR (value = 2000))", - "SELECT name, value FROM report WHERE name IN ('Alice', 'Bob', 'Carol') AND ((value = 1000) OR (value = 2000))", - 3 - }, - - // PREWHERE - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - 4 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')", - 3 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol')", - "SELECT name, value FROM report PREWHERE (value = 1000) OR name IN ('Alice', 'Bob', 'Carol')", - 2 - }, - - { - "SELECT name, value FROM report PREWHERE (name = 'Alice') OR (value = 1000) OR (name = 'Bob') OR (name = 'Carol') OR (value = 2000)", - "SELECT name, value FROM report PREWHERE name IN ('Alice', 'Bob', 'Carol') OR value IN (1000, 2000)", - 2 - }, - - { - "SELECT value FROM report PREWHERE ((value + 1) = 1000) OR ((2 * value) = 2000) OR ((2 * value) = 4000) OR ((value + 1) = 3000)", - "SELECT value FROM report PREWHERE (value + 1) IN (1000, 3000) OR (2 * value) IN (2000, 4000)", - 2 - }, - - // HAVING - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number = 1", - 2 - }, - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING number IN (1, 2)", - 2 - }, - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING (number = 1) OR (number = 2)", - 3 - }, - - { - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 1) = 1) OR ((number + 1) = 2) OR ((number + 3) = 7)", - "SELECT number, count() FROM (SELECT * FROM system.numbers LIMIT 10) GROUP BY number HAVING ((number + 3) = 7) OR (number + 1) IN (1, 2)", - 2 - }, - - // PREWHERE + WHERE + HAVING - - { - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) " - "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2) " - "WHERE T IN (1, 2) GROUP BY number HAVING number IN (1, 2)", - 2 - }, - - { - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE (U = 1) OR (U = 2) OR (U = 3) " - "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)", - "SELECT number, count(), 1 AS T, 2 AS U FROM (SELECT * FROM system.numbers LIMIT 10) PREWHERE U IN (1, 2, 3) " - "WHERE (T = 1) OR (T = 2) GROUP BY number HAVING (number = 1) OR (number = 2)", - 3 - }, - - { - "SELECT x = 1 OR x=2 OR (x = 3 AS x3) AS y, 4 AS x", - "SELECT x IN (1, 2, 3) AS y, 4 AS x", - 2 - } - }; - - performTests(entries); -} - -void performTests(const TestEntries & entries) -{ - unsigned int count = 0; - unsigned int i = 1; - - for (const auto & entry : entries) - { - auto res = check(entry); - if (res.first) - { - ++count; - std::cout << "Test " << i << " passed.\n"; - } - else - std::cout << "Test " << i << " failed. Expected: " << entry.expected_output << ". Received: " << res.second << "\n"; - - ++i; - } - std::cout << count << " out of " << entries.size() << " test(s) passed.\n"; -} - -TestResult check(const TestEntry & entry) -{ - try - { - /// Parse and optimize the incoming query. - DB::ASTPtr ast_input; - if (!parse(ast_input, entry.input)) - return TestResult(false, "parse error"); - - auto select_query = typeid_cast(&*ast_input); - - DB::Settings settings; - settings.optimize_min_equality_disjunction_chain_length = entry.limit; - - DB::LogicalExpressionsOptimizer optimizer(select_query, settings); - optimizer.perform(); - - /// Parse the expected result. - DB::ASTPtr ast_expected; - if (!parse(ast_expected, entry.expected_output)) - return TestResult(false, "parse error"); - - /// Compare the optimized query and the expected result. - bool res = equals(ast_input, ast_expected); - std::string output = DB::queryToString(ast_input); - - return TestResult(res, output); - } - catch (DB::Exception & e) - { - return TestResult(false, e.displayText()); - } -} - -bool parse(DB::ASTPtr & ast, const std::string & query) -{ - DB::ParserSelectQuery parser; - std::string message; - auto begin = query.data(); - auto end = begin + query.size(); - ast = DB::tryParseQuery(parser, begin, end, message, false, "", false, 0); - return ast != nullptr; -} - -bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs) -{ - DB::ASTPtr lhs_reordered = lhs->clone(); - reorder(&*lhs_reordered); - - DB::ASTPtr rhs_reordered = rhs->clone(); - reorder(&*rhs_reordered); - - return lhs_reordered->getTreeHash() == rhs_reordered->getTreeHash(); -} - -void reorderImpl(DB::IAST * ast) -{ - if (ast == nullptr) - return; - - auto & children = ast->children; - if (children.empty()) - return; - - for (auto & child : children) - reorderImpl(&*child); - - std::sort(children.begin(), children.end(), [](const DB::ASTPtr & lhs, const DB::ASTPtr & rhs) - { - return lhs->getTreeHash() < rhs->getTreeHash(); - }); -} - -void reorder(DB::IAST * ast) -{ - if (ast == nullptr) - return; - - auto select_query = typeid_cast(ast); - if (select_query == nullptr) - return; - - reorderImpl(select_query->where_expression.get()); - reorderImpl(select_query->prewhere_expression.get()); - reorderImpl(select_query->having_expression.get()); -} - -} - -int main() -{ - run(); - return 0; -} diff --git a/dbms/src/Interpreters/tests/select_query.cpp b/dbms/src/Interpreters/tests/select_query.cpp deleted file mode 100644 index 2d57d3bee89..00000000000 --- a/dbms/src/Interpreters/tests/select_query.cpp +++ /dev/null @@ -1,69 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include -#include - - -using namespace DB; - -int main(int, char **) -try -{ - Poco::AutoPtr channel = new Poco::ConsoleChannel(std::cerr); - Poco::Logger::root().setChannel(channel); - Poco::Logger::root().setLevel("trace"); - - /// Pre-initialize the `DateLUT` so that the first initialization does not affect the measured execution speed. - DateLUT::instance(); - - Context context = Context::createGlobal(); - - context.setPath("./"); - - loadMetadata(context); - - DatabasePtr system = std::make_shared("system", "./metadata/system/", context); - context.addDatabase("system", system); - system->loadTables(context, nullptr, false); - attachSystemTablesLocal(*context.getDatabase("system")); - context.setCurrentDatabase("default"); - - ReadBufferFromFileDescriptor in(STDIN_FILENO); - WriteBufferFromFileDescriptor out(STDOUT_FILENO); - - executeQuery(in, out, /* allow_into_outfile = */ false, context, {}); - - return 0; -} -catch (const Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl - << std::endl - << "Stack trace:" << std::endl - << e.getStackTrace().toString(); - return 1; -} diff --git a/dbms/src/Interpreters/tests/two_level_hash_map.cpp b/dbms/src/Interpreters/tests/two_level_hash_map.cpp deleted file mode 100644 index 5d7d67d71c2..00000000000 --- a/dbms/src/Interpreters/tests/two_level_hash_map.cpp +++ /dev/null @@ -1,149 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include - -#include - -#include -#include - -//#define DBMS_HASH_MAP_DEBUG_RESIZES - -#include -#include - -#include -#include -#include -#include -#include - - -using Key = UInt64; -using Value = UInt64; - - -int main(int argc, char ** argv) -{ - if (argc < 2) - { - std::cerr << "Usage: program n\n"; - return 1; - } - - size_t n = atoi(argv[1]); - - std::vector data(n); - - std::cerr << "sizeof(Key) = " << sizeof(Key) << ", sizeof(Value) = " << sizeof(Value) << std::endl; - - { - Stopwatch watch; - DB::ReadBufferFromFileDescriptor in1(STDIN_FILENO); - DB::CompressedReadBuffer in2(in1); - - in2.readStrict(reinterpret_cast(&data[0]), sizeof(data[0]) * n); - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "Vector. Size: " << n - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - } - - { - Stopwatch watch; - - std::cerr << sizeof(HashMapCell>) << std::endl; - - using Map = TwoLevelHashTable>, DefaultHash, HashTableGrower<8>, HashTableAllocator>; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(data[i], it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - - size_t sum_counts = 0; - size_t elems = 0; - for (const auto & kv : map) - { - sum_counts += kv.second; - ++elems; - } - - std::cerr << "sum_counts: " << sum_counts << ", elems: " << elems << std::endl; - } - - { - Stopwatch watch; - - using Map = TwoLevelHashTable>, DefaultHash, HashTableGrower<8>, HashTableAllocator>; - //using Map = HashMap; - - Map map; - Map::iterator it; - bool inserted; - - for (size_t i = 0; i < n; ++i) - { - map.emplace(i, it, inserted); - if (inserted) - it->second = 0; - ++it->second; - } - - watch.stop(); - std::cerr << std::fixed << std::setprecision(2) - << "HashMap. Size: " << map.size() - << ", elapsed: " << watch.elapsedSeconds() - << " (" << n / watch.elapsedSeconds() << " elem/sec.)" - << std::endl; - - size_t sum_counts = 0; - size_t elems = 0; - for (const auto & kv : map) - { - sum_counts += kv.second; - ++elems; - - if (kv.first > n) - std::cerr << kv.first << std::endl; - } - - std::cerr << "sum_counts: " << sum_counts << ", elems: " << elems << std::endl; - - if (sum_counts != n) - std::cerr << "!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!" << std::endl; - } - - return 0; -} diff --git a/dbms/src/Interpreters/tests/users.cpp b/dbms/src/Interpreters/tests/users.cpp deleted file mode 100644 index 60df6af4d71..00000000000 --- a/dbms/src/Interpreters/tests/users.cpp +++ /dev/null @@ -1,299 +0,0 @@ -// Copyright 2022 PingCAP, Ltd. -// -// Licensed 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 -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace -{ - -namespace fs = boost::filesystem; - -struct TestEntry -{ - std::string user_name; - std::string database_name; - bool is_allowed; -}; - -using TestEntries = std::vector; - -struct TestDescriptor -{ - const char * config_content; - TestEntries entries; -}; - -using TestSet = std::vector; - -/// Tests description. - -TestSet test_set = -{ - { - "" - " " - " " - " " - " defaultdefault" - " " - " default" - " test" - " " - " " - " " - " defaultdefault" - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", true }, - { "default", "stats", false }, - { "web", "default", true }, - { "web", "test", true }, - { "web", "stats", true }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - }, - - { - "" - " " - " " - " " - " defaultdefault" - " " - " default" - " " - " " - " " - " defaultdefault" - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", false }, - { "default", "stats", false }, - { "web", "default", true }, - { "web", "test", true }, - { "web", "stats", true }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - }, - - { - "" - " " - " " - " " - " defaultdefault" - " " - " " - " " - " " - " defaultdefault" - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", true }, - { "default", "stats", true }, - { "web", "default", true }, - { "web", "test", true }, - { "web", "stats", true }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - }, - - { - "" - " " - " " - " " - " defaultdefault" - " " - " default" - " " - " " - " " - " defaultdefault" - " " - " test" - " " - " " - " " - " " - "", - - { - { "default", "default", true }, - { "default", "test", false }, - { "default", "stats", false }, - { "web", "default", false }, - { "web", "test", true }, - { "web", "stats", false }, - { "analytics", "default", false }, - { "analytics", "test", false }, - { "analytics", "stats", false } - } - } -}; - -std::string createTmpPath(const std::string & filename); -void createFile(const std::string & filename, const char * data); -void runOneTest(const TestDescriptor & test_descriptor); -auto runTestSet(const TestSet & test_set); - -std::string createTmpPath(const std::string & filename) -{ - char pattern[] = "/tmp/fileXXXXXX"; - char * dir = mkdtemp(pattern); - if (dir == nullptr) - throw std::runtime_error("Could not create directory"); - - return std::string(dir) + "/" + filename; -} - -void createFile(const std::string & filename, const char * data) -{ - std::ofstream ofs(filename.c_str()); - if (!ofs.is_open()) - throw std::runtime_error("Could not open file " + filename); - ofs << data; -} - -void runOneTest(const TestDescriptor & test_descriptor) -{ - const auto path_name = createTmpPath("users.xml"); - createFile(path_name, test_descriptor.config_content); - - ConfigurationPtr config; - - try - { - config = ConfigProcessor(path_name).loadConfig().configuration; - } - catch (const Poco::Exception & ex) - { - std::ostringstream os; - os << "Error: " << ex.what() << ": " << ex.displayText(); - throw std::runtime_error(os.str()); - } - - DB::SecurityManager security_manager; - - try - { - security_manager.loadFromConfig(*config); - } - catch (const Poco::Exception & ex) - { - std::ostringstream os; - os << "Error: " << ex.what() << ": " << ex.displayText(); - throw std::runtime_error(os.str()); - } - - for (const auto & entry : test_descriptor.entries) - { - bool res; - - try - { - res = security_manager.hasAccessToDatabase(entry.user_name, entry.database_name); - } - catch (const Poco::Exception &) - { - res = false; - } - - if (res != entry.is_allowed) - { - auto to_string = [](bool access){ return (access ? "'granted'" : "'denied'"); }; - std::ostringstream os; - os << "(user=" << entry.user_name << ", database=" << entry.database_name << "): "; - os << "Expected " << to_string(entry.is_allowed) << " but got " << to_string(res); - throw std::runtime_error(os.str()); - } - } - - fs::remove_all(fs::path(path_name).parent_path().string()); -} - -auto runTestSet(const TestSet & test_set) -{ - size_t test_num = 1; - size_t failure_count = 0; - - for (const auto & test_descriptor : test_set) - { - try - { - runOneTest(test_descriptor); - std::cout << "Test " << test_num << " passed\n"; - } - catch (const std::runtime_error & ex) - { - std::cerr << "Test " << test_num << " failed with reason: " << ex.what() << "\n"; - ++failure_count; - } - catch (...) - { - std::cerr << "Test " << test_num << " failed with unknown reason\n"; - ++failure_count; - } - - ++test_num; - } - - return std::make_tuple(test_set.size(), failure_count); -} - -} - -int main() -{ - size_t test_count; - size_t failure_count; - - std::tie(test_count, failure_count) = runTestSet(test_set); - - std::cout << (test_count - failure_count) << " test(s) passed out of " << test_count << "\n"; - - return (failure_count == 0) ? 0 : EXIT_FAILURE; -} diff --git a/dbms/src/Server/Server.cpp b/dbms/src/Server/Server.cpp index 87f1b55d29f..5e71ee79684 100644 --- a/dbms/src/Server/Server.cpp +++ b/dbms/src/Server/Server.cpp @@ -49,7 +49,6 @@ #include #include #include -#include #include #include #include @@ -976,8 +975,8 @@ int Server::main(const std::vector & /*args*/) global_context->initializePageStorageMode(global_context->getPathPool(), STORAGE_FORMAT_CURRENT.page); // Use pd address to define which default_database we use by default. - // For mock test, we use "default". For deployed with pd/tidb/tikv use "system", which is always exist in TiFlash. - std::string default_database = config().getString("default_database", raft_config.pd_addrs.empty() ? "default" : "system"); + // For deployed with pd/tidb/tikv use "system", which is always exist in TiFlash. + std::string default_database = config().getString("default_database", "system"); Strings all_normal_path = storage_config.getAllNormalPaths(); const std::string path = all_normal_path[0]; global_context->setPath(path); @@ -1206,27 +1205,8 @@ int Server::main(const std::vector & /*args*/) global_context->initializeSharedBlockSchemas(); - { - // Note that this must do before initialize schema sync service. - do - { - // Check whether we need to upgrade directories hierarchy - // If some database can not find in TiDB, they will be dropped - // if theirs name is not in reserved_databases. - // Besides, database engine in reserved_databases just keep as - // what they are. - IDAsPathUpgrader upgrader( - *global_context, - /*is_mock=*/raft_config.pd_addrs.empty(), - /*reserved_databases=*/raft_config.ignore_databases); - if (!upgrader.needUpgrade()) - break; - upgrader.doUpgrade(); - } while (false); - - /// Then, load remaining databases - loadMetadata(*global_context); - } + // Load remaining databases + loadMetadata(*global_context); LOG_DEBUG(log, "Load metadata done."); if (!global_context->isDisaggregatedComputeMode()) diff --git a/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h b/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h index 48c8424ea13..60740424562 100644 --- a/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h +++ b/dbms/src/Storages/DeltaMerge/DeltaIndexManager.h @@ -50,7 +50,7 @@ class DeltaIndexManager size_t current_size = 0; const size_t max_size; - Poco::Logger * log; + LoggerPtr log; std::mutex mutex; @@ -60,7 +60,7 @@ class DeltaIndexManager public: explicit DeltaIndexManager(size_t max_size_) : max_size(max_size_) - , log(&Poco::Logger::get("DeltaIndexManager")) + , log(Logger::get()) {} /// Note that if isLimit() is false, than this method always return 0. diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp index c3dee70e21a..b27e5908cf6 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.cpp +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include #include @@ -102,13 +103,11 @@ String DMFile::ngcPath() const DMFilePtr DMFile::create(UInt64 file_id, const String & parent_path, DMConfigurationOpt configuration) { - Poco::Logger * log = &Poco::Logger::get("DMFile"); // On create, ref_id is the same as file_id. DMFilePtr new_dmfile(new DMFile(file_id, file_id, parent_path, Status::WRITABLE, - log, std::move(configuration))); auto path = new_dmfile->path(); @@ -116,7 +115,7 @@ DMFilePtr DMFile::create(UInt64 file_id, const String & parent_path, DMConfigura if (file.exists()) { file.remove(true); - LOG_WARNING(log, "Existing dmfile, removed: {}", path); + LOG_WARNING(Logger::get(), "Existing dmfile, removed: {}", path); } file.createDirectories(); @@ -146,8 +145,7 @@ DMFilePtr DMFile::restore( file_id, page_id, parent_path, - Status::READABLE, - &Poco::Logger::get("DMFile"))); + Status::READABLE)); if (!read_meta_mode.isNone()) { dmfile->readConfiguration(file_provider); @@ -395,7 +393,7 @@ void DMFile::readColumnStat(const FileProviderPtr & file_provider, const MetaPac } else { - log->warning(fmt::format("checksum for {} not found", name)); + LOG_WARNING(log, "checksum for {} not found", name); } } @@ -486,7 +484,7 @@ void DMFile::readPackProperty(const FileProviderPtr & file_provider, const MetaP } else { - log->warning(fmt::format("checksum for {} not found", name)); + LOG_WARNING(log, "checksum for {} not found", name); } } } @@ -533,7 +531,7 @@ void DMFile::finalizeForFolderMode(const FileProviderPtr & file_provider, const { if (STORAGE_FORMAT_CURRENT.dm_file >= DMFileFormat::V2 && !configuration) { - log->warning("checksum disabled due to lack of configuration"); + LOG_WARNING(log, "checksum disabled due to lack of configuration"); } writeMetadata(file_provider, write_limiter); if (unlikely(status != Status::WRITING)) @@ -569,7 +567,6 @@ std::set DMFile::listAllInPath( std::vector file_names; folder.list(file_names); std::set file_ids; - Poco::Logger * log = &Poco::Logger::get("DMFile"); auto try_parse_file_id = [](const String & name) -> std::optional { std::vector ss; @@ -597,7 +594,7 @@ std::set DMFile::listAllInPath( auto res = try_parse_file_id(name); if (!res) { - LOG_INFO(log, "Unrecognized temporary or dropped dmfile, ignored: {}", name); + LOG_INFO(Logger::get(), "Unrecognized temporary or dropped dmfile, ignored: {}", name); continue; } UInt64 file_id = *res; @@ -606,7 +603,7 @@ std::set DMFile::listAllInPath( const auto full_path = parent_path + "/" + name; if (Poco::File file(full_path); file.exists()) file.remove(true); - LOG_WARNING(log, "Existing temporary or dropped dmfile, removed: {}", full_path); + LOG_WARNING(Logger::get(), "Existing temporary or dropped dmfile, removed: {}", full_path); continue; } } @@ -619,7 +616,7 @@ std::set DMFile::listAllInPath( auto res = try_parse_file_id(name); if (!res) { - LOG_INFO(log, "Unrecognized DM file, ignored: {}", name); + LOG_INFO(Logger::get(), "Unrecognized DM file, ignored: {}", name); continue; } UInt64 file_id = *res; diff --git a/dbms/src/Storages/DeltaMerge/File/DMFile.h b/dbms/src/Storages/DeltaMerge/File/DMFile.h index 06cf1f9502f..3a45597921a 100644 --- a/dbms/src/Storages/DeltaMerge/File/DMFile.h +++ b/dbms/src/Storages/DeltaMerge/File/DMFile.h @@ -287,14 +287,13 @@ class DMFile : private boost::noncopyable UInt64 page_id_, String parent_path_, Status status_, - Poco::Logger * log_, DMConfigurationOpt configuration_ = std::nullopt) : file_id(file_id_) , page_id(page_id_) , parent_path(std::move(parent_path_)) , status(status_) , configuration(std::move(configuration_)) - , log(log_) + , log(Logger::get()) { } @@ -389,7 +388,7 @@ class DMFile : private boost::noncopyable SubFileStats sub_file_stats; - Poco::Logger * log; + LoggerPtr log; friend class DMFileWriter; friend class DMFileReader; diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp index a946768795c..5f8684893e3 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp +++ b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.cpp @@ -11,10 +11,11 @@ // 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 +#include #include #include -#include #include #include @@ -93,7 +94,7 @@ std::vector> getLinuxNumaNodes() return numa_nodes; } -std::vector> getNumaNodes(Poco::Logger * log) +std::vector> getNumaNodes(const LoggerPtr & log) { #ifndef __APPLE__ // Apple macbooks does not support NUMA try diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h index 2d1f564a43a..27439770ec2 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/CPU.h @@ -13,12 +13,16 @@ // limitations under the License. #pragma once -#include - #include +namespace DB +{ +class Logger; +using LoggerPtr = std::shared_ptr; +} // namespace DB + namespace DB::DM { // `getNumaNodes` returns cpus of each Numa node. -std::vector> getNumaNodes(Poco::Logger * log); -} // namespace DB::DM \ No newline at end of file +std::vector> getNumaNodes(const LoggerPtr & log); +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h b/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h index 42e88f01f1e..128ef537cf5 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/MergedTask.h @@ -47,7 +47,7 @@ class MergedTask , inited(false) , cur_idx(-1) , finished_count(0) - , log(&Poco::Logger::get("MergedTask")) + , log(Logger::get()) { passive_merged_segments.fetch_add(units.size() - 1, std::memory_order_relaxed); GET_METRIC(tiflash_storage_read_thread_gauge, type_merged_task).Increment(); @@ -142,7 +142,7 @@ class MergedTask bool inited; int cur_idx; size_t finished_count; - Poco::Logger * log; + LoggerPtr log; Stopwatch sw; inline static std::atomic passive_merged_segments{0}; }; @@ -156,7 +156,7 @@ class MergedTaskPool { public: MergedTaskPool() - : log(&Poco::Logger::get("MergedTaskPool")) + : log(Logger::get()) {} MergedTaskPtr pop(uint64_t pool_id); @@ -166,6 +166,6 @@ class MergedTaskPool private: std::mutex mtx; std::list merged_task_pool; - Poco::Logger * log; + LoggerPtr log; }; -} // namespace DB::DM \ No newline at end of file +} // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp index 99dbbc8caab..11d5e7da6a2 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp +++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.cpp @@ -19,7 +19,7 @@ namespace DB::DM { SegmentReadTaskScheduler::SegmentReadTaskScheduler() : stop(false) - , log(&Poco::Logger::get("SegmentReadTaskScheduler")) + , log(Logger::get()) { sched_thread = std::thread(&SegmentReadTaskScheduler::schedLoop, this); } diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h index c319275fd0f..5373e226609 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReadTaskScheduler.h @@ -78,6 +78,6 @@ class SegmentReadTaskScheduler std::atomic stop; std::thread sched_thread; - Poco::Logger * log; + LoggerPtr log; }; } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp index 2b090e7913d..5981ec05eb9 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp +++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.cpp @@ -11,6 +11,8 @@ // 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 #include #include #include @@ -29,7 +31,7 @@ class SegmentReader SegmentReader(WorkQueue & task_queue_, const std::vector & cpus_) : task_queue(task_queue_) , stop(false) - , log(&Poco::Logger::get(name)) + , log(Logger::get()) , cpus(cpus_) { t = std::thread(&SegmentReader::run, this); @@ -155,7 +157,7 @@ class SegmentReader WorkQueue & task_queue; std::atomic stop; - Poco::Logger * log; + LoggerPtr log; std::thread t; std::vector cpus; }; @@ -171,7 +173,7 @@ void SegmentReaderPool::addTask(MergedTaskPtr && task) } SegmentReaderPool::SegmentReaderPool(int thread_count, const std::vector & cpus) - : log(&Poco::Logger::get("SegmentReaderPool")) + : log(Logger::get()) { LOG_INFO(log, "Create start, thread_count={} cpus={}", thread_count, cpus); for (int i = 0; i < thread_count; i++) @@ -203,7 +205,7 @@ std::vector SegmentReaderPool::getReaderIds() const // ===== SegmentReaderPoolManager ===== // SegmentReaderPoolManager::SegmentReaderPoolManager() - : log(&Poco::Logger::get("SegmentReaderPoolManager")) + : log(Logger::get()) {} SegmentReaderPoolManager::~SegmentReaderPoolManager() = default; diff --git a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h index 65d24865d3d..f57095fa623 100644 --- a/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h +++ b/dbms/src/Storages/DeltaMerge/ReadThread/SegmentReader.h @@ -17,7 +17,12 @@ #include #include #include -#include + +namespace DB +{ +class Logger; +using LoggerPtr = std::shared_ptr; +} // namespace DB namespace DB::DM { @@ -43,7 +48,7 @@ class SegmentReaderPool WorkQueue task_queue; std::vector readers; - Poco::Logger * log; + LoggerPtr log; }; // SegmentReaderPoolManager is a NUMA-aware singleton that manages several SegmentReaderPool objects. @@ -74,7 +79,7 @@ class SegmentReaderPoolManager SegmentReaderPoolManager(); std::vector> reader_pools; std::unordered_set reader_ids; - Poco::Logger * log; + LoggerPtr log; }; } // namespace DB::DM diff --git a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp index dd26d1b0637..52dbfe25fee 100644 --- a/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp +++ b/dbms/src/Storages/DeltaMerge/SchemaUpdate.cpp @@ -221,8 +221,6 @@ void applyAlter(ColumnDefines & table_columns, /// Caller should ensure the command is legal. /// eg. The column to modify/drop/rename must exist, the column to add must not exist, the new column name of rename must not exists. - Poco::Logger * log = &Poco::Logger::get("SchemaUpdate"); - if (command.type == AlterCommand::MODIFY_COLUMN) { // find column define and then apply modify @@ -248,7 +246,7 @@ void applyAlter(ColumnDefines & table_columns, { // Fall back to find column by name, this path should only call by tests. LOG_WARNING( - log, + Logger::get(), "Try to apply alter to column: {}, id: {}," " but not found by id, fall back locating col by name.", command.column_name, @@ -311,7 +309,7 @@ void applyAlter(ColumnDefines & table_columns, } else { - LOG_WARNING(log, "receive unknown alter command, type: {}", static_cast(command.type)); + LOG_WARNING(Logger::get(), "receive unknown alter command, type: {}", static_cast(command.type)); } } diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 1d3016e2853..87271b0e889 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -61,7 +61,7 @@ class IStorage : public std::enable_shared_from_this , public ITableDeclaration { public: - /// The main name of the table type (for example, StorageMergeTree). + /// The main name of the table type (for example, StorageDeltaMerge). virtual std::string getName() const = 0; /** The name of the table. diff --git a/dbms/src/Storages/MarkCache.h b/dbms/src/Storages/MarkCache.h index fe96636a046..c3167486181 100644 --- a/dbms/src/Storages/MarkCache.h +++ b/dbms/src/Storages/MarkCache.h @@ -42,7 +42,7 @@ struct MarksWeightFunction }; -/** Cache of 'marks' for StorageMergeTree. +/** Cache of 'marks' for StorageDeltaMerge. * Marks is an index structure that addresses ranges in column file, corresponding to ranges of primary key. */ class MarkCache : public LRUCache, MarksWeightFunction> diff --git a/dbms/src/Storages/Page/V2/PageFile.cpp b/dbms/src/Storages/Page/V2/PageFile.cpp index e99f96d7cc0..b6dfd57fde8 100644 --- a/dbms/src/Storages/Page/V2/PageFile.cpp +++ b/dbms/src/Storages/Page/V2/PageFile.cpp @@ -94,7 +94,7 @@ std::pair genWriteData( // meta_write_bytes += sizeof(WBSize) + sizeof(PageFormat::Version) + sizeof(WriteBatch::SequenceID); - for (auto & write : wb.getWrites()) + for (auto & write : wb.getMutWrites()) { meta_write_bytes += sizeof(IsPut); // We don't serialize `PUT_EXTERNAL` for V2, just convert it to `PUT` @@ -138,7 +138,7 @@ std::pair genWriteData( // PageUtil::put(meta_pos, wb.getSequence()); PageOffset page_data_file_off = page_file.getDataFileAppendPos(); - for (auto & write : wb.getWrites()) + for (auto & write : wb.getMutWrites()) { // We don't serialize `PUT_EXTERNAL` for V2, just convert it to `PUT` if (write.type == WriteBatchWriteType::PUT_EXTERNAL) diff --git a/dbms/src/Storages/Page/V3/BlobStore.cpp b/dbms/src/Storages/Page/V3/BlobStore.cpp index 8da5bcafd5b..4aa683d6bb1 100644 --- a/dbms/src/Storages/Page/V3/BlobStore.cpp +++ b/dbms/src/Storages/Page/V3/BlobStore.cpp @@ -168,7 +168,7 @@ typename BlobStore::PageEntriesEdit BlobStore::handleLargeWrite(typename Trait::WriteBatch & wb, const WriteLimiterPtr & write_limiter) { PageEntriesEdit edit; - for (auto & write : wb.getWrites()) + for (auto & write : wb.getMutWrites()) { switch (write.type) { @@ -315,7 +315,7 @@ BlobStore::write(typename Trait::WriteBatch & wb, const WriteLimiterPtr & size_t offset_in_allocated = 0; - for (auto & write : wb.getWrites()) + for (auto & write : wb.getMutWrites()) { switch (write.type) { diff --git a/dbms/src/Storages/Page/V3/PageDirectory.cpp b/dbms/src/Storages/Page/V3/PageDirectory.cpp index dfb9f1c740c..1bb6d5e0dce 100644 --- a/dbms/src/Storages/Page/V3/PageDirectory.cpp +++ b/dbms/src/Storages/Page/V3/PageDirectory.cpp @@ -31,6 +31,7 @@ #include #include +#include #include #include #include @@ -442,7 +443,7 @@ VersionedPageEntries::resolveToPageId(UInt64 seq, bool ignore_delete, Pag } else { - LOG_WARNING(&Poco::Logger::get("VersionedPageEntries"), "Can't resolve the EditRecordType {}", static_cast(type)); + LOG_WARNING(Logger::get(), "Can't resolve the EditRecordType {}", magic_enum::enum_name(type)); } return {ResolveResult::FAIL, Trait::PageIdTrait::getInvalidID(), PageVersion(0)}; diff --git a/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h b/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h index bfb09307e40..14fe6b891dc 100644 --- a/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h +++ b/dbms/src/Storages/Page/V3/Universal/UniversalWriteBatch.h @@ -61,17 +61,14 @@ class UniversalWriteBatch : private boost::noncopyable offsets.emplace_back(off, 0); off += data_sz; } - if (unlikely(!data_sizes.empty() && off != size)) - { - throw Exception(fmt::format( - "Try to put Page with fields, but page size and fields total size not match " - "[page_id={}] [num_fields={}] [page_size={}] [all_fields_size={}]", - page_id, - data_sizes.size(), - size, - off), - ErrorCodes::LOGICAL_ERROR); - } + + RUNTIME_CHECK_MSG(data_sizes.empty() || off == size, + "Try to put Page with fields, but page size and fields total size not match " + "[page_id={}] [num_fields={}] [page_size={}] [all_fields_size={}]", + page_id, + data_sizes.size(), + size, + off); Write w{WriteBatchWriteType::PUT, page_id, tag, read_buffer, size, "", std::move(offsets)}; total_data_size += size; @@ -113,7 +110,7 @@ class UniversalWriteBatch : private boost::noncopyable { return writes; } - Writes & getWrites() + Writes & getMutWrites() { return writes; } diff --git a/dbms/src/Storages/Page/WriteBatch.h b/dbms/src/Storages/Page/WriteBatch.h index cb6be591be3..209322ef68b 100644 --- a/dbms/src/Storages/Page/WriteBatch.h +++ b/dbms/src/Storages/Page/WriteBatch.h @@ -173,7 +173,7 @@ class WriteBatch : private boost::noncopyable { return writes; } - Writes & getWrites() + Writes & getMutWrites() { return writes; } diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 0e572b13573..bfe7cbc125d 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -1156,11 +1156,12 @@ void BaseDaemon::initialize(Application & self) void BaseDaemon::logRevision() const { - Logger::root().information("Welcome to TiFlash"); - Logger::root().information("Starting daemon with revision " + Poco::NumberFormatter::format(ClickHouseRevision::get())); + auto * log = &Logger::root(); + LOG_INFO(log, "Welcome to TiFlash"); + LOG_INFO(log, "Starting daemon with revision " + Poco::NumberFormatter::format(ClickHouseRevision::get())); std::stringstream ss; TiFlashBuildInfo::outputDetail(ss); - Logger::root().information("TiFlash build info: " + ss.str()); + LOG_INFO(log, "TiFlash build info: {}", ss.str()); } /// Used for exitOnTaskError() diff --git a/tests/_env.sh b/tests/_env.sh index 7803181c4a1..09e20cb5671 100644 --- a/tests/_env.sh +++ b/tests/_env.sh @@ -36,7 +36,7 @@ export storage_server="127.0.0.1" export storage_port="9000" # Default database for scripts -export storage_db="default" +export storage_db="system" # TiDB address export tidb_server="127.0.0.1" diff --git a/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test b/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test index b7a70d25fa6..a956f9a2775 100644 --- a/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test +++ b/tests/delta-merge-test/query/misc/timestamp_rough_set_filter.test @@ -18,11 +18,14 @@ => DBGInvoke __enable_schema_sync_service('true') => DBGInvoke __drop_tidb_table(default, test) +=> DBGInvoke __drop_tidb_db(default) => drop table if exists default.test +=> drop database if exists default => DBGInvoke __set_flush_threshold(1000000, 1000000) # Data. +=> DBGInvoke __mock_tidb_db(default) => DBGInvoke __mock_tidb_table(default, test, 'col_1 Int64, col_2 default \'asTiDBType|timestamp(5)\'') => DBGInvoke __refresh_schemas() => DBGInvoke __put_region(4, 0, 100, default, test) diff --git a/tests/delta-merge-test/run.sh b/tests/delta-merge-test/run.sh index 53e49d2d418..cd841ebd949 100755 --- a/tests/delta-merge-test/run.sh +++ b/tests/delta-merge-test/run.sh @@ -22,11 +22,7 @@ set -xe check_env -# We need to separate mock-test for dt and tmt, since this behavior -# is different in some tests -# * "tmt" engine ONLY support disable_bg_flush = false. -# * "dt" engine ONLY support disable_bg_flush = true. -# (only tics0 up) (for engine DetlaTree) +# (only tics0 up) docker-compose -f mock-test-dt.yaml down clean_data_log diff --git a/tests/docker/_env.sh b/tests/docker/_env.sh index 1b92af55bb9..abe39c3c699 100755 --- a/tests/docker/_env.sh +++ b/tests/docker/_env.sh @@ -26,7 +26,7 @@ export storage_server="127.0.0.1" export storage_port="9000" # Default database for scripts -export storage_db="default" +export storage_db="system" # TiDB address export tidb_server="tidb0" diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test%2Dtbl/stable/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test%2Dtbl/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/data/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/log/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/meta/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test%2Ddb/test-tbl/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test%2Dtbl/stable/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test%2Dtbl/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/data/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/log/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/meta/.gitignore b/tests/testdata/flash-1136-v3.1.0/data0/data/test/test-tbl/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb.sql deleted file mode 100644 index c97c56faf67..00000000000 --- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH DATABASE `test-db` -ENGINE = Ordinary diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb/test%2Dtbl.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb/test%2Dtbl.sql deleted file mode 100644 index 18be51be751..00000000000 --- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test%2Ddb/test%2Dtbl.sql +++ /dev/null @@ -1,20 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE `test-tbl` -( - pk Nullable(Int32), - _tidb_rowid Int64 -) -ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":66,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}') diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test.sql deleted file mode 100644 index 897b444f60b..00000000000 --- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH DATABASE `test` -ENGINE = Ordinary diff --git a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test/test%2Dtbl.sql b/tests/testdata/flash-1136-v3.1.0/data0/metadata/test/test%2Dtbl.sql deleted file mode 100644 index f6e5c321a91..00000000000 --- a/tests/testdata/flash-1136-v3.1.0/data0/metadata/test/test%2Dtbl.sql +++ /dev/null @@ -1,20 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE `test-tbl` -( - pk Nullable(Int32), - _tidb_rowid Int64 -) -ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":666,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}') diff --git a/tests/testdata/flash-1136-v3.1.0/data1/data/test%2Ddb/test%2Dtbl/stable/.gitignore b/tests/testdata/flash-1136-v3.1.0/data1/data/test%2Ddb/test%2Dtbl/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/data/.gitignore b/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/log/.gitignore b/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/meta/.gitignore b/tests/testdata/flash-1136/data0/data/test%2Ddb/test-tbl/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test-db/test-tbl/stable/.gitignore b/tests/testdata/flash-1136/data0/data/test-db/test-tbl/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/data/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/log/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/meta/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/data/test/test-tbl/stable/.gitignore b/tests/testdata/flash-1136/data0/data/test/test-tbl/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/flash-1136/data0/metadata/test%2Ddb.sql b/tests/testdata/flash-1136/data0/metadata/test%2Ddb.sql deleted file mode 100644 index c97c56faf67..00000000000 --- a/tests/testdata/flash-1136/data0/metadata/test%2Ddb.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH DATABASE `test-db` -ENGINE = Ordinary diff --git a/tests/testdata/flash-1136/data0/metadata/test%2Ddb/test%2Dtbl.sql b/tests/testdata/flash-1136/data0/metadata/test%2Ddb/test%2Dtbl.sql deleted file mode 100644 index 18be51be751..00000000000 --- a/tests/testdata/flash-1136/data0/metadata/test%2Ddb/test%2Dtbl.sql +++ /dev/null @@ -1,20 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE `test-tbl` -( - pk Nullable(Int32), - _tidb_rowid Int64 -) -ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":66,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}') diff --git a/tests/testdata/flash-1136/data0/metadata/test.sql b/tests/testdata/flash-1136/data0/metadata/test.sql deleted file mode 100644 index 897b444f60b..00000000000 --- a/tests/testdata/flash-1136/data0/metadata/test.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH DATABASE `test` -ENGINE = Ordinary diff --git a/tests/testdata/flash-1136/data0/metadata/test/test%2Dtbl.sql b/tests/testdata/flash-1136/data0/metadata/test/test%2Dtbl.sql deleted file mode 100644 index f6e5c321a91..00000000000 --- a/tests/testdata/flash-1136/data0/metadata/test/test%2Dtbl.sql +++ /dev/null @@ -1,20 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE `test-tbl` -( - pk Nullable(Int32), - _tidb_rowid Int64 -) -ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":666,"name":{"L":"test-tbl","O":"test-tbl"},"partition":null,"pk_is_handle":false,"schema_version":36,"state":5,"update_timestamp":417293448885567496}') diff --git a/tests/testdata/flash-1136/data1/data/test-db/test-tbl/stable/.gitignore b/tests/testdata/flash-1136/data1/data/test-db/test-tbl/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/data/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/log/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/meta/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/issue-941/data0/data/test/#hello-world/stable/.gitignore b/tests/testdata/issue-941/data0/data/test/#hello-world/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/issue-941/data0/metadata/test.sql b/tests/testdata/issue-941/data0/metadata/test.sql deleted file mode 100644 index 897b444f60b..00000000000 --- a/tests/testdata/issue-941/data0/metadata/test.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH DATABASE `test` -ENGINE = Ordinary diff --git a/tests/testdata/issue-941/data0/metadata/test/%23hello%2Dworld.sql b/tests/testdata/issue-941/data0/metadata/test/%23hello%2Dworld.sql deleted file mode 100644 index f93ab0a91b4..00000000000 --- a/tests/testdata/issue-941/data0/metadata/test/%23hello%2Dworld.sql +++ /dev/null @@ -1,20 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE `#hello-world` -( - t Nullable(Int32), - _tidb_rowid Int64 -) -ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"t","O":"t"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":45,"name":{"L":"#hello-world","O":"#hello-world"},"partition":null,"pk_is_handle":false,"schema_version":23,"state":5,"update_timestamp":418360557052624904}') diff --git a/tests/testdata/issue-941/data1/data/test/#hello-world/stable/.gitignore b/tests/testdata/issue-941/data1/data/test/#hello-world/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/abc/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/abc/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/abc/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/abc/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/abc/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_49/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_49/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_50/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_50/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_51/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_51/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/data/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/data/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/log/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/log/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/meta/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/meta/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/data/test/emp_bak_52/stable/.gitignore b/tests/testdata/oncall-1651/db/data/test/emp_bak_52/stable/.gitignore deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/tests/testdata/oncall-1651/db/metadata/test.sql b/tests/testdata/oncall-1651/db/metadata/test.sql deleted file mode 100644 index a92a7db72c5..00000000000 --- a/tests/testdata/oncall-1651/db/metadata/test.sql +++ /dev/null @@ -1,16 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH DATABASE test -ENGINE = Ordinary diff --git a/tests/testdata/oncall-1651/db/metadata/test/abc.sql b/tests/testdata/oncall-1651/db/metadata/test/abc.sql deleted file mode 100644 index 9ea2b02b274..00000000000 --- a/tests/testdata/oncall-1651/db/metadata/test/abc.sql +++ /dev/null @@ -1,20 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE abc -( - pk Nullable(Int32), - _tidb_rowid Int64 -) -ENGINE = DeltaMerge(_tidb_rowid, '{"cols":[{"comment":"","default":null,"id":1,"name":{"L":"pk","O":"pk"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":0,"Flen":11,"Tp":3}}],"comment":"","id":45,"name":{"L":"aaa","O":"aaa"},"partition":null,"pk_is_handle":false,"schema_version":23,"state":5,"update_timestamp":417160204721061891}') diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak.sql deleted file mode 100644 index 4be86529f02..00000000000 --- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak.sql +++ /dev/null @@ -1,23 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE emp_bak -( - id Int32, - fname String, - lname String, - store_id Int32, - department_id Int32 -) -ENGINE = DeltaMerge(id, '{"belonging_table_id":-1,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":48,"name":{"L":"emp","O":"emp"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":31,"state":5,"update_timestamp":417160573256204297}') diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_49.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_49.sql deleted file mode 100644 index 315b7f24731..00000000000 --- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_49.sql +++ /dev/null @@ -1,23 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE emp_bak_49 -( - id Int32, - fname String, - lname String, - store_id Int32, - department_id Int32 -) -ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":49,"is_partition_sub_table":true,"name":{"L":"employees_49","O":"employees_49"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}') diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_50.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_50.sql deleted file mode 100644 index 055be9e4d91..00000000000 --- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_50.sql +++ /dev/null @@ -1,23 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE emp_bak_50 -( - id Int32, - fname String, - lname String, - store_id Int32, - department_id Int32 -) -ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":50,"is_partition_sub_table":true,"name":{"L":"employees_50","O":"employees_50"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}') diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_51.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_51.sql deleted file mode 100644 index f96af626e07..00000000000 --- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_51.sql +++ /dev/null @@ -1,23 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE emp_bak_51 -( - id Int32, - fname String, - lname String, - store_id Int32, - department_id Int32 -) -ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":51,"is_partition_sub_table":true,"name":{"L":"employees_51","O":"employees_51"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}') diff --git a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_52.sql b/tests/testdata/oncall-1651/db/metadata/test/emp_bak_52.sql deleted file mode 100644 index 6e1bb90b0fb..00000000000 --- a/tests/testdata/oncall-1651/db/metadata/test/emp_bak_52.sql +++ /dev/null @@ -1,23 +0,0 @@ --- Copyright 2022 PingCAP, Ltd. --- --- Licensed 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. - -ATTACH TABLE emp_bak_52 -( - id Int32, - fname String, - lname String, - store_id Int32, - department_id Int32 -) -ENGINE = DeltaMerge(id, '{"belonging_table_id":48,"cols":[{"comment":"","default":null,"id":1,"name":{"L":"id","O":"id"},"offset":0,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":515,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":2,"name":{"L":"fname","O":"fname"},"offset":1,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":3,"name":{"L":"lname","O":"lname"},"offset":2,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":25,"Tp":15}},{"comment":"","default":null,"id":4,"name":{"L":"store_id","O":"store_id"},"offset":3,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}},{"comment":"","default":null,"id":5,"name":{"L":"department_id","O":"department_id"},"offset":4,"origin_default":null,"state":5,"type":{"Decimal":0,"Elems":null,"Flag":4097,"Flen":11,"Tp":3}}],"comment":"","id":52,"is_partition_sub_table":true,"name":{"L":"employees_52","O":"employees_52"},"partition":{"definitions":[{"comment":"","id":49,"name":{"L":"p0","O":"p0"}},{"comment":"","id":50,"name":{"L":"p1","O":"p1"}},{"comment":"","id":51,"name":{"L":"p2","O":"p2"}},{"comment":"","id":52,"name":{"L":"p3","O":"p3"}}],"enable":true,"expr":"`id`","num":4,"type":1},"pk_is_handle":true,"schema_version":25,"state":5,"update_timestamp":417160265315647498}') From 76e725e107f44c768ac82e61186987265aee6330 Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 9 Feb 2023 13:09:59 +0800 Subject: [PATCH 12/23] fix tsan fail in unit test `JoinExecutorTestRunner.MultiJoin` (#6770) close pingcap/tiflash#6765 --- dbms/src/Interpreters/Join.cpp | 1 + dbms/src/Interpreters/Join.h | 12 ++++++++++-- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index f54b8bd51c3..b1eb6139128 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -2136,6 +2136,7 @@ bool Join::needReturnNonJoinedData() const void Join::joinTotals(Block & block) const { + std::shared_lock lock(rwlock); Block totals_without_keys = totals; if (totals_without_keys) diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index c3e6e218b5c..abae6268430 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -122,8 +122,16 @@ class Join /** Keep "totals" (separate part of dataset, see WITH TOTALS) to use later. */ - void setTotals(const Block & block) { totals = block; } - bool hasTotals() const { return static_cast(totals); }; + void setTotals(const Block & block) + { + std::unique_lock lock(rwlock); + totals = block; + } + bool hasTotals() const + { + std::shared_lock lock(rwlock); + return static_cast(totals); + }; void joinTotals(Block & block) const; From c5ea6786a1ac95efa0e1a889c599483e5ace06da Mon Sep 17 00:00:00 2001 From: SeaRise Date: Thu, 9 Feb 2023 14:18:04 +0800 Subject: [PATCH 13/23] Refine the hostname backlist in `MetricsPrometheus` (#6784) ref pingcap/tiflash#6678 --- dbms/src/Server/MetricsPrometheus.cpp | 13 +++++++++++-- etc/config-template.toml | 4 ++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/dbms/src/Server/MetricsPrometheus.cpp b/dbms/src/Server/MetricsPrometheus.cpp index 2a1c1bae42d..c3248632e84 100644 --- a/dbms/src/Server/MetricsPrometheus.cpp +++ b/dbms/src/Server/MetricsPrometheus.cpp @@ -59,8 +59,17 @@ std::string getInstanceValue(const Poco::Util::AbstractConfiguration & conf) auto service_addr = conf.getString("flash.service_addr"); if (service_addr.empty()) return getHostName(); - // "0.0.0.0", "127.x.x.x", "locallhost", "0:0:0:0:0:0:0:0", "0:0:0:0:0:0:0:1", "::", "::1", ":${port}" - static const std::vector blacklist{"0.0.0.0", "127.", "locallhost", "0:0:0:0:0:0:0", ":"}; + // "0.0.0.0", "127.x.x.x", "localhost", "0:0:0:0:0:0:0:0", "0:0:0:0:0:0:0:1", "::", "::1", ":${port}" + static const std::vector blacklist{ + // ivp4 + "0.0.0.0", + "127.", + "localhost", + // ipv6 + "0:0:0:0:0:0:0", + "[0:0:0:0:0:0:0", + ":", + "[:"}; for (const auto & prefix : blacklist) { if (startsWith(service_addr, prefix)) diff --git a/etc/config-template.toml b/etc/config-template.toml index 0c89597c5bf..ad13fca53d0 100644 --- a/etc/config-template.toml +++ b/etc/config-template.toml @@ -122,6 +122,10 @@ # advertise-status-addr = "tiflash0:20292" # engine-addr = "tiflash0:3930" +# [cluster] +# Used to specify which cluster the tiflash compute node belongs to. +# cluster_id = "${cluster_id}" + [logger] # log = "/tmp/tiflash/log/server.log" # level = "debug" From 0d25d4da382182a62b6a899672944b33db7c656e Mon Sep 17 00:00:00 2001 From: xufei Date: Thu, 9 Feb 2023 15:20:00 +0800 Subject: [PATCH 14/23] Add more information to the error message in MPPTunnel (#6787) ref pingcap/tiflash#6233 --- dbms/src/Flash/Mpp/ExchangeReceiver.cpp | 31 +++++++++++++------- dbms/src/Flash/Mpp/MPPTunnel.cpp | 18 ++++++------ dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp | 22 +++++++------- 3 files changed, 41 insertions(+), 30 deletions(-) diff --git a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp index 6e254bd62dc..9f0bd8e99dc 100644 --- a/dbms/src/Flash/Mpp/ExchangeReceiver.cpp +++ b/dbms/src/Flash/Mpp/ExchangeReceiver.cpp @@ -834,6 +834,7 @@ void ExchangeReceiverBase::connectionDone( const LoggerPtr & log) { Int32 copy_live_connections; + String first_err_msg = local_err_msg; { std::lock_guard lock(mu); @@ -843,29 +844,39 @@ void ExchangeReceiverBase::connectionDone( state = ExchangeReceiverState::ERROR; if (err_msg.empty()) err_msg = local_err_msg; + else + first_err_msg = err_msg; } copy_live_connections = --live_connections; } - LOG_DEBUG( - log, - "connection end. meet error: {}, err msg: {}, current alive connections: {}", - meet_error, - local_err_msg, - copy_live_connections); - + if (meet_error) + { + LOG_WARNING( + log, + "connection end. meet error: {}, err msg: {}, current alive connections: {}", + meet_error, + local_err_msg, + copy_live_connections); + } + else + { + LOG_DEBUG( + log, + "connection end. Current alive connections: {}", + copy_live_connections); + } + assert(copy_live_connections >= 0); if (copy_live_connections == 0) { LOG_DEBUG(log, "All threads end in ExchangeReceiver"); cv.notify_all(); } - else if (copy_live_connections < 0) - throw Exception("alive_connection_num should not be less than 0!"); if (meet_error || copy_live_connections == 0) { - LOG_INFO(exc_log, "receiver channels finished"); + LOG_INFO(exc_log, "receiver channels finished, meet error: {}, error message: {}", meet_error, first_err_msg); finishAllMsgChannels(); } } diff --git a/dbms/src/Flash/Mpp/MPPTunnel.cpp b/dbms/src/Flash/Mpp/MPPTunnel.cpp index 5a56f47368e..4978e6f4f27 100644 --- a/dbms/src/Flash/Mpp/MPPTunnel.cpp +++ b/dbms/src/Flash/Mpp/MPPTunnel.cpp @@ -157,7 +157,7 @@ void MPPTunnel::write(TrackedMppDataPacketPtr && data) { std::unique_lock lk(mu); waitUntilConnectedOrFinished(lk); - RUNTIME_CHECK_MSG(tunnel_sender != nullptr, "write to tunnel which is already closed."); + RUNTIME_CHECK_MSG(tunnel_sender != nullptr, "write to tunnel {} which is already closed.", tunnel_id); } auto pushed_data_size = data->getPacket().ByteSizeLong(); @@ -167,7 +167,7 @@ void MPPTunnel::write(TrackedMppDataPacketPtr && data) updateConnProfileInfo(pushed_data_size); return; } - throw Exception(fmt::format("write to tunnel which is already closed,{}", tunnel_sender->isConsumerFinished() ? tunnel_sender->getConsumerFinishMsg() : "")); + throw Exception(fmt::format("write to tunnel {} which is already closed, {}", tunnel_id, tunnel_sender->isConsumerFinished() ? tunnel_sender->getConsumerFinishMsg() : "")); } /// done normally and being called exactly once after writing all packets @@ -179,7 +179,7 @@ void MPPTunnel::writeDone() /// make sure to finish the tunnel after it is connected waitUntilConnectedOrFinished(lk); if (tunnel_sender == nullptr) - throw Exception(fmt::format("write to tunnel which is already closed.")); + throw Exception(fmt::format("write to tunnel {} which is already closed.", tunnel_id)); } tunnel_sender->finish(); waitForSenderFinish(/*allow_throw=*/true); @@ -208,8 +208,8 @@ void MPPTunnel::connectLocalV2(size_t source_index, LocalRequestHandler & local_ { { std::unique_lock lk(mu); - RUNTIME_CHECK_MSG(status == TunnelStatus::Unconnected, fmt::format("MPPTunnel has connected or finished: {}", statusToString())); - RUNTIME_CHECK_MSG(mode == TunnelSenderMode::LOCAL, "This should be a local tunnel"); + RUNTIME_CHECK_MSG(status == TunnelStatus::Unconnected, fmt::format("MPPTunnel {} has connected or finished: {}", tunnel_id, statusToString())); + RUNTIME_CHECK_MSG(mode == TunnelSenderMode::LOCAL, "{} should be a local tunnel", tunnel_id); LOG_TRACE(log, "ready to connect local tunnel version 2"); if (is_fine_grained) @@ -233,7 +233,7 @@ void MPPTunnel::connectAsync(IAsyncCallData * call_data) { { std::unique_lock lk(mu); - RUNTIME_CHECK_MSG(status == TunnelStatus::Unconnected, "MPPTunnel has connected or finished: {}", statusToString()); + RUNTIME_CHECK_MSG(status == TunnelStatus::Unconnected, "MPPTunnel {} has connected or finished: {}", tunnel_id, statusToString()); LOG_TRACE(log, "ready to connect async"); RUNTIME_ASSERT(mode == TunnelSenderMode::ASYNC_GRPC, log, "mode {} is not async grpc in connectAsync", magic_enum::enum_name(mode)); @@ -283,7 +283,7 @@ void MPPTunnel::waitForSenderFinish(bool allow_throw) status = TunnelStatus::Finished; } if (allow_throw && !err_msg.empty()) - throw Exception("Consumer exits unexpected, " + err_msg); + throw Exception(fmt::format("{}: consumer exits unexpected, error message: {} ", tunnel_id, err_msg)); LOG_TRACE(log, "end wait for consumer finish!"); } @@ -311,7 +311,7 @@ void MPPTunnel::waitUntilConnectedOrFinished(std::unique_lock & lk) LOG_TRACE(log, "end waitUntilConnectedOrFinished"); } if (status == TunnelStatus::Unconnected) - throw Exception("MPPTunnel can not be connected because MPPTask is cancelled"); + throw Exception(fmt::format("MPPTunnel {} can not be connected because MPPTask is cancelled", tunnel_id)); } StringRef MPPTunnel::statusToString() @@ -399,7 +399,7 @@ void MPPTunnel::connectLocalV1(PacketWriter * writer) { std::unique_lock lk(mu); if (status != TunnelStatus::Unconnected) - throw Exception(fmt::format("MPPTunnel has connected or finished: {}", statusToString())); + throw Exception(fmt::format("MPPTunnel {} has connected or finished: {}", tunnel_id, statusToString())); LOG_TRACE(log, "ready to connect local tunnel version 1"); diff --git a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp index 5fa98e93a26..28bdc3921ab 100644 --- a/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp +++ b/dbms/src/Flash/Mpp/tests/gtest_mpptunnel.cpp @@ -377,7 +377,7 @@ TEST_F(TestMPPTunnel, SyncWriteAfterUnconnectFinished) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Check tunnel_sender != nullptr failed: write to tunnel which is already closed."); + GTEST_ASSERT_EQ(e.message(), "Check tunnel_sender != nullptr failed: write to tunnel 0000_0001 which is already closed."); } } @@ -392,7 +392,7 @@ TEST_F(TestMPPTunnel, SyncWriteDoneAfterUnconnectFinished) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "write to tunnel which is already closed."); + GTEST_ASSERT_EQ(e.message(), "write to tunnel 0000_0001 which is already closed."); } } @@ -460,7 +460,7 @@ TEST_F(TestMPPTunnel, SyncWriteError) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed."); + GTEST_ASSERT_EQ(e.message(), "0000_0001: consumer exits unexpected, error message: 0000_0001 meet error: grpc writes failed. "); } } @@ -481,7 +481,7 @@ TEST_F(TestMPPTunnel, SyncWriteAfterFinished) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "write to tunnel which is already closed,"); + GTEST_ASSERT_EQ(e.message(), "write to tunnel 0000_0001 which is already closed, "); } if (mpp_tunnel_ptr != nullptr) mpp_tunnel_ptr->waitForFinish(); @@ -573,7 +573,7 @@ TEST_F(TestMPPTunnel, AsyncWriteError) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, 0000_0001 meet error: grpc writes failed."); + GTEST_ASSERT_EQ(e.message(), "0000_0001: consumer exits unexpected, error message: 0000_0001 meet error: grpc writes failed. "); } } @@ -660,7 +660,7 @@ try } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Check status == TunnelStatus::Unconnected failed: MPPTunnel has connected or finished: Finished"); + GTEST_ASSERT_EQ(e.message(), "Check status == TunnelStatus::Unconnected failed: MPPTunnel 0000_0001 has connected or finished: Finished"); } TEST_F(TestMPPTunnel, LocalConnectWhenConnected) @@ -679,7 +679,7 @@ try } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Check status == TunnelStatus::Unconnected failed: MPPTunnel has connected or finished: Connected"); + GTEST_ASSERT_EQ(e.message(), "Check status == TunnelStatus::Unconnected failed: MPPTunnel 0000_0001 has connected or finished: Connected"); } TEST_F(TestMPPTunnel, LocalCloseBeforeConnect) @@ -713,7 +713,7 @@ try } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Check tunnel_sender != nullptr failed: write to tunnel which is already closed."); + GTEST_ASSERT_EQ(e.message(), "Check tunnel_sender != nullptr failed: write to tunnel 0000_0001 which is already closed."); } TEST_F(TestMPPTunnel, LocalWriteDoneAfterUnconnectFinished) @@ -726,7 +726,7 @@ try } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "write to tunnel which is already closed."); + GTEST_ASSERT_EQ(e.message(), "write to tunnel 0000_0001 which is already closed."); } TEST_F(TestMPPTunnel, LocalWriteError) @@ -756,7 +756,7 @@ try } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "Consumer exits unexpected, err"); + GTEST_ASSERT_EQ(e.message(), "0000_0001: consumer exits unexpected, error message: err "); } TEST_F(TestMPPTunnel, LocalWriteAfterFinished) @@ -775,7 +775,7 @@ TEST_F(TestMPPTunnel, LocalWriteAfterFinished) } catch (Exception & e) { - GTEST_ASSERT_EQ(e.message(), "write to tunnel which is already closed,"); + GTEST_ASSERT_EQ(e.message(), "write to tunnel 0000_0001 which is already closed, "); } if (tunnel != nullptr) tunnel->waitForFinish(); From 206b807e24d50cbc99e1ddaa26a69d9b13154ac8 Mon Sep 17 00:00:00 2001 From: Zhi Qi <30543181+LittleFall@users.noreply.github.com> Date: Thu, 9 Feb 2023 16:35:59 +0800 Subject: [PATCH 15/23] fix the issue that decimal divide not round. (#6471) close pingcap/tiflash#4488, close pingcap/tiflash#6393, close pingcap/tiflash#6462 --- dbms/src/Functions/divide.cpp | 37 ++++- .../tests/gtest_arithmetic_functions.cpp | 137 ++++++++++++++++++ tests/fullstack-test/expr/decimal_divide.test | 137 ++++++++++++++++++ .../tidb-ci/fullstack-test-dt/issue_1425.test | 10 +- 4 files changed, 314 insertions(+), 7 deletions(-) create mode 100644 tests/fullstack-test/expr/decimal_divide.test diff --git a/dbms/src/Functions/divide.cpp b/dbms/src/Functions/divide.cpp index 53c5de79448..132f6fae623 100644 --- a/dbms/src/Functions/divide.cpp +++ b/dbms/src/Functions/divide.cpp @@ -60,9 +60,36 @@ struct TiDBDivideFloatingImpl using ResultType = typename NumberTraits::ResultOfFloatingPointDivision::Type; template - static Result apply(A a, B b) + static Result apply(A x, B d) { - return static_cast(a) / b; + /// ref https://github.com/pingcap/tiflash/issues/6462 + /// For division of Decimal/Decimal or Int/Decimal or Decimal/Int, we should round the result to make compatible with TiDB. + /// basically refer to https://stackoverflow.com/a/71634489 + if constexpr (std::is_integral_v || std::is_same_v) + { + /// 1. do division first, get the quotient and mod, todo:(perf) find a unified `divmod` function to speed up this. + Result quotient = x / d; + Result mod = x % d; + /// 2. get the half of divisor, which is threshold to decide whether to round up or down. + /// note: don't directly use bit operation here, it may cause unexpected result. + Result half = (d / 2) + (d % 2); + + /// 3. compare the abstract values of mod and half, if mod >= half, then round up. + Result abs_m = mod < 0 ? -mod : mod; + Result abs_h = half < 0 ? -half : half; + if (abs_m >= abs_h) + { + /// 4. now we need to round up, i.e., add 1 to the quotient's absolute value. + /// if the signs of dividend and divisor are the same, then the quotient should be positive, otherwise negative. + if ((x < 0) == (d < 0)) // same_sign, i.e., quotient >= 0 + quotient = quotient + 1; + else + quotient = quotient - 1; + } + return quotient; + } + else + return static_cast(x) / d; } template static Result apply(A a, B b, UInt8 & res_null) @@ -75,7 +102,7 @@ struct TiDBDivideFloatingImpl res_null = 1; return static_cast(0); } - return static_cast(a) / b; + return apply(a, b); } }; @@ -102,7 +129,7 @@ struct TiDBDivideFloatingImpl res_null = 1; return static_cast(0); } - return static_cast(a) / static_cast(b); + return apply(a, b); } }; @@ -332,4 +359,4 @@ void registerFunctionDivideIntegralOrZero(FunctionFactory & factory) factory.registerFunction(); } -} // namespace DB \ No newline at end of file +} // namespace DB diff --git a/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp b/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp index 1d548a4c2d2..4ba24d677f3 100644 --- a/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp +++ b/dbms/src/Functions/tests/gtest_arithmetic_functions.cpp @@ -19,7 +19,9 @@ #include #include #include +#include +#include #include #include #include @@ -103,6 +105,141 @@ class TestBinaryArithmeticFunctions : public DB::tests::FunctionTest } }; +template +void doTiDBDivideDecimalRoundInternalTest() +{ + auto apply = static_cast(&TiDBDivideFloatingImpl::apply); + + constexpr TYPE max = std::numeric_limits::max(); + // note: Int256's min is not equal to -max-1 + // according to https://www.boost.org/doc/libs/1_60_0/libs/multiprecision/doc/html/boost_multiprecision/tut/ints/cpp_int.html + constexpr TYPE min = std::numeric_limits::min(); + + // clang-format off + const std::vector> cases = { + {1, 2, 1}, {1, -2, -1}, {-1, 2, -1}, {-1, -2, 1}, + + {0, 3, 0}, {0, -3, 0}, {0, 3, 0}, {0, -3, 0}, + {1, 3, 0}, {1, -3, 0}, {-1, 3, 0}, {-1, -3, 0}, + {2, 3, 1}, {2, -3, -1}, {-2, 3, -1}, {-2, -3, 1}, + {3, 3, 1}, {3, -3, -1}, {-3, 3, -1}, {-3, -3, 1}, + {4, 3, 1}, {4, -3, -1}, {-4, 3, -1}, {-4, -3, 1}, + {5, 3, 2}, {5, -3, -2}, {-5, 3, -2}, {-5, -3, 2}, + + // ±max as divisor + {0, max, 0}, {max/2-1, max, 0}, {max/2, max, 0}, {max/2+1, max, 1}, {max-1, max, 1}, {max, max, 1}, + {-1, max, 0}, {-max/2+1, max, 0}, {-max/2, max, 0}, {-max/2-1, max, -1}, {-max+1, max, -1}, {-max, max, -1}, {min, max, -1}, + {0, -max, 0}, {max/2-1, -max, 0}, {max/2, -max, 0}, {max/2+1, -max, -1}, {max-1, -max, -1}, {max, -max, -1}, + {-1, -max, 0}, {-max/2+1, -max, 0}, {-max/2, -max, 0}, {-max/2-1, -max, 1}, {-max+1, -max, 1}, {-max, -max, 1}, {min, -max, 1}, + + // ±max as dividend + {max, 1, max}, {max, 2, max/2+1}, {max, max/2-1, 2}, {max, max/2, 2}, {max, max/2+1, 2}, {max, max-1, 1}, + {max, -1, -max}, {max, -2, -max/2-1}, {max, -max/2+1, -2}, {max, -max/2, -2}, {max, -max/2-1, -2}, {max, -max+1, -1}, + {-max, 1, -max}, {-max, 2, -max/2-1}, {-max, max/2+1, -2}, {-max, max/2, -2}, {-max, max/2-1, -2}, {-max, max-1, -1}, + {-max, -1, max}, {-max, -2, max/2+1}, {-max, -max/2-1, 2}, {-max, -max/2, 2}, {-max, -max/2+1, 2}, {-max, -max+1, 1}, + }; + // clang-format on + + for (const auto & expect : cases) + { + std::array actual = {expect[0], expect[1], apply(expect[0], expect[1])}; + ASSERT_EQ(expect, actual); + } +} + +TEST_F(TestBinaryArithmeticFunctions, TiDBDivideDecimalRoundInternal) +try +{ + doTiDBDivideDecimalRoundInternalTest(); + doTiDBDivideDecimalRoundInternalTest(); + doTiDBDivideDecimalRoundInternalTest(); + doTiDBDivideDecimalRoundInternalTest(); +} +CATCH + +TEST_F(TestBinaryArithmeticFunctions, TiDBDivideDecimalRound) +try +{ + const String func_name = "tidbDivide"; + + // decimal32 + { + // int and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(18, 4), {DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(0, 4)}), + executeFunction( + func_name, + createColumn({1, 1, 1, 1, 1}), + createColumn(std::make_tuple(20, 4), {DecimalField32(100000000, 4), DecimalField32(100010000, 4), DecimalField32(199990000, 4), DecimalField32(200000000, 4), DecimalField32(200010000, 4)}))); + + // decimal and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(26, 8), {DecimalField128(10000, 8), DecimalField128(9999, 8), DecimalField128(5000, 8), DecimalField128(5000, 8), DecimalField128(5000, 8)}), + executeFunction( + func_name, + createColumn(std::make_tuple(18, 4), {DecimalField32(10000, 4), DecimalField32(10000, 4), DecimalField32(10000, 4), DecimalField32(10000, 4), DecimalField32(10000, 4)}), + createColumn(std::make_tuple(18, 4), {DecimalField32(100000000, 4), DecimalField32(100010000, 4), DecimalField32(199990000, 4), DecimalField32(200000000, 4), DecimalField32(200010000, 4)}))); + } + + // decimal64 + { + // int and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(18, 4), {DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(0, 4)}), + executeFunction( + func_name, + createColumn({1, 1, 1, 1, 1}), + createColumn(std::make_tuple(20, 4), {DecimalField64(100000000, 4), DecimalField64(100010000, 4), DecimalField64(199990000, 4), DecimalField64(200000000, 4), DecimalField64(200010000, 4)}))); + + // decimal and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(26, 8), {DecimalField128(10000, 8), DecimalField128(9999, 8), DecimalField128(5000, 8), DecimalField128(5000, 8), DecimalField128(5000, 8)}), + executeFunction( + func_name, + createColumn(std::make_tuple(18, 4), {DecimalField64(10000, 4), DecimalField64(10000, 4), DecimalField64(10000, 4), DecimalField64(10000, 4), DecimalField64(10000, 4)}), + createColumn(std::make_tuple(18, 4), {DecimalField64(100000000, 4), DecimalField64(100010000, 4), DecimalField64(199990000, 4), DecimalField64(200000000, 4), DecimalField64(200010000, 4)}))); + } + + // decimal128 + { + // int and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(18, 4), {DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(0, 4)}), + executeFunction( + func_name, + createColumn({1, 1, 1, 1, 1}), + createColumn(std::make_tuple(20, 4), {DecimalField128(100000000, 4), DecimalField128(100010000, 4), DecimalField128(199990000, 4), DecimalField128(200000000, 4), DecimalField128(200010000, 4)}))); + + // decimal and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(26, 8), {DecimalField128(10000, 8), DecimalField128(9999, 8), DecimalField128(5000, 8), DecimalField128(5000, 8), DecimalField128(5000, 8)}), + executeFunction( + func_name, + createColumn(std::make_tuple(18, 4), {DecimalField128(10000, 4), DecimalField128(10000, 4), DecimalField128(10000, 4), DecimalField128(10000, 4), DecimalField128(10000, 4)}), + createColumn(std::make_tuple(18, 4), {DecimalField128(100000000, 4), DecimalField128(100010000, 4), DecimalField128(199990000, 4), DecimalField128(200000000, 4), DecimalField128(200010000, 4)}))); + } + + // decimal256 + { + // int and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(18, 4), {DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(1, 4), DecimalField64(0, 4)}), + executeFunction( + func_name, + createColumn({1, 1, 1, 1, 1}), + createColumn(std::make_tuple(20, 4), {DecimalField256(Int256(100000000), 4), DecimalField256(Int256(100010000), 4), DecimalField256(Int256(199990000), 4), DecimalField256(Int256(200000000), 4), DecimalField256(Int256(200010000), 4)}))); + + // decimal and decimal + ASSERT_COLUMN_EQ( + createColumn>(std::make_tuple(26, 8), {DecimalField128(10000, 8), DecimalField128(9999, 8), DecimalField128(5000, 8), DecimalField128(5000, 8), DecimalField128(5000, 8)}), + executeFunction( + func_name, + createColumn(std::make_tuple(18, 4), {DecimalField256(Int256(10000), 4), DecimalField256(Int256(10000), 4), DecimalField256(Int256(10000), 4), DecimalField256(Int256(10000), 4), DecimalField256(Int256(10000), 4)}), + createColumn(std::make_tuple(18, 4), {DecimalField256(Int256(100000000), 4), DecimalField256(Int256(100010000), 4), DecimalField256(Int256(199990000), 4), DecimalField256(Int256(200000000), 4), DecimalField256(Int256(200010000), 4)}))); + } +} +CATCH + TEST_F(TestBinaryArithmeticFunctions, TiDBDivideDecimal) try { diff --git a/tests/fullstack-test/expr/decimal_divide.test b/tests/fullstack-test/expr/decimal_divide.test new file mode 100644 index 00000000000..98056ca556f --- /dev/null +++ b/tests/fullstack-test/expr/decimal_divide.test @@ -0,0 +1,137 @@ +# Copyright 2023 PingCAP, Ltd. +# +# Licensed 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. + +# decimal / decimal +mysql> drop table if exists test.t; +mysql> create table test.t(a decimal(4,0), b decimal(40, 20)); +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values (1, 10000), (1, 10001), (1, 20000), (1, 20001); +func> wait_table test t +mysql> set tidb_enforce_mpp=1; select a, b, a/b from test.t order by b; ++------+----------------------------+--------+ +| a | b | a/b | ++------+----------------------------+--------+ +| 1 | 10000.00000000000000000000 | 0.0001 | +| 1 | 10001.00000000000000000000 | 0.0001 | +| 1 | 20000.00000000000000000000 | 0.0001 | +| 1 | 20001.00000000000000000000 | 0.0000 | ++------+----------------------------+--------+ + +# int / decimal +mysql> drop table if exists test.t; +mysql> create table test.t(a int, b decimal(40, 20)); +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values (1, 10000), (1, 10001), (1, 20000), (1, 20001); +func> wait_table test t +mysql> set tidb_enforce_mpp=1; select a, b, a/b from test.t order by b; ++------+----------------------------+--------+ +| a | b | a/b | ++------+----------------------------+--------+ +| 1 | 10000.00000000000000000000 | 0.0001 | +| 1 | 10001.00000000000000000000 | 0.0001 | +| 1 | 20000.00000000000000000000 | 0.0001 | +| 1 | 20001.00000000000000000000 | 0.0000 | ++------+----------------------------+--------+ + +# decimal / int +mysql> drop table if exists test.t; +mysql> create table test.t(a int, b decimal(40, 20)); +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values (1, 10000), (1, 10001), (1, 20000), (1, 20001); +func> wait_table test t +mysql> set tidb_enforce_mpp=1; select a, b, a/b from test.t order by b; ++------+----------------------------+--------+ +| a | b | a/b | ++------+----------------------------+--------+ +| 1 | 10000.00000000000000000000 | 0.0001 | +| 1 | 10001.00000000000000000000 | 0.0001 | +| 1 | 20000.00000000000000000000 | 0.0001 | +| 1 | 20001.00000000000000000000 | 0.0000 | ++------+----------------------------+--------+ + +# int / int +mysql> drop table if exists test.t; +mysql> create table test.t(a int, b int); +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values (1, 10000), (1, 10001), (1, 20000), (1, 20001); +func> wait_table test t +mysql> set tidb_enforce_mpp=1; select a, b, a/b from test.t order by b; ++------+-------+--------+ +| a | b | a/b | ++------+-------+--------+ +| 1 | 10000 | 0.0001 | +| 1 | 10001 | 0.0001 | +| 1 | 20000 | 0.0001 | +| 1 | 20001 | 0.0000 | ++------+-------+--------+ + +mysql> drop table if exists test.t; +mysql> create table test.t(a decimal(10,0), b decimal(10,0)); +mysql> alter table test.t set tiflash replica 1 +mysql> insert into test.t values (2147483647, 1), (2147483647, 1073741823), (2147483647, 1073741824), (2147483647, 2147483646), (2147483647, 2147483647); +mysql> insert into test.t values (-2147483647, 1), (-2147483647, 1073741823), (-2147483647, 1073741824), (-2147483647, 2147483646), (-2147483647, 2147483647); +mysql> insert into test.t values (-2147483647, -1), (-2147483647, -1073741823), (-2147483647, -1073741824), (-2147483647, -2147483646), (-2147483647, -2147483647); +mysql> insert into test.t values (2147483647, -1), (2147483647, -1073741823), (2147483647, -1073741824), (2147483647, -2147483646), (2147483647, -2147483647); +func> wait_table test t +mysql> set tidb_enforce_mpp=1; select b, a, b/(a*10000) from test.t where a/b order by b; ++-------------+-------------+-------------+ +| b | a | b/(a*10000) | ++-------------+-------------+-------------+ +| -2147483647 | 2147483647 | -0.0001 | +| -2147483647 | -2147483647 | 0.0001 | +| -2147483646 | 2147483647 | -0.0001 | +| -2147483646 | -2147483647 | 0.0001 | +| -1073741824 | 2147483647 | -0.0001 | +| -1073741824 | -2147483647 | 0.0001 | +| -1073741823 | -2147483647 | 0.0000 | +| -1073741823 | 2147483647 | 0.0000 | +| -1 | 2147483647 | 0.0000 | +| -1 | -2147483647 | 0.0000 | +| 1 | -2147483647 | 0.0000 | +| 1 | 2147483647 | 0.0000 | +| 1073741823 | -2147483647 | 0.0000 | +| 1073741823 | 2147483647 | 0.0000 | +| 1073741824 | -2147483647 | -0.0001 | +| 1073741824 | 2147483647 | 0.0001 | +| 2147483646 | -2147483647 | -0.0001 | +| 2147483646 | 2147483647 | 0.0001 | +| 2147483647 | -2147483647 | -0.0001 | +| 2147483647 | 2147483647 | 0.0001 | ++-------------+-------------+-------------+ +mysql> delete from test.t; +mysql> insert into test.t values (2147483647, 9999999999), (9999999999, 2147483647), (1, 9999999999), (4999999999, 9999999999), (5000000000, 9999999999); +mysql> insert into test.t values (-2147483647, 9999999999), (-9999999999, 2147483647), (-1, 9999999999), (-4999999999, 9999999999), (-5000000000, 9999999999); +mysql> insert into test.t values (-2147483647, -9999999999), (-9999999999, -2147483647), (-1, -9999999999), (-4999999999, -9999999999), (-5000000000, -9999999999); +mysql> insert into test.t values (2147483647, -9999999999), (9999999999, -2147483647), (1, -9999999999), (4999999999, -9999999999), (5000000000, -9999999999); +mysql> set tidb_enforce_mpp=1; select b, a, b/(a*10000) from test.t where a/b order by b; ++-------------+-------------+-------------+ +| b | a | b/(a*10000) | ++-------------+-------------+-------------+ +| -9999999999 | 2147483647 | -0.0005 | +| -9999999999 | -4999999999 | 0.0002 | +| -9999999999 | 5000000000 | -0.0002 | +| -9999999999 | 4999999999 | -0.0002 | +| -9999999999 | -2147483647 | 0.0005 | +| -9999999999 | -5000000000 | 0.0002 | +| -2147483647 | -9999999999 | 0.0000 | +| -2147483647 | 9999999999 | 0.0000 | +| 2147483647 | 9999999999 | 0.0000 | +| 2147483647 | -9999999999 | 0.0000 | +| 9999999999 | -4999999999 | -0.0002 | +| 9999999999 | -2147483647 | -0.0005 | +| 9999999999 | -5000000000 | -0.0002 | +| 9999999999 | 2147483647 | 0.0005 | +| 9999999999 | 5000000000 | 0.0002 | +| 9999999999 | 4999999999 | 0.0002 | ++-------------+-------------+-------------+ diff --git a/tests/tidb-ci/fullstack-test-dt/issue_1425.test b/tests/tidb-ci/fullstack-test-dt/issue_1425.test index ca9106cc397..994e7d8abeb 100644 --- a/tests/tidb-ci/fullstack-test-dt/issue_1425.test +++ b/tests/tidb-ci/fullstack-test-dt/issue_1425.test @@ -16,14 +16,20 @@ mysql> drop table if exists test.t; mysql> create table test.t (id int, value decimal(7,4), c1 int, c2 int); -mysql> insert into test.t values(1,1.9286,54,28); +mysql> insert into test.t values (1,1.9285,54,28), (1,1.9286,54,28); mysql> alter table test.t set tiflash replica 1; func> wait_table test t +# note: ref to https://github.com/pingcap/tiflash/issues/1682, +# The precision of tiflash results is different from that of tidb, which is a compatibility issue mysql> use test; set session tidb_isolation_read_engines='tiflash'; select * from t where value = 54/28; - mysql> use test; set session tidb_isolation_read_engines='tiflash'; select * from t where value = c1/c2; ++------+--------+------+------+ +| id | value | c1 | c2 | ++------+--------+------+------+ +| 1 | 1.9286 | 54 | 28 | ++------+--------+------+------+ mysql> drop table if exists test.t; From ff2949f17e66021d3afb67e3a362aa765426a698 Mon Sep 17 00:00:00 2001 From: xufei Date: Fri, 10 Feb 2023 07:47:59 +0800 Subject: [PATCH 16/23] Refine the spill threshold for aggregation/sort, refine the memory usage calculation for aggregation (#6708) ref pingcap/tiflash#6528 --- dbms/src/Common/ThresholdUtils.h | 27 +++++++ .../AggregatingBlockInputStream.cpp | 1 + .../ParallelAggregatingBlockInputStream.cpp | 7 +- .../ParallelAggregatingBlockInputStream.h | 1 - .../AggregationInterpreterHelper.cpp | 9 ++- .../AggregationInterpreterHelper.h | 1 + .../Coprocessor/DAGQueryBlockInterpreter.cpp | 2 + dbms/src/Flash/Coprocessor/DAGUtils.cpp | 1 - dbms/src/Flash/Coprocessor/DAGUtils.h | 1 - .../Flash/Coprocessor/InterpreterUtils.cpp | 3 +- .../Planner/Plans/PhysicalAggregation.cpp | 1 + dbms/src/Interpreters/Aggregator.cpp | 64 ++++++--------- dbms/src/Interpreters/Aggregator.h | 81 +++++++++++++------ 13 files changed, 127 insertions(+), 72 deletions(-) create mode 100644 dbms/src/Common/ThresholdUtils.h diff --git a/dbms/src/Common/ThresholdUtils.h b/dbms/src/Common/ThresholdUtils.h new file mode 100644 index 00000000000..255e3788572 --- /dev/null +++ b/dbms/src/Common/ThresholdUtils.h @@ -0,0 +1,27 @@ +// Copyright 2023 PingCAP, Ltd. +// +// Licensed 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 + +inline size_t getAverageThreshold(size_t threshold, size_t concurrency) +{ + assert(concurrency > 0); + if (threshold == 0) + return 0; + return std::max(static_cast(1), threshold / concurrency); +} diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp index dc91a49a096..ad47d57f056 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp @@ -34,6 +34,7 @@ Block AggregatingBlockInputStream::readImpl() return this->isCancelled(); }; aggregator.setCancellationHook(hook); + aggregator.initThresholdByAggregatedDataVariantsSize(1); aggregator.execute(children.back(), *data_variants); diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index 86ce5724597..a2514b78f82 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -115,8 +115,7 @@ void ParallelAggregatingBlockInputStream::Handler::onBlock(Block & block, size_t block, *parent.many_data[thread_num], parent.threads_data[thread_num].key_columns, - parent.threads_data[thread_num].aggregate_columns, - parent.threads_data[thread_num].local_delta_memory); + parent.threads_data[thread_num].aggregate_columns); parent.threads_data[thread_num].src_rows += block.rows(); parent.threads_data[thread_num].src_bytes += block.bytes(); @@ -173,6 +172,7 @@ void ParallelAggregatingBlockInputStream::execute() for (size_t i = 0; i < max_threads; ++i) threads_data.emplace_back(keys_size, aggregates_size); + aggregator.initThresholdByAggregatedDataVariantsSize(many_data.size()); LOG_TRACE(log, "Aggregating"); @@ -226,8 +226,7 @@ void ParallelAggregatingBlockInputStream::execute() children.at(0)->getHeader(), *many_data[0], threads_data[0].key_columns, - threads_data[0].aggregate_columns, - threads_data[0].local_delta_memory); + threads_data[0].aggregate_columns); } void ParallelAggregatingBlockInputStream::appendInfo(FmtBuffer & buffer) const diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h index a23b7f7a516..6575a58de0a 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.h @@ -84,7 +84,6 @@ class ParallelAggregatingBlockInputStream : public IProfilingBlockInputStream { size_t src_rows = 0; size_t src_bytes = 0; - Int64 local_delta_memory = 0; ColumnRawPtrs key_columns; Aggregator::AggregateColumns aggregate_columns; diff --git a/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.cpp b/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.cpp index 8ee50a111fe..bc0b5bb0c1f 100644 --- a/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.cpp +++ b/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -76,6 +77,7 @@ Aggregator::Params buildParams( const Context & context, const Block & before_agg_header, size_t before_agg_streams_size, + size_t agg_streams_size, const Names & key_names, const TiDB::TiDBCollators & collators, const AggregateDescriptions & aggregate_descriptions, @@ -91,6 +93,7 @@ Aggregator::Params buildParams( const Settings & settings = context.getSettingsRef(); bool allow_to_use_two_level_group_by = isAllowToUseTwoLevelGroupBy(before_agg_streams_size, settings); + auto total_two_level_threshold_bytes = allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0); bool has_collator = std::any_of(begin(collators), end(collators), [](const auto & p) { return p != nullptr; }); @@ -98,9 +101,11 @@ Aggregator::Params buildParams( before_agg_header, keys, aggregate_descriptions, + /// do not use the average value for key count threshold, because for a random distributed data, the key count + /// in every threads should almost be the same allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), - allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), - settings.max_bytes_before_external_group_by, + getAverageThreshold(total_two_level_threshold_bytes, agg_streams_size), + getAverageThreshold(settings.max_bytes_before_external_group_by, agg_streams_size), !is_final_agg, spill_config, context.getSettingsRef().max_block_size, diff --git a/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.h b/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.h index 48b6afeeab5..d3fa500a893 100644 --- a/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.h +++ b/dbms/src/Flash/Coprocessor/AggregationInterpreterHelper.h @@ -37,6 +37,7 @@ Aggregator::Params buildParams( const Context & context, const Block & before_agg_header, size_t before_agg_streams_size, + size_t agg_streams_size, const Names & key_names, const TiDB::TiDBCollators & collators, const AggregateDescriptions & aggregate_descriptions, diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index ee4f564a0e4..77a3402fa82 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -13,6 +13,7 @@ // limitations under the License. #include +#include #include #include #include @@ -407,6 +408,7 @@ void DAGQueryBlockInterpreter::executeAggregation( context, before_agg_header, pipeline.streams.size(), + enable_fine_grained_shuffle ? pipeline.streams.size() : 1, key_names, collators, aggregate_descriptions, diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.cpp b/dbms/src/Flash/Coprocessor/DAGUtils.cpp index a7462671c77..180ba343d30 100755 --- a/dbms/src/Flash/Coprocessor/DAGUtils.cpp +++ b/dbms/src/Flash/Coprocessor/DAGUtils.cpp @@ -1447,5 +1447,4 @@ tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name) throw Exception(fmt::format("Unsupported function {}", name)); return func_name_sig_map[name]; } - } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGUtils.h b/dbms/src/Flash/Coprocessor/DAGUtils.h index 5776edf0098..1fe665e9c89 100644 --- a/dbms/src/Flash/Coprocessor/DAGUtils.h +++ b/dbms/src/Flash/Coprocessor/DAGUtils.h @@ -105,5 +105,4 @@ class UniqueNameGenerator tipb::DAGRequest getDAGRequestFromStringWithRetry(const String & s); tipb::EncodeType analyzeDAGEncodeType(DAGContext & dag_context); tipb::ScalarFuncSig reverseGetFuncSigByFuncName(const String & name); - } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 5fd559ec9b8..3ac42f29079 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -12,6 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include @@ -121,7 +122,7 @@ void orderStreams( order_descr, settings.max_block_size, limit, - settings.max_bytes_before_external_sort, + getAverageThreshold(settings.max_bytes_before_external_sort, pipeline.streams.size()), SpillConfig(context.getTemporaryPath(), fmt::format("{}_sort", log->identifier()), settings.max_spilled_size_per_spill, context.getFileProvider()), log->identifier()); stream->setExtraInfo(String(enableFineGrainedShuffleExtraInfo)); diff --git a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp index 90a4f441e35..6efdf523b6a 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalAggregation.cpp @@ -99,6 +99,7 @@ void PhysicalAggregation::buildBlockInputStreamImpl(DAGPipeline & pipeline, Cont context, before_agg_header, pipeline.streams.size(), + fine_grained_shuffle.enable() ? pipeline.streams.size() : 1, aggregation_keys, aggregation_collators, aggregate_descriptions, diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 1c73d3e28e9..3dc1c98e8b5 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -15,17 +15,12 @@ #include #include #include -#include -#include #include -#include #include #include +#include #include -#include -#include #include -#include #include #include #include @@ -34,14 +29,10 @@ #include #include #include -#include #include #include -#include #include -#include -#include namespace DB { @@ -177,6 +168,7 @@ void AggregatedDataVariants::convertToTwoLevel() default: throw Exception("Wrong data variant passed.", ErrorCodes::LOGICAL_ERROR); } + aggregator->useTwoLevelHashTable(); } @@ -240,9 +232,6 @@ Aggregator::Aggregator(const Params & params_, const String & req_id) , log(Logger::get(req_id)) , is_cancelled([]() { return false; }) { - if (current_memory_tracker) - memory_usage_before_aggregation = current_memory_tracker->get(); - aggregate_functions.resize(params.aggregates_size); for (size_t i = 0; i < params.aggregates_size; ++i) aggregate_functions[i] = params.aggregates[i].function.get(); @@ -735,8 +724,7 @@ bool Aggregator::executeOnBlock( const Block & block, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, - Int64 & local_delta_memory) + AggregateColumns & aggregate_columns) { if (is_cancelled()) return true; @@ -814,21 +802,14 @@ bool Aggregator::executeOnBlock( } size_t result_size = result.size(); - Int64 current_memory_usage = 0; - if (current_memory_tracker) - { - current_memory_usage = current_memory_tracker->get(); - auto updated_local_delta_memory = CurrentMemoryTracker::getLocalDeltaMemory(); - auto local_delta_memory_diff = updated_local_delta_memory - local_delta_memory; - current_memory_usage += (local_memory_usage.fetch_add(local_delta_memory_diff) + local_delta_memory_diff); - local_delta_memory = updated_local_delta_memory; - } - - auto result_size_bytes = current_memory_usage - memory_usage_before_aggregation; /// Here all the results in the sum are taken into account, from different threads. + auto result_size_bytes = result.bytesCount(); + /// worth_convert_to_two_level is set to true if + /// 1. some other threads already convert to two level + /// 2. the result size exceeds threshold bool worth_convert_to_two_level - = (params.group_by_two_level_threshold && result_size >= params.group_by_two_level_threshold) - || (params.group_by_two_level_threshold_bytes && result_size_bytes >= static_cast(params.group_by_two_level_threshold_bytes)); + = use_two_level_hash_table || (group_by_two_level_threshold && result_size >= group_by_two_level_threshold) + || (group_by_two_level_threshold_bytes && result_size_bytes >= group_by_two_level_threshold_bytes); /** Converting to a two-level data structure. * It allows you to make, in the subsequent, an effective merge - either economical from memory or parallel. @@ -837,16 +818,14 @@ bool Aggregator::executeOnBlock( result.convertToTwoLevel(); /** Flush data to disk if too much RAM is consumed. - * Data can only be flushed to disk if a two-level aggregation structure is used. + * Data can only be flushed to disk if a two-level aggregation is supported. */ - if (params.max_bytes_before_external_group_by - && result.isTwoLevel() - && current_memory_usage > static_cast(params.max_bytes_before_external_group_by) - && worth_convert_to_two_level) - { - /// todo: the memory usage is calculated by memory_tracker, it is not accurate since memory tracker - /// will tracker all the memory usage for the task/query, need to record and maintain the memory usage - /// in Aggregator directly. + if (max_bytes_before_external_group_by + && (result.isTwoLevel() || result.isConvertibleToTwoLevel()) + && result_size_bytes > max_bytes_before_external_group_by) + { + if (!result.isTwoLevel()) + result.convertToTwoLevel(); spill(result); } @@ -866,6 +845,13 @@ BlockInputStreams Aggregator::restoreSpilledData() return spiller->restoreBlocks(0); } +void Aggregator::initThresholdByAggregatedDataVariantsSize(size_t aggregated_data_variants_size) +{ + group_by_two_level_threshold = params.getGroupByTwoLevelThreshold(); + group_by_two_level_threshold_bytes = getAverageThreshold(params.getGroupByTwoLevelThresholdBytes(), aggregated_data_variants_size); + max_bytes_before_external_group_by = getAverageThreshold(params.getMaxBytesBeforeExternalGroupBy(), aggregated_data_variants_size); +} + void Aggregator::spill(AggregatedDataVariants & data_variants) { /// Flush only two-level data and possibly overflow data. @@ -992,14 +978,14 @@ void Aggregator::execute(const BlockInputStreamPtr & stream, AggregatedDataVaria src_rows += block.rows(); src_bytes += block.bytes(); - if (!executeOnBlock(block, result, key_columns, aggregate_columns, params.local_delta_memory)) + if (!executeOnBlock(block, result, key_columns, aggregate_columns)) break; } /// If there was no data, and we aggregate without keys, and we must return single row with the result of empty aggregation. /// To do this, we pass a block with zero rows to aggregate. if (result.empty() && params.keys_size == 0 && !params.empty_result_for_aggregation_by_empty_set) - executeOnBlock(stream->getHeader(), result, key_columns, aggregate_columns, params.local_delta_memory); + executeOnBlock(stream->getHeader(), result, key_columns, aggregate_columns); double elapsed_seconds = watch.elapsedSeconds(); size_t rows = result.size(); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index e8dbf0d4d69..087f338eb96 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -718,7 +718,7 @@ struct AggregatedDataVariants : private boost::noncopyable case Type::NAME: \ { \ const auto * ptr = reinterpret_cast(aggregation_method_impl); \ - return ptr->data.size() + (without_key != nullptr); \ + return ptr->data.size(); \ } APPLY_FOR_AGGREGATED_VARIANTS(M) @@ -729,6 +729,34 @@ struct AggregatedDataVariants : private boost::noncopyable } } + size_t bytesCount() const + { + size_t bytes_count = 0; + switch (type) + { + case Type::EMPTY: + case Type::without_key: + break; + +#define M(NAME, IS_TWO_LEVEL) \ + case Type::NAME: \ + { \ + const auto * ptr = reinterpret_cast(aggregation_method_impl); \ + bytes_count = ptr->data.getBufferSizeInBytes(); \ + break; \ + } + + APPLY_FOR_AGGREGATED_VARIANTS(M) +#undef M + + default: + throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); + } + for (const auto & pool : aggregates_pools) + bytes_count += pool->size(); + return bytes_count; + } + const char * getMethodName() const { return getMethodName(type); @@ -888,23 +916,10 @@ class Aggregator AggregateDescriptions aggregates; size_t keys_size; size_t aggregates_size; - Int64 local_delta_memory = 0; - - /// Two-level aggregation settings (used for a large number of keys). - /** With how many keys or the size of the aggregation state in bytes, - * two-level aggregation begins to be used. Enough to reach of at least one of the thresholds. - * 0 - the corresponding threshold is not specified. - */ - const size_t group_by_two_level_threshold; - const size_t group_by_two_level_threshold_bytes; - - /// Settings to flush temporary data to the filesystem (external aggregation). - const size_t max_bytes_before_external_group_by; /// 0 - do not use external aggregation. /// Return empty result when aggregating without keys on empty set. bool empty_result_for_aggregation_by_empty_set; - const std::string tmp_path; SpillConfig spill_config; UInt64 max_block_size; @@ -926,13 +941,13 @@ class Aggregator , aggregates(aggregates_) , keys_size(keys.size()) , aggregates_size(aggregates.size()) - , group_by_two_level_threshold(group_by_two_level_threshold_) - , group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_) - , max_bytes_before_external_group_by(max_bytes_before_external_group_by_) , empty_result_for_aggregation_by_empty_set(empty_result_for_aggregation_by_empty_set_) , spill_config(spill_config_) , max_block_size(max_block_size_) , collators(collators_) + , group_by_two_level_threshold(group_by_two_level_threshold_) + , group_by_two_level_threshold_bytes(group_by_two_level_threshold_bytes_) + , max_bytes_before_external_group_by(max_bytes_before_external_group_by_) { } @@ -962,6 +977,17 @@ class Aggregator /// Calculate the column numbers in `keys` and `aggregates`. void calculateColumnNumbers(const Block & block); + + size_t getGroupByTwoLevelThreshold() const { return group_by_two_level_threshold; } + size_t getGroupByTwoLevelThresholdBytes() const { return group_by_two_level_threshold_bytes; } + size_t getMaxBytesBeforeExternalGroupBy() const { return max_bytes_before_external_group_by; } + + private: + /// Note these thresholds should not be used directly, they are only used to + /// init the threshold in Aggregator + const size_t group_by_two_level_threshold; + const size_t group_by_two_level_threshold_bytes; + const size_t max_bytes_before_external_group_by; /// 0 - do not use external aggregation. }; @@ -980,8 +1006,8 @@ class Aggregator const Block & block, AggregatedDataVariants & result, ColumnRawPtrs & key_columns, - AggregateColumns & aggregate_columns, /// Passed to not create them anew for each block - Int64 & local_delta_memory); + AggregateColumns & aggregate_columns /// Passed to not create them anew for each block + ); /** Convert the aggregation data structure into a block. * If final = false, then ColumnAggregateFunction is created as the aggregation columns with the state of the calculations, @@ -1020,6 +1046,8 @@ class Aggregator void finishSpill(); BlockInputStreams restoreSpilledData(); bool hasSpilledData() const { return spiller != nullptr && spiller->hasSpilledData(); } + void useTwoLevelHashTable() { use_two_level_hash_table = true; } + void initThresholdByAggregatedDataVariantsSize(size_t aggregated_data_variants_size); /// Get data structure of the result. Block getHeader(bool final) const; @@ -1064,10 +1092,7 @@ class Aggregator bool all_aggregates_has_trivial_destructor = false; - /// How many RAM were used to process the query before processing the first block. - Int64 memory_usage_before_aggregation = 0; - - std::atomic local_memory_usage = 0; + std::atomic use_two_level_hash_table = false; std::mutex mutex; @@ -1076,6 +1101,16 @@ class Aggregator /// Returns true if you can abort the current task. CancellationHook is_cancelled; + /// Two-level aggregation settings (used for a large number of keys). + /** With how many keys or the size of the aggregation state in bytes, + * two-level aggregation begins to be used. Enough to reach of at least one of the thresholds. + * 0 - the corresponding threshold is not specified. + */ + size_t group_by_two_level_threshold = 0; + size_t group_by_two_level_threshold_bytes = 0; + /// Settings to flush temporary data to the filesystem (external aggregation). + size_t max_bytes_before_external_group_by = 0; + /// For external aggregation. std::unique_ptr spiller; From 8de01676ff0dcdf7aedc90b779cb9f393d6bedfb Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Fri, 10 Feb 2023 12:27:39 +0800 Subject: [PATCH 17/23] fix format Signed-off-by: guo-shaoge --- .../GeneratedColumnPlaceholderInputStream.h | 19 ++++++++++--------- .../Flash/Coprocessor/InterpreterUtils.cpp | 4 ++-- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 2 +- 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h index ee147734dbb..37b374caf41 100644 --- a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h @@ -15,12 +15,12 @@ #pragma once #include -#include #include +#include #include -#include -#include #include +#include +#include namespace DB { @@ -28,14 +28,14 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt { public: GeneratedColumnPlaceholderBlockInputStream( - const BlockInputStreamPtr & input, - const std::vector> & generated_column_infos_, - const String & req_id_) + const BlockInputStreamPtr & input, + const std::vector> & generated_column_infos_, + const String & req_id_) : generated_column_infos(generated_column_infos_) , log(Logger::get(req_id_)) - { - children.push_back(input); - } + { + children.push_back(input); + } String getName() const override { return NAME; } Block getHeader() const override @@ -49,6 +49,7 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt { return "generated_column_" + std::to_string(col_index); } + protected: void readPrefix() override { diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 3ac42f29079..1c9ad60dec8 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -228,8 +228,8 @@ void executeGeneratedColumnPlaceholder(const std::vector(stream, generated_column_infos, log->identifier()); - stream->setExtraInfo("generated column placeholder above table scan"); + stream = std::make_shared(stream, generated_column_infos, log->identifier()); + stream->setExtraInfo("generated column placeholder above table scan"); }); } } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 1cabc41572e..0e9edb638d5 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -80,7 +80,7 @@ void executePushedDownFilter( DAGPipeline & pipeline); void executeGeneratedColumnPlaceholder( - const std::vector> & generated_column_infos, + const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline); } // namespace DB From c20403898d9891567e2ba6dbf29402f2766faaac Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Sun, 12 Feb 2023 14:30:47 +0800 Subject: [PATCH 18/23] trivial fix Signed-off-by: guo-shaoge --- dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h | 2 +- dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h index 37b374caf41..efac4365c38 100644 --- a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h @@ -15,8 +15,8 @@ #pragma once #include -#include #include +#include #include #include #include diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index a7b26d62756..b3486cd926d 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -339,6 +339,7 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) recordProfileStreams(pipeline, table_scan.getTableScanExecutorID()); executeGeneratedColumnPlaceholder(generated_column_infos, log, pipeline); + recordProfileStreams(pipeline, table_scan.getTableScanExecutorID()); /// handle filter conditions for local and remote table scan. if (filter_conditions.hasValue()) From 9ca51bc364ffbed744ad9ece7dcef546e24b83ff Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Sun, 12 Feb 2023 21:04:39 +0800 Subject: [PATCH 19/23] fix nullable column bug Signed-off-by: guo-shaoge --- .../GeneratedColumnPlaceholderInputStream.h | 18 +++++++----------- .../Coprocessor/DAGStorageInterpreter.cpp | 1 + 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h index efac4365c38..6581149b3fa 100644 --- a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h @@ -41,7 +41,7 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt Block getHeader() const override { Block block = children.back()->getHeader(); - insertColumns(block, /*insert_null=*/false); + insertColumns(block, /*insert_data=*/false); return block; } @@ -64,29 +64,25 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt Block readImpl() override { Block block = children.back()->read(); - insertColumns(block, /*insert_null=*/true); + insertColumns(block, /*insert_data=*/true); return block; } private: - void insertColumns(Block & block, bool insert_null) const + void insertColumns(Block & block, bool insert_data) const { if (!block) return; - ColumnPtr nested_column = ColumnString::create(); for (const auto & ele : generated_column_infos) { auto col_index = ele.first; auto data_type = ele.second; - ColumnPtr null_map; - if (insert_null) - null_map = ColumnUInt8::create(block.rows(), 1); + ColumnPtr column = nullptr; + if (insert_data) + column = data_type->createColumnConstWithDefaultValue(block.rows()); else - null_map = ColumnUInt8::create(); - if (!data_type->isNullable()) - data_type = std::make_shared(data_type); - ColumnPtr column = ColumnNullable::create(nested_column, std::move(null_map)); + column = data_type->createColumn(); block.insert(col_index, ColumnWithTypeAndName{column, data_type, getColumnName(col_index)}); } } diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index b3486cd926d..77442087868 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -946,6 +946,7 @@ std::tuple> DAGStorageIn if (ci.hasGeneratedColumnFlag()) { + LOG_DEBUG(log, "got column({}) with generated column flag", i); auto data_type = getDataTypeByColumnInfoForComputingLayer(ci); generated_column_infos.push_back(std::make_pair(i, data_type)); source_columns_tmp.emplace_back(NameAndTypePair{GeneratedColumnPlaceholderBlockInputStream::getColumnName(i), data_type}); From 79d7ec1d483696d91965ce54eb3050e6c43f388d Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Sun, 12 Feb 2023 23:18:38 +0800 Subject: [PATCH 20/23] fix fmt Signed-off-by: guo-shaoge --- dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h index 6581149b3fa..c35f4ffe932 100644 --- a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h @@ -15,8 +15,8 @@ #pragma once #include -#include #include +#include #include #include #include From f1bb9e4fba80f1986f083264ef1b5a77e31714e8 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 13 Feb 2023 11:56:06 +0800 Subject: [PATCH 21/23] fix remote stream Signed-off-by: guo-shaoge --- .../Flash/Coprocessor/DAGQueryBlockInterpreter.cpp | 2 +- .../src/Flash/Coprocessor/DAGStorageInterpreter.cpp | 5 ++--- dbms/src/Flash/Coprocessor/InterpreterUtils.cpp | 13 ++++++++++--- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 1 + 4 files changed, 14 insertions(+), 7 deletions(-) diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 77a3402fa82..7c9cccca017 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -194,7 +194,7 @@ void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_s generated_column_infos.push_back(std::make_pair(i, data_type)); } } - executeGeneratedColumnPlaceholder(generated_column_infos, log, pipeline); + executeGeneratedColumnPlaceholder(pipeline.streams.size(), generated_column_infos, log, pipeline); } diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 77442087868..d69389292d8 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -336,9 +336,8 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) /// handle timezone/duration cast for local and remote table scan. executeCastAfterTableScan(remote_read_streams_start_index, pipeline); - recordProfileStreams(pipeline, table_scan.getTableScanExecutorID()); - - executeGeneratedColumnPlaceholder(generated_column_infos, log, pipeline); + /// handle generated column if necessary. + executeGeneratedColumnPlaceholder(remote_read_streams_start_index, generated_column_infos, log, pipeline); recordProfileStreams(pipeline, table_scan.getTableScanExecutorID()); /// handle filter conditions for local and remote table scan. diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 1c9ad60dec8..35a4bcd263b 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -223,13 +223,20 @@ void executePushedDownFilter( } } -void executeGeneratedColumnPlaceholder(const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline) +void executeGeneratedColumnPlaceholder( + size_t remote_read_streams_start_index, + const std::vector> & generated_column_infos, + LoggerPtr log, + DAGPipeline & pipeline) { if (generated_column_infos.empty()) return; - pipeline.transform([&](auto & stream) { + assert(remote_read_streams_start_index <= pipeline.streams.size()); + for (size_t i = 0; i < remote_read_streams_start_index; ++i) + { + auto & stream = pipeline.streams[i]; stream = std::make_shared(stream, generated_column_infos, log->identifier()); stream->setExtraInfo("generated column placeholder above table scan"); - }); + } } } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 0e9edb638d5..dc24b289ba3 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -80,6 +80,7 @@ void executePushedDownFilter( DAGPipeline & pipeline); void executeGeneratedColumnPlaceholder( + size_t remote_read_streams_start_index, const std::vector> & generated_column_infos, LoggerPtr log, DAGPipeline & pipeline); From 2c24e58dba782645f75691e44744a734a3a6695f Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 13 Feb 2023 13:32:59 +0800 Subject: [PATCH 22/23] add getGeneratedColumnInfos Signed-off-by: guo-shaoge --- ...GeneratedColumnPlaceholderBlockInputStream.h} | 16 ++++++++++++++++ .../Coprocessor/DAGQueryBlockInterpreter.cpp | 14 ++++---------- .../Flash/Coprocessor/DAGStorageInterpreter.cpp | 4 ++-- dbms/src/Flash/Coprocessor/InterpreterUtils.cpp | 8 ++++---- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 2 +- .../Planner/Plans/PhysicalMockTableScan.cpp | 4 ++++ 6 files changed, 31 insertions(+), 17 deletions(-) rename dbms/src/DataStreams/{GeneratedColumnPlaceholderInputStream.h => GeneratedColumnPlaceholderBlockInputStream.h} (82%) diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderBlockInputStream.h similarity index 82% rename from dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h rename to dbms/src/DataStreams/GeneratedColumnPlaceholderBlockInputStream.h index c35f4ffe932..4823e1cdc25 100644 --- a/dbms/src/DataStreams/GeneratedColumnPlaceholderInputStream.h +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderBlockInputStream.h @@ -21,6 +21,7 @@ #include #include #include +#include namespace DB { @@ -50,6 +51,20 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt return "generated_column_" + std::to_string(col_index); } + static std::vector> getGeneratedColumnInfos(const TiDBTableScan & table_scan) + { + std::vector> generated_column_infos; + for (Int64 i = 0; i < table_scan.getColumnSize(); ++i) + { + if (table_scan.getColumns()[i].hasGeneratedColumnFlag()) + { + auto data_type = getDataTypeByColumnInfoForComputingLayer(table_scan.getColumns()[i]); + generated_column_infos.push_back(std::make_pair(i, data_type)); + } + } + return generated_column_infos; + } + protected: void readPrefix() override { @@ -91,4 +106,5 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt const std::vector> generated_column_infos; const LoggerPtr log; }; + } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index 7c9cccca017..eaf17f2238d 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -19,6 +19,7 @@ #include #include #include +#include #include #include #include @@ -185,16 +186,9 @@ void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_s analyzer = std::make_unique(std::move(names_and_types), context); pipeline.streams.insert(pipeline.streams.end(), mock_table_scan_streams.begin(), mock_table_scan_streams.end()); } - std::vector> generated_column_infos; - for (Int64 i = 0; i < table_scan.getColumnSize(); i++) - { - if (table_scan.getColumns()[i].hasGeneratedColumnFlag()) - { - auto data_type = getDataTypeByColumnInfoForComputingLayer(table_scan.getColumns()[i]); - generated_column_infos.push_back(std::make_pair(i, data_type)); - } - } - executeGeneratedColumnPlaceholder(pipeline.streams.size(), generated_column_infos, log, pipeline); + + auto generated_column_infos = GeneratedColumnPlaceholderBlockInputStream::getGeneratedColumnInfos(table_scan); + executeGeneratedColumnPlaceholder(pipeline.streams.size(), generated_column_infos, log, pipeline.streams); } diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index d69389292d8..4c04f0c0f1d 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -19,7 +19,7 @@ #include #include #include -#include +#include #include #include #include @@ -337,7 +337,7 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) /// handle timezone/duration cast for local and remote table scan. executeCastAfterTableScan(remote_read_streams_start_index, pipeline); /// handle generated column if necessary. - executeGeneratedColumnPlaceholder(remote_read_streams_start_index, generated_column_infos, log, pipeline); + executeGeneratedColumnPlaceholder(remote_read_streams_start_index, generated_column_infos, log, pipeline.streams); recordProfileStreams(pipeline, table_scan.getTableScanExecutorID()); /// handle filter conditions for local and remote table scan. diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 35a4bcd263b..9e3bef7faf0 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -16,7 +16,7 @@ #include #include #include -#include +#include #include #include #include @@ -227,14 +227,14 @@ void executeGeneratedColumnPlaceholder( size_t remote_read_streams_start_index, const std::vector> & generated_column_infos, LoggerPtr log, - DAGPipeline & pipeline) + BlockInputStreams & streams) { if (generated_column_infos.empty()) return; - assert(remote_read_streams_start_index <= pipeline.streams.size()); + assert(remote_read_streams_start_index <= streams.size()); for (size_t i = 0; i < remote_read_streams_start_index; ++i) { - auto & stream = pipeline.streams[i]; + auto & stream = streams[i]; stream = std::make_shared(stream, generated_column_infos, log->identifier()); stream->setExtraInfo("generated column placeholder above table scan"); } diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index dc24b289ba3..384157d684d 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -83,5 +83,5 @@ void executeGeneratedColumnPlaceholder( size_t remote_read_streams_start_index, const std::vector> & generated_column_infos, LoggerPtr log, - DAGPipeline & pipeline); + BlockInputStreams & streams); } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index 4a1143686f5..b6aa1c7e775 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -12,10 +12,12 @@ // See the License for the specific language governing permissions and // limitations under the License. +#include #include #include #include #include +#include #include #include #include @@ -67,6 +69,8 @@ std::pair mockSchemaAndStreams( assert(!schema.empty()); assert(!mock_streams.empty()); + auto generated_column_infos = GeneratedColumnPlaceholderBlockInputStream::getGeneratedColumnInfos(table_scan); + executeGeneratedColumnPlaceholder(mock_streams.size(), generated_column_infos, log, mock_streams); return {std::move(schema), std::move(mock_streams)}; } } // namespace From ec1914d7b36f5e5db1fa67d8ac74ef6593f71bb5 Mon Sep 17 00:00:00 2001 From: guo-shaoge Date: Mon, 13 Feb 2023 13:46:56 +0800 Subject: [PATCH 23/23] del getGeneratedColumnInfos Signed-off-by: guo-shaoge --- ...neratedColumnPlaceholderBlockInputStream.h | 27 +++++-------------- .../Coprocessor/DAGQueryBlockInterpreter.cpp | 4 +-- .../Coprocessor/DAGStorageInterpreter.cpp | 9 ++++--- .../Flash/Coprocessor/DAGStorageInterpreter.h | 2 +- .../Flash/Coprocessor/InterpreterUtils.cpp | 8 +++--- dbms/src/Flash/Coprocessor/InterpreterUtils.h | 4 +-- .../Planner/Plans/PhysicalMockTableScan.cpp | 4 +-- 7 files changed, 21 insertions(+), 37 deletions(-) diff --git a/dbms/src/DataStreams/GeneratedColumnPlaceholderBlockInputStream.h b/dbms/src/DataStreams/GeneratedColumnPlaceholderBlockInputStream.h index 4823e1cdc25..f220f1377d4 100644 --- a/dbms/src/DataStreams/GeneratedColumnPlaceholderBlockInputStream.h +++ b/dbms/src/DataStreams/GeneratedColumnPlaceholderBlockInputStream.h @@ -30,7 +30,7 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt public: GeneratedColumnPlaceholderBlockInputStream( const BlockInputStreamPtr & input, - const std::vector> & generated_column_infos_, + const std::vector> & generated_column_infos_, const String & req_id_) : generated_column_infos(generated_column_infos_) , log(Logger::get(req_id_)) @@ -51,20 +51,6 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt return "generated_column_" + std::to_string(col_index); } - static std::vector> getGeneratedColumnInfos(const TiDBTableScan & table_scan) - { - std::vector> generated_column_infos; - for (Int64 i = 0; i < table_scan.getColumnSize(); ++i) - { - if (table_scan.getColumns()[i].hasGeneratedColumnFlag()) - { - auto data_type = getDataTypeByColumnInfoForComputingLayer(table_scan.getColumns()[i]); - generated_column_infos.push_back(std::make_pair(i, data_type)); - } - } - return generated_column_infos; - } - protected: void readPrefix() override { @@ -72,7 +58,7 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt // Validation check. for (size_t i = 1; i < generated_column_infos.size(); ++i) { - RUNTIME_CHECK(generated_column_infos[i].first > generated_column_infos[i - 1].first); + RUNTIME_CHECK(std::get<0>(generated_column_infos[i]) > std::get<0>(generated_column_infos[i - 1])); } } @@ -91,19 +77,20 @@ class GeneratedColumnPlaceholderBlockInputStream : public IProfilingBlockInputSt for (const auto & ele : generated_column_infos) { - auto col_index = ele.first; - auto data_type = ele.second; + const auto & col_index = std::get<0>(ele); + const auto & col_name = std::get<1>(ele); + const auto & data_type = std::get<2>(ele); ColumnPtr column = nullptr; if (insert_data) column = data_type->createColumnConstWithDefaultValue(block.rows()); else column = data_type->createColumn(); - block.insert(col_index, ColumnWithTypeAndName{column, data_type, getColumnName(col_index)}); + block.insert(col_index, ColumnWithTypeAndName{column, data_type, col_name}); } } static constexpr auto NAME = "GeneratedColumnPlaceholder"; - const std::vector> generated_column_infos; + const std::vector> generated_column_infos; const LoggerPtr log; }; diff --git a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp index eaf17f2238d..e16b711c8f1 100644 --- a/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGQueryBlockInterpreter.cpp @@ -19,7 +19,6 @@ #include #include #include -#include #include #include #include @@ -187,8 +186,7 @@ void DAGQueryBlockInterpreter::handleMockTableScan(const TiDBTableScan & table_s pipeline.streams.insert(pipeline.streams.end(), mock_table_scan_streams.begin(), mock_table_scan_streams.end()); } - auto generated_column_infos = GeneratedColumnPlaceholderBlockInputStream::getGeneratedColumnInfos(table_scan); - executeGeneratedColumnPlaceholder(pipeline.streams.size(), generated_column_infos, log, pipeline.streams); + // Ignore handling GeneratedColumnPlaceholderBlockInputStream for now, because we don't support generated column in test framework. } diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp index 4c04f0c0f1d..ab963e9d9a0 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.cpp @@ -337,7 +337,7 @@ void DAGStorageInterpreter::executeImpl(DAGPipeline & pipeline) /// handle timezone/duration cast for local and remote table scan. executeCastAfterTableScan(remote_read_streams_start_index, pipeline); /// handle generated column if necessary. - executeGeneratedColumnPlaceholder(remote_read_streams_start_index, generated_column_infos, log, pipeline.streams); + executeGeneratedColumnPlaceholder(remote_read_streams_start_index, generated_column_infos, log, pipeline); recordProfileStreams(pipeline, table_scan.getTableScanExecutorID()); /// handle filter conditions for local and remote table scan. @@ -946,9 +946,10 @@ std::tuple> DAGStorageIn if (ci.hasGeneratedColumnFlag()) { LOG_DEBUG(log, "got column({}) with generated column flag", i); - auto data_type = getDataTypeByColumnInfoForComputingLayer(ci); - generated_column_infos.push_back(std::make_pair(i, data_type)); - source_columns_tmp.emplace_back(NameAndTypePair{GeneratedColumnPlaceholderBlockInputStream::getColumnName(i), data_type}); + const auto & data_type = getDataTypeByColumnInfoForComputingLayer(ci); + const auto & col_name = GeneratedColumnPlaceholderBlockInputStream::getColumnName(i); + generated_column_infos.push_back(std::make_tuple(i, col_name, data_type)); + source_columns_tmp.emplace_back(NameAndTypePair{col_name, data_type}); continue; } // Column ID -1 return the handle column diff --git a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h index a07de68aabd..c9be8e00201 100644 --- a/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h +++ b/dbms/src/Flash/Coprocessor/DAGStorageInterpreter.h @@ -135,7 +135,7 @@ class DAGStorageInterpreter Names required_columns; NamesAndTypes source_columns; // For generated column, just need a placeholder, and TiDB will fill this column. - std::vector> generated_column_infos; + std::vector> generated_column_infos; }; } // namespace DB diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp index 9e3bef7faf0..669163d44b4 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.cpp @@ -225,16 +225,16 @@ void executePushedDownFilter( void executeGeneratedColumnPlaceholder( size_t remote_read_streams_start_index, - const std::vector> & generated_column_infos, + const std::vector> & generated_column_infos, LoggerPtr log, - BlockInputStreams & streams) + DAGPipeline & pipeline) { if (generated_column_infos.empty()) return; - assert(remote_read_streams_start_index <= streams.size()); + assert(remote_read_streams_start_index <= pipeline.streams.size()); for (size_t i = 0; i < remote_read_streams_start_index; ++i) { - auto & stream = streams[i]; + auto & stream = pipeline.streams[i]; stream = std::make_shared(stream, generated_column_infos, log->identifier()); stream->setExtraInfo("generated column placeholder above table scan"); } diff --git a/dbms/src/Flash/Coprocessor/InterpreterUtils.h b/dbms/src/Flash/Coprocessor/InterpreterUtils.h index 384157d684d..3d2ebd33afc 100644 --- a/dbms/src/Flash/Coprocessor/InterpreterUtils.h +++ b/dbms/src/Flash/Coprocessor/InterpreterUtils.h @@ -81,7 +81,7 @@ void executePushedDownFilter( void executeGeneratedColumnPlaceholder( size_t remote_read_streams_start_index, - const std::vector> & generated_column_infos, + const std::vector> & generated_column_infos, LoggerPtr log, - BlockInputStreams & streams); + DAGPipeline & pipeline); } // namespace DB diff --git a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp index b6aa1c7e775..1411dce0f70 100644 --- a/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp +++ b/dbms/src/Flash/Planner/Plans/PhysicalMockTableScan.cpp @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -#include #include #include #include @@ -69,8 +68,7 @@ std::pair mockSchemaAndStreams( assert(!schema.empty()); assert(!mock_streams.empty()); - auto generated_column_infos = GeneratedColumnPlaceholderBlockInputStream::getGeneratedColumnInfos(table_scan); - executeGeneratedColumnPlaceholder(mock_streams.size(), generated_column_infos, log, mock_streams); + // Ignore handling GeneratedColumnPlaceholderBlockInputStream for now, because we don't support generated column in test framework. return {std::move(schema), std::move(mock_streams)}; } } // namespace