From 36a3425a0fb91866a61c672933b6d223533348e4 Mon Sep 17 00:00:00 2001 From: "Sergei Tsetlin (rekub)" Date: Mon, 18 Jun 2018 15:52:27 +0300 Subject: [PATCH 01/62] multiply aggregate state --- dbms/src/Functions/FunctionsArithmetic.h | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index 865f1f934e1..5c3633a6d1a 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -1,5 +1,6 @@ #pragma once +#include #include #include #include @@ -779,6 +780,14 @@ class FunctionBinaryArithmetic : public IFunction return castType(left, [&](const auto & left) { return castType(right, [&](const auto & right) { return f(left, right); }); }); } + bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const + { + return std::is_same_v, MultiplyImpl> + && checkDataType(type0.get()) + && (checkDataType(type1.get()) + || checkDataType(type1.get())); + } + FunctionBuilderPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1) const { /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. @@ -835,6 +844,10 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { + /// Special case when multiply aggregate function state + if (isAggregateMultiply(arguments[0], arguments[1])) + return arguments[0]; + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1])) { @@ -875,6 +888,16 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { + if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) + { + auto c = block.getByPosition(arguments[0]).column->cloneEmpty(); + size_t m = block.getByPosition(arguments[1]).column->getUInt(0); + for (size_t i = 0; i < m; ++i) + c->insertRangeFrom(*(block.getByPosition(arguments[0]).column.get()), 0, input_rows_count); + block.getByPosition(result).column = std::move(c); + return; + } + /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) { From 32557a36c56d5a810a6ee1a544dc30ba11fd008a Mon Sep 17 00:00:00 2001 From: "Sergei Tsetlin (rekub)" Date: Tue, 19 Jun 2018 14:21:44 +0300 Subject: [PATCH 02/62] test aggregation state multiply --- .../0_stateless/00647_multiply_aggregation_state.reference | 3 +++ .../queries/0_stateless/00647_multiply_aggregation_state.sql | 3 +++ 2 files changed, 6 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference create mode 100644 dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql diff --git a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference new file mode 100644 index 00000000000..604440660c8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference @@ -0,0 +1,3 @@ +2 +0 +33 diff --git a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql new file mode 100644 index 00000000000..1264e8b991b --- /dev/null +++ b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql @@ -0,0 +1,3 @@ +SELECT countMerge(x) AS y FROM ( SELECT countState() * 2 AS x FROM ( SELECT 1 )); +SELECT countMerge(x) AS y FROM ( SELECT countState() * 0 AS x FROM ( SELECT 1 UNION ALL SELECT 2)); +SELECT sumMerge(y) AS z FROM ( SELECT sumState(x) * 11 AS y FROM ( SELECT 1 AS x UNION ALL SELECT 2 AS x)); From 987644e1e7dea470b0b292e30aaa8115845385b9 Mon Sep 17 00:00:00 2001 From: "Sergei Tsetlin (rekub)" Date: Tue, 19 Jun 2018 16:36:53 +0300 Subject: [PATCH 03/62] shift arguments, some problem seems found! --- dbms/src/Functions/FunctionsArithmetic.h | 43 ++++++++++++++----- ...00647_multiply_aggregation_state.reference | 3 ++ .../00647_multiply_aggregation_state.sql | 3 ++ 3 files changed, 39 insertions(+), 10 deletions(-) diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index 5c3633a6d1a..a9e6079f946 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -780,12 +780,33 @@ class FunctionBinaryArithmetic : public IFunction return castType(left, [&](const auto & left) { return castType(right, [&](const auto & right) { return f(left, right); }); }); } - bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const + bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1, bool & shift) const { return std::is_same_v, MultiplyImpl> - && checkDataType(type0.get()) - && (checkDataType(type1.get()) - || checkDataType(type1.get())); + && + ( + ( + checkDataType(type0.get()) + && + ( + checkDataType(type1.get()) + || checkDataType(type1.get()) + ) + && + !(shift = false) + ) + || + ( + checkDataType(type1.get()) + && + ( + checkDataType(type0.get()) + || checkDataType(type0.get()) + ) + && + (shift = true) + ) + ); } FunctionBuilderPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1) const @@ -845,8 +866,9 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { /// Special case when multiply aggregate function state - if (isAggregateMultiply(arguments[0], arguments[1])) - return arguments[0]; + bool shift; + if (isAggregateMultiply(arguments[0], arguments[1], shift)) + return arguments[shift?1:0]; /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1])) @@ -888,12 +910,13 @@ public: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) + bool shift; + if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type, shift)) { - auto c = block.getByPosition(arguments[0]).column->cloneEmpty(); - size_t m = block.getByPosition(arguments[1]).column->getUInt(0); + auto c = block.getByPosition(arguments[shift?1:0]).column->cloneEmpty(); + size_t m = block.getByPosition(arguments[shift?0:1]).column->getUInt(0); for (size_t i = 0; i < m; ++i) - c->insertRangeFrom(*(block.getByPosition(arguments[0]).column.get()), 0, input_rows_count); + c->insertRangeFrom(*(block.getByPosition(arguments[shift?1:0]).column.get()), 0, input_rows_count); block.getByPosition(result).column = std::move(c); return; } diff --git a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference index 604440660c8..3ebd70dce2f 100644 --- a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference +++ b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.reference @@ -1,3 +1,6 @@ 2 0 33 +2 +0 +18 diff --git a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql index 1264e8b991b..209a84e3548 100644 --- a/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql +++ b/dbms/tests/queries/0_stateless/00647_multiply_aggregation_state.sql @@ -1,3 +1,6 @@ SELECT countMerge(x) AS y FROM ( SELECT countState() * 2 AS x FROM ( SELECT 1 )); SELECT countMerge(x) AS y FROM ( SELECT countState() * 0 AS x FROM ( SELECT 1 UNION ALL SELECT 2)); SELECT sumMerge(y) AS z FROM ( SELECT sumState(x) * 11 AS y FROM ( SELECT 1 AS x UNION ALL SELECT 2 AS x)); +SELECT countMerge(x) AS y FROM ( SELECT 2 * countState() AS x FROM ( SELECT 1 )); +SELECT countMerge(x) AS y FROM ( SELECT 0 * countState() AS x FROM ( SELECT 1 UNION ALL SELECT 2)); +SELECT sumMerge(y) AS z FROM ( SELECT 3 * sumState(x) * 2 AS y FROM ( SELECT 1 AS x UNION ALL SELECT 2 AS x)); From df017f973868d845e83d9e5b58d0342c4e224729 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 7 Aug 2018 12:03:15 +0300 Subject: [PATCH 04/62] test --- test | 0 1 file changed, 0 insertions(+), 0 deletions(-) create mode 100644 test diff --git a/test b/test new file mode 100644 index 00000000000..e69de29bb2d From 8f974538b272ee6dc1d76889fd0a82b37b995028 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 7 Aug 2018 12:06:38 +0300 Subject: [PATCH 05/62] Revert "test" This reverts commit df017f973868d845e83d9e5b58d0342c4e224729. --- test | 0 1 file changed, 0 insertions(+), 0 deletions(-) delete mode 100644 test diff --git a/test b/test deleted file mode 100644 index e69de29bb2d..00000000000 From a9b32792654381ea33de4370b143fbef5337d9d2 Mon Sep 17 00:00:00 2001 From: The-Alchemist Date: Tue, 7 Aug 2018 15:09:30 -0400 Subject: [PATCH 06/62] Support for UUID in ODBC dictionaries --- dbms/src/Dictionaries/ExternalResultDescription.cpp | 3 +++ dbms/src/Dictionaries/ExternalResultDescription.h | 3 ++- dbms/src/Dictionaries/ODBCBlockInputStream.cpp | 2 ++ 3 files changed, 7 insertions(+), 1 deletion(-) diff --git a/dbms/src/Dictionaries/ExternalResultDescription.cpp b/dbms/src/Dictionaries/ExternalResultDescription.cpp index 18788606be7..4ac883d1c39 100644 --- a/dbms/src/Dictionaries/ExternalResultDescription.cpp +++ b/dbms/src/Dictionaries/ExternalResultDescription.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include @@ -55,6 +56,8 @@ void ExternalResultDescription::init(const Block & sample_block_) types.push_back(ValueType::Date); else if (typeid_cast(type)) types.push_back(ValueType::DateTime); + else if (typeid_cast(type)) + types.push_back(ValueType::UUID); else throw Exception{"Unsupported type " + type->getName(), ErrorCodes::UNKNOWN_TYPE}; diff --git a/dbms/src/Dictionaries/ExternalResultDescription.h b/dbms/src/Dictionaries/ExternalResultDescription.h index f3bfdf14fcc..ff9426ade57 100644 --- a/dbms/src/Dictionaries/ExternalResultDescription.h +++ b/dbms/src/Dictionaries/ExternalResultDescription.h @@ -24,7 +24,8 @@ struct ExternalResultDescription Float64, String, Date, - DateTime + DateTime, + UUID }; Block sample_block; diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.cpp b/dbms/src/Dictionaries/ODBCBlockInputStream.cpp index 6ecc60c9925..56a829a8923 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.cpp +++ b/dbms/src/Dictionaries/ODBCBlockInputStream.cpp @@ -59,6 +59,8 @@ namespace case ValueType::String: static_cast(column).insert(value.convert()); break; case ValueType::Date: static_cast(column).insert(UInt16{LocalDate{value.convert()}.getDayNum()}); break; case ValueType::DateTime: static_cast(column).insert(time_t{LocalDateTime{value.convert()}}); break; + case ValueType::UUID: static_cast(column).insert(value.convert()); break; + } } From 4534d449c15d3c0a4c68b745bf483f7f4b4cfdb1 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 21 Aug 2018 19:08:45 +0300 Subject: [PATCH 07/62] working on rollup --- .../DataStreams/RollupBlockInputStream.cpp | 255 ++++++++++++++++++ dbms/src/DataStreams/RollupBlockInputStream.h | 63 +++++ dbms/src/Interpreters/Aggregator.cpp | 6 + dbms/src/Interpreters/Aggregator.h | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 57 +++- .../src/Interpreters/InterpreterSelectQuery.h | 1 + dbms/src/Parsers/ASTSelectQuery.h | 1 + dbms/src/Parsers/ParserSelectQuery.cpp | 9 +- 8 files changed, 387 insertions(+), 7 deletions(-) create mode 100644 dbms/src/DataStreams/RollupBlockInputStream.cpp create mode 100644 dbms/src/DataStreams/RollupBlockInputStream.h diff --git a/dbms/src/DataStreams/RollupBlockInputStream.cpp b/dbms/src/DataStreams/RollupBlockInputStream.cpp new file mode 100644 index 00000000000..db9113bb8fc --- /dev/null +++ b/dbms/src/DataStreams/RollupBlockInputStream.cpp @@ -0,0 +1,255 @@ +#include +#include +#include +#include +#include +#include + +#include +#include + + +namespace DB +{ + +RollupBlockInputStream::RollupBlockInputStream( + const BlockInputStreamPtr & input_, const Aggregator::Params & params_) : params(params_), aggregator(params) +{ + children.push_back(input_); + + LOG_DEBUG(&Logger::get("Rollup"), "children: " << children.size()); + + LOG_DEBUG(&Logger::get("Rollup"), "input columns: " << input_->getHeader().columns()); + + Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); }; + aggregator.setCancellationHook(hook); + + + /// Initialize current totals with initial state. + + arena = std::make_shared(); + Block source_header = children.at(0)->getHeader(); + + current_totals.reserve(source_header.columns()); + for (const auto & elem : source_header) + { + if (const ColumnAggregateFunction * column = typeid_cast(elem.column.get())) + { + /// Create ColumnAggregateFunction with initial aggregate function state. + + auto target = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena)); + current_totals.emplace_back(std::move(target)); + } + else + { + + /// Not an aggregate function state. Just create a column with default value. + + MutableColumnPtr new_column = elem.type->createColumn(); + current_totals.emplace_back(std::move(new_column)); + ++not_aggregate_columns; + } + } +} + + +static void finalize(Block & block) +{ + for (size_t i = 0; i < block.columns(); ++i) + { + ColumnWithTypeAndName & current = block.getByPosition(i); + const DataTypeAggregateFunction * unfinalized_type = typeid_cast(current.type.get()); + + if (unfinalized_type) + { + current.type = unfinalized_type->getReturnType(); + if (current.column) + current.column = typeid_cast(*current.column).convertToValues(); + } + } +} + + + +Block RollupBlockInputStream::getHeader() const +{ + Block res = children.at(0)->getHeader(); + finalize(res); + return res; +} + + +Block RollupBlockInputStream::readImpl() +{ + Block finalized; + Block block; + + while(1) + { + + block = children[0]->read(); + + /// Block with values not included in `max_rows_to_group_by`. We'll postpone it. + if (!block) + return finalized; + + blocks.push_back(block); + auto rollup_block = std::move(block); + + for (int i = static_cast(params.keys_size) - 1; i >= 0; --i) + { + auto & current = rollup_block.getByPosition(params.keys[i]); + current.column = current.column->cloneEmpty()->cloneResized(rollup_block.rows()); + + Aggregator aggregator(params); + + auto result = std::make_shared(); + + StringRefs key(params.keys_size); + ColumnRawPtrs key_columns(params.keys_size); + AggregateColumns aggregate_columns(params.aggregates_size); + + bool no_more_keys = false; + + aggregator.executeOnBlock(rollup_block, *result, key_columns, aggregate_columns, key, no_more_keys); + aggregator.mergeStream(children.back(), *result, 1); + auto current_blocks = aggregator.convertToBlocks(*result, true, 1); + blocks.insert(blocks.end(), current_blocks.begin(), current_blocks.end()); + } + + finalized = aggregator.mergeBlocks(blocks, false); + finalize(finalized); + + total_keys += finalized.rows(); + // createRollupScheme(block); + // addToTotals(block); + // executeRollup(block); + + if (!finalized) + continue; + + passed_keys += finalized.rows(); + return finalized; + } +} + +void RollupBlockInputStream::createRollupScheme(const Block & block) +{ + size_t num_columns = block.columns(); + group_borders.resize(not_aggregate_columns); + size_t total = 1; + for(size_t i = 0; i + 1 < not_aggregate_columns; ++i) + { + const ColumnWithTypeAndName & current = block.getByPosition(i); + const auto & column = current.column.get(); + size_t size = column->size(); + for (size_t j = 1; j < size; ++j) + { + if (column->getDataAt(j) != column->getDataAt(j - 1)) + { + ++total; + group_borders[i].push_back(j); + } + } + ++total; + group_borders[i].push_back(block.rows()); + } + + for (size_t i = 0; i < num_columns; ++i) + { + const ColumnWithTypeAndName & current = block.getByPosition(i); + if (const ColumnAggregateFunction * column = typeid_cast(current.column.get())) + { + IAggregateFunction * function = column->getAggregateFunction().get(); + auto & target = typeid_cast(*current_totals[i]); + for (size_t j = 0; j < total; ++j) + { + AggregateDataPtr data = arena->alloc(function->sizeOfData()); + function->create(data); + target.getData().push_back(data); + } + } + else + { + for (size_t j = 0; j < total; ++j) + current.type->insertDefaultInto(*current_totals[i]); + } + } + // LOG_DEBUG(&Logger::get("Rollup"), "rollup size: " << rollups.size()); + // for(auto p : rollups) + // LOG_DEBUG(&Logger::get("Rollup"), "rollup: (" << p.first << ", " << p.second << ")"); +} + +void RollupBlockInputStream::executeRollup(const Block & block ) +{ + // if (!not_aggregate_columns) + // return; + + LOG_DEBUG(&Logger::get("Rollup"), "execute rollup"); + for (size_t i = not_aggregate_columns; i < block.columns(); ++i) + { + const ColumnWithTypeAndName & current = block.getByPosition(i); + const ColumnAggregateFunction * column = typeid_cast(current.column.get()); + + LOG_DEBUG(&Logger::get("Rollup"), "name: " << current.name); + + auto & target = typeid_cast(*current_totals[i]); + IAggregateFunction * function = target.getAggregateFunction().get(); + AggregateDataPtr data = target.getData()[0]; + + const ColumnAggregateFunction::Container & vec = column->getData(); + size_t size = vec.size(); + LOG_DEBUG(&Logger::get("Rollup"), "size: " << size); + int current_pos = 0; + + for (int j = static_cast(not_aggregate_columns) - 2; j >= 0; --j) + { + size_t ptr = 0; + for (size_t k = 0; k < size; ++k) + { + if (k == group_borders[j][ptr]) + { + ++ptr; + ++current_pos; + data = target.getData()[current_pos]; + } + function->merge(data, vec[k], arena.get()); + } + ++current_pos; + } + data = target.getData()[current_pos]; + for (size_t j = 0; j < size; ++j) + function->merge(data, vec[j], arena.get()); + } +} + + +void RollupBlockInputStream::addToTotals(const Block & block) +{ + LOG_DEBUG(&Logger::get("Rollup"), "add totals in rollup"); + LOG_DEBUG(&Logger::get("Rollup"), "block rows : " << block.rows()); + for (size_t i = 0, num_columns = block.columns(); i < num_columns; ++i) + { + const ColumnWithTypeAndName & current = block.getByPosition(i); + LOG_DEBUG(&Logger::get("Rollup"), "name: " + current.name); + if (const ColumnAggregateFunction * column = typeid_cast(current.column.get())) + { + auto & target = typeid_cast(*current_totals[i]); + IAggregateFunction * function = target.getAggregateFunction().get(); + AggregateDataPtr data = target.getData()[0]; + + /// Accumulate all aggregate states into that value. + + + const ColumnAggregateFunction::Container & vec = column->getData(); + size_t size = vec.size(); + + for (size_t j = 0; j < size; ++j) + { + function->merge(data, vec[j], arena.get()); + } + } + } +} + +} diff --git a/dbms/src/DataStreams/RollupBlockInputStream.h b/dbms/src/DataStreams/RollupBlockInputStream.h new file mode 100644 index 00000000000..fed1f17589a --- /dev/null +++ b/dbms/src/DataStreams/RollupBlockInputStream.h @@ -0,0 +1,63 @@ +#pragma once + +#include +#include +#include + + +namespace DB +{ + +class ExpressionActions; + + +/** Takes blocks after grouping, with non-finalized aggregate functions. + * Calculates total values according to totals_mode. + * If necessary, evaluates the expression from HAVING and filters rows. Returns the finalized and filtered blocks. + */ +class RollupBlockInputStream : public IProfilingBlockInputStream +{ +private: + using ExpressionActionsPtr = std::shared_ptr; + using AggregateColumns = std::vector; +public: + /// expression may be nullptr + RollupBlockInputStream( + const BlockInputStreamPtr & input_, const Aggregator::Params & params_); + + String getName() const override { return "Rollup"; } + + Block getHeader() const override; + +protected: + Block readImpl() override; + +private: + size_t passed_keys = 0; + size_t total_keys = 0; + size_t not_aggregate_columns = 0; + std::vector> group_borders; + + Aggregator::Params params; + Aggregator aggregator; + + Block result_block; + BlocksList blocks; + + /** Here are the values that did not pass max_rows_to_group_by. + * They are added or not added to the current_totals, depending on the totals_mode. + */ + + /// Here, total values are accumulated. After the work is finished, they will be placed in IProfilingBlockInputStream::totals. + MutableColumns current_totals; + /// Arena for aggregate function states in totals. + ArenaPtr arena; + std::unique_ptr impl; + + /// If filter == nullptr - add all rows. Otherwise, only the rows that pass the filter (HAVING). + void addToTotals(const Block & block); + void createRollupScheme(const Block & block); + void executeRollup(const Block & block); +}; + +} diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 2413db3e204..32612c7e080 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -98,6 +98,12 @@ Block Aggregator::getHeader(bool final) const if (params.src_header) { + + for (const auto & key : params.keys) + LOG_DEBUG(log, "key: " << key); + + LOG_DEBUG(log, "key_size: " << params.keys_size); + for (size_t i = 0; i < params.keys_size; ++i) res.insert(params.src_header.safeGetByPosition(params.keys[i]).cloneEmpty()); diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 5919a296085..cbe124511e2 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -844,7 +844,7 @@ struct AggregatedDataVariants : private boost::noncopyable throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } } - + /// The size without taking into account the row in which data is written for the calculation of TOTALS. size_t sizeWithoutOverflowRow() const { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 720d10d0a1a..5ac600e64fc 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include @@ -467,7 +468,12 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt executeMergeAggregated(pipeline, aggregate_overflow_row, aggregate_final); if (!aggregate_final) - executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row); + { + if (query.group_by_with_rollup) + executeRollup(pipeline); + if (query.group_by_with_totals) + executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row); + } else if (expressions.has_having) executeHaving(pipeline, expressions.before_having); @@ -482,6 +488,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (query.group_by_with_totals && !aggregate_final) executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row); + if (query.group_by_with_rollup && !aggregate_final) + executeRollup(pipeline); } if (expressions.has_order_by) @@ -627,7 +635,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline size_t max_block_size = settings.max_block_size; - /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size, + /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size, * then as the block size we will use limit + offset (not to read more from the table than requested), * and also set the number of threads to 1. */ @@ -790,9 +798,9 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header.getPositionByName(name)); - + const Settings & settings = context.getSettingsRef(); - + /** Two-level aggregation is useful in two cases: * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. @@ -910,6 +918,47 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold); } +void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) +{ + executeUnion(pipeline); + + Names key_names; + AggregateDescriptions aggregates; + query_analyzer->getAggregateInfo(key_names, aggregates); + + Block header = pipeline.firstStream()->getHeader(); + ColumnNumbers keys; + + for (const auto & name : key_names) + keys.push_back(header.getPositionByName(name)); + + for (auto & descr : aggregates) + if (descr.arguments.empty()) + for (const auto & elem : header) + { + if (typeid_cast(elem.column.get())) + descr.arguments.push_back(header.getPositionByName(elem.name)); + } + + const Settings & settings = context.getSettingsRef(); + + /** Two-level aggregation is useful in two cases: + * 1. Parallel aggregation is done, and the results should be merged in parallel. + * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. + */ + bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.max_bytes_before_external_group_by != 0; + + Aggregator::Params params(header, keys, aggregates, + false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, + settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, + 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, settings.empty_result_for_aggregation_by_empty_set, + context.getTemporaryPath()); + + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); +} + void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression) { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 26f6c2f15ab..7ed1437596a 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -187,6 +187,7 @@ private: void executeDistinct(Pipeline & pipeline, bool before_order, Names columns); void executeExtremes(Pipeline & pipeline); void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map & subqueries_for_sets); + void executeRollup(Pipeline & pipeline); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. * diff --git a/dbms/src/Parsers/ASTSelectQuery.h b/dbms/src/Parsers/ASTSelectQuery.h index 91d8d52172c..7422c70212c 100644 --- a/dbms/src/Parsers/ASTSelectQuery.h +++ b/dbms/src/Parsers/ASTSelectQuery.h @@ -28,6 +28,7 @@ public: ASTPtr where_expression; ASTPtr group_expression_list; bool group_by_with_totals = false; + bool group_by_with_rollup = false; ASTPtr having_expression; ASTPtr order_expression_list; ASTPtr limit_by_value; diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 4bf89d49984..5e94ba5b4ff 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -38,6 +38,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_limit("LIMIT"); ParserKeyword s_settings("SETTINGS"); ParserKeyword s_by("BY"); + ParserKeyword s_rollup("ROLLUP"); ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list_for_with_clause(false, true); /// Set prefer_alias_to_column_name for each alias. @@ -97,10 +98,14 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// WITH TOTALS if (s_with.ignore(pos, expected)) { - if (!s_totals.ignore(pos, expected)) + if (!s_totals.ignore(pos, expected) && !s_rollup.ignore(pos, expected)) return false; - select_query->group_by_with_totals = true; + // if (s_totals.ignore(pos, expected)) + select_query->group_by_with_totals = true; + + // if (s_rollup.ignore(pos, expected)) + select_query->group_by_with_rollup = true; } /// HAVING expr From c18ad18d963efcf4dcd5e6e162448ccb8c0c9e4d Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 24 Aug 2018 18:00:00 +0300 Subject: [PATCH 08/62] Add rollup modificator --- .../DataStreams/RollupBlockInputStream.cpp | 226 +++--------------- dbms/src/DataStreams/RollupBlockInputStream.h | 21 -- .../TotalsHavingBlockInputStream.cpp | 10 +- .../TotalsHavingBlockInputStream.h | 3 +- dbms/src/Interpreters/Aggregator.cpp | 7 +- .../Interpreters/InterpreterSelectQuery.cpp | 45 ++-- .../src/Interpreters/InterpreterSelectQuery.h | 4 +- dbms/src/Parsers/ParserSelectQuery.cpp | 28 ++- 8 files changed, 85 insertions(+), 259 deletions(-) diff --git a/dbms/src/DataStreams/RollupBlockInputStream.cpp b/dbms/src/DataStreams/RollupBlockInputStream.cpp index db9113bb8fc..e8519199cdf 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.cpp +++ b/dbms/src/DataStreams/RollupBlockInputStream.cpp @@ -1,58 +1,12 @@ #include -#include #include #include #include #include -#include -#include - - namespace DB { -RollupBlockInputStream::RollupBlockInputStream( - const BlockInputStreamPtr & input_, const Aggregator::Params & params_) : params(params_), aggregator(params) -{ - children.push_back(input_); - - LOG_DEBUG(&Logger::get("Rollup"), "children: " << children.size()); - - LOG_DEBUG(&Logger::get("Rollup"), "input columns: " << input_->getHeader().columns()); - - Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); }; - aggregator.setCancellationHook(hook); - - - /// Initialize current totals with initial state. - - arena = std::make_shared(); - Block source_header = children.at(0)->getHeader(); - - current_totals.reserve(source_header.columns()); - for (const auto & elem : source_header) - { - if (const ColumnAggregateFunction * column = typeid_cast(elem.column.get())) - { - /// Create ColumnAggregateFunction with initial aggregate function state. - - auto target = ColumnAggregateFunction::create(column->getAggregateFunction(), Arenas(1, arena)); - current_totals.emplace_back(std::move(target)); - } - else - { - - /// Not an aggregate function state. Just create a column with default value. - - MutableColumnPtr new_column = elem.type->createColumn(); - current_totals.emplace_back(std::move(new_column)); - ++not_aggregate_columns; - } - } -} - - static void finalize(Block & block) { for (size_t i = 0; i < block.columns(); ++i) @@ -69,6 +23,11 @@ static void finalize(Block & block) } } +RollupBlockInputStream::RollupBlockInputStream( + const BlockInputStreamPtr & input_, const Aggregator::Params & params_) : params(params_) +{ + children.push_back(input_); +} Block RollupBlockInputStream::getHeader() const @@ -81,175 +40,48 @@ Block RollupBlockInputStream::getHeader() const Block RollupBlockInputStream::readImpl() { - Block finalized; Block block; + BlocksList blocks; while(1) { - block = children[0]->read(); - /// Block with values not included in `max_rows_to_group_by`. We'll postpone it. if (!block) - return finalized; - - blocks.push_back(block); - auto rollup_block = std::move(block); + return block; + + Aggregator aggregator(params); + + Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); }; + aggregator.setCancellationHook(hook); + + StringRefs key(params.keys_size); + ColumnRawPtrs key_columns(params.keys_size); + AggregateColumns aggregate_columns(params.aggregates_size); + + Block rollup_block = block; for (int i = static_cast(params.keys_size) - 1; i >= 0; --i) { auto & current = rollup_block.getByPosition(params.keys[i]); current.column = current.column->cloneEmpty()->cloneResized(rollup_block.rows()); - - Aggregator aggregator(params); - - auto result = std::make_shared(); - - StringRefs key(params.keys_size); - ColumnRawPtrs key_columns(params.keys_size); - AggregateColumns aggregate_columns(params.aggregates_size); - - bool no_more_keys = false; - - aggregator.executeOnBlock(rollup_block, *result, key_columns, aggregate_columns, key, no_more_keys); - aggregator.mergeStream(children.back(), *result, 1); - auto current_blocks = aggregator.convertToBlocks(*result, true, 1); - blocks.insert(blocks.end(), current_blocks.begin(), current_blocks.end()); + BlocksList rollup_blocks = { rollup_block }; + blocks.push_back(aggregator.mergeBlocks(rollup_blocks, true)); } - finalized = aggregator.mergeBlocks(blocks, false); - finalize(finalized); + finalize(block); - total_keys += finalized.rows(); - // createRollupScheme(block); - // addToTotals(block); - // executeRollup(block); + for (size_t i = 0; i < block.columns(); ++i) + { + MutableColumnPtr column = block.getByPosition(i).column->assumeMutable(); + for (const auto & current_block : blocks) + column->insertRangeFrom(*current_block.getByPosition(i).column.get(), 0, current_block.rows()); + } - if (!finalized) + if (!block) continue; - passed_keys += finalized.rows(); - return finalized; + return block; } } - -void RollupBlockInputStream::createRollupScheme(const Block & block) -{ - size_t num_columns = block.columns(); - group_borders.resize(not_aggregate_columns); - size_t total = 1; - for(size_t i = 0; i + 1 < not_aggregate_columns; ++i) - { - const ColumnWithTypeAndName & current = block.getByPosition(i); - const auto & column = current.column.get(); - size_t size = column->size(); - for (size_t j = 1; j < size; ++j) - { - if (column->getDataAt(j) != column->getDataAt(j - 1)) - { - ++total; - group_borders[i].push_back(j); - } - } - ++total; - group_borders[i].push_back(block.rows()); - } - - for (size_t i = 0; i < num_columns; ++i) - { - const ColumnWithTypeAndName & current = block.getByPosition(i); - if (const ColumnAggregateFunction * column = typeid_cast(current.column.get())) - { - IAggregateFunction * function = column->getAggregateFunction().get(); - auto & target = typeid_cast(*current_totals[i]); - for (size_t j = 0; j < total; ++j) - { - AggregateDataPtr data = arena->alloc(function->sizeOfData()); - function->create(data); - target.getData().push_back(data); - } - } - else - { - for (size_t j = 0; j < total; ++j) - current.type->insertDefaultInto(*current_totals[i]); - } - } - // LOG_DEBUG(&Logger::get("Rollup"), "rollup size: " << rollups.size()); - // for(auto p : rollups) - // LOG_DEBUG(&Logger::get("Rollup"), "rollup: (" << p.first << ", " << p.second << ")"); -} - -void RollupBlockInputStream::executeRollup(const Block & block ) -{ - // if (!not_aggregate_columns) - // return; - - LOG_DEBUG(&Logger::get("Rollup"), "execute rollup"); - for (size_t i = not_aggregate_columns; i < block.columns(); ++i) - { - const ColumnWithTypeAndName & current = block.getByPosition(i); - const ColumnAggregateFunction * column = typeid_cast(current.column.get()); - - LOG_DEBUG(&Logger::get("Rollup"), "name: " << current.name); - - auto & target = typeid_cast(*current_totals[i]); - IAggregateFunction * function = target.getAggregateFunction().get(); - AggregateDataPtr data = target.getData()[0]; - - const ColumnAggregateFunction::Container & vec = column->getData(); - size_t size = vec.size(); - LOG_DEBUG(&Logger::get("Rollup"), "size: " << size); - int current_pos = 0; - - for (int j = static_cast(not_aggregate_columns) - 2; j >= 0; --j) - { - size_t ptr = 0; - for (size_t k = 0; k < size; ++k) - { - if (k == group_borders[j][ptr]) - { - ++ptr; - ++current_pos; - data = target.getData()[current_pos]; - } - function->merge(data, vec[k], arena.get()); - } - ++current_pos; - } - data = target.getData()[current_pos]; - for (size_t j = 0; j < size; ++j) - function->merge(data, vec[j], arena.get()); - } -} - - -void RollupBlockInputStream::addToTotals(const Block & block) -{ - LOG_DEBUG(&Logger::get("Rollup"), "add totals in rollup"); - LOG_DEBUG(&Logger::get("Rollup"), "block rows : " << block.rows()); - for (size_t i = 0, num_columns = block.columns(); i < num_columns; ++i) - { - const ColumnWithTypeAndName & current = block.getByPosition(i); - LOG_DEBUG(&Logger::get("Rollup"), "name: " + current.name); - if (const ColumnAggregateFunction * column = typeid_cast(current.column.get())) - { - auto & target = typeid_cast(*current_totals[i]); - IAggregateFunction * function = target.getAggregateFunction().get(); - AggregateDataPtr data = target.getData()[0]; - - /// Accumulate all aggregate states into that value. - - - const ColumnAggregateFunction::Container & vec = column->getData(); - size_t size = vec.size(); - - for (size_t j = 0; j < size; ++j) - { - function->merge(data, vec[j], arena.get()); - } - } - } -} - } diff --git a/dbms/src/DataStreams/RollupBlockInputStream.h b/dbms/src/DataStreams/RollupBlockInputStream.h index fed1f17589a..775668afe02 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.h +++ b/dbms/src/DataStreams/RollupBlockInputStream.h @@ -35,29 +35,8 @@ protected: private: size_t passed_keys = 0; size_t total_keys = 0; - size_t not_aggregate_columns = 0; - std::vector> group_borders; Aggregator::Params params; - Aggregator aggregator; - - Block result_block; - BlocksList blocks; - - /** Here are the values that did not pass max_rows_to_group_by. - * They are added or not added to the current_totals, depending on the totals_mode. - */ - - /// Here, total values are accumulated. After the work is finished, they will be placed in IProfilingBlockInputStream::totals. - MutableColumns current_totals; - /// Arena for aggregate function states in totals. - ArenaPtr arena; - std::unique_ptr impl; - - /// If filter == nullptr - add all rows. Otherwise, only the rows that pass the filter (HAVING). - void addToTotals(const Block & block); - void createRollupScheme(const Block & block); - void executeRollup(const Block & block); }; } diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp index 8689e29e77b..8f301b759b3 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.cpp @@ -13,10 +13,10 @@ namespace DB TotalsHavingBlockInputStream::TotalsHavingBlockInputStream( const BlockInputStreamPtr & input_, bool overflow_row_, const ExpressionActionsPtr & expression_, - const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_) + const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_) : overflow_row(overflow_row_), expression(expression_), filter_column_name(filter_column_), totals_mode(totals_mode_), - auto_include_threshold(auto_include_threshold_) + auto_include_threshold(auto_include_threshold_), final(final_) { children.push_back(input_); @@ -99,7 +99,8 @@ Block TotalsHavingBlockInputStream::getTotals() Block TotalsHavingBlockInputStream::getHeader() const { Block res = children.at(0)->getHeader(); - finalize(res); + if (final) + finalize(res); if (expression) expression->execute(res); return res; @@ -126,7 +127,8 @@ Block TotalsHavingBlockInputStream::readImpl() return finalized; finalized = block; - finalize(finalized); + if (final) + finalize(finalized); total_keys += finalized.rows(); diff --git a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h index 5ec4724ddd7..4926ea8afe7 100644 --- a/dbms/src/DataStreams/TotalsHavingBlockInputStream.h +++ b/dbms/src/DataStreams/TotalsHavingBlockInputStream.h @@ -24,7 +24,7 @@ public: TotalsHavingBlockInputStream( const BlockInputStreamPtr & input_, bool overflow_row_, const ExpressionActionsPtr & expression_, - const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_); + const std::string & filter_column_, TotalsMode totals_mode_, double auto_include_threshold_, bool final_); String getName() const override { return "TotalsHaving"; } @@ -41,6 +41,7 @@ private: String filter_column_name; TotalsMode totals_mode; double auto_include_threshold; + bool final; size_t passed_keys = 0; size_t total_keys = 0; diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index eb127bdc151..52632ca68ef 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -97,12 +97,6 @@ Block Aggregator::getHeader(bool final) const if (params.src_header) { - - for (const auto & key : params.keys) - LOG_DEBUG(log, "key: " << key); - - LOG_DEBUG(log, "key_size: " << params.keys_size); - for (size_t i = 0; i < params.keys_size; ++i) res.insert(params.src_header.safeGetByPosition(params.keys[i]).cloneEmpty()); @@ -667,6 +661,7 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, StringRefs & key, bool & no_more_keys) { + LOG_DEBUG(log, "on block structure: " + block.dumpStructure()); if (isCancelled()) return true; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 1d45d272d81..1f342d88017 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -479,7 +479,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt bool aggregate_final = expressions.need_aggregate && to_stage > QueryProcessingStage::WithMergeableState && - !query.group_by_with_totals; + !query.group_by_with_totals && !query.group_by_with_rollup; if (expressions.first_stage) { @@ -536,14 +536,17 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt if (!aggregate_final) { - if (query.group_by_with_rollup) - executeRollup(pipeline); if (query.group_by_with_totals) - executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row); + executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, !query.group_by_with_rollup); + + if (query.group_by_with_rollup) + executeRollup(pipeline, expressions.before_aggregation); } else if (expressions.has_having) executeHaving(pipeline, expressions.before_having); + + executeExpression(pipeline, expressions.before_order_and_select); executeDistinct(pipeline, true, expressions.selected_columns); @@ -552,11 +555,12 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt else { need_second_distinct_pass = query.distinct && pipeline.hasMoreThanOneStream(); - + if (query.group_by_with_totals && !aggregate_final) - executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row); + executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row, !query.group_by_with_rollup); + if (query.group_by_with_rollup && !aggregate_final) - executeRollup(pipeline); + executeRollup(pipeline, expressions.before_aggregation); } if (expressions.has_order_by) @@ -854,6 +858,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header.getPositionByName(name)); + const Settings & settings = context.getSettingsRef(); @@ -963,7 +968,7 @@ void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const Expression } -void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row) +void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { executeUnion(pipeline); @@ -971,11 +976,17 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha pipeline.firstStream() = std::make_shared( pipeline.firstStream(), overflow_row, expression, - has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold); + has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final); } -void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) +void InterpreterSelectQuery::executeRollup(Pipeline & pipeline, const ExpressionActionsPtr & expression) { + pipeline.transform([&](auto & stream) + { + stream = std::make_shared( + std::make_shared(stream, expression)); + }); + executeUnion(pipeline); Names key_names; @@ -983,18 +994,14 @@ void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) query_analyzer->getAggregateInfo(key_names, aggregates); Block header = pipeline.firstStream()->getHeader(); + ColumnNumbers keys; for (const auto & name : key_names) keys.push_back(header.getPositionByName(name)); - - for (auto & descr : aggregates) - if (descr.arguments.empty()) - for (const auto & elem : header) - { - if (typeid_cast(elem.column.get())) - descr.arguments.push_back(header.getPositionByName(elem.name)); - } + // for (auto & descr : aggregates) + // if (descr.arguments.empty()) + // descr.arguments.push_back(header.getPositionByName(descr.column_name)); const Settings & settings = context.getSettingsRef(); @@ -1012,7 +1019,7 @@ void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath()); - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 3d9c366bbbb..677c2c6942f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -177,7 +177,7 @@ private: void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); - void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row); + void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); void executeOrder(Pipeline & pipeline); @@ -190,7 +190,7 @@ private: void executeDistinct(Pipeline & pipeline, bool before_order, Names columns); void executeExtremes(Pipeline & pipeline); void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map & subqueries_for_sets); - void executeRollup(Pipeline & pipeline); + void executeRollup(Pipeline & pipeline, const ExpressionActionsPtr & expression); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. * diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 7263b2eba5d..16e354b238f 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -11,6 +11,10 @@ #include +#include +#include + + namespace DB { @@ -49,6 +53,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserExpressionWithOptionalAlias exp_elem(false); ParserOrderByExpressionList order_list; + ParserToken open_bracket(TokenType::OpeningRoundBracket); + ParserToken close_bracket(TokenType::ClosingRoundBracket); + /// WITH expr list { if (s_with.ignore(pos, expected)) @@ -68,8 +75,6 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (s_top.ignore(pos, expected)) { - ParserToken open_bracket(TokenType::OpeningRoundBracket); - ParserToken close_bracket(TokenType::ClosingRoundBracket); ParserNumber num; if (open_bracket.ignore(pos, expected)) @@ -114,21 +119,26 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) /// GROUP BY expr list if (s_group_by.ignore(pos, expected)) { + if (s_rollup.ignore(pos, expected)) + { + select_query->group_by_with_rollup = true; + if (!open_bracket.ignore(pos, expected)) + return false; + } + if (!exp_list.parse(pos, select_query->group_expression_list, expected)) return false; + + if (select_query->group_by_with_rollup && !close_bracket.ignore(pos, expected)) + return false; } /// WITH TOTALS if (s_with.ignore(pos, expected)) { - if (!s_totals.ignore(pos, expected) && !s_rollup.ignore(pos, expected)) + if (!s_totals.ignore(pos, expected)) return false; - - // if (s_totals.ignore(pos, expected)) - select_query->group_by_with_totals = true; - - // if (s_rollup.ignore(pos, expected)) - select_query->group_by_with_rollup = true; + select_query->group_by_with_totals = true; } /// HAVING expr From bdedf1df635208896d37b20f005b0e2249f97edd Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 24 Aug 2018 20:34:56 +0300 Subject: [PATCH 09/62] Add test --- .../0_stateless/00701_rollup.reference | 19 +++++++++++++++++++ .../queries/0_stateless/00701_rollup.sql | 17 +++++++++++++++++ 2 files changed, 36 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00701_rollup.reference create mode 100644 dbms/tests/queries/0_stateless/00701_rollup.sql diff --git a/dbms/tests/queries/0_stateless/00701_rollup.reference b/dbms/tests/queries/0_stateless/00701_rollup.reference new file mode 100644 index 00000000000..267ac5f5bd7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00701_rollup.reference @@ -0,0 +1,19 @@ + 0 120 8 +a 0 70 4 +a 1 25 2 +a 2 45 2 +b 0 50 4 +b 1 15 2 +b 2 35 2 + 0 120 8 +a 0 70 4 +a 1 25 2 +a 2 45 2 +b 0 50 4 +b 1 15 2 +b 2 35 2 + + 0 120 8 + 120 8 +a 70 4 +b 50 4 diff --git a/dbms/tests/queries/0_stateless/00701_rollup.sql b/dbms/tests/queries/0_stateless/00701_rollup.sql new file mode 100644 index 00000000000..ad225231ec5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00701_rollup.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS test.rollup; +CREATE TABLE test.rollup(a String, b Int32, s Int32) ENGINE = Memory; + +INSERT INTO test.rollup VALUES('a', 1, 10); +INSERT INTO test.rollup VALUES('a', 1, 15); +INSERT INTO test.rollup VALUES('a', 2, 20); +INSERT INTO test.rollup VALUES('a', 2, 25); +INSERT INTO test.rollup VALUES('b', 1, 10); +INSERT INTO test.rollup VALUES('b', 1, 5); +INSERT INTO test.rollup VALUES('b', 2, 20); +INSERT INTO test.rollup VALUES('b', 2, 15); + +SELECT a, b, sum(s), count() from test.rollup GROUP BY ROLLUP(a, b) ORDER BY a, b; + +SELECT a, b, sum(s), count() from test.rollup GROUP BY ROLLUP(a, b) WITH TOTALS ORDER BY a, b; + +SELECT a, sum(s), count() from test.rollup GROUP BY ROLLUP(a) ORDER BY a; From cf3fbb8c51faabeb58333aa840bd2a121536b21f Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 24 Aug 2018 20:37:08 +0300 Subject: [PATCH 10/62] minor changes --- .../DataStreams/RollupBlockInputStream.cpp | 6 ++++- dbms/src/Interpreters/Aggregator.cpp | 1 - .../Interpreters/InterpreterSelectQuery.cpp | 24 ++++--------------- .../src/Interpreters/InterpreterSelectQuery.h | 2 +- dbms/src/Parsers/ParserSelectQuery.cpp | 5 +--- 5 files changed, 11 insertions(+), 27 deletions(-) diff --git a/dbms/src/DataStreams/RollupBlockInputStream.cpp b/dbms/src/DataStreams/RollupBlockInputStream.cpp index e8519199cdf..4f808ce4e52 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.cpp +++ b/dbms/src/DataStreams/RollupBlockInputStream.cpp @@ -65,11 +65,15 @@ Block RollupBlockInputStream::readImpl() { auto & current = rollup_block.getByPosition(params.keys[i]); current.column = current.column->cloneEmpty()->cloneResized(rollup_block.rows()); + BlocksList rollup_blocks = { rollup_block }; - blocks.push_back(aggregator.mergeBlocks(rollup_blocks, true)); + rollup_block = aggregator.mergeBlocks(rollup_blocks, false); + blocks.push_back(rollup_block); } finalize(block); + for (auto & current_block : blocks) + finalize(current_block); for (size_t i = 0; i < block.columns(); ++i) { diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index 52632ca68ef..dc9dde6acb8 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -661,7 +661,6 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re ColumnRawPtrs & key_columns, AggregateColumns & aggregate_columns, StringRefs & key, bool & no_more_keys) { - LOG_DEBUG(log, "on block structure: " + block.dumpStructure()); if (isCancelled()) return true; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 1f342d88017..cfc1dde883c 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -540,7 +540,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt executeTotalsAndHaving(pipeline, expressions.has_having, expressions.before_having, aggregate_overflow_row, !query.group_by_with_rollup); if (query.group_by_with_rollup) - executeRollup(pipeline, expressions.before_aggregation); + executeRollup(pipeline); } else if (expressions.has_having) executeHaving(pipeline, expressions.before_having); @@ -560,7 +560,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row, !query.group_by_with_rollup); if (query.group_by_with_rollup && !aggregate_final) - executeRollup(pipeline, expressions.before_aggregation); + executeRollup(pipeline); } if (expressions.has_order_by) @@ -979,14 +979,8 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold, final); } -void InterpreterSelectQuery::executeRollup(Pipeline & pipeline, const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) { - pipeline.transform([&](auto & stream) - { - stream = std::make_shared( - std::make_shared(stream, expression)); - }); - executeUnion(pipeline); Names key_names; @@ -999,23 +993,13 @@ void InterpreterSelectQuery::executeRollup(Pipeline & pipeline, const Expression for (const auto & name : key_names) keys.push_back(header.getPositionByName(name)); - // for (auto & descr : aggregates) - // if (descr.arguments.empty()) - // descr.arguments.push_back(header.getPositionByName(descr.column_name)); const Settings & settings = context.getSettingsRef(); - /** Two-level aggregation is useful in two cases: - * 1. Parallel aggregation is done, and the results should be merged in parallel. - * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. - */ - bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.max_bytes_before_external_group_by != 0; - Aggregator::Params params(header, keys, aggregates, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile, - 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), + SettingUInt64(0), SettingUInt64(0), settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath()); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 677c2c6942f..8e8289e471b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -190,7 +190,7 @@ private: void executeDistinct(Pipeline & pipeline, bool before_order, Names columns); void executeExtremes(Pipeline & pipeline); void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map & subqueries_for_sets); - void executeRollup(Pipeline & pipeline, const ExpressionActionsPtr & expression); + void executeRollup(Pipeline & pipeline); /** If there is a SETTINGS section in the SELECT query, then apply settings from it. * diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 16e354b238f..5fb2fe42b4f 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -11,10 +11,6 @@ #include -#include -#include - - namespace DB { @@ -138,6 +134,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) { if (!s_totals.ignore(pos, expected)) return false; + select_query->group_by_with_totals = true; } From a04c37251bd62eb2151744552f8a2d15ea3aab25 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 27 Aug 2018 12:09:47 +0300 Subject: [PATCH 11/62] remove redundant code --- dbms/src/DataStreams/RollupBlockInputStream.cpp | 4 ---- dbms/src/DataStreams/RollupBlockInputStream.h | 7 +------ 2 files changed, 1 insertion(+), 10 deletions(-) diff --git a/dbms/src/DataStreams/RollupBlockInputStream.cpp b/dbms/src/DataStreams/RollupBlockInputStream.cpp index 4f808ce4e52..bbd18d345a2 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.cpp +++ b/dbms/src/DataStreams/RollupBlockInputStream.cpp @@ -55,10 +55,6 @@ Block RollupBlockInputStream::readImpl() Aggregator::CancellationHook hook = [&]() { return this->isCancelled(); }; aggregator.setCancellationHook(hook); - StringRefs key(params.keys_size); - ColumnRawPtrs key_columns(params.keys_size); - AggregateColumns aggregate_columns(params.aggregates_size); - Block rollup_block = block; for (int i = static_cast(params.keys_size) - 1; i >= 0; --i) diff --git a/dbms/src/DataStreams/RollupBlockInputStream.h b/dbms/src/DataStreams/RollupBlockInputStream.h index 775668afe02..a4107fd5b11 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.h +++ b/dbms/src/DataStreams/RollupBlockInputStream.h @@ -12,8 +12,7 @@ class ExpressionActions; /** Takes blocks after grouping, with non-finalized aggregate functions. - * Calculates total values according to totals_mode. - * If necessary, evaluates the expression from HAVING and filters rows. Returns the finalized and filtered blocks. + * Calculates subtotals and grand totals values for a set of columns. */ class RollupBlockInputStream : public IProfilingBlockInputStream { @@ -21,7 +20,6 @@ private: using ExpressionActionsPtr = std::shared_ptr; using AggregateColumns = std::vector; public: - /// expression may be nullptr RollupBlockInputStream( const BlockInputStreamPtr & input_, const Aggregator::Params & params_); @@ -33,9 +31,6 @@ protected: Block readImpl() override; private: - size_t passed_keys = 0; - size_t total_keys = 0; - Aggregator::Params params; }; From b48a144d78cadd0c1064609f32fbfb004091f995 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 27 Aug 2018 12:50:30 +0300 Subject: [PATCH 12/62] add mysql like rollup syntax --- dbms/src/Parsers/ParserSelectQuery.cpp | 22 ++++++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 5fb2fe42b4f..8bdeaeb2e4d 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -129,13 +130,26 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - /// WITH TOTALS + ASTPtr totals_and_rollup_list; + const String identifier_totals = "identifier_totals"; + const String identifier_rollup = "identifier_rollup"; + + /// WITH TOTALS and/or ROLLUP if (s_with.ignore(pos, expected)) { - if (!s_totals.ignore(pos, expected)) + if (!exp_list.parse(pos, totals_and_rollup_list, expected)) return false; - - select_query->group_by_with_totals = true; + + for (const auto & child : totals_and_rollup_list->children) + { + String id = Poco::toLower(child->getID()); + if (id == identifier_totals) + select_query->group_by_with_totals = true; + else if (id == identifier_rollup) + select_query->group_by_with_rollup = true; + else + return false; + } } /// HAVING expr From 0e28195e7a21fc0384cf9f16e42fecb536d52bc2 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 27 Aug 2018 19:43:29 +0300 Subject: [PATCH 13/62] update syntax and test --- dbms/src/Parsers/ParserSelectQuery.cpp | 27 ++++++++++--------- .../0_stateless/00701_rollup.reference | 8 ++++++ .../queries/0_stateless/00701_rollup.sql | 4 +++ 3 files changed, 26 insertions(+), 13 deletions(-) diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 8bdeaeb2e4d..3c44517728f 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -1,5 +1,4 @@ #include -#include #include #include #include @@ -134,22 +133,24 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) const String identifier_totals = "identifier_totals"; const String identifier_rollup = "identifier_rollup"; - /// WITH TOTALS and/or ROLLUP + /// WITH ROLLUP if (s_with.ignore(pos, expected)) { - if (!exp_list.parse(pos, totals_and_rollup_list, expected)) + if (s_rollup.ignore(pos, expected)) + select_query->group_by_with_rollup = true; + else if (s_totals.ignore(pos, expected)) + select_query->group_by_with_totals = true; + else + return false; + } + + /// WITH TOTALS + if (s_with.ignore(pos, expected)) + { + if (select_query->group_by_with_totals || !s_totals.ignore(pos, expected)) return false; - for (const auto & child : totals_and_rollup_list->children) - { - String id = Poco::toLower(child->getID()); - if (id == identifier_totals) - select_query->group_by_with_totals = true; - else if (id == identifier_rollup) - select_query->group_by_with_rollup = true; - else - return false; - } + select_query->group_by_with_totals = true; } /// HAVING expr diff --git a/dbms/tests/queries/0_stateless/00701_rollup.reference b/dbms/tests/queries/0_stateless/00701_rollup.reference index 267ac5f5bd7..ec07ad52cae 100644 --- a/dbms/tests/queries/0_stateless/00701_rollup.reference +++ b/dbms/tests/queries/0_stateless/00701_rollup.reference @@ -17,3 +17,11 @@ b 2 35 2 120 8 a 70 4 b 50 4 + 120 8 +a 70 4 +b 50 4 + 120 8 +a 70 4 +b 50 4 + + 120 8 diff --git a/dbms/tests/queries/0_stateless/00701_rollup.sql b/dbms/tests/queries/0_stateless/00701_rollup.sql index ad225231ec5..1a52ccd09dc 100644 --- a/dbms/tests/queries/0_stateless/00701_rollup.sql +++ b/dbms/tests/queries/0_stateless/00701_rollup.sql @@ -15,3 +15,7 @@ SELECT a, b, sum(s), count() from test.rollup GROUP BY ROLLUP(a, b) ORDER BY a, SELECT a, b, sum(s), count() from test.rollup GROUP BY ROLLUP(a, b) WITH TOTALS ORDER BY a, b; SELECT a, sum(s), count() from test.rollup GROUP BY ROLLUP(a) ORDER BY a; + +SELECT a, sum(s), count() from test.rollup GROUP BY a WITH ROLLUP ORDER BY a; + +SELECT a, sum(s), count() from test.rollup GROUP BY a WITH ROLLUP WITH TOTALS ORDER BY a; From a603595120f146f9ad123bd0ee7ee49284639f44 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 28 Aug 2018 12:27:17 +0300 Subject: [PATCH 14/62] remove redundant code --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 7 +++++-- dbms/src/Parsers/ParserSelectQuery.cpp | 6 +----- 2 files changed, 6 insertions(+), 7 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index cfc1dde883c..b7402a06a76 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -705,9 +705,12 @@ void InterpreterSelectQuery::executeFetchColumns( max_streams = settings.max_distributed_connections; } - size_t max_block_size = settings.max_block_size; + size_t max_block_size = settings.max_block_size; - /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size, + if (!max_block_size) + throw Exception("Setting 'max_block_size' cannot be zero", ErrorCodes::PARAMETER_OUT_OF_BOUND); + + /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size, * then as the block size we will use limit + offset (not to read more from the table than requested), * and also set the number of threads to 1. */ diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 3c44517728f..24924e27d7b 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -129,11 +129,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - ASTPtr totals_and_rollup_list; - const String identifier_totals = "identifier_totals"; - const String identifier_rollup = "identifier_rollup"; - - /// WITH ROLLUP + /// WITH ROLLUP if (s_with.ignore(pos, expected)) { if (s_rollup.ignore(pos, expected)) From 3bc743c50d06c5845ea29a4d5e8dac4d3fd71420 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 3 Sep 2018 12:49:04 +0300 Subject: [PATCH 15/62] do not merge blocks into one --- .../DataStreams/RollupBlockInputStream.cpp | 20 +++++++++---------- dbms/src/DataStreams/RollupBlockInputStream.h | 1 + dbms/src/Interpreters/Aggregator.h | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 8 ++++---- dbms/src/Parsers/ParserSelectQuery.cpp | 2 +- 5 files changed, 16 insertions(+), 17 deletions(-) diff --git a/dbms/src/DataStreams/RollupBlockInputStream.cpp b/dbms/src/DataStreams/RollupBlockInputStream.cpp index bbd18d345a2..a8f0c90c45d 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.cpp +++ b/dbms/src/DataStreams/RollupBlockInputStream.cpp @@ -41,10 +41,17 @@ Block RollupBlockInputStream::getHeader() const Block RollupBlockInputStream::readImpl() { Block block; - BlocksList blocks; while(1) { + if (!blocks.empty()) + { + auto finalized = std::move(blocks.front()); + finalize(finalized); + blocks.pop_front(); + return finalized; + } + block = children[0]->read(); if (!block) @@ -57,7 +64,7 @@ Block RollupBlockInputStream::readImpl() Block rollup_block = block; - for (int i = static_cast(params.keys_size) - 1; i >= 0; --i) + for (ssize_t i = params.keys_size - 1; i >= 0; --i) { auto & current = rollup_block.getByPosition(params.keys[i]); current.column = current.column->cloneEmpty()->cloneResized(rollup_block.rows()); @@ -68,15 +75,6 @@ Block RollupBlockInputStream::readImpl() } finalize(block); - for (auto & current_block : blocks) - finalize(current_block); - - for (size_t i = 0; i < block.columns(); ++i) - { - MutableColumnPtr column = block.getByPosition(i).column->assumeMutable(); - for (const auto & current_block : blocks) - column->insertRangeFrom(*current_block.getByPosition(i).column.get(), 0, current_block.rows()); - } if (!block) continue; diff --git a/dbms/src/DataStreams/RollupBlockInputStream.h b/dbms/src/DataStreams/RollupBlockInputStream.h index a4107fd5b11..f80e4df6332 100644 --- a/dbms/src/DataStreams/RollupBlockInputStream.h +++ b/dbms/src/DataStreams/RollupBlockInputStream.h @@ -32,6 +32,7 @@ protected: private: Aggregator::Params params; + BlocksList blocks; }; } diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index cbe124511e2..5919a296085 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -844,7 +844,7 @@ struct AggregatedDataVariants : private boost::noncopyable throw Exception("Unknown aggregated data variant.", ErrorCodes::UNKNOWN_AGGREGATED_DATA_VARIANT); } } - + /// The size without taking into account the row in which data is written for the calculation of TOTALS. size_t sizeWithoutOverflowRow() const { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index b7402a06a76..89ac6933fad 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -61,6 +61,7 @@ namespace ErrorCodes extern const int TOO_MANY_COLUMNS; extern const int LOGICAL_ERROR; extern const int NOT_IMPLEMENTED; + extern const int PARAMETER_OUT_OF_BOUND; } InterpreterSelectQuery::InterpreterSelectQuery( @@ -555,7 +556,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt else { need_second_distinct_pass = query.distinct && pipeline.hasMoreThanOneStream(); - + if (query.group_by_with_totals && !aggregate_final) executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row, !query.group_by_with_rollup); @@ -861,10 +862,9 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre if (descr.arguments.empty()) for (const auto & name : descr.argument_names) descr.arguments.push_back(header.getPositionByName(name)); - const Settings & settings = context.getSettingsRef(); - + /** Two-level aggregation is useful in two cases: * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. @@ -1006,7 +1006,7 @@ void InterpreterSelectQuery::executeRollup(Pipeline & pipeline) settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, context.getTemporaryPath()); - pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params); } diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 24924e27d7b..480cb32b8bd 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -140,7 +140,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - /// WITH TOTALS + /// WITH TOTALS if (s_with.ignore(pos, expected)) { if (select_query->group_by_with_totals || !s_totals.ignore(pos, expected)) From 8d8aeb51e5cfa4a414a58a527f97823a1dc42dc3 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 4 Sep 2018 17:08:35 +0300 Subject: [PATCH 16/62] fix and optimize multiplying aggregate states --- dbms/src/Functions/FunctionsArithmetic.h | 99 ++++++++++++++---------- 1 file changed, 60 insertions(+), 39 deletions(-) diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index a9e6079f946..8d2f8d06c14 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -5,9 +5,11 @@ #include #include #include +#include #include #include #include +#include #include #include #include @@ -780,35 +782,6 @@ class FunctionBinaryArithmetic : public IFunction return castType(left, [&](const auto & left) { return castType(right, [&](const auto & right) { return f(left, right); }); }); } - bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1, bool & shift) const - { - return std::is_same_v, MultiplyImpl> - && - ( - ( - checkDataType(type0.get()) - && - ( - checkDataType(type1.get()) - || checkDataType(type1.get()) - ) - && - !(shift = false) - ) - || - ( - checkDataType(type1.get()) - && - ( - checkDataType(type0.get()) - || checkDataType(type0.get()) - ) - && - (shift = true) - ) - ); - } - FunctionBuilderPtr getFunctionForIntervalArithmetic(const DataTypePtr & type0, const DataTypePtr & type1) const { /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. @@ -866,9 +839,12 @@ public: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { /// Special case when multiply aggregate function state - bool shift; - if (isAggregateMultiply(arguments[0], arguments[1], shift)) - return arguments[shift?1:0]; + if (isAggregateMultiply(arguments[0], arguments[1])) + { + if (checkDataType(arguments[0].get())) + return arguments[0]; + return arguments[1]; + } /// Special case when the function is plus or minus, one of arguments is Date/DateTime and another is Interval. if (auto function_builder = getFunctionForIntervalArithmetic(arguments[0], arguments[1])) @@ -908,16 +884,61 @@ public: return type_res; } + bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const + { + if constexpr (!std::is_same_v, MultiplyImpl>) + return false; + auto is_uint_type = [](const DataTypePtr & type) + { + return checkDataType(type.get()) || checkDataType(type.get()) + || checkDataType(type.get()) || checkDataType(type.get()); + }; + return ((checkDataType(type0.get()) && is_uint_type(type1)) + || (is_uint_type(type0) && checkDataType(type1.get()))); + } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - bool shift; - if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type, shift)) + /// Special case when multiply aggregate function state + if (isAggregateMultiply(block.getByPosition(arguments[0]).type, block.getByPosition(arguments[1]).type)) { - auto c = block.getByPosition(arguments[shift?1:0]).column->cloneEmpty(); - size_t m = block.getByPosition(arguments[shift?0:1]).column->getUInt(0); - for (size_t i = 0; i < m; ++i) - c->insertRangeFrom(*(block.getByPosition(arguments[shift?1:0]).column.get()), 0, input_rows_count); - block.getByPosition(result).column = std::move(c); + ColumnNumbers new_arguments = arguments; + if (checkDataType(block.getByPosition(new_arguments[1]).type.get())) + std::swap(new_arguments[0], new_arguments[1]); + + const ColumnAggregateFunction * column = typeid_cast(block.getByPosition(new_arguments[0]).column.get()); + IAggregateFunction * function = column->getAggregateFunction().get(); + + MutableColumnPtr current = column->cloneEmpty(); + auto arena = std::make_shared(); + auto & res = typeid_cast(*current); + auto & vec_to = res.getData(); + const auto & vec_from = column->getData(); + + for (size_t i = 0; i < input_rows_count; ++i) + res.insertDefault(); + + size_t m = block.getByPosition(new_arguments[1]).column->getUInt(0); + + /// We use exponentiation by squaring algorithm to perform multiplying aggregate states by N in O(log(N)) operations + /// https://en.wikipedia.org/wiki/Exponentiation_by_squaring + while (m) + { + if (m % 2) + { + for (size_t i = 0; i < input_rows_count; ++i) + function->merge(vec_to[i], vec_from[i], arena.get()); + --m; + } + else + { + for (size_t i = 0; i < input_rows_count; ++i) + function->merge(vec_from[i], vec_from[i], arena.get()); + m /= 2; + } + } + + block.getByPosition(result).column = std::move(current); return; } From 3c6db8ebc150c11db3e0d4074f9349c11213b413 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Mon, 3 Sep 2018 20:24:46 +0300 Subject: [PATCH 17/62] Fix prewhere with aliased columns. [#CLICKHOUSE-3944] --- dbms/src/Interpreters/ExpressionActions.cpp | 41 ++++++++- dbms/src/Interpreters/ExpressionActions.h | 3 + dbms/src/Interpreters/ExpressionAnalyzer.cpp | 14 +-- dbms/src/Interpreters/ExpressionAnalyzer.h | 5 +- .../Interpreters/InterpreterSelectQuery.cpp | 85 +++++++++++++++++-- .../MergeTreeBaseBlockInputStream.cpp | 13 ++- .../MergeTree/MergeTreeBlockInputStream.cpp | 5 +- .../MergeTree/MergeTreeRangeReader.cpp | 9 +- .../Storages/MergeTree/MergeTreeRangeReader.h | 5 +- .../Storages/MergeTree/MergeTreeReadPool.cpp | 5 +- dbms/src/Storages/SelectQueryInfo.h | 4 +- 11 files changed, 164 insertions(+), 25 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 2158784a81b..10710ac9685 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -133,6 +133,14 @@ ExpressionAction ExpressionAction::project(const Names & projected_columns_) return a; } +ExpressionAction ExpressionAction::addAliases(const NamesWithAliases & aliased_columns_) +{ + ExpressionAction a; + a.type = ADD_ALIASES; + a.projection = aliased_columns_; + return a; +} + ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context) { if (array_joined_columns.empty()) @@ -265,6 +273,19 @@ void ExpressionAction::prepare(Block & sample_block) break; } + case ADD_ALIASES: + { + for (size_t i = 0; i < projection.size(); ++i) + { + const std::string & name = projection[i].first; + const std::string & alias = projection[i].second; + const ColumnWithTypeAndName & column = sample_block.getByName(name); + if (alias != "" && !sample_block.has(alias)) + sample_block.insert({column.column, column.type, alias}); + } + break; + } + case REMOVE_COLUMN: { sample_block.erase(source_name); @@ -448,6 +469,19 @@ void ExpressionAction::execute(Block & block, std::unordered_map join_, const Names & join_key_names_left, const NamesAndTypesList & columns_added_by_join_); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 0bd7efaded9..f4bf946f042 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -2894,7 +2894,7 @@ void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, Express } -ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result) +ExpressionActionsPtr ExpressionAnalyzer::getActions(bool add_aliases, bool project_result) { ExpressionActionsPtr actions = std::make_shared(source_columns, settings); NamesWithAliases result_columns; @@ -2911,7 +2911,7 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result) { std::string name = asts[i]->getColumnName(); std::string alias; - if (project_result) + if (add_aliases) alias = asts[i]->getAliasOrColumnName(); else alias = name; @@ -2920,11 +2920,15 @@ ExpressionActionsPtr ExpressionAnalyzer::getActions(bool project_result) getRootActions(asts[i], false, false, actions); } - if (project_result) + if (add_aliases) { - actions->add(ExpressionAction::project(result_columns)); + if (project_result) + actions->add(ExpressionAction::project(result_columns)); + else + actions->add(ExpressionAction::addAliases(result_columns)); } - else + + if (!project_result) { /// We will not delete the original columns. for (const auto & column_name_type : source_columns) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index cf056698379..4b68d647040 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -155,9 +155,10 @@ public: void appendProjectResult(ExpressionActionsChain & chain) const; /// If `ast` is not a SELECT query, just gets all the actions to evaluate the expression. - /// If project_result, only the calculated values in the desired order, renamed to aliases, remain in the output block. + /// If add_aliases, only the calculated values in the desired order and add aliases. + /// If also project_result, than only aliases remain in the output block. /// Otherwise, only temporary columns will be deleted from the block. - ExpressionActionsPtr getActions(bool project_result); + ExpressionActionsPtr getActions(bool add_aliases, bool project_result = true); /// Actions that can be performed on an empty block: adding constants and applying functions that depend only on constants. /// Does not execute subqueries. diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 9333d6d8cf4..1e78d35a887 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -627,6 +627,9 @@ static void getLimitLengthAndOffset(ASTSelectQuery & query, size_t & length, siz void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, Pipeline & pipeline, const PrewhereInfoPtr & prewhere_info) { + + const Settings & settings = context.getSettingsRef(); + /// Actions to calculate ALIAS if required. ExpressionActionsPtr alias_actions; /// Are ALIAS columns required for query execution? @@ -647,26 +650,98 @@ void InterpreterSelectQuery::executeFetchColumns( if (alias_columns_required) { + + NameSet required_prewhere_columns; + NameSet required_prewhere_aliases; + Block prewhere_actions_result; + if (prewhere_info) + { + auto required_columns = prewhere_info->prewhere_actions->getRequiredColumns(); + required_prewhere_columns.insert(required_columns.begin(), required_columns.end()); + prewhere_actions_result = prewhere_info->prewhere_actions->getSampleBlock(); + } + /// We will create an expression to return all the requested columns, with the calculation of the required ALIAS columns. auto required_columns_expr_list = std::make_shared(); + auto required_prewhere_columns_expr_list = std::make_shared(); + + auto source_columns = storage->getColumns().getAllPhysical(); for (const auto & column : required_columns) { + ASTPtr column_expr; const auto default_it = column_defaults.find(column); - if (default_it != std::end(column_defaults) && default_it->second.kind == ColumnDefaultKind::Alias) - required_columns_expr_list->children.emplace_back(setAlias(default_it->second.expression->clone(), column)); + bool is_alias = default_it != std::end(column_defaults) && default_it->second.kind == ColumnDefaultKind::Alias; + if (is_alias) + column_expr = setAlias(default_it->second.expression->clone(), column); else - required_columns_expr_list->children.emplace_back(std::make_shared(column)); + column_expr = std::make_shared(column); + + if (required_prewhere_columns.count(column)) + { + required_prewhere_columns_expr_list->children.emplace_back(std::move(column_expr)); + + if (is_alias) + required_prewhere_aliases.insert(column); + } + else + required_columns_expr_list->children.emplace_back(std::move(column_expr)); } - alias_actions = ExpressionAnalyzer(required_columns_expr_list, context, storage).getActions(true); + for (const auto & column : prewhere_actions_result) + { + if (prewhere_info->remove_prewhere_column && column.name == prewhere_info->prewhere_column_name) + continue; + + required_columns_expr_list->children.emplace_back(std::make_shared(column.name)); + source_columns.emplace_back(column.name, column.type); + } + + alias_actions = ExpressionAnalyzer(required_columns_expr_list, context, nullptr, source_columns).getActions(true); /// The set of required columns could be added as a result of adding an action to calculate ALIAS. required_columns = alias_actions->getRequiredColumns(); + + if (prewhere_info && prewhere_info->remove_prewhere_column) + if (required_columns.end() + != std::find(required_columns.begin(), required_columns.end(), prewhere_info->prewhere_column_name)) + prewhere_info->remove_prewhere_column = false; + + for (size_t i = 0; i < required_columns.size(); ++i) + { + if (!storage->getColumns().hasPhysical(required_columns[i])) + { + std::swap(required_columns[i], required_columns.back()); + required_columns.pop_back(); + } + } + + if (prewhere_info) + { + auto new_actions = std::make_shared(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), settings); + for (const auto & action : prewhere_info->prewhere_actions->getActions()) + { + if (action.type != ExpressionAction::REMOVE_COLUMN + || required_columns.end() == std::find(required_columns.begin(), required_columns.end(), action.source_name)) + new_actions->add(action); + } + prewhere_info->prewhere_actions = std::move(new_actions); + + prewhere_info->alias_actions = ExpressionAnalyzer(required_prewhere_columns_expr_list, context, storage).getActions(true, false); + auto required_aliased_columns = prewhere_info->alias_actions->getRequiredColumns(); + for (auto & column : required_aliased_columns) + if (!prewhere_actions_result.has(column)) + if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column)) + required_columns.push_back(column); + + for (const auto & column : required_prewhere_columns) + if (required_prewhere_aliases.count(column) == 0) + if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column)) + required_columns.push_back(column); + } } } - const Settings & settings = context.getSettingsRef(); /// Limitation on the number of columns to read. /// It's not applied in 'only_analyze' mode, because the query could be analyzed without removal of unnecessary columns. diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index ba137aa108f..c90c37fb9b4 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -121,26 +121,28 @@ Block MergeTreeBaseBlockInputStream::readFromPart() if (reader->getColumns().empty()) { task->range_reader = MergeTreeRangeReader( - pre_reader.get(), index_granularity, nullptr, prewhere_info->prewhere_actions, + pre_reader.get(), index_granularity, nullptr, + prewhere_info->alias_actions, prewhere_info->prewhere_actions, &prewhere_info->prewhere_column_name, &task->ordered_names, task->should_reorder, task->remove_prewhere_column, true); } else { task->pre_range_reader = MergeTreeRangeReader( - pre_reader.get(), index_granularity, nullptr, prewhere_info->prewhere_actions, + pre_reader.get(), index_granularity, nullptr, + prewhere_info->alias_actions, prewhere_info->prewhere_actions, &prewhere_info->prewhere_column_name, &task->ordered_names, task->should_reorder, task->remove_prewhere_column, false); task->range_reader = MergeTreeRangeReader( - reader.get(), index_granularity, &task->pre_range_reader, nullptr, + reader.get(), index_granularity, &task->pre_range_reader, nullptr, nullptr, nullptr, &task->ordered_names, true, false, true); } } else { task->range_reader = MergeTreeRangeReader( - reader.get(), index_granularity, nullptr, nullptr, + reader.get(), index_granularity, nullptr, nullptr, nullptr, nullptr, &task->ordered_names, task->should_reorder, false, true); } } @@ -218,6 +220,9 @@ void MergeTreeBaseBlockInputStream::executePrewhereActions(Block & block, const { if (prewhere_info) { + if (prewhere_info->alias_actions) + prewhere_info->alias_actions->execute(block); + prewhere_info->prewhere_actions->execute(block); if (prewhere_info->remove_prewhere_column) block.erase(prewhere_info->prewhere_column_name); diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp index 9e545360e02..8a3fccc5e43 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp @@ -109,7 +109,10 @@ try if (prewhere_info) { - pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); + if (prewhere_info->alias_actions) + pre_column_names = prewhere_info->alias_actions->getRequiredColumns(); + else + pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); if (pre_column_names.empty()) pre_column_names.push_back(column_names[0]); diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp index db0d9f9076a..d9aaf9acd84 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.cpp @@ -366,13 +366,13 @@ void MergeTreeRangeReader::ReadResult::setFilter(const ColumnPtr & new_filter) MergeTreeRangeReader::MergeTreeRangeReader( - MergeTreeReader * merge_tree_reader, size_t index_granularity, - MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions, + MergeTreeReader * merge_tree_reader, size_t index_granularity, MergeTreeRangeReader * prev_reader, + ExpressionActionsPtr alias_actions, ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, const Names * ordered_names, bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain) : index_granularity(index_granularity), merge_tree_reader(merge_tree_reader) , prev_reader(prev_reader), prewhere_column_name(prewhere_column_name) - , ordered_names(ordered_names), prewhere_actions(std::move(prewhere_actions)) + , ordered_names(ordered_names), alias_actions(alias_actions), prewhere_actions(std::move(prewhere_actions)) , always_reorder(always_reorder), remove_prewhere_column(remove_prewhere_column) , last_reader_in_chain(last_reader_in_chain), is_initialized(true) { @@ -571,6 +571,9 @@ void MergeTreeRangeReader::executePrewhereActionsAndFilterColumns(ReadResult & r if (!prewhere_actions) return; + if (alias_actions) + alias_actions->execute(result.block); + prewhere_actions->execute(result.block); auto & prewhere_column = result.block.getByName(*prewhere_column_name); size_t prev_rows = result.block.rows(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h index 21267b517e9..103ec2fb162 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeRangeReader.h @@ -19,8 +19,8 @@ class MergeTreeReader; class MergeTreeRangeReader { public: - MergeTreeRangeReader(MergeTreeReader * merge_tree_reader, size_t index_granularity, - MergeTreeRangeReader * prev_reader, ExpressionActionsPtr prewhere_actions, + MergeTreeRangeReader(MergeTreeReader * merge_tree_reader, size_t index_granularity, MergeTreeRangeReader * prev_reader, + ExpressionActionsPtr alias_actions, ExpressionActionsPtr prewhere_actions, const String * prewhere_column_name, const Names * ordered_names, bool always_reorder, bool remove_prewhere_column, bool last_reader_in_chain); @@ -175,6 +175,7 @@ private: const String * prewhere_column_name = nullptr; const Names * ordered_names = nullptr; + ExpressionActionsPtr alias_actions = nullptr; /// If not nullptr, calculate aliases. ExpressionActionsPtr prewhere_actions = nullptr; /// If not nullptr, calculate filter. Stream stream; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp index 55ba286dc08..16630e0d5c3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReadPool.cpp @@ -196,7 +196,10 @@ std::vector MergeTreeReadPool::fillPerPartInfo( if (prewhere_info) { /// collect columns required for PREWHERE evaluation - required_pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); + if (prewhere_info->alias_actions) + required_pre_column_names = prewhere_info->alias_actions->getRequiredColumns(); + else + required_pre_column_names = prewhere_info->prewhere_actions->getRequiredColumns(); /// there must be at least one column required for PREWHERE if (required_pre_column_names.empty()) diff --git a/dbms/src/Storages/SelectQueryInfo.h b/dbms/src/Storages/SelectQueryInfo.h index a448f2dc0bb..a6e40e4c27d 100644 --- a/dbms/src/Storages/SelectQueryInfo.h +++ b/dbms/src/Storages/SelectQueryInfo.h @@ -21,6 +21,8 @@ using PreparedSets = std::unordered_map; From 7bb92227cdea49dd77275119b9fac35e7efbe0bf Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Sep 2018 21:38:13 +0300 Subject: [PATCH 18/62] Added test. [#CLICKHOUSE-3944] --- .../00712_prewhere_with_alias.reference | 30 ++++++++++++++++ .../0_stateless/00712_prewhere_with_alias.sql | 35 +++++++++++++++++++ 2 files changed, 65 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00712_prewhere_with_alias.reference create mode 100644 dbms/tests/queries/0_stateless/00712_prewhere_with_alias.sql diff --git a/dbms/tests/queries/0_stateless/00712_prewhere_with_alias.reference b/dbms/tests/queries/0_stateless/00712_prewhere_with_alias.reference new file mode 100644 index 00000000000..6ee9c223dfd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00712_prewhere_with_alias.reference @@ -0,0 +1,30 @@ +- +1 +1 +1 +2 +- +1 +1 +1 +2 +- +1 +1 +1 +2 +- +1 +1 +1 +2 +- +1 +1 +1 +2 +- +1 +1 +1 +2 diff --git a/dbms/tests/queries/0_stateless/00712_prewhere_with_alias.sql b/dbms/tests/queries/0_stateless/00712_prewhere_with_alias.sql new file mode 100644 index 00000000000..5cfc2318f3d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00712_prewhere_with_alias.sql @@ -0,0 +1,35 @@ +drop table if exists test.prewhere_alias; +CREATE TABLE test.prewhere_alias (a UInt8, b Int32, c UInt8 ALIAS a, d Int64 ALIAS b + 1, e Int32 alias a + b) ENGINE = MergeTree ORDER BY tuple(); +insert into test.prewhere_alias values (1, 1); +select '-'; +select a from test.prewhere_alias prewhere a = 1; +select b from test.prewhere_alias prewhere a = 1; +select c from test.prewhere_alias prewhere a = 1; +select d from test.prewhere_alias prewhere a = 1; +select '-'; +select a from test.prewhere_alias prewhere b = 1; +select b from test.prewhere_alias prewhere b = 1; +select c from test.prewhere_alias prewhere b = 1; +select d from test.prewhere_alias prewhere b = 1; +select '-'; +select a from test.prewhere_alias prewhere c = 1; +select b from test.prewhere_alias prewhere c = 1; +select c from test.prewhere_alias prewhere c = 1; +select d from test.prewhere_alias prewhere c = 1; +select '-'; +select a from test.prewhere_alias prewhere d = 2; +select b from test.prewhere_alias prewhere d = 2; +select c from test.prewhere_alias prewhere d = 2; +select d from test.prewhere_alias prewhere d = 2; +select '-'; +select a from test.prewhere_alias prewhere a; +select b from test.prewhere_alias prewhere a; +select c from test.prewhere_alias prewhere a; +select d from test.prewhere_alias prewhere a; +select '-'; +select a from test.prewhere_alias prewhere c; +select b from test.prewhere_alias prewhere c; +select c from test.prewhere_alias prewhere c; +select d from test.prewhere_alias prewhere c; +drop table if exists test.prewhere_alias; + From 8b9937ec518c7b380b675cfe44b97f752da5d43a Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Sep 2018 21:50:19 +0300 Subject: [PATCH 19/62] More comments. [#CLICKHOUSE-3944] --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 1e78d35a887..577dd380c22 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -650,8 +650,9 @@ void InterpreterSelectQuery::executeFetchColumns( if (alias_columns_required) { - + /// Columns required for prewhere actions. NameSet required_prewhere_columns; + /// Columns required for prewhere actions which are aliases in storage. NameSet required_prewhere_aliases; Block prewhere_actions_result; if (prewhere_info) @@ -663,8 +664,10 @@ void InterpreterSelectQuery::executeFetchColumns( /// We will create an expression to return all the requested columns, with the calculation of the required ALIAS columns. auto required_columns_expr_list = std::make_shared(); + /// Separate expression for columns used in prewhere. auto required_prewhere_columns_expr_list = std::make_shared(); + /// Columns which we will get after prewhere execution. auto source_columns = storage->getColumns().getAllPhysical(); for (const auto & column : required_columns) @@ -688,6 +691,7 @@ void InterpreterSelectQuery::executeFetchColumns( required_columns_expr_list->children.emplace_back(std::move(column_expr)); } + /// Add columns which will be added by prewhere (otherwise we will remove them in project action). for (const auto & column : prewhere_actions_result) { if (prewhere_info->remove_prewhere_column && column.name == prewhere_info->prewhere_column_name) @@ -702,11 +706,13 @@ void InterpreterSelectQuery::executeFetchColumns( /// The set of required columns could be added as a result of adding an action to calculate ALIAS. required_columns = alias_actions->getRequiredColumns(); + /// Do not remove prewhere filter if it is a column which is used as alias. if (prewhere_info && prewhere_info->remove_prewhere_column) if (required_columns.end() != std::find(required_columns.begin(), required_columns.end(), prewhere_info->prewhere_column_name)) prewhere_info->remove_prewhere_column = false; + /// Remove columns which will be added by prewhere. for (size_t i = 0; i < required_columns.size(); ++i) { if (!storage->getColumns().hasPhysical(required_columns[i])) @@ -718,6 +724,7 @@ void InterpreterSelectQuery::executeFetchColumns( if (prewhere_info) { + /// Don't remove columns which are needed to be aliased. auto new_actions = std::make_shared(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), settings); for (const auto & action : prewhere_info->prewhere_actions->getActions()) { @@ -728,12 +735,15 @@ void InterpreterSelectQuery::executeFetchColumns( prewhere_info->prewhere_actions = std::move(new_actions); prewhere_info->alias_actions = ExpressionAnalyzer(required_prewhere_columns_expr_list, context, storage).getActions(true, false); + + /// Add columns required by alias actions. auto required_aliased_columns = prewhere_info->alias_actions->getRequiredColumns(); for (auto & column : required_aliased_columns) if (!prewhere_actions_result.has(column)) if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column)) required_columns.push_back(column); + /// Add columns required by prewhere actions. for (const auto & column : required_prewhere_columns) if (required_prewhere_aliases.count(column) == 0) if (required_columns.end() == std::find(required_columns.begin(), required_columns.end(), column)) From eedd2c50de9da9cdbe47966be6b1d5d9fe54beab Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 4 Sep 2018 21:59:34 +0300 Subject: [PATCH 20/62] Fix test. [#CLICKHOUSE-3944] --- dbms/src/Interpreters/ExpressionActions.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 10710ac9685..57b50ceaaae 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -264,6 +264,7 @@ void ExpressionAction::prepare(Block & sample_block) const std::string & name = projection[i].first; const std::string & alias = projection[i].second; ColumnWithTypeAndName column = sample_block.getByName(name); + column.column = (*std::move(column.column)).mutate(); if (alias != "") column.name = alias; new_block.insert(std::move(column)); @@ -459,6 +460,7 @@ void ExpressionAction::execute(Block & block, std::unordered_map Date: Tue, 4 Sep 2018 22:48:05 +0300 Subject: [PATCH 21/62] Update ExpressionActions.cpp --- dbms/src/Interpreters/ExpressionActions.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 57b50ceaaae..19fd092ac0f 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -565,7 +565,7 @@ std::string ExpressionAction::toString() const } break; - case PROJECT: + case PROJECT: [[fallthrough]]; case ADD_ALIASES: ss << (type == PROJECT ? "PROJECT " : "ADD_ALIASES "); for (size_t i = 0; i < projection.size(); ++i) From 8e806d4e3d6b6e2aa241244f4d434d850f07e4d8 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 5 Sep 2018 09:21:33 +0300 Subject: [PATCH 22/62] Fix test. [#CLICKHOUSE-3944] --- dbms/src/Interpreters/ExpressionActions.cpp | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 19fd092ac0f..46b26bb16fb 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -264,7 +264,8 @@ void ExpressionAction::prepare(Block & sample_block) const std::string & name = projection[i].first; const std::string & alias = projection[i].second; ColumnWithTypeAndName column = sample_block.getByName(name); - column.column = (*std::move(column.column)).mutate(); + if (column.column) + column.column = (*std::move(column.column)).mutate(); if (alias != "") column.name = alias; new_block.insert(std::move(column)); @@ -460,7 +461,8 @@ void ExpressionAction::execute(Block & block, std::unordered_map Date: Wed, 5 Sep 2018 12:45:57 +0300 Subject: [PATCH 23/62] Revert "Merge branch 'master' into CLICKHOUSE-3893" This reverts commit 0704da57567f4e9c912ad86424b5adfb415fb31f, reversing changes made to 4534d449c15d3c0a4c68b745bf483f7f4b4cfdb1. --- .gitignore | 1 + .gitmodules | 11 +- .travis.yml | 107 +- CHANGELOG.draft.md | 30 - CHANGELOG.md | 42 - CHANGELOG_RU.md | 50 - CMakeLists.txt | 62 +- ci/install-libraries.sh | 3 + ci/install-os-packages.sh | 28 + ci/jobs/quick-build/run.sh | 2 +- cmake/Modules/FindODBC.cmake | 88 - cmake/arch.cmake | 17 +- cmake/find_capnp.cmake | 2 +- cmake/find_cpuid.cmake | 2 +- cmake/find_execinfo.cmake | 2 +- cmake/find_llvm.cmake | 6 - cmake/find_ltdl.cmake | 8 +- cmake/find_odbc.cmake | 117 +- cmake/find_poco.cmake | 3 +- cmake/find_rdkafka.cmake | 8 +- cmake/find_rt.cmake | 2 +- cmake/find_ssl.cmake | 2 +- cmake/find_zlib.cmake | 2 +- cmake/sanitize.cmake | 60 +- contrib/jemalloc | 1 - contrib/jemalloc-cmake/CMakeLists.txt | 52 - contrib/jemalloc-cmake/README | 1 - .../include/jemalloc/jemalloc.h | 16 - .../include/jemalloc/jemalloc_rename.h | 29 - .../include_linux_x86_64/README | 7 - .../internal/jemalloc_internal_defs.h | 373 - .../jemalloc/internal/jemalloc_preamble.h | 194 - .../jemalloc/jemalloc_defs.h | 43 - .../jemalloc/jemalloc_macros.h | 122 - .../jemalloc/jemalloc_protos.h | 66 - .../jemalloc/jemalloc_typedefs.h | 77 - contrib/libcpuid/CMakeLists.txt | 2 +- contrib/libmurmurhash2/CMakeLists.txt | 6 + contrib/libmurmurhash2/LICENSE | 1 + contrib/{murmurhash => libmurmurhash2}/README | 0 .../include/murmurhash2.h | 0 .../src/murmurhash2.cpp | 0 contrib/librdkafka-cmake/config.h | 74 - contrib/librdkafka-cmake/include/README | 1 - contrib/mariadb-connector-c | 1 - .../mariadb-connector-c-cmake/CMakeLists.txt | 66 - .../common/include/mysql/mysql.h | 1 - .../common/include/mysql/mysqld_error.h | 1 - .../linux_x86_64/include/config.h | 269 - .../linux_x86_64/include/ma_config.h | 269 - .../linux_x86_64/include/mariadb_version.h | 36 - .../libmariadb/ma_client_plugin.c | 499 - contrib/murmurhash/CMakeLists.txt | 7 - contrib/murmurhash/LICENSE | 1 - contrib/murmurhash/include/murmurhash3.h | 37 - contrib/murmurhash/src/murmurhash3.cpp | 331 - contrib/re2_st/CMakeLists.txt | 3 +- contrib/unixodbc | 1 - .../linux_x86_64/libltdl/config.h | 181 - .../linux_x86_64/libltdl/libltdlcS.c | 53 - .../linux_x86_64/private/config.h | 496 - .../linux_x86_64/unixodbc_conf.h | 60 - dbms/CMakeLists.txt | 7 +- dbms/cmake/version.cmake | 27 +- dbms/programs/CMakeLists.txt | 15 +- dbms/programs/clang/CMakeLists.txt | 4 +- .../clang/Compiler-5.0.0/CMakeLists.txt | 3 - .../clang/Compiler-6.0.0/CMakeLists.txt | 3 - .../clang/Compiler-7.0.0/CMakeLists.txt | 3 - dbms/programs/client/Client.cpp | 356 +- dbms/programs/client/TestHint.h | 118 - dbms/programs/copier/ClusterCopier.cpp | 4 - dbms/programs/main.cpp | 8 - dbms/programs/odbc-bridge/CMakeLists.txt | 31 - dbms/programs/odbc-bridge/ColumnInfoHandler.h | 30 - dbms/programs/odbc-bridge/HandlerFactory.cpp | 34 - dbms/programs/odbc-bridge/HandlerFactory.h | 38 - dbms/programs/odbc-bridge/MainHandler.h | 49 - dbms/programs/odbc-bridge/ODBCBridge.cpp | 205 - dbms/programs/odbc-bridge/ODBCBridge.h | 41 - dbms/programs/odbc-bridge/PingHandler.cpp | 22 - dbms/programs/odbc-bridge/PingHandler.h | 17 - dbms/programs/odbc-bridge/README.md | 38 - dbms/programs/odbc-bridge/odbc-bridge.cpp | 2 - .../programs/odbc-bridge/tests/CMakeLists.txt | 2 - .../tests/validate-odbc-connection-string.cpp | 24 - .../validate-odbc-connection-string.reference | 39 - .../tests/validate-odbc-connection-string.sh | 41 - .../validateODBCConnectionString.cpp | 241 - .../validateODBCConnectionString.h | 21 - .../performance-test/PerformanceTest.cpp | 5 +- dbms/programs/server/CMakeLists.txt | 2 +- dbms/programs/server/HTTPHandler.cpp | 72 +- dbms/programs/server/MetricsTransmitter.cpp | 2 +- dbms/programs/server/config.d/listen.xml | 1 - dbms/programs/server/config.xml | 4 +- .../AggregateFunctionFactory.cpp | 21 +- dbms/src/AggregateFunctions/QuantileExact.h | 2 +- .../QuantileExactWeighted.h | 2 +- .../registerAggregateFunctions.cpp | 2 - dbms/src/Client/Connection.cpp | 101 +- dbms/src/Client/Connection.h | 16 +- .../src/Client/ConnectionPoolWithFailover.cpp | 5 +- dbms/src/Client/MultiplexedConnections.cpp | 3 +- dbms/src/Columns/Collator.cpp | 3 + dbms/src/Columns/ColumnAggregateFunction.cpp | 20 - dbms/src/Columns/ColumnAggregateFunction.h | 5 - dbms/src/Columns/ColumnArray.cpp | 38 - dbms/src/Columns/ColumnArray.h | 2 - dbms/src/Columns/ColumnConst.cpp | 17 - dbms/src/Columns/ColumnConst.h | 3 - dbms/src/Columns/ColumnFixedString.cpp | 27 - dbms/src/Columns/ColumnFixedString.h | 5 - dbms/src/Columns/ColumnFunction.cpp | 9 - dbms/src/Columns/ColumnFunction.h | 1 - dbms/src/Columns/ColumnNullable.cpp | 7 - dbms/src/Columns/ColumnNullable.h | 1 - dbms/src/Columns/ColumnString.cpp | 41 - dbms/src/Columns/ColumnString.h | 5 - dbms/src/Columns/ColumnTuple.cpp | 11 - dbms/src/Columns/ColumnTuple.h | 1 - dbms/src/Columns/ColumnVector.cpp | 15 +- dbms/src/Columns/ColumnVector.h | 82 +- dbms/src/Columns/ColumnWithDictionary.cpp | 619 -- dbms/src/Columns/ColumnsCommon.cpp | 22 - dbms/src/Columns/ColumnsCommon.h | 39 - dbms/src/Columns/FilterDescription.cpp | 15 +- dbms/src/Columns/IColumn.h | 10 - dbms/src/Columns/IColumnDummy.h | 8 - dbms/src/Columns/IColumnUnique.h | 129 - dbms/src/Columns/ReverseIndex.h | 412 - dbms/src/Common/ActionBlocker.h | 8 +- dbms/src/Common/ActionLock.h | 2 +- .../BackgroundSchedulePool.cpp | 0 .../{Core => Common}/BackgroundSchedulePool.h | 0 dbms/src/Common/COWPtr.h | 10 + dbms/src/Common/ConcurrentBoundedQueue.h | 2 +- dbms/src/Common/Config/ConfigProcessor.cpp | 3 + dbms/src/Common/CurrentThread.cpp | 80 - dbms/src/Common/CurrentThread.h | 83 - dbms/src/Common/DNSResolver.cpp | 21 - dbms/src/Common/DNSResolver.h | 3 - dbms/src/Common/ErrorCodes.cpp | 12 +- dbms/src/Common/Exception.cpp | 4 +- dbms/src/Common/ExternalTable.h | 225 + dbms/src/Common/FieldVisitors.cpp | 50 +- dbms/src/Common/FieldVisitors.h | 108 +- dbms/src/Common/FileChecker.cpp | 3 +- dbms/src/Common/HashTable/HashMap.h | 2 - dbms/src/Common/HashTable/HashSet.h | 1 - dbms/src/Common/HashTable/HashTable.h | 20 +- dbms/src/Common/IFactoryWithAliases.h | 4 +- dbms/src/Common/MemoryTracker.cpp | 80 +- dbms/src/Common/MemoryTracker.h | 60 +- dbms/src/Common/NaNUtils.h | 8 - dbms/src/Common/ODBCBridgeHelper.h | 52 - dbms/src/Common/PODArray.h | 15 + dbms/src/Common/ProfileEvents.cpp | 105 +- dbms/src/Common/ProfileEvents.h | 84 +- dbms/src/Common/RWLockFIFO.cpp | 2 +- dbms/src/Common/SimpleActionBlocker.h | 79 - dbms/src/Common/SpaceSaving.h | 2 +- dbms/src/Common/StatusFile.cpp | 2 +- dbms/src/Common/Stopwatch.h | 64 +- dbms/src/Common/Throttler.h | 15 +- dbms/src/Common/UInt128.h | 11 +- dbms/src/Common/VariableContext.h | 12 - dbms/src/Common/ZooKeeper/KeeperException.h | 23 +- dbms/src/Common/ZooKeeper/LeaderElection.h | 2 +- dbms/src/Common/ZooKeeper/ZooKeeper.cpp | 6 +- dbms/src/Common/ZooKeeper/ZooKeeperHolder.h | 2 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp | 30 +- dbms/src/Common/ZooKeeper/ZooKeeperImpl.h | 8 +- dbms/src/Common/formatIPv6.cpp | 5 +- dbms/src/Common/localBackup.cpp | 2 +- dbms/src/Common/setThreadName.h | 3 - dbms/src/Common/tests/CMakeLists.txt | 2 +- .../Common/tests/gtest_rw_lock_fifo.cpp.cpp | 2 +- dbms/src/Core/AccurateComparison.h | 32 +- dbms/src/Core/Block.cpp | 20 +- dbms/src/Core/Block.h | 1 - dbms/src/Core/Defines.h | 2 - dbms/src/Core/ExternalTable.cpp | 182 - dbms/src/Core/ExternalTable.h | 111 - dbms/src/Core/Field.cpp | 20 - dbms/src/Core/Field.h | 74 +- dbms/src/Core/Names.h | 1 - dbms/src/Core/Protocol.h | 6 +- dbms/src/Core/Types.h | 126 +- dbms/src/Core/UUID.h | 2 +- .../AggregatingSortedBlockInputStream.cpp | 2 +- .../AsynchronousBlockInputStream.cpp | 84 - .../AsynchronousBlockInputStream.h | 59 +- dbms/src/DataStreams/BlockIO.h | 16 +- .../CollapsingSortedBlockInputStream.cpp | 73 +- .../CollapsingSortedBlockInputStream.h | 2 +- ...lumnWithDictionaryToFullBlockInputStream.h | 49 - .../ConvertingBlockInputStream.cpp | 18 +- .../DataStreams/CountingBlockOutputStream.h | 4 +- .../CreatingSetsBlockInputStream.cpp | 15 - .../DataStreams/FilterBlockInputStream.cpp | 27 +- dbms/src/DataStreams/FilterBlockInputStream.h | 7 +- .../GraphiteRollupSortedBlockInputStream.cpp | 2 +- .../IProfilingBlockInputStream.cpp | 36 +- .../DataStreams/IProfilingBlockInputStream.h | 7 +- .../InternalTextLogsRowOutputStream.cpp | 82 - .../InternalTextLogsRowOutputStream.h | 32 - .../MergeSortingBlockInputStream.cpp | 4 +- .../MergeSortingBlockInputStream.h | 2 +- ...ggregatedMemoryEfficientBlockInputStream.h | 3 +- .../MergingSortedBlockInputStream.cpp | 6 +- .../MergingSortedBlockInputStream.h | 11 +- .../DataStreams/NativeBlockInputStream.cpp | 10 +- .../DataStreams/NativeBlockOutputStream.cpp | 11 +- .../src/DataStreams/ParallelInputsProcessor.h | 12 +- .../DataStreams/RemoteBlockInputStream.cpp | 18 +- .../DataStreams/RemoteBlockOutputStream.cpp | 75 +- .../ReplacingSortedBlockInputStream.cpp | 2 +- .../SummingSortedBlockInputStream.cpp | 15 +- .../SummingSortedBlockInputStream.h | 4 +- ...sionedCollapsingSortedBlockInputStream.cpp | 11 +- ...ersionedCollapsingSortedBlockInputStream.h | 4 +- .../DataStreams/tests/expression_stream.cpp | 2 +- dbms/src/DataStreams/tests/filter_stream.cpp | 2 +- .../tests/filter_stream_hitlog.cpp | 0 dbms/src/DataStreams/tests/native_streams.cpp | 0 dbms/src/DataStreams/tests/sorting_stream.cpp | 0 dbms/src/DataStreams/tests/union_stream2.cpp | 2 +- .../DataTypes/DataTypeAggregateFunction.cpp | 4 +- dbms/src/DataTypes/DataTypeArray.cpp | 66 +- dbms/src/DataTypes/DataTypeArray.h | 35 +- dbms/src/DataTypes/DataTypeEnum.cpp | 38 +- dbms/src/DataTypes/DataTypeEnum.h | 10 +- dbms/src/DataTypes/DataTypeFactory.cpp | 22 - dbms/src/DataTypes/DataTypeFixedString.cpp | 4 +- dbms/src/DataTypes/DataTypeNullable.cpp | 62 +- dbms/src/DataTypes/DataTypeNullable.h | 37 +- dbms/src/DataTypes/DataTypeNumberBase.h | 1 - dbms/src/DataTypes/DataTypeString.cpp | 4 +- dbms/src/DataTypes/DataTypeTuple.cpp | 131 +- dbms/src/DataTypes/DataTypeTuple.h | 35 +- dbms/src/DataTypes/DataTypeWithDictionary.h | 170 - dbms/src/DataTypes/DataTypesDecimal.cpp | 220 - dbms/src/DataTypes/DataTypesDecimal.h | 272 - dbms/src/DataTypes/FieldToDataType.cpp | 6 - dbms/src/DataTypes/FieldToDataType.h | 1 - dbms/src/DataTypes/IDataType.cpp | 2 - dbms/src/DataTypes/IDataType.h | 84 +- dbms/src/DataTypes/NestedUtils.h | 2 +- dbms/src/DataTypes/tests/data_type_string.cpp | 16 +- .../tests/data_types_number_fixed.cpp | 7 +- dbms/src/Databases/DatabaseDictionary.cpp | 6 +- dbms/src/Dictionaries/CacheDictionary.cpp | 20 +- dbms/src/Dictionaries/CatBoostModel.cpp | 38 +- dbms/src/Dictionaries/CatBoostModel.h | 2 +- ...exKeyCacheDictionary_setAttributeValue.cpp | 10 +- .../Dictionaries/DictionarySourceFactory.cpp | 3 + dbms/src/Dictionaries/DictionaryStructure.cpp | 18 +- .../Dictionaries/MongoDBBlockInputStream.cpp | 11 +- .../Dictionaries/MongoDBDictionarySource.cpp | 13 +- .../Dictionaries/MySQLDictionarySource.cpp | 16 +- dbms/src/Dictionaries/ODBCBlockInputStream.h | 9 +- .../src/Dictionaries/ODBCDictionarySource.cpp | 106 +- dbms/src/Dictionaries/ODBCDictionarySource.h | 19 +- dbms/src/Dictionaries/tests/CMakeLists.txt | 0 dbms/src/Formats/FormatFactory.cpp | 3 - dbms/src/Formats/FormatSettings.h | 1 - .../Formats/JSONEachRowRowOutputStream.cpp | 2 +- dbms/src/Formats/JSONRowOutputStream.cpp | 8 +- .../Formats/ODBCDriver2BlockOutputStream.cpp | 96 - .../Formats/ODBCDriver2BlockOutputStream.h | 44 - dbms/src/Functions/CMakeLists.txt | 4 +- dbms/src/Functions/FunctionHelpers.h | 150 - dbms/src/Functions/FunctionsArithmetic.h | 447 +- dbms/src/Functions/FunctionsArray.h | 3 +- dbms/src/Functions/FunctionsCoding.h | 2 +- dbms/src/Functions/FunctionsComparison.h | 329 +- dbms/src/Functions/FunctionsConversion.h | 103 +- dbms/src/Functions/FunctionsDateTime.h | 6 +- .../Functions/FunctionsEmbeddedDictionaries.h | 8 +- .../Functions/FunctionsExternalDictionaries.h | 14 +- dbms/src/Functions/FunctionsExternalModels.h | 2 +- dbms/src/Functions/FunctionsHashing.cpp | 7 +- dbms/src/Functions/FunctionsHashing.h | 165 +- dbms/src/Functions/FunctionsMath.h | 20 +- dbms/src/Functions/FunctionsMiscellaneous.cpp | 163 +- dbms/src/Functions/FunctionsMiscellaneous.h | 1 - dbms/src/Functions/FunctionsRandom.h | 2 +- dbms/src/Functions/GeoUtils.h | 41 +- dbms/src/Functions/IFunction.h | 41 +- dbms/src/IO/CascadeWriteBuffer.h | 2 +- dbms/src/IO/InterserverWriteBuffer.h | 2 +- dbms/src/IO/LZ4_decompress_faster.cpp | 16 +- dbms/src/IO/LZ4_decompress_faster.h | 4 +- dbms/src/IO/LimitReadBuffer.cpp | 21 +- .../src/IO/MMapReadBufferFromFileDescriptor.h | 2 +- dbms/src/IO/MemoryReadWriteBuffer.cpp | 2 +- dbms/src/IO/ReadBufferFromPocoSocket.cpp | 13 +- dbms/src/IO/ReadHelpers.h | 1 - dbms/src/IO/WriteBufferFromFileDescriptor.cpp | 5 - .../IO/WriteBufferFromHTTPServerResponse.h | 2 +- dbms/src/IO/WriteBufferFromPocoSocket.cpp | 14 +- dbms/src/IO/WriteHelpers.h | 19 +- dbms/src/IO/WriteIntText.h | 33 +- dbms/src/IO/ZlibDeflatingWriteBuffer.cpp | 4 +- dbms/src/IO/ZlibInflatingReadBuffer.cpp | 4 +- dbms/src/IO/readFloatText.h | 80 - dbms/src/IO/tests/CMakeLists.txt | 5 +- dbms/src/IO/tests/limit_read_buffer.cpp | 4 +- dbms/src/IO/tests/limit_read_buffer2.cpp | 6 +- dbms/src/Interpreters/Aggregator.cpp | 46 +- dbms/src/Interpreters/AsynchronousMetrics.cpp | 34 - dbms/src/Interpreters/CMakeLists.txt | 6 +- dbms/src/Interpreters/ClientInfo.cpp | 15 - dbms/src/Interpreters/ClientInfo.h | 1 - dbms/src/Interpreters/Cluster.cpp | 95 +- dbms/src/Interpreters/Cluster.h | 2 +- .../ClusterProxy/DescribeStreamFactory.cpp | 59 + .../ClusterProxy/DescribeStreamFactory.h | 23 + .../ClusterProxy/SelectStreamFactory.cpp | 21 +- dbms/src/Interpreters/Context.cpp | 174 +- dbms/src/Interpreters/Context.h | 43 +- dbms/src/Interpreters/DDLWorker.cpp | 8 +- dbms/src/Interpreters/DNSCacheUpdater.cpp | 6 +- dbms/src/Interpreters/DictionaryFactory.cpp | 2 +- dbms/src/Interpreters/ExpressionActions.cpp | 26 +- dbms/src/Interpreters/ExpressionActions.h | 14 +- dbms/src/Interpreters/ExpressionAnalyzer.h | 84 +- dbms/src/Interpreters/ExpressionJIT.cpp | 4 +- dbms/src/Interpreters/ExternalLoader.cpp | 16 +- dbms/src/Interpreters/ExternalLoader.h | 7 +- .../Interpreters/InternalTextLogsQueue.cpp | 68 - dbms/src/Interpreters/InternalTextLogsQueue.h | 32 - .../Interpreters/InterpreterAlterQuery.cpp | 2 - .../Interpreters/InterpreterCheckQuery.cpp | 196 +- .../Interpreters/InterpreterCreateQuery.cpp | 40 - .../src/Interpreters/InterpreterCreateQuery.h | 2 - .../src/Interpreters/InterpreterDropQuery.cpp | 10 +- .../Interpreters/InterpreterSelectQuery.cpp | 146 +- .../src/Interpreters/InterpreterSelectQuery.h | 9 +- .../InterpreterSelectWithUnionQuery.cpp | 15 +- .../InterpreterSelectWithUnionQuery.h | 2 +- .../Interpreters/InterpreterSystemQuery.cpp | 32 +- dbms/src/Interpreters/Join.cpp | 18 +- dbms/src/Interpreters/Join.h | 6 +- dbms/src/Interpreters/ProcessList.cpp | 193 +- dbms/src/Interpreters/ProcessList.h | 146 +- dbms/src/Interpreters/ProfileEventsExt.cpp | 59 - dbms/src/Interpreters/ProfileEventsExt.h | 12 - dbms/src/Interpreters/QueryLog.cpp | 137 +- dbms/src/Interpreters/QueryLog.h | 12 - dbms/src/Interpreters/QueryThreadLog.cpp | 116 - dbms/src/Interpreters/QueryThreadLog.h | 61 - dbms/src/Interpreters/Settings.cpp | 39 - dbms/src/Interpreters/SystemLog.cpp | 1 - dbms/src/Interpreters/SystemLog.h | 58 +- dbms/src/Interpreters/ThreadStatusExt.cpp | 256 - dbms/src/Interpreters/convertFieldToType.cpp | 55 +- dbms/src/Parsers/ASTSystemQuery.cpp | 2 - dbms/src/Parsers/ASTSystemQuery.h | 1 - dbms/src/Parsers/ASTWithAlias.h | 2 +- dbms/src/Parsers/ParserUnionQueryElement.cpp | 4 +- dbms/src/Parsers/StringRange.h | 2 +- dbms/src/Storages/AlterCommands.h | 2 - .../DistributedBlockOutputStream.cpp | 15 +- dbms/src/Storages/IStorage.h | 36 +- dbms/src/Storages/Kafka/KafkaSettings.cpp | 44 - dbms/src/Storages/Kafka/KafkaSettings.h | 43 - .../MergeTree/BackgroundProcessingPool.h | 4 - .../Storages/MergeTree/DataPartsExchange.cpp | 5 +- .../Storages/MergeTree/DataPartsExchange.h | 1 - dbms/src/Storages/MergeTree/KeyCondition.cpp | 4 + dbms/src/Storages/MergeTree/MergeList.cpp | 13 +- dbms/src/Storages/MergeTree/MergeList.h | 5 +- .../MergeTreeBaseBlockInputStream.cpp | 37 +- .../MergeTree/MergeTreeBaseBlockInputStream.h | 11 +- .../MergeTree/MergeTreeBlockInputStream.cpp | 31 +- .../MergeTree/MergeTreeBlockInputStream.h | 6 +- dbms/src/Storages/MergeTree/MergeTreeData.cpp | 17 +- dbms/src/Storages/MergeTree/MergeTreeData.h | 4 - .../MergeTree/MergeTreeDataFormatVersion.h | 2 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 32 +- .../Storages/MergeTree/MergeTreeDataPart.cpp | 10 +- .../MergeTree/MergeTreeDataSelectExecutor.cpp | 45 +- .../MergeTree/MergeTreeDataSelectExecutor.h | 8 +- .../MergeTree/MergeTreeMutationEntry.h | 2 - .../MergeTree/MergeTreeRangeReader.cpp | 36 +- .../Storages/MergeTree/MergeTreeReadPool.cpp | 34 +- .../Storages/MergeTree/MergeTreeReadPool.h | 14 +- .../Storages/MergeTree/MergeTreeReader.cpp | 83 +- dbms/src/Storages/MergeTree/MergeTreeReader.h | 8 +- .../Storages/MergeTree/MergeTreeSettings.h | 6 +- .../MergeTreeThreadBlockInputStream.cpp | 16 +- .../MergeTreeThreadBlockInputStream.h | 7 +- .../MergeTree/MergedBlockOutputStream.h | 10 +- .../MergeTree/ReplicatedMergeTreeAddress.cpp | 9 +- .../MergeTree/ReplicatedMergeTreeAddress.h | 1 - .../ReplicatedMergeTreeAlterThread.h | 2 +- .../ReplicatedMergeTreeBlockOutputStream.cpp | 5 +- .../ReplicatedMergeTreeCleanupThread.cpp | 60 - .../ReplicatedMergeTreeCleanupThread.h | 6 +- .../ReplicatedMergeTreePartCheckThread.cpp | 2 +- .../ReplicatedMergeTreePartCheckThread.h | 2 +- .../MergeTree/ReplicatedMergeTreeQueue.cpp | 86 +- .../MergeTree/ReplicatedMergeTreeQueue.h | 21 +- .../ReplicatedMergeTreeRestartingThread.cpp | 2 +- .../ReplicatedMergeTreeRestartingThread.h | 2 +- .../MergeTree/StorageFromMergeTreeDataPart.h | 4 +- dbms/src/Storages/MergeTree/checkDataPart.cpp | 28 +- dbms/src/Storages/SelectQueryInfo.h | 19 - dbms/src/Storages/StorageBuffer.cpp | 21 +- dbms/src/Storages/StorageBuffer.h | 4 +- dbms/src/Storages/StorageCatBoostPool.cpp | 11 +- dbms/src/Storages/StorageCatBoostPool.h | 2 +- dbms/src/Storages/StorageDictionary.cpp | 4 +- dbms/src/Storages/StorageDictionary.h | 2 +- dbms/src/Storages/StorageDistributed.cpp | 62 +- dbms/src/Storages/StorageDistributed.h | 8 +- dbms/src/Storages/StorageFactory.cpp | 12 +- dbms/src/Storages/StorageFile.cpp | 5 +- dbms/src/Storages/StorageFile.h | 2 +- dbms/src/Storages/StorageJoin.cpp | 8 +- .../src/Storages/{Kafka => }/StorageKafka.cpp | 189 +- dbms/src/Storages/{Kafka => }/StorageKafka.h | 2 +- dbms/src/Storages/StorageLog.cpp | 126 +- dbms/src/Storages/StorageLog.h | 2 +- dbms/src/Storages/StorageMaterializedView.cpp | 33 +- dbms/src/Storages/StorageMaterializedView.h | 9 +- dbms/src/Storages/StorageMemory.cpp | 6 +- dbms/src/Storages/StorageMemory.h | 2 +- dbms/src/Storages/StorageMerge.cpp | 83 +- dbms/src/Storages/StorageMerge.h | 4 +- dbms/src/Storages/StorageMergeTree.cpp | 67 +- dbms/src/Storages/StorageMergeTree.h | 8 +- dbms/src/Storages/StorageMySQL.cpp | 4 +- dbms/src/Storages/StorageMySQL.h | 2 +- dbms/src/Storages/StorageNull.h | 5 +- dbms/src/Storages/StorageODBC.h | 62 +- .../src/Storages/StorageReplicatedMergeTree.h | 8 +- dbms/src/Storages/StorageSet.cpp | 6 +- dbms/src/Storages/StorageSet.h | 2 +- dbms/src/Storages/StorageStripeLog.cpp | 8 +- dbms/src/Storages/StorageStripeLog.h | 2 +- dbms/src/Storages/StorageTinyLog.cpp | 58 +- dbms/src/Storages/StorageTinyLog.h | 2 +- dbms/src/Storages/StorageURL.cpp | 62 +- dbms/src/Storages/StorageURL.h | 56 +- dbms/src/Storages/StorageView.cpp | 4 +- dbms/src/Storages/StorageView.h | 2 +- .../Storages/System/IStorageSystemOneBlock.h | 4 +- .../Storages/System/StorageSystemColumns.cpp | 20 +- .../System/StorageSystemDatabases.cpp | 11 +- .../Storages/System/StorageSystemEvents.cpp | 2 +- .../System/StorageSystemMergeTreeSettings.cpp | 31 - .../System/StorageSystemMergeTreeSettings.h | 29 - .../Storages/System/StorageSystemMerges.cpp | 37 +- .../System/StorageSystemMutations.cpp | 11 +- .../Storages/System/StorageSystemNumbers.cpp | 6 +- .../Storages/System/StorageSystemNumbers.h | 2 +- dbms/src/Storages/System/StorageSystemOne.cpp | 6 +- dbms/src/Storages/System/StorageSystemOne.h | 2 +- .../System/StorageSystemPartsBase.cpp | 10 +- .../Storages/System/StorageSystemPartsBase.h | 2 +- .../System/StorageSystemProcesses.cpp | 52 +- .../Storages/System/StorageSystemReplicas.cpp | 16 +- .../Storages/System/StorageSystemReplicas.h | 2 +- .../System/StorageSystemReplicationQueue.cpp | 12 +- .../Storages/System/StorageSystemTables.cpp | 38 +- .../src/Storages/System/StorageSystemTables.h | 2 +- .../Storages/System/attachSystemTables.cpp | 2 - dbms/src/Storages/tests/hit_log.cpp | 0 dbms/src/Storages/tests/storage_log.cpp | 2 +- dbms/src/Storages/tests/system_numbers.cpp | 2 +- dbms/src/TableFunctions/CMakeLists.txt | 10 + dbms/src/TableFunctions/TableFunctionODBC.h | 6 + dbms/tests/clickhouse-test-server | 3 +- dbms/tests/instructions/sanitizers.md | 10 +- .../configs/config_allow_databases.xml | 20 - .../test_config_substitutions/test.py | 17 - .../test_https_replication/__init__.py | 0 .../test_https_replication/configs/config.xml | 364 - .../configs/no_ssl_conf.xml | 3 - .../configs/remote_servers.xml | 49 - .../test_https_replication/configs/server.crt | 19 - .../test_https_replication/configs/server.key | 28 - .../configs/ssl_conf.xml | 18 - .../test_https_replication/test.py | 103 - .../configs/remote_servers.xml | 12 - .../test_insert_into_distributed/test.py | 58 +- .../test_replace_partition/__init__.py | 0 .../configs/remote_servers.xml | 43 - .../test_replace_partition/test.py | 163 - .../integration/test_storage_kafka/test.py | 68 +- .../0_stateless/00002_system_numbers.sql | 2 - .../00043_summing_empty_part.reference | 1 + .../0_stateless/00050_any_left_join.reference | 20 +- .../0_stateless/00050_any_left_join.sql | 2 +- .../00051_any_inner_join.reference | 10 +- .../0_stateless/00051_any_inner_join.sql | 2 +- .../0_stateless/00052_all_left_join.reference | 30 +- .../0_stateless/00052_all_left_join.sql | 2 +- .../00053_all_inner_join.reference | 20 +- .../0_stateless/00053_all_inner_join.sql | 2 +- .../0_stateless/00054_join_string.reference | 30 +- .../queries/0_stateless/00054_join_string.sql | 2 +- ...80_show_tables_and_system_tables.reference | 1 - .../00080_show_tables_and_system_tables.sql | 4 - .../0_stateless/00098_1_union_all.reference | 2 - .../queries/0_stateless/00098_1_union_all.sql | 6 - .../queries/0_stateless/00155_long_merges.sh | 2 +- .../00158_buffer_and_nonexistent_table.sql | 1 - ...1_aggregate_functions_statistics.reference | 8 - .../00181_aggregate_functions_statistics.sql | 13 - .../00183_skip_unavailable_shards.sql | 1 - .../0_stateless/00336_shard_stack_trace.sh | 6 +- .../00443_optimize_final_vertical_merge.sh | 4 +- .../0_stateless/00474_readonly_settings.sh | 4 +- .../00492_drop_temporary_table.sql | 2 - .../00500_point_in_polygon_bug.reference | 1 - .../00500_point_in_polygon_bug.sql | 6 - .../00534_functions_bad_arguments.lib | 3 + .../0_stateless/00550_join_insert_select.sh | 2 +- .../00553_buff_exists_materlized_column.sql | 2 +- .../00555_right_join_excessive_rows.reference | 22 +- ..._versioned_collapsing_merge_tree.reference | 20 + .../0_stateless/00568_compile_catch_throw.sh | 2 +- ...column_exception_when_drop_depen_column.sh | 2 +- ...query_aggregation_column_removal.reference | 8 +- .../0_stateless/00595_insert_into_view.sh | 2 +- .../00596_limit_on_expanded_ast.sh | 2 +- .../queries/0_stateless/00602_throw_if.sh | 4 +- .../0_stateless/00606_quantiles_and_nans.sql | 17 +- ...remote_node_in_distributed_query.reference | 2 - ...l_and_remote_node_in_distributed_query.sql | 7 - .../00623_truncate_table_throw_exception.sh | 2 +- .../00626_replace_partition_from_table.sql | 2 +- ...replace_partition_from_table_zookeeper.sql | 2 +- ..._introspection_and_logging_shard.reference | 5 - ...ormance_introspection_and_logging_shard.sh | 135 - ...51_default_database_on_client_reconnect.sh | 2 +- .../0_stateless/00672_arrayDistinct.reference | 1 - .../0_stateless/00672_arrayDistinct.sql | 1 - .../00674_join_on_syntax.reference | 73 - .../0_stateless/00674_join_on_syntax.sql | 107 - .../0_stateless/00678_murmurhash.reference | 26 - .../0_stateless/00678_murmurhash2.reference | 15 + ...8_murmurhash.sql => 00678_murmurhash2.sql} | 15 - .../00679_replace_asterisk.reference | 4 - .../0_stateless/00679_replace_asterisk.sql | 4 - .../0_stateless/00679_uuid_in_key.reference | 6 - .../queries/0_stateless/00679_uuid_in_key.sql | 21 - ...plicate_columns_inside_union_all.reference | 8 - ...680_duplicate_columns_inside_union_all.sql | 4 - ...s_inside_union_all_stas_sviridov.reference | 2 - ...columns_inside_union_all_stas_sviridov.sql | 14 - ...rmat_json_escape_forward_slashes.reference | 2 - ...put_format_json_escape_forward_slashes.sql | 4 - .../00686_client_exit_code.reference | 1 - .../0_stateless/00686_client_exit_code.sh | 9 - .../00687_insert_into_mv.reference | 6 - .../0_stateless/00687_insert_into_mv.sql | 31 - .../00687_top_and_offset.reference | 8 - .../0_stateless/00687_top_and_offset.sh | 28 - .../00688_low_cardinality_defaults.reference | 4 - .../00688_low_cardinality_defaults.sql | 4 - ...ality_dictionary_deserialization.reference | 1 - ...cardinality_dictionary_deserialization.sql | 7 - ...88_low_cardinality_serialization.reference | 25 - .../00688_low_cardinality_serialization.sql | 30 - .../00688_low_cardinality_syntax.reference | 20 - .../00688_low_cardinality_syntax.sql | 74 - .../00689_catboost_pool_files.reference | 1 - .../0_stateless/00689_catboost_pool_files.sh | 30 - dbms/tests/queries/0_stateless/00689_file.txt | 1 - ...ect_converting_exception_message.reference | 2 - ...ert_select_converting_exception_message.sh | 15 - ...0699_materialized_view_mutations.reference | 8 - .../00699_materialized_view_mutations.sh | 51 - .../00700_decimal_arithm.reference | 25 - .../00700_decimal_bounds.reference | 21 - .../0_stateless/00700_decimal_bounds.sql | 96 - .../00700_decimal_compare.reference | 37 - .../0_stateless/00700_decimal_compare.sql | 71 - .../0_stateless/mergetree_mutations.lib | 20 - .../1_stateful/00001_count_hits.reference | 1 - .../queries/1_stateful/00001_count_hits.sql | 1 - .../1_stateful/00002_count_visits.reference | 1 - .../queries/1_stateful/00002_count_visits.sql | 1 - .../00003_count_mouse_clicks.reference | 1 - .../1_stateful/00003_count_mouse_clicks.sql | 1 - .../1_stateful/00004_top_counters.reference | 10 - .../queries/1_stateful/00004_top_counters.sql | 1 - .../1_stateful/00005_filtering.reference | 1 - .../queries/1_stateful/00005_filtering.sql | 2 - .../1_stateful/00006_agregates.reference | 1 - .../queries/1_stateful/00006_agregates.sql | 1 - .../queries/1_stateful/00007_uniq.reference | 10 - dbms/tests/queries/1_stateful/00007_uniq.sql | 1 - .../queries/1_stateful/00008_uniq.reference | 1 - dbms/tests/queries/1_stateful/00008_uniq.sql | 1 - .../00009_uniq_distributed.reference | 1 - .../1_stateful/00009_uniq_distributed.sql | 1 - .../00010_quantiles_segfault.reference | 3 - .../1_stateful/00010_quantiles_segfault.sql | 1 - .../1_stateful/00011_sorting.reference | 10 - .../queries/1_stateful/00011_sorting.sql | 1 - .../00012_sorting_distributed.reference | 10 - .../1_stateful/00012_sorting_distributed.sql | 1 - .../00013_sorting_of_nested.reference | 10 - .../1_stateful/00013_sorting_of_nested.sql | 2 - .../00014_filtering_arrays.reference | 10 - .../1_stateful/00014_filtering_arrays.sql | 1 - ...otals_and_no_aggregate_functions.reference | 27 - ...0015_totals_and_no_aggregate_functions.sql | 1 - ...if_distributed_cond_always_false.reference | 1 - ...6_any_if_distributed_cond_always_false.sql | 1 - ...aggregation_uninitialized_memory.reference | 47 - ...00017_aggregation_uninitialized_memory.sql | 2 - ...20_distinct_order_by_distributed.reference | 1 - .../00020_distinct_order_by_distributed.sql | 2 - .../00021_1_select_with_in.reference | 1 - .../1_stateful/00021_1_select_with_in.sql | 1 - .../00021_2_select_with_in.reference | 1 - .../1_stateful/00021_2_select_with_in.sql | 1 - .../00021_3_select_with_in.reference | 2 - .../1_stateful/00021_3_select_with_in.sql | 3 - .../1_stateful/00022_merge_prewhere.reference | 2 - .../1_stateful/00022_merge_prewhere.sql | 5 - .../1_stateful/00023_totals_limit.reference | 24 - .../queries/1_stateful/00023_totals_limit.sql | 2 - .../00024_random_counters.reference | 992 -- .../1_stateful/00024_random_counters.sql | 992 -- .../00030_array_enumerate_uniq.reference | 1 - .../1_stateful/00030_array_enumerate_uniq.sql | 1 - .../00031_array_enumerate_uniq.reference | 20 - .../1_stateful/00031_array_enumerate_uniq.sql | 20 - .../00032_aggregate_key64.reference | 10 - .../1_stateful/00032_aggregate_key64.sql | 1 - .../00033_aggregate_key_string.reference | 10 - .../1_stateful/00033_aggregate_key_string.sql | 1 - ...00034_aggregate_key_fixed_string.reference | 10 - .../00034_aggregate_key_fixed_string.sql | 1 - .../00035_aggregate_keys128.reference | 10 - .../1_stateful/00035_aggregate_keys128.sql | 1 - .../00036_aggregate_hashed.reference | 10 - .../1_stateful/00036_aggregate_hashed.sql | 1 - .../00037_uniq_state_merge1.reference | 100 - .../1_stateful/00037_uniq_state_merge1.sql | 1 - .../00038_uniq_state_merge2.reference | 100 - .../1_stateful/00038_uniq_state_merge2.sql | 1 - .../1_stateful/00039_primary_key.reference | 2 - .../queries/1_stateful/00039_primary_key.sql | 2 - ...40_aggregating_materialized_view.reference | 21 - .../00040_aggregating_materialized_view.sql | 44 - ...41_aggregating_materialized_view.reference | 21 - .../00041_aggregating_materialized_view.sql | 73 - .../1_stateful/00042_any_left_join.reference | 10 - .../1_stateful/00042_any_left_join.sql | 21 - .../1_stateful/00043_any_left_join.reference | 10 - .../1_stateful/00043_any_left_join.sql | 15 - .../00044_any_left_join_string.reference | 10 - .../1_stateful/00044_any_left_join_string.sql | 23 - .../1_stateful/00045_uniq_upto.reference | 0 .../queries/1_stateful/00045_uniq_upto.sql | 1 - .../00046_uniq_upto_distributed.reference | 0 .../00046_uniq_upto_distributed.sql | 1 - .../queries/1_stateful/00047_bar.reference | 100 - dbms/tests/queries/1_stateful/00047_bar.sql | 1 - .../1_stateful/00048_min_max.reference | 1 - .../queries/1_stateful/00048_min_max.sql | 1 - .../1_stateful/00049_max_string_if.reference | 20 - .../1_stateful/00049_max_string_if.sql | 1 - .../1_stateful/00050_min_max.reference | 20 - .../queries/1_stateful/00050_min_max.sql | 1 - .../1_stateful/00051_min_max_array.reference | 20 - .../1_stateful/00051_min_max_array.sql | 1 - .../1_stateful/00052_group_by_in.reference | 14 - .../queries/1_stateful/00052_group_by_in.sql | 4 - .../00053_replicate_segfault.reference | 1 - .../1_stateful/00053_replicate_segfault.sql | 1 - .../00054_merge_tree_partitions.reference | 12 - .../00054_merge_tree_partitions.sql | 36 - .../1_stateful/00055_index_and_not.reference | 1 - .../1_stateful/00055_index_and_not.sql | 1 - .../queries/1_stateful/00056_view.reference | 31 - dbms/tests/queries/1_stateful/00056_view.sql | 7 - ...merge_sorting_empty_array_joined.reference | 0 ...00059_merge_sorting_empty_array_joined.sql | 1 - .../00060_move_to_prewhere_and_sets.reference | 1 - .../00060_move_to_prewhere_and_sets.sql | 2 - .../1_stateful/00061_storage_buffer.reference | 4 - .../1_stateful/00061_storage_buffer.sql | 16 - .../1_stateful/00062_loyalty.reference | 12 - .../queries/1_stateful/00062_loyalty.sql | 1 - .../1_stateful/00063_loyalty_joins.reference | 51 - .../1_stateful/00063_loyalty_joins.sql | 94 - .../00065_loyalty_with_storage_join.reference | 24 - .../00065_loyalty_with_storage_join.sql | 34 - ...orting_distributed_many_replicas.reference | 10 - ...0066_sorting_distributed_many_replicas.sql | 2 - .../1_stateful/00067_union_all.reference | 20 - .../queries/1_stateful/00067_union_all.sql | 13 - .../00068_subquery_in_prewhere.reference | 1 - .../1_stateful/00068_subquery_in_prewhere.sql | 1 - ...00069_duplicate_aggregation_keys.reference | 52 - .../00069_duplicate_aggregation_keys.sql | 1 - .../00071_merge_tree_optimize_aio.reference | 100 - .../00071_merge_tree_optimize_aio.sql | 17 - ...72_compare_date_and_string_index.reference | 26 - .../00072_compare_date_and_string_index.sql | 37 - .../1_stateful/00073_uniq_array.reference | 7 - .../queries/1_stateful/00073_uniq_array.sql | 1 - .../1_stateful/00074_full_join.reference | 60 - .../queries/1_stateful/00074_full_join.sql | 106 - .../00075_left_array_join.reference | 200 - .../1_stateful/00075_left_array_join.sql | 2 - .../00076_system_columns_bytes.reference | 1 - .../1_stateful/00076_system_columns_bytes.sql | 1 - .../00077_log_tinylog_stripelog.reference | 10 - .../00077_log_tinylog_stripelog.sql | 28 - .../00078_group_by_arrays.reference | 40 - .../1_stateful/00078_group_by_arrays.sql | 4 - ...rray_join_not_used_joined_column.reference | 104 - ...0079_array_join_not_used_joined_column.sql | 3 - .../00080_array_join_and_union.reference | 1 - .../1_stateful/00080_array_join_and_union.sql | 1 - ..._group_by_without_key_and_totals.reference | 60 - .../00081_group_by_without_key_and_totals.sql | 15 - .../1_stateful/00082_quantiles.reference | 80 - .../queries/1_stateful/00082_quantiles.sql | 12 - .../1_stateful/00083_array_filter.reference | 2 - .../queries/1_stateful/00083_array_filter.sql | 2 - .../00084_external_aggregation.reference | 20 - .../1_stateful/00084_external_aggregation.sql | 10 - ...85_monotonic_evaluation_segfault.reference | 1 - .../00085_monotonic_evaluation_segfault.sql | 1 - .../1_stateful/00086_array_reduce.reference | 0 .../queries/1_stateful/00086_array_reduce.sql | 1 - .../1_stateful/00087_where_0.reference | 0 .../queries/1_stateful/00087_where_0.sql | 3 - ..._one_shard_and_rows_before_limit.reference | 26 - ...bal_in_one_shard_and_rows_before_limit.sql | 2 - ..._functions_with_non_constant_arg.reference | 9 - ...sition_functions_with_non_constant_arg.sql | 8 - .../00090_thread_pool_deadlock.reference | 10 - .../1_stateful/00090_thread_pool_deadlock.sh | 21 - .../queries/1_stateful/00139_like.reference | 4 - dbms/tests/queries/1_stateful/00139_like.sql | 5 - .../queries/1_stateful/00140_rename.reference | 12 - .../tests/queries/1_stateful/00140_rename.sql | 32 - .../1_stateful/00141_transform.reference | 3 - .../queries/1_stateful/00141_transform.sql | 1 - .../1_stateful/00142_system_columns.reference | 133 - .../1_stateful/00142_system_columns.sql | 1 - ...0143_transform_non_const_default.reference | 10 - .../00143_transform_non_const_default.sql | 1 - ..._functions_of_aggregation_states.reference | 7 - .../00144_functions_of_aggregation_states.sql | 1 - ...5_aggregate_functions_statistics.reference | 21 - .../00145_aggregate_functions_statistics.sql | 28 - .../00146_aggregate_function_uniq.reference | 19 - .../00146_aggregate_function_uniq.sql | 3 - ...147_global_in_aggregate_function.reference | 2 - .../00147_global_in_aggregate_function.sql | 2 - ...48_monotonic_functions_and_index.reference | 39 - .../00148_monotonic_functions_and_index.sql | 59 - ...149_quantiles_timing_distributed.reference | 1 - .../00149_quantiles_timing_distributed.sql | 1 - ...00150_quantiles_timing_precision.reference | 1 - .../00150_quantiles_timing_precision.sql | 1 - dbms/tests/queries/shell_config.sh | 2 - debian/changelog | 4 +- debian/clickhouse-server.postinst | 5 - debian/clickhouse-server.service | 1 - debian/control | 6 +- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- docs/en/data_types/boolean.md | 2 +- docs/en/data_types/datetime.md | 4 +- docs/en/data_types/float.md | 2 +- docs/en/data_types/index.md | 2 +- docs/en/data_types/int_uint.md | 4 +- .../nested_data_structures/index.md | 2 +- .../en/data_types/special_data_types/index.md | 2 +- docs/en/development/architecture.md | 8 +- docs/en/development/build.md | 91 +- docs/en/development/build_osx.md | 17 +- docs/en/development/style.md | 12 +- docs/en/faq/general.md | 4 +- .../example_datasets/criteo.md | 2 +- .../example_datasets/nyc_taxi.md | 8 +- docs/en/getting_started/index.md | 10 +- docs/en/index.md | 18 +- docs/en/interfaces/cli.md | 6 +- docs/en/interfaces/formats.md | 2 +- docs/en/interfaces/http_interface.md | 4 +- docs/en/interfaces/jdbc.md | 2 +- docs/en/interfaces/tcp.md | 2 +- .../third-party_client_libraries.md | 2 +- docs/en/interfaces/third-party_gui.md | 2 +- docs/en/introduction/distinctive_features.md | 24 +- .../features_considered_disadvantages.md | 2 +- docs/en/introduction/performance.md | 8 +- docs/en/introduction/ya_metrika_task.md | 6 +- docs/en/operations/access_rights.md | 2 +- docs/en/operations/configuration_files.md | 2 +- .../table_engines/custom_partitioning_key.md | 2 +- .../operations/table_engines/external_data.md | 2 +- docs/en/operations/table_engines/file.md | 6 +- .../table_engines/graphitemergetree.md | 2 +- docs/en/operations/table_engines/index.md | 2 +- docs/en/operations/table_engines/kafka.md | 59 +- docs/en/operations/table_engines/merge.md | 2 +- .../operations/table_engines/replication.md | 10 +- .../table_engines/summingmergetree.md | 2 +- docs/en/operations/table_engines/url.md | 4 +- docs/en/operations/tips.md | 16 +- docs/en/operations/utils/index.md | 2 +- .../agg_functions/parametric_functions.md | 30 - docs/en/query_language/alter.md | 12 +- docs/en/query_language/create.md | 4 +- .../en/query_language/dicts/external_dicts.md | 2 +- .../dicts/external_dicts_dict.md | 2 +- .../dicts/external_dicts_dict_layout.md | 6 +- .../dicts/external_dicts_dict_lifetime.md | 2 +- .../dicts/external_dicts_dict_sources.md | 10 +- .../dicts/external_dicts_dict_structure.md | 6 +- .../en/query_language/dicts/internal_dicts.md | 2 +- docs/en/query_language/index.md | 2 +- docs/en/query_language/insert_into.md | 4 +- docs/en/query_language/misc.md | 2 +- docs/en/query_language/operators.md | 30 +- docs/en/query_language/select.md | 44 +- docs/en/query_language/syntax.md | 8 +- docs/en/security_changelog.md | 21 - docs/fa/data_types/array.md | 7 - docs/fa/data_types/boolean.md | 7 - docs/fa/data_types/date.md | 9 - docs/fa/data_types/datetime.md | 15 - docs/fa/data_types/enum.md | 33 - docs/fa/data_types/fixedstring.md | 9 - docs/fa/data_types/float.md | 89 - docs/fa/data_types/index.md | 11 - docs/fa/data_types/int_uint.md | 22 - .../aggregatefunction.md | 7 - .../nested_data_structures/index.md | 5 - .../nested_data_structures/nested.md | 113 - .../special_data_types/expression.md | 7 - .../fa/data_types/special_data_types/index.md | 7 - docs/fa/data_types/special_data_types/set.md | 7 - docs/fa/data_types/string.md | 11 - docs/fa/data_types/tuple.md | 9 - .../example_datasets/amplab_benchmark.md | 136 - .../example_datasets/criteo.md | 89 - .../example_datasets/nyc_taxi.md | 411 - .../example_datasets/ontime.md | 393 - .../example_datasets/star_schema.md | 98 - .../example_datasets/wikistat.md | 35 - docs/fa/getting_started/index.md | 170 - docs/fa/index.md | 149 - docs/fa/interfaces/cli.md | 122 - docs/fa/interfaces/formats.md | 582 -- docs/fa/interfaces/http_interface.md | 282 - docs/fa/interfaces/index.md | 9 - docs/fa/interfaces/jdbc.md | 14 - docs/fa/interfaces/tcp.md | 7 - .../third-party_client_libraries.md | 50 - docs/fa/interfaces/third-party_gui.md | 38 - docs/fa/introduction/distinctive_features.md | 66 - .../features_considered_disadvantages.md | 9 - docs/fa/introduction/performance.md | 27 - docs/fa/introduction/ya_metrika_task.md | 48 - docs/redirects.txt | 1 - docs/ru/data_types/array.md | 2 +- docs/ru/data_types/datetime.md | 3 +- docs/ru/data_types/int_uint.md | 6 +- docs/ru/interfaces/jdbc.md | 3 +- docs/ru/interfaces/third-party_gui.md | 9 +- docs/ru/introduction/distinctive_features.md | 4 +- .../ru/operations/server_settings/settings.md | 2 +- docs/ru/operations/settings/index.md | 11 +- docs/ru/operations/settings/settings.md | 66 +- docs/ru/operations/system_tables.md | 50 +- .../table_engines/aggregatingmergetree.md | 4 +- docs/ru/operations/table_engines/kafka.md | 60 +- docs/ru/operations/table_engines/mysql.md | 14 +- .../operations/table_engines/replication.md | 2 +- .../table_engines/summingmergetree.md | 2 +- .../agg_functions/parametric_functions.md | 57 - docs/ru/query_language/alter.md | 7 +- .../functions/conditional_functions.md | 2 +- docs/ru/query_language/functions/geo.md | 33 +- docs/ru/query_language/misc.md | 2 +- .../query_language/table_functions/numbers.md | 9 +- docs/ru/security_changelog.md | 21 - docs/toc_en.yml | 9 +- docs/toc_ru.yml | 11 +- docs/tools/build.py | 80 +- .../javascripts/application.5165553b.js | 1 + .../assets/javascripts/application.js | 8978 ----------------- .../assets/javascripts/lunr/lunr.da.js | 2 +- .../assets/javascripts/lunr/lunr.de.js | 2 +- .../assets/javascripts/lunr/lunr.du.js | 2 +- .../assets/javascripts/lunr/lunr.es.js | 2 +- .../assets/javascripts/lunr/lunr.fi.js | 2 +- .../assets/javascripts/lunr/lunr.fr.js | 2 +- .../assets/javascripts/lunr/lunr.hu.js | 2 +- .../assets/javascripts/lunr/lunr.it.js | 2 +- .../assets/javascripts/lunr/lunr.jp.js | 2 +- .../assets/javascripts/lunr/lunr.js | 2987 ------ .../assets/javascripts/lunr/lunr.no.js | 2 +- .../assets/javascripts/lunr/lunr.pt.js | 2 +- .../assets/javascripts/lunr/lunr.ro.js | 2 +- .../assets/javascripts/lunr/lunr.ru.js | 2 +- .../javascripts/lunr/lunr.stemmer.support.js | 2 +- .../assets/javascripts/lunr/lunr.sv.js | 2 +- .../assets/javascripts/lunr/lunr.th.js | 97 - .../assets/javascripts/lunr/lunr.tr.js | 2 +- .../assets/javascripts/lunr/tinyseg.js | 2 +- .../assets/javascripts/modernizr.1aa3b519.js | 1 + .../assets/javascripts/modernizr.js | 91 - .../stylesheets/application.ac64251e.css | 2 +- docs/tools/mkdocs-material-theme/base.html | 11 +- .../partials/header.html | 6 +- .../partials/nav-item.html | 4 +- docs/tools/requirements.txt | 5 +- libs/libcommon/CMakeLists.txt | 14 +- libs/libcommon/cmake/find_gperftools.cmake | 15 +- libs/libcommon/cmake/find_jemalloc.cmake | 37 +- libs/libcommon/include/common/DateLUTImpl.h | 2 +- libs/libcommon/include/common/JSON.h | 16 +- libs/libcommon/include/common/StringRef.h | 2 +- libs/libcommon/include/common/intExp.h | 93 - libs/libcommon/include/ext/bit_cast.h | 4 +- libs/libcommon/include/ext/collection_cast.h | 2 +- libs/libcommon/include/ext/map.h | 4 +- libs/libcommon/src/JSON.cpp | 8 +- libs/libdaemon/CMakeLists.txt | 6 - libs/libdaemon/include/daemon/BaseDaemon.h | 9 +- .../include/daemon/ExtendedLogChannel.h | 38 - .../include/daemon/OwnPatternFormatter.h | 5 +- .../include/daemon/OwnSplitChannel.h | 35 - libs/libdaemon/src/BaseDaemon.cpp | 122 +- libs/libdaemon/src/ExtendedLogChannel.cpp | 27 - libs/libdaemon/src/OwnFormattingChannel.cpp | 33 - libs/libdaemon/src/OwnPatternFormatter.cpp | 50 +- libs/libdaemon/src/OwnSplitChannel.cpp | 64 - .../glibc-compatibility.c | 27 - libs/libmysqlxx/CMakeLists.txt | 38 +- libs/libmysqlxx/cmake/find_mysqlclient.cmake | 79 +- libs/libmysqlxx/include/mysqlxx/Connection.h | 7 +- libs/libmysqlxx/include/mysqlxx/Row.h | 2 +- libs/libmysqlxx/src/Connection.cpp | 16 +- libs/libmysqlxx/src/Pool.cpp | 3 +- libs/libmysqlxx/src/PoolWithFailover.cpp | 3 +- .../include/Poco/Ext/LevelFilterChannel.h | 53 +- libs/libpocoext/src/LevelFilterChannel.cpp | 84 +- release | 39 +- release_lib.sh | 23 +- utils/build/build_debian.sh | 2 +- utils/build/build_debian_unbundled.sh | 2 +- utils/fill-factor/main.cpp | 6 +- utils/iotest/iotest.cpp | 23 +- utils/iotest/iotest_aio.cpp | 27 +- utils/iotest/iotest_nonblock.cpp | 23 +- utils/travis/pbuilder.sh | 4 +- 967 files changed, 3929 insertions(+), 38374 deletions(-) delete mode 100644 cmake/Modules/FindODBC.cmake delete mode 160000 contrib/jemalloc delete mode 100644 contrib/jemalloc-cmake/CMakeLists.txt delete mode 100644 contrib/jemalloc-cmake/README delete mode 100644 contrib/jemalloc-cmake/include/jemalloc/jemalloc.h delete mode 100644 contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/README delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h delete mode 100644 contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h create mode 100644 contrib/libmurmurhash2/CMakeLists.txt create mode 100644 contrib/libmurmurhash2/LICENSE rename contrib/{murmurhash => libmurmurhash2}/README (100%) rename contrib/{murmurhash => libmurmurhash2}/include/murmurhash2.h (100%) rename contrib/{murmurhash => libmurmurhash2}/src/murmurhash2.cpp (100%) delete mode 100644 contrib/librdkafka-cmake/config.h delete mode 100644 contrib/librdkafka-cmake/include/README delete mode 160000 contrib/mariadb-connector-c delete mode 100644 contrib/mariadb-connector-c-cmake/CMakeLists.txt delete mode 100644 contrib/mariadb-connector-c-cmake/common/include/mysql/mysql.h delete mode 100644 contrib/mariadb-connector-c-cmake/common/include/mysql/mysqld_error.h delete mode 100644 contrib/mariadb-connector-c-cmake/linux_x86_64/include/config.h delete mode 100644 contrib/mariadb-connector-c-cmake/linux_x86_64/include/ma_config.h delete mode 100644 contrib/mariadb-connector-c-cmake/linux_x86_64/include/mariadb_version.h delete mode 100644 contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c delete mode 100644 contrib/murmurhash/CMakeLists.txt delete mode 100644 contrib/murmurhash/LICENSE delete mode 100644 contrib/murmurhash/include/murmurhash3.h delete mode 100644 contrib/murmurhash/src/murmurhash3.cpp delete mode 160000 contrib/unixodbc delete mode 100644 contrib/unixodbc-cmake/linux_x86_64/libltdl/config.h delete mode 100644 contrib/unixodbc-cmake/linux_x86_64/libltdl/libltdlcS.c delete mode 100644 contrib/unixodbc-cmake/linux_x86_64/private/config.h delete mode 100644 contrib/unixodbc-cmake/linux_x86_64/unixodbc_conf.h delete mode 100644 dbms/programs/client/TestHint.h delete mode 100644 dbms/programs/odbc-bridge/CMakeLists.txt delete mode 100644 dbms/programs/odbc-bridge/ColumnInfoHandler.h delete mode 100644 dbms/programs/odbc-bridge/HandlerFactory.cpp delete mode 100644 dbms/programs/odbc-bridge/HandlerFactory.h delete mode 100644 dbms/programs/odbc-bridge/MainHandler.h delete mode 100644 dbms/programs/odbc-bridge/ODBCBridge.cpp delete mode 100644 dbms/programs/odbc-bridge/ODBCBridge.h delete mode 100644 dbms/programs/odbc-bridge/PingHandler.cpp delete mode 100644 dbms/programs/odbc-bridge/PingHandler.h delete mode 100644 dbms/programs/odbc-bridge/README.md delete mode 100644 dbms/programs/odbc-bridge/odbc-bridge.cpp delete mode 100644 dbms/programs/odbc-bridge/tests/CMakeLists.txt delete mode 100644 dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.cpp delete mode 100644 dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.reference delete mode 100755 dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.sh delete mode 100644 dbms/programs/odbc-bridge/validateODBCConnectionString.cpp delete mode 100644 dbms/programs/odbc-bridge/validateODBCConnectionString.h delete mode 100644 dbms/programs/server/config.d/listen.xml delete mode 100644 dbms/src/Columns/ColumnWithDictionary.cpp delete mode 100644 dbms/src/Columns/IColumnUnique.h delete mode 100644 dbms/src/Columns/ReverseIndex.h rename dbms/src/{Core => Common}/BackgroundSchedulePool.cpp (100%) rename dbms/src/{Core => Common}/BackgroundSchedulePool.h (100%) delete mode 100644 dbms/src/Common/CurrentThread.cpp delete mode 100644 dbms/src/Common/CurrentThread.h create mode 100644 dbms/src/Common/ExternalTable.h delete mode 100644 dbms/src/Common/ODBCBridgeHelper.h delete mode 100644 dbms/src/Common/SimpleActionBlocker.h delete mode 100644 dbms/src/Common/VariableContext.h delete mode 100644 dbms/src/Core/ExternalTable.cpp delete mode 100644 dbms/src/Core/ExternalTable.h delete mode 100644 dbms/src/DataStreams/AsynchronousBlockInputStream.cpp delete mode 100644 dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h delete mode 100644 dbms/src/DataStreams/InternalTextLogsRowOutputStream.cpp delete mode 100644 dbms/src/DataStreams/InternalTextLogsRowOutputStream.h delete mode 100644 dbms/src/DataStreams/tests/filter_stream_hitlog.cpp delete mode 100644 dbms/src/DataStreams/tests/native_streams.cpp delete mode 100644 dbms/src/DataStreams/tests/sorting_stream.cpp delete mode 100644 dbms/src/DataTypes/DataTypeWithDictionary.h delete mode 100644 dbms/src/DataTypes/DataTypesDecimal.cpp delete mode 100644 dbms/src/DataTypes/DataTypesDecimal.h delete mode 100644 dbms/src/Dictionaries/tests/CMakeLists.txt delete mode 100644 dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp delete mode 100644 dbms/src/Formats/ODBCDriver2BlockOutputStream.h create mode 100644 dbms/src/Interpreters/ClusterProxy/DescribeStreamFactory.cpp create mode 100644 dbms/src/Interpreters/ClusterProxy/DescribeStreamFactory.h delete mode 100644 dbms/src/Interpreters/InternalTextLogsQueue.cpp delete mode 100644 dbms/src/Interpreters/InternalTextLogsQueue.h delete mode 100644 dbms/src/Interpreters/ProfileEventsExt.cpp delete mode 100644 dbms/src/Interpreters/ProfileEventsExt.h delete mode 100644 dbms/src/Interpreters/QueryThreadLog.cpp delete mode 100644 dbms/src/Interpreters/QueryThreadLog.h delete mode 100644 dbms/src/Interpreters/ThreadStatusExt.cpp delete mode 100644 dbms/src/Storages/Kafka/KafkaSettings.cpp delete mode 100644 dbms/src/Storages/Kafka/KafkaSettings.h rename dbms/src/Storages/{Kafka => }/StorageKafka.cpp (77%) rename dbms/src/Storages/{Kafka => }/StorageKafka.h (98%) delete mode 100644 dbms/src/Storages/System/StorageSystemMergeTreeSettings.cpp delete mode 100644 dbms/src/Storages/System/StorageSystemMergeTreeSettings.h delete mode 100644 dbms/src/Storages/tests/hit_log.cpp delete mode 100644 dbms/tests/integration/test_config_substitutions/configs/config_allow_databases.xml delete mode 100644 dbms/tests/integration/test_https_replication/__init__.py delete mode 100644 dbms/tests/integration/test_https_replication/configs/config.xml delete mode 100644 dbms/tests/integration/test_https_replication/configs/no_ssl_conf.xml delete mode 100644 dbms/tests/integration/test_https_replication/configs/remote_servers.xml delete mode 100644 dbms/tests/integration/test_https_replication/configs/server.crt delete mode 100644 dbms/tests/integration/test_https_replication/configs/server.key delete mode 100644 dbms/tests/integration/test_https_replication/configs/ssl_conf.xml delete mode 100644 dbms/tests/integration/test_https_replication/test.py delete mode 100644 dbms/tests/integration/test_replace_partition/__init__.py delete mode 100644 dbms/tests/integration/test_replace_partition/configs/remote_servers.xml delete mode 100644 dbms/tests/integration/test_replace_partition/test.py delete mode 100644 dbms/tests/queries/0_stateless/00500_point_in_polygon_bug.reference delete mode 100644 dbms/tests/queries/0_stateless/00500_point_in_polygon_bug.sql delete mode 100644 dbms/tests/queries/0_stateless/00614_shard_same_header_for_local_and_remote_node_in_distributed_query.reference delete mode 100644 dbms/tests/queries/0_stateless/00614_shard_same_header_for_local_and_remote_node_in_distributed_query.sql delete mode 100644 dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging_shard.reference delete mode 100755 dbms/tests/queries/0_stateless/00634_performance_introspection_and_logging_shard.sh delete mode 100644 dbms/tests/queries/0_stateless/00674_join_on_syntax.reference delete mode 100644 dbms/tests/queries/0_stateless/00674_join_on_syntax.sql delete mode 100644 dbms/tests/queries/0_stateless/00678_murmurhash.reference create mode 100644 dbms/tests/queries/0_stateless/00678_murmurhash2.reference rename dbms/tests/queries/0_stateless/{00678_murmurhash.sql => 00678_murmurhash2.sql} (70%) delete mode 100644 dbms/tests/queries/0_stateless/00679_replace_asterisk.reference delete mode 100644 dbms/tests/queries/0_stateless/00679_replace_asterisk.sql delete mode 100644 dbms/tests/queries/0_stateless/00679_uuid_in_key.reference delete mode 100644 dbms/tests/queries/0_stateless/00679_uuid_in_key.sql delete mode 100644 dbms/tests/queries/0_stateless/00680_duplicate_columns_inside_union_all.reference delete mode 100644 dbms/tests/queries/0_stateless/00680_duplicate_columns_inside_union_all.sql delete mode 100644 dbms/tests/queries/0_stateless/00681_duplicate_columns_inside_union_all_stas_sviridov.reference delete mode 100644 dbms/tests/queries/0_stateless/00681_duplicate_columns_inside_union_all_stas_sviridov.sql delete mode 100644 dbms/tests/queries/0_stateless/00685_output_format_json_escape_forward_slashes.reference delete mode 100644 dbms/tests/queries/0_stateless/00685_output_format_json_escape_forward_slashes.sql delete mode 100644 dbms/tests/queries/0_stateless/00686_client_exit_code.reference delete mode 100755 dbms/tests/queries/0_stateless/00686_client_exit_code.sh delete mode 100644 dbms/tests/queries/0_stateless/00687_insert_into_mv.reference delete mode 100644 dbms/tests/queries/0_stateless/00687_insert_into_mv.sql delete mode 100644 dbms/tests/queries/0_stateless/00687_top_and_offset.reference delete mode 100755 dbms/tests/queries/0_stateless/00687_top_and_offset.sh delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_defaults.reference delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_defaults.sql delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_dictionary_deserialization.reference delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_dictionary_deserialization.sql delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_serialization.reference delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_serialization.sql delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_syntax.reference delete mode 100644 dbms/tests/queries/0_stateless/00688_low_cardinality_syntax.sql delete mode 100644 dbms/tests/queries/0_stateless/00689_catboost_pool_files.reference delete mode 100755 dbms/tests/queries/0_stateless/00689_catboost_pool_files.sh delete mode 100644 dbms/tests/queries/0_stateless/00689_file.txt delete mode 100644 dbms/tests/queries/0_stateless/00690_insert_select_converting_exception_message.reference delete mode 100755 dbms/tests/queries/0_stateless/00690_insert_select_converting_exception_message.sh delete mode 100644 dbms/tests/queries/0_stateless/00699_materialized_view_mutations.reference delete mode 100755 dbms/tests/queries/0_stateless/00699_materialized_view_mutations.sh delete mode 100644 dbms/tests/queries/0_stateless/00700_decimal_arithm.reference delete mode 100644 dbms/tests/queries/0_stateless/00700_decimal_bounds.reference delete mode 100644 dbms/tests/queries/0_stateless/00700_decimal_bounds.sql delete mode 100644 dbms/tests/queries/0_stateless/00700_decimal_compare.reference delete mode 100644 dbms/tests/queries/0_stateless/00700_decimal_compare.sql delete mode 100644 dbms/tests/queries/0_stateless/mergetree_mutations.lib delete mode 100644 dbms/tests/queries/1_stateful/00001_count_hits.reference delete mode 100644 dbms/tests/queries/1_stateful/00001_count_hits.sql delete mode 100644 dbms/tests/queries/1_stateful/00002_count_visits.reference delete mode 100644 dbms/tests/queries/1_stateful/00002_count_visits.sql delete mode 100644 dbms/tests/queries/1_stateful/00003_count_mouse_clicks.reference delete mode 100644 dbms/tests/queries/1_stateful/00003_count_mouse_clicks.sql delete mode 100644 dbms/tests/queries/1_stateful/00004_top_counters.reference delete mode 100644 dbms/tests/queries/1_stateful/00004_top_counters.sql delete mode 100644 dbms/tests/queries/1_stateful/00005_filtering.reference delete mode 100644 dbms/tests/queries/1_stateful/00005_filtering.sql delete mode 100644 dbms/tests/queries/1_stateful/00006_agregates.reference delete mode 100644 dbms/tests/queries/1_stateful/00006_agregates.sql delete mode 100644 dbms/tests/queries/1_stateful/00007_uniq.reference delete mode 100644 dbms/tests/queries/1_stateful/00007_uniq.sql delete mode 100644 dbms/tests/queries/1_stateful/00008_uniq.reference delete mode 100644 dbms/tests/queries/1_stateful/00008_uniq.sql delete mode 100644 dbms/tests/queries/1_stateful/00009_uniq_distributed.reference delete mode 100644 dbms/tests/queries/1_stateful/00009_uniq_distributed.sql delete mode 100644 dbms/tests/queries/1_stateful/00010_quantiles_segfault.reference delete mode 100644 dbms/tests/queries/1_stateful/00010_quantiles_segfault.sql delete mode 100644 dbms/tests/queries/1_stateful/00011_sorting.reference delete mode 100644 dbms/tests/queries/1_stateful/00011_sorting.sql delete mode 100644 dbms/tests/queries/1_stateful/00012_sorting_distributed.reference delete mode 100644 dbms/tests/queries/1_stateful/00012_sorting_distributed.sql delete mode 100644 dbms/tests/queries/1_stateful/00013_sorting_of_nested.reference delete mode 100644 dbms/tests/queries/1_stateful/00013_sorting_of_nested.sql delete mode 100644 dbms/tests/queries/1_stateful/00014_filtering_arrays.reference delete mode 100644 dbms/tests/queries/1_stateful/00014_filtering_arrays.sql delete mode 100644 dbms/tests/queries/1_stateful/00015_totals_and_no_aggregate_functions.reference delete mode 100644 dbms/tests/queries/1_stateful/00015_totals_and_no_aggregate_functions.sql delete mode 100644 dbms/tests/queries/1_stateful/00016_any_if_distributed_cond_always_false.reference delete mode 100644 dbms/tests/queries/1_stateful/00016_any_if_distributed_cond_always_false.sql delete mode 100644 dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.reference delete mode 100644 dbms/tests/queries/1_stateful/00017_aggregation_uninitialized_memory.sql delete mode 100644 dbms/tests/queries/1_stateful/00020_distinct_order_by_distributed.reference delete mode 100644 dbms/tests/queries/1_stateful/00020_distinct_order_by_distributed.sql delete mode 100644 dbms/tests/queries/1_stateful/00021_1_select_with_in.reference delete mode 100644 dbms/tests/queries/1_stateful/00021_1_select_with_in.sql delete mode 100644 dbms/tests/queries/1_stateful/00021_2_select_with_in.reference delete mode 100644 dbms/tests/queries/1_stateful/00021_2_select_with_in.sql delete mode 100644 dbms/tests/queries/1_stateful/00021_3_select_with_in.reference delete mode 100644 dbms/tests/queries/1_stateful/00021_3_select_with_in.sql delete mode 100644 dbms/tests/queries/1_stateful/00022_merge_prewhere.reference delete mode 100644 dbms/tests/queries/1_stateful/00022_merge_prewhere.sql delete mode 100644 dbms/tests/queries/1_stateful/00023_totals_limit.reference delete mode 100644 dbms/tests/queries/1_stateful/00023_totals_limit.sql delete mode 100644 dbms/tests/queries/1_stateful/00024_random_counters.reference delete mode 100644 dbms/tests/queries/1_stateful/00024_random_counters.sql delete mode 100644 dbms/tests/queries/1_stateful/00030_array_enumerate_uniq.reference delete mode 100644 dbms/tests/queries/1_stateful/00030_array_enumerate_uniq.sql delete mode 100644 dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.reference delete mode 100644 dbms/tests/queries/1_stateful/00031_array_enumerate_uniq.sql delete mode 100644 dbms/tests/queries/1_stateful/00032_aggregate_key64.reference delete mode 100644 dbms/tests/queries/1_stateful/00032_aggregate_key64.sql delete mode 100644 dbms/tests/queries/1_stateful/00033_aggregate_key_string.reference delete mode 100644 dbms/tests/queries/1_stateful/00033_aggregate_key_string.sql delete mode 100644 dbms/tests/queries/1_stateful/00034_aggregate_key_fixed_string.reference delete mode 100644 dbms/tests/queries/1_stateful/00034_aggregate_key_fixed_string.sql delete mode 100644 dbms/tests/queries/1_stateful/00035_aggregate_keys128.reference delete mode 100644 dbms/tests/queries/1_stateful/00035_aggregate_keys128.sql delete mode 100644 dbms/tests/queries/1_stateful/00036_aggregate_hashed.reference delete mode 100644 dbms/tests/queries/1_stateful/00036_aggregate_hashed.sql delete mode 100644 dbms/tests/queries/1_stateful/00037_uniq_state_merge1.reference delete mode 100644 dbms/tests/queries/1_stateful/00037_uniq_state_merge1.sql delete mode 100644 dbms/tests/queries/1_stateful/00038_uniq_state_merge2.reference delete mode 100644 dbms/tests/queries/1_stateful/00038_uniq_state_merge2.sql delete mode 100644 dbms/tests/queries/1_stateful/00039_primary_key.reference delete mode 100644 dbms/tests/queries/1_stateful/00039_primary_key.sql delete mode 100644 dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.reference delete mode 100644 dbms/tests/queries/1_stateful/00040_aggregating_materialized_view.sql delete mode 100644 dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.reference delete mode 100644 dbms/tests/queries/1_stateful/00041_aggregating_materialized_view.sql delete mode 100644 dbms/tests/queries/1_stateful/00042_any_left_join.reference delete mode 100644 dbms/tests/queries/1_stateful/00042_any_left_join.sql delete mode 100644 dbms/tests/queries/1_stateful/00043_any_left_join.reference delete mode 100644 dbms/tests/queries/1_stateful/00043_any_left_join.sql delete mode 100644 dbms/tests/queries/1_stateful/00044_any_left_join_string.reference delete mode 100644 dbms/tests/queries/1_stateful/00044_any_left_join_string.sql delete mode 100644 dbms/tests/queries/1_stateful/00045_uniq_upto.reference delete mode 100644 dbms/tests/queries/1_stateful/00045_uniq_upto.sql delete mode 100644 dbms/tests/queries/1_stateful/00046_uniq_upto_distributed.reference delete mode 100644 dbms/tests/queries/1_stateful/00046_uniq_upto_distributed.sql delete mode 100644 dbms/tests/queries/1_stateful/00047_bar.reference delete mode 100644 dbms/tests/queries/1_stateful/00047_bar.sql delete mode 100644 dbms/tests/queries/1_stateful/00048_min_max.reference delete mode 100644 dbms/tests/queries/1_stateful/00048_min_max.sql delete mode 100644 dbms/tests/queries/1_stateful/00049_max_string_if.reference delete mode 100644 dbms/tests/queries/1_stateful/00049_max_string_if.sql delete mode 100644 dbms/tests/queries/1_stateful/00050_min_max.reference delete mode 100644 dbms/tests/queries/1_stateful/00050_min_max.sql delete mode 100644 dbms/tests/queries/1_stateful/00051_min_max_array.reference delete mode 100644 dbms/tests/queries/1_stateful/00051_min_max_array.sql delete mode 100644 dbms/tests/queries/1_stateful/00052_group_by_in.reference delete mode 100644 dbms/tests/queries/1_stateful/00052_group_by_in.sql delete mode 100644 dbms/tests/queries/1_stateful/00053_replicate_segfault.reference delete mode 100644 dbms/tests/queries/1_stateful/00053_replicate_segfault.sql delete mode 100644 dbms/tests/queries/1_stateful/00054_merge_tree_partitions.reference delete mode 100644 dbms/tests/queries/1_stateful/00054_merge_tree_partitions.sql delete mode 100644 dbms/tests/queries/1_stateful/00055_index_and_not.reference delete mode 100644 dbms/tests/queries/1_stateful/00055_index_and_not.sql delete mode 100644 dbms/tests/queries/1_stateful/00056_view.reference delete mode 100644 dbms/tests/queries/1_stateful/00056_view.sql delete mode 100644 dbms/tests/queries/1_stateful/00059_merge_sorting_empty_array_joined.reference delete mode 100644 dbms/tests/queries/1_stateful/00059_merge_sorting_empty_array_joined.sql delete mode 100644 dbms/tests/queries/1_stateful/00060_move_to_prewhere_and_sets.reference delete mode 100644 dbms/tests/queries/1_stateful/00060_move_to_prewhere_and_sets.sql delete mode 100644 dbms/tests/queries/1_stateful/00061_storage_buffer.reference delete mode 100644 dbms/tests/queries/1_stateful/00061_storage_buffer.sql delete mode 100644 dbms/tests/queries/1_stateful/00062_loyalty.reference delete mode 100644 dbms/tests/queries/1_stateful/00062_loyalty.sql delete mode 100644 dbms/tests/queries/1_stateful/00063_loyalty_joins.reference delete mode 100644 dbms/tests/queries/1_stateful/00063_loyalty_joins.sql delete mode 100644 dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.reference delete mode 100644 dbms/tests/queries/1_stateful/00065_loyalty_with_storage_join.sql delete mode 100644 dbms/tests/queries/1_stateful/00066_sorting_distributed_many_replicas.reference delete mode 100644 dbms/tests/queries/1_stateful/00066_sorting_distributed_many_replicas.sql delete mode 100644 dbms/tests/queries/1_stateful/00067_union_all.reference delete mode 100644 dbms/tests/queries/1_stateful/00067_union_all.sql delete mode 100644 dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.reference delete mode 100644 dbms/tests/queries/1_stateful/00068_subquery_in_prewhere.sql delete mode 100644 dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.reference delete mode 100644 dbms/tests/queries/1_stateful/00069_duplicate_aggregation_keys.sql delete mode 100644 dbms/tests/queries/1_stateful/00071_merge_tree_optimize_aio.reference delete mode 100644 dbms/tests/queries/1_stateful/00071_merge_tree_optimize_aio.sql delete mode 100644 dbms/tests/queries/1_stateful/00072_compare_date_and_string_index.reference delete mode 100644 dbms/tests/queries/1_stateful/00072_compare_date_and_string_index.sql delete mode 100644 dbms/tests/queries/1_stateful/00073_uniq_array.reference delete mode 100644 dbms/tests/queries/1_stateful/00073_uniq_array.sql delete mode 100644 dbms/tests/queries/1_stateful/00074_full_join.reference delete mode 100644 dbms/tests/queries/1_stateful/00074_full_join.sql delete mode 100644 dbms/tests/queries/1_stateful/00075_left_array_join.reference delete mode 100644 dbms/tests/queries/1_stateful/00075_left_array_join.sql delete mode 100644 dbms/tests/queries/1_stateful/00076_system_columns_bytes.reference delete mode 100644 dbms/tests/queries/1_stateful/00076_system_columns_bytes.sql delete mode 100644 dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.reference delete mode 100644 dbms/tests/queries/1_stateful/00077_log_tinylog_stripelog.sql delete mode 100644 dbms/tests/queries/1_stateful/00078_group_by_arrays.reference delete mode 100644 dbms/tests/queries/1_stateful/00078_group_by_arrays.sql delete mode 100644 dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.reference delete mode 100644 dbms/tests/queries/1_stateful/00079_array_join_not_used_joined_column.sql delete mode 100644 dbms/tests/queries/1_stateful/00080_array_join_and_union.reference delete mode 100644 dbms/tests/queries/1_stateful/00080_array_join_and_union.sql delete mode 100644 dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.reference delete mode 100644 dbms/tests/queries/1_stateful/00081_group_by_without_key_and_totals.sql delete mode 100644 dbms/tests/queries/1_stateful/00082_quantiles.reference delete mode 100644 dbms/tests/queries/1_stateful/00082_quantiles.sql delete mode 100644 dbms/tests/queries/1_stateful/00083_array_filter.reference delete mode 100644 dbms/tests/queries/1_stateful/00083_array_filter.sql delete mode 100644 dbms/tests/queries/1_stateful/00084_external_aggregation.reference delete mode 100644 dbms/tests/queries/1_stateful/00084_external_aggregation.sql delete mode 100644 dbms/tests/queries/1_stateful/00085_monotonic_evaluation_segfault.reference delete mode 100644 dbms/tests/queries/1_stateful/00085_monotonic_evaluation_segfault.sql delete mode 100644 dbms/tests/queries/1_stateful/00086_array_reduce.reference delete mode 100644 dbms/tests/queries/1_stateful/00086_array_reduce.sql delete mode 100644 dbms/tests/queries/1_stateful/00087_where_0.reference delete mode 100644 dbms/tests/queries/1_stateful/00087_where_0.sql delete mode 100644 dbms/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.reference delete mode 100644 dbms/tests/queries/1_stateful/00088_global_in_one_shard_and_rows_before_limit.sql delete mode 100644 dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.reference delete mode 100644 dbms/tests/queries/1_stateful/00089_position_functions_with_non_constant_arg.sql delete mode 100644 dbms/tests/queries/1_stateful/00090_thread_pool_deadlock.reference delete mode 100755 dbms/tests/queries/1_stateful/00090_thread_pool_deadlock.sh delete mode 100644 dbms/tests/queries/1_stateful/00139_like.reference delete mode 100644 dbms/tests/queries/1_stateful/00139_like.sql delete mode 100644 dbms/tests/queries/1_stateful/00140_rename.reference delete mode 100644 dbms/tests/queries/1_stateful/00140_rename.sql delete mode 100644 dbms/tests/queries/1_stateful/00141_transform.reference delete mode 100644 dbms/tests/queries/1_stateful/00141_transform.sql delete mode 100644 dbms/tests/queries/1_stateful/00142_system_columns.reference delete mode 100644 dbms/tests/queries/1_stateful/00142_system_columns.sql delete mode 100644 dbms/tests/queries/1_stateful/00143_transform_non_const_default.reference delete mode 100644 dbms/tests/queries/1_stateful/00143_transform_non_const_default.sql delete mode 100644 dbms/tests/queries/1_stateful/00144_functions_of_aggregation_states.reference delete mode 100644 dbms/tests/queries/1_stateful/00144_functions_of_aggregation_states.sql delete mode 100644 dbms/tests/queries/1_stateful/00145_aggregate_functions_statistics.reference delete mode 100644 dbms/tests/queries/1_stateful/00145_aggregate_functions_statistics.sql delete mode 100644 dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.reference delete mode 100644 dbms/tests/queries/1_stateful/00146_aggregate_function_uniq.sql delete mode 100644 dbms/tests/queries/1_stateful/00147_global_in_aggregate_function.reference delete mode 100644 dbms/tests/queries/1_stateful/00147_global_in_aggregate_function.sql delete mode 100644 dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.reference delete mode 100644 dbms/tests/queries/1_stateful/00148_monotonic_functions_and_index.sql delete mode 100644 dbms/tests/queries/1_stateful/00149_quantiles_timing_distributed.reference delete mode 100644 dbms/tests/queries/1_stateful/00149_quantiles_timing_distributed.sql delete mode 100644 dbms/tests/queries/1_stateful/00150_quantiles_timing_precision.reference delete mode 100644 dbms/tests/queries/1_stateful/00150_quantiles_timing_precision.sql delete mode 100644 docs/en/security_changelog.md delete mode 100644 docs/fa/data_types/array.md delete mode 100644 docs/fa/data_types/boolean.md delete mode 100644 docs/fa/data_types/date.md delete mode 100644 docs/fa/data_types/datetime.md delete mode 100644 docs/fa/data_types/enum.md delete mode 100644 docs/fa/data_types/fixedstring.md delete mode 100644 docs/fa/data_types/float.md delete mode 100644 docs/fa/data_types/index.md delete mode 100644 docs/fa/data_types/int_uint.md delete mode 100644 docs/fa/data_types/nested_data_structures/aggregatefunction.md delete mode 100644 docs/fa/data_types/nested_data_structures/index.md delete mode 100644 docs/fa/data_types/nested_data_structures/nested.md delete mode 100644 docs/fa/data_types/special_data_types/expression.md delete mode 100644 docs/fa/data_types/special_data_types/index.md delete mode 100644 docs/fa/data_types/special_data_types/set.md delete mode 100644 docs/fa/data_types/string.md delete mode 100644 docs/fa/data_types/tuple.md delete mode 100644 docs/fa/getting_started/example_datasets/amplab_benchmark.md delete mode 100644 docs/fa/getting_started/example_datasets/criteo.md delete mode 100644 docs/fa/getting_started/example_datasets/nyc_taxi.md delete mode 100644 docs/fa/getting_started/example_datasets/ontime.md delete mode 100644 docs/fa/getting_started/example_datasets/star_schema.md delete mode 100644 docs/fa/getting_started/example_datasets/wikistat.md delete mode 100644 docs/fa/getting_started/index.md delete mode 100644 docs/fa/index.md delete mode 100644 docs/fa/interfaces/cli.md delete mode 100644 docs/fa/interfaces/formats.md delete mode 100644 docs/fa/interfaces/http_interface.md delete mode 100644 docs/fa/interfaces/index.md delete mode 100644 docs/fa/interfaces/jdbc.md delete mode 100644 docs/fa/interfaces/tcp.md delete mode 100644 docs/fa/interfaces/third-party_client_libraries.md delete mode 100644 docs/fa/interfaces/third-party_gui.md delete mode 100644 docs/fa/introduction/distinctive_features.md delete mode 100644 docs/fa/introduction/features_considered_disadvantages.md delete mode 100644 docs/fa/introduction/performance.md delete mode 100644 docs/fa/introduction/ya_metrika_task.md delete mode 100644 docs/ru/security_changelog.md create mode 100644 docs/tools/mkdocs-material-theme/assets/javascripts/application.5165553b.js delete mode 100644 docs/tools/mkdocs-material-theme/assets/javascripts/application.js delete mode 100644 docs/tools/mkdocs-material-theme/assets/javascripts/lunr/lunr.js delete mode 100644 docs/tools/mkdocs-material-theme/assets/javascripts/lunr/lunr.th.js create mode 100644 docs/tools/mkdocs-material-theme/assets/javascripts/modernizr.1aa3b519.js delete mode 100644 docs/tools/mkdocs-material-theme/assets/javascripts/modernizr.js delete mode 100644 libs/libdaemon/include/daemon/ExtendedLogChannel.h delete mode 100644 libs/libdaemon/include/daemon/OwnSplitChannel.h delete mode 100644 libs/libdaemon/src/ExtendedLogChannel.cpp delete mode 100644 libs/libdaemon/src/OwnFormattingChannel.cpp delete mode 100644 libs/libdaemon/src/OwnSplitChannel.cpp diff --git a/.gitignore b/.gitignore index 6c0865d1959..291eb35c67a 100644 --- a/.gitignore +++ b/.gitignore @@ -178,6 +178,7 @@ utils/zookeeper-create-entry-to-download-part/zookeeper-create-entry-to-download utils/zookeeper-dump-tree/zookeeper-dump-tree utils/zookeeper-remove-by-list/zookeeper-remove-by-list dbms/src/Storages/tests/remove_symlink_directory +dbms/tests/queries/1_stateful debian/control debian/copyright debian/tmp/ diff --git a/.gitmodules b/.gitmodules index c43b754dba8..1f392b73c83 100644 --- a/.gitmodules +++ b/.gitmodules @@ -15,7 +15,7 @@ url = https://github.com/google/cctz.git [submodule "contrib/zlib-ng"] path = contrib/zlib-ng - url = https://github.com/ClickHouse-Extras/zlib-ng.git + url = https://github.com/Dead2/zlib-ng.git [submodule "contrib/googletest"] path = contrib/googletest url = https://github.com/google/googletest.git @@ -37,12 +37,3 @@ [submodule "contrib/llvm"] path = contrib/llvm url = https://github.com/ClickHouse-Extras/llvm -[submodule "contrib/mariadb-connector-c"] - path = contrib/mariadb-connector-c - url = https://github.com/MariaDB/mariadb-connector-c.git -[submodule "contrib/jemalloc"] - path = contrib/jemalloc - url = https://github.com/jemalloc/jemalloc.git -[submodule "contrib/unixodbc"] - path = contrib/unixodbc - url = https://github.com/ClickHouse-Extras/UnixODBC.git diff --git a/.travis.yml b/.travis.yml index d658b8d285c..705b6977114 100644 --- a/.travis.yml +++ b/.travis.yml @@ -3,6 +3,26 @@ language: generic matrix: fast_finish: true include: +# - os: linux +# +# cache: +# ccache: true +# timeout: 1000 +# +# addons: +# apt: +# update: true +# sources: +# - ubuntu-toolchain-r-test +# packages: [ g++-7, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo, openssl ] +# +# env: +# - MATRIX_EVAL="export CC=gcc-7 && export CXX=g++-7" +# +# script: +# - env TEST_RUN= utils/travis/normal.sh + + # We need to have gcc7 headers to compile c++17 code on clang - os: linux @@ -21,11 +41,33 @@ matrix: packages: [ ninja-build, g++-7, clang-5.0, lld-5.0, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo, openssl] env: - - MATRIX_EVAL="export CC=clang-5.0 CXX=clang++-5.0" + - MATRIX_EVAL="export CC=clang-5.0 && export CXX=clang++-5.0" script: - utils/travis/normal.sh + +# TODO: fix internal compiler +# - os: linux +# +# sudo: required +# +# cache: +# timeout: 1000 +# directories: +# - /var/cache/pbuilder/ccache +# +# addons: +# apt: +# packages: [ pbuilder, fakeroot, debhelper ] +# +# env: +# - MATRIX_EVAL="export DEB_CC=clang-5.0 && export DEB_CXX=clang++-5.0" +# +# script: +# - utils/travis/pbuilder.sh + + - os: linux sudo: required @@ -43,6 +85,69 @@ matrix: script: - utils/travis/pbuilder.sh + +# - os: linux +# +# sudo: required +# +# cache: +# timeout: 1000 +# directories: +# - /var/cache/pbuilder/ccache +# +# addons: +# apt: +# update: true +# packages: [ pbuilder, fakeroot, debhelper ] +# +# env: +# - MATRIX_EVAL="export ARCH=i386" +# +# script: +# - env PBUILDER_TIMEOUT=40m TEST_TRUE=true TEST_RUN= utils/travis/pbuilder.sh + + +# TODO: Can't bootstrap bionic on trusty host +# - os: linux +# +# sudo: required +# +# cache: +# timeout: 1000 +# directories: +# - /var/cache/pbuilder/ccache +# +# addons: +# apt: +# update: true +# packages: [ pbuilder, fakeroot, debhelper ] +# +# env: +# - MATRIX_EVAL="export DEB_CC=clang-6.0 && export DEB_CXX=clang++-6.0 && export DIST=bionic && export EXTRAPACKAGES='clang-6.0 lld-6.0'" +# +# script: +# - utils/travis/pbuilder.sh + + +# Cant fit to time limit (48min) +# - os: osx +# osx_image: xcode9.2 +# +# cache: +# ccache: true +# timeout: 1000 +# +# before_install: +# - brew install unixodbc gcc ccache libtool gettext zlib readline double-conversion gperftools google-sparsehash lz4 zstd || true +# - brew link --overwrite gcc || true +# +# env: +# - MATRIX_EVAL="export CC=gcc-8 && export CXX=g++-8" +# +# script: +# - env CMAKE_FLAGS="-DUSE_INTERNAL_BOOST_LIBRARY=1" utils/travis/normal.sh + + allow_failures: - os: osx diff --git a/CHANGELOG.draft.md b/CHANGELOG.draft.md index 5c7dc6cef09..8b137891791 100644 --- a/CHANGELOG.draft.md +++ b/CHANGELOG.draft.md @@ -1,31 +1 @@ -## RU -## ClickHouse release 18.10.3, 2018-08-13 - -### Новые возможности: -* поддержка межсерверной репликации по HTTPS -* MurmurHash -* ODBCDriver2 с поддержкой NULL-ов -* поддержка UUID в ключевых колонках (экспериментально) - -### Улучшения: -* добавлена поддержка SETTINGS для движка Kafka -* поддежка пустых кусков после мержей в движках Summing, Collapsing and VersionedCollapsing -* удаление старых записей о полностью выполнившихся мутациях -* исправлена логика REPLACE PARTITION для движка RplicatedMergeTree -* добавлена системная таблица system.merge_tree_settings -* в системную таблицу system.tables добавлены столбцы зависимостей: dependencies_database и dependencies_table -* заменен аллокатор, теперь используется jemalloc вместо tcmalloc -* улучшена валидация connection string ODBC -* удалена поддержка CHECK TABLE для распределенных таблиц -* добавлены stateful тесты (пока без данных) -* добавлена опция конфига max_partition_size_to_drop -* добавлена настройка output_format_json_escape_slashes -* добавлена настройка max_fetch_partition_retries_count -* добавлена настройка prefer_localhost_replica -* добавлены libressl, unixodbc и mariadb-connector-c как сабмодули - -### Исправление ошибок: -* #2786 -* #2777 -* #2795 diff --git a/CHANGELOG.md b/CHANGELOG.md index 99994b0621d..ed71baf8046 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,45 +1,3 @@ -## ClickHouse release 18.5.1, 2018-07-31 - -### New features: - -* Added the hash function `murmurHash2_32` [#2756](https://github.com/yandex/ClickHouse/pull/2756). - -### Improvements: - -* Now you can use the `from_env` attribute to set values in config files from environment variables [#2741](https://github.com/yandex/ClickHouse/pull/2741). -* Added case-insensitive versions of the `coalesce`, `ifNull`, and `nullIf functions` [#2752](https://github.com/yandex/ClickHouse/pull/2752). - -### Bug fixes: - -* Fixed a possible bug when starting a replica [#2759](https://github.com/yandex/ClickHouse/pull/2759). - -## ClickHouse release 18.4.0, 2018-07-28 - -### New features: - -* Added system tables: `formats`, `data_type_families`, `aggregate_function_combinators`, `table_functions`, `table_engines`, `collations` [#2721](https://github.com/yandex/ClickHouse/pull/2721). -* Added the ability to use a table function instead of a table as an argument of a `remote` or `cluster` table function [#2708](https://github.com/yandex/ClickHouse/pull/2708). -* Support for `HTTP Basic` authentication in the replication protocol [#2727](https://github.com/yandex/ClickHouse/pull/2727). -* The `has` function now allows searching for a numeric value in an array of `Enum` values [Maxim Khrisanfov](https://github.com/yandex/ClickHouse/pull/2699). -* Support for adding arbitrary message separators when reading from `Kafka` [Amos Bird](https://github.com/yandex/ClickHouse/pull/2701). - -### Improvements: - -* The `ALTER TABLE t DELETE WHERE` query does not rewrite data chunks that were not affected by the WHERE condition [#2694](https://github.com/yandex/ClickHouse/pull/2694). -* The `use_minimalistic_checksums_in_zookeeper` option for `ReplicatedMergeTree` tables is enabled by default. This setting was added in version 1.1.54378, 2018-04-16. Versions that are older than 1.1.54378 can no longer be installed. -* Support for running `KILL` and `OPTIMIZE` queries that specify `ON CLUSTER` [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2689). - -### Bug fixes: - -* Fixed the error `Column ... is not under an aggregate function and not in GROUP BY` for aggregation with an IN expression. This bug appeared in version 18.1.0. ([bbdd780b](https://github.com/yandex/ClickHouse/commit/bbdd780be0be06a0f336775941cdd536878dd2c2)) -* Fixed a bug in the `windowFunnel` aggregate function [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2735). -* Fixed a bug in the `anyHeavy` aggregate function ([a2101df2](https://github.com/yandex/ClickHouse/commit/a2101df25a6a0fba99aa71f8793d762af2b801ee)) -* Fixed server crash when using the `countArray()` aggregate function. - -### Backward incompatible changes: - -* Parameters for `Kafka` engine was changed from `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_schema, kafka_num_consumers])` to `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_row_delimiter, kafka_schema, kafka_num_consumers])`. If your tables use `kafka_schema` or `kafka_num_consumers` parameters, you have to manually edit the metadata files `path/metadata/database/table.sql` and add `kafka_row_delimiter` parameter with `''` value. - ## ClickHouse release 18.1.0, 2018-07-23 ### New features: diff --git a/CHANGELOG_RU.md b/CHANGELOG_RU.md index 7988e7850c6..8150e1f5a57 100644 --- a/CHANGELOG_RU.md +++ b/CHANGELOG_RU.md @@ -1,53 +1,3 @@ -## ClickHouse release 18.6.0, 2018-08-02 - -### Новые возможности: -* Добавлена поддержка ON выражений для JOIN ON синтаксиса: -`JOIN ON Expr([table.]column, ...) = Expr([table.]column, ...) [AND Expr([table.]column, ...) = Expr([table.]column, ...) ...]` -Выражение должно представлять из себя цепочку равенств, объединенных оператором AND. Каждая часть равенства может являться произвольным выражением над столбцами одной из таблиц. Поддержана возможность использования fully qualified имен столбцов (`table.name`, `database.table.name`, `table_alias.name`, `subquery_alias.name`) для правой таблицы. [#2742](https://github.com/yandex/ClickHouse/pull/2742) -* Добавлена возможность включить HTTPS для репликации. [#2760](https://github.com/yandex/ClickHouse/pull/2760) - -### Улучшения: -* Сервер передаёт на клиент также patch-компонент своей версии. Данные о patch компоненте версии добавлены в `system.processes` и `query_log`. [#2646](https://github.com/yandex/ClickHouse/pull/2646) - - -## ClickHouse release 18.5.1, 2018-07-31 - -### Новые возможности: -* Добавлена функция хеширования `murmurHash2_32` [#2756](https://github.com/yandex/ClickHouse/pull/2756). - -### Улучшения: -* Добавлена возможность указывать значения в конфигурационных файлах из переменных окружения с помощью атрибута `from_env` [#2741](https://github.com/yandex/ClickHouse/pull/2741). -* Добавлены регистронезависимые версии функций `coalesce`, `ifNull`, `nullIf` [#2752](https://github.com/yandex/ClickHouse/pull/2752). - -### Исправление ошибок: -* Исправлена возможная ошибка при старте реплики [#2759](https://github.com/yandex/ClickHouse/pull/2759). - - -## ClickHouse release 18.4.0, 2018-07-28 - -### Новые возможности: -* Добавлены системные таблицы `formats`, `data_type_families`, `aggregate_function_combinators`, `table_functions`, `table_engines`, `collations` [#2721](https://github.com/yandex/ClickHouse/pull/2721). -* Добавлена возможность использования табличной функции вместо таблицы в качестве аргумента табличной функции `remote` и `cluster` [#2708](https://github.com/yandex/ClickHouse/pull/2708). -* Поддержка `HTTP Basic` аутентификации в протоколе репликации [#2727](https://github.com/yandex/ClickHouse/pull/2727). -* В функции `has` добавлена возможность поиска в массиве значений типа `Enum` по числовому значению [Maxim Khrisanfov](https://github.com/yandex/ClickHouse/pull/2699). -* Поддержка добавления произвольных разделителей сообщений в процессе чтения из `Kafka` [Amos Bird](https://github.com/yandex/ClickHouse/pull/2701). - -### Улучшения: -* Запрос `ALTER TABLE t DELETE WHERE` не перезаписывает куски данных, которые не были затронуты условием WHERE [#2694](https://github.com/yandex/ClickHouse/pull/2694). -* Настройка `use_minimalistic_checksums_in_zookeeper` таблиц семейства `ReplicatedMergeTree` включена по-умолчанию. Эта настройка была добавлена в версии 1.1.54378, 2018-04-16. Установка версий, более старых, чем 1.1.54378, становится невозможной. -* Поддерживается запуск запросов `KILL` и `OPTIMIZE` с указанием `ON CLUSTER` [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2689). - -### Исправление ошибок: -* Исправлена ошибка `Column ... is not under aggregate function and not in GROUP BY` в случае агрегации по выражению с оператором IN. Ошибка появилась в версии 18.1.0. ([bbdd780b](https://github.com/yandex/ClickHouse/commit/bbdd780be0be06a0f336775941cdd536878dd2c2)) -* Исправлена ошибка в агрегатной функции `windowFunnel` [Winter Zhang](https://github.com/yandex/ClickHouse/pull/2735). -* Исправлена ошибка в агрегатной функции `anyHeavy` ([a2101df2](https://github.com/yandex/ClickHouse/commit/a2101df25a6a0fba99aa71f8793d762af2b801ee)) -* Исправлено падение сервера при использовании функции `countArray()`. - -### Обратно несовместимые изменения: - -* Список параметров для таблиц `Kafka` был изменён с `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_schema, kafka_num_consumers])` на `Kafka(kafka_broker_list, kafka_topic_list, kafka_group_name, kafka_format[, kafka_row_delimiter, kafka_schema, kafka_num_consumers])`. Если вы использовали параметры `kafka_schema` или `kafka_num_consumers`, вам необходимо вручную отредактировать файлы с метаданными `path/metadata/database/table.sql`, добавив параметр `kafka_row_delimiter` со значением `''` в соответствующее место. - - ## ClickHouse release 18.1.0, 2018-07-23 ### Новые возможности: diff --git a/CMakeLists.txt b/CMakeLists.txt index bec91ce705a..b5188588d06 100644 --- a/CMakeLists.txt +++ b/CMakeLists.txt @@ -1,5 +1,5 @@ project (ClickHouse) -cmake_minimum_required (VERSION 3.3) +cmake_minimum_required (VERSION 2.8) set(CMAKE_MODULE_PATH ${CMAKE_MODULE_PATH} "${ClickHouse_SOURCE_DIR}/cmake/Modules/") @@ -34,9 +34,10 @@ endif () string(TOUPPER ${CMAKE_BUILD_TYPE} CMAKE_BUILD_TYPE_UC) message (STATUS "CMAKE_BUILD_TYPE: " ${CMAKE_BUILD_TYPE} ) -set (CMAKE_CONFIGURATION_TYPES "RelWithDebInfo;Debug;Release;MinSizeRel" CACHE STRING "" FORCE) - -include (cmake/sanitize.cmake) +# ASan - build type with address sanitizer +# UBSan - build type with undefined behaviour sanitizer +# TSan is not supported due to false positive errors in libstdc++ and necessity to rebuild libstdc++ with TSan +set (CMAKE_CONFIGURATION_TYPES "RelWithDebInfo;Debug;Release;MinSizeRel;ASan;UBSan" CACHE STRING "" FORCE) include (cmake/arch.cmake) @@ -60,8 +61,12 @@ if (CMAKE_CXX_COMPILER_ID STREQUAL "Clang") set (COMMON_WARNING_FLAGS "${COMMON_WARNING_FLAGS} -Wno-unused-command-line-argument") endif () +if (ARCH_LINUX) + set (CXX11_ABI "ENABLE" CACHE STRING "Use C++11 ABI: DEFAULT, ENABLE, DISABLE") +endif () + option (TEST_COVERAGE "Enables flags for test coverage" OFF) -option (ENABLE_TESTS "Enables tests" ON) +option (ENABLE_TESTS "Enables tests" ${NOT_MSVC}) option (USE_STATIC_LIBRARIES "Set to FALSE to use shared libraries" ON) option (MAKE_STATIC_LIBRARIES "Set to FALSE to make shared libraries" ${USE_STATIC_LIBRARIES}) @@ -81,7 +86,7 @@ endif () if (CMAKE_SYSTEM_PROCESSOR MATCHES "amd64|x86_64") option (USE_INTERNAL_MEMCPY "Use internal implementation of 'memcpy' function instead of provided by libc. Only for x86_64." ON) - if (OS_LINUX) + if (ARCH_LINUX) option (GLIBC_COMPATIBILITY "Set to TRUE to enable compatibility with older glibc libraries. Only for x86_64, Linux. Implies USE_INTERNAL_MEMCPY." ON) endif() endif () @@ -90,7 +95,15 @@ if (GLIBC_COMPATIBILITY) set (USE_INTERNAL_MEMCPY ON) endif () -set (COMPILER_FLAGS "${COMPILER_FLAGS}") +if (CXX11_ABI STREQUAL ENABLE) + set (CXX11_ABI_FLAGS "-D_GLIBCXX_USE_CXX11_ABI=1") +elseif (CXX11_ABI STREQUAL DISABLE) + set (CXX11_ABI_FLAGS "-D_GLIBCXX_USE_CXX11_ABI=0") +else () + set (CXX11_ABI_FLAGS "") +endif () + +set (COMPILER_FLAGS "${COMPILER_FLAGS} ${CXX11_ABI_FLAGS}") string(REGEX MATCH "-?[0-9]+(.[0-9]+)?$" COMPILER_POSTFIX ${CMAKE_CXX_COMPILER}) @@ -137,29 +150,26 @@ else () endif () set (CMAKE_BUILD_COLOR_MAKEFILE ON) -set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${PLATFORM_EXTRA_CXX_FLAG} -fno-omit-frame-pointer ${COMMON_WARNING_FLAGS} ${CXX_WARNING_FLAGS}") +set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${COMPILER_FLAGS} ${PLATFORM_EXTRA_CXX_FLAG} -fno-omit-frame-pointer ${COMMON_WARNING_FLAGS} ${CXX_WARNING_FLAGS} ${GLIBC_COMPATIBILITY_COMPILE_FLAGS}") #set (CMAKE_CXX_FLAGS_RELEASE "${CMAKE_CXX_FLAGS_RELEASE} ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_CXX_FLAGS_RELWITHDEBINFO "${CMAKE_CXX_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_CXX_FLAGS_ADD}") set (CMAKE_CXX_FLAGS_DEBUG "${CMAKE_CXX_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_CXX_FLAGS_ADD}") -set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fno-omit-frame-pointer ${COMMON_WARNING_FLAGS} ${CMAKE_C_FLAGS_ADD}") +set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${COMPILER_FLAGS} -fno-omit-frame-pointer ${COMMON_WARNING_FLAGS} ${GLIBC_COMPATIBILITY_COMPILE_FLAGS} ${CMAKE_C_FLAGS_ADD}") #set (CMAKE_C_FLAGS_RELEASE "${CMAKE_C_FLAGS_RELEASE} ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_RELWITHDEBINFO "${CMAKE_C_FLAGS_RELWITHDEBINFO} -O3 ${CMAKE_C_FLAGS_ADD}") set (CMAKE_C_FLAGS_DEBUG "${CMAKE_C_FLAGS_DEBUG} -O0 -g3 -ggdb3 -fno-inline ${CMAKE_C_FLAGS_ADD}") -if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (CMAKE_CXX_COMPILER_ID STREQUAL "Clang" AND OS_FREEBSD)) +if (MAKE_STATIC_LIBRARIES AND NOT APPLE AND NOT (CMAKE_CXX_COMPILER_ID STREQUAL "Clang" AND ARCH_FREEBSD)) set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libgcc -static-libstdc++") - - # Along with executables, we also build example of shared library for "library dictionary source"; and it also should be self-contained. - set (CMAKE_SHARED_LINKER_FLAGS "${CMAKE_SHARED_LINKER_FLAGS} -static-libgcc -static-libstdc++") endif () set(THREADS_PREFER_PTHREAD_FLAG ON) include (cmake/test_compiler.cmake) -if (OS_LINUX AND CMAKE_CXX_COMPILER_ID STREQUAL "Clang") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS}") +if (ARCH_LINUX AND CMAKE_CXX_COMPILER_ID STREQUAL "Clang") + set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} ${GLIBC_COMPATIBILITY_LINK_FLAGS} ${CXX11_ABI_FLAGS}") option (USE_LIBCXX "Use libc++ and libc++abi instead of libstdc++ (only make sense on Linux with Clang)" ${HAVE_LIBCXX}) set (LIBCXX_PATH "" CACHE STRING "Use custom path for libc++. It should be used for MSan.") @@ -174,7 +184,7 @@ if (OS_LINUX AND CMAKE_CXX_COMPILER_ID STREQUAL "Clang") endif () if (LIBCXX_PATH) -# include_directories (SYSTEM BEFORE "${LIBCXX_PATH}/include" "${LIBCXX_PATH}/include/c++/v1") +# include_directories (BEFORE SYSTEM "${LIBCXX_PATH}/include" "${LIBCXX_PATH}/include/c++/v1") link_directories ("${LIBCXX_PATH}/lib") endif () endif () @@ -189,6 +199,8 @@ if (NOT MAKE_STATIC_LIBRARIES) set(CMAKE_POSITION_INDEPENDENT_CODE ON) endif () +include (cmake/sanitize.cmake) + # Using "include-what-you-use" tool. option (USE_INCLUDE_WHAT_YOU_USE "Use 'include-what-you-use' tool" OFF) if (USE_INCLUDE_WHAT_YOU_USE) @@ -218,14 +230,14 @@ else () set (CLICKHOUSE_ETC_DIR "${CMAKE_INSTALL_PREFIX}/etc") endif () -option (UNBUNDLED "Try find all libraries in system. We recommend to avoid this mode for production builds, because we cannot guarantee exact versions and variants of libraries your system has installed. This mode exists for enthusiastic developers who search for trouble. Also it is useful for maintainers of OS packages." OFF) +option (UNBUNDLED "Try find all libraries in system (if fail - use bundled from contrib/)" OFF) if (UNBUNDLED) set(NOT_UNBUNDLED 0) else () set(NOT_UNBUNDLED 1) endif () # Using system libs can cause lot of warnings in includes. -if (UNBUNDLED OR NOT (OS_LINUX OR APPLE) OR ARCH_32) +if (UNBUNDLED OR NOT (ARCH_LINUX OR APPLE) OR ARCH_32) option (NO_WERROR "Disable -Werror compiler option" ON) endif () @@ -234,15 +246,24 @@ message (STATUS "Building for: ${CMAKE_SYSTEM} ${CMAKE_SYSTEM_PROCESSOR} ${CMAKE include(GNUInstallDirs) include (cmake/find_ssl.cmake) +if (NOT OPENSSL_FOUND) + message (FATAL_ERROR "Need openssl for build. debian tip: sudo apt install libssl-dev") +endif () + include (cmake/lib_name.cmake) include (cmake/find_icu4c.cmake) include (cmake/find_boost.cmake) +# openssl, zlib before poco include (cmake/find_zlib.cmake) include (cmake/find_zstd.cmake) include (cmake/find_ltdl.cmake) # for odbc include (cmake/find_termcap.cmake) -include (cmake/find_odbc.cmake) -# openssl, zlib, odbc before poco +if (EXISTS ${CMAKE_CURRENT_SOURCE_DIR}/contrib/poco/cmake/FindODBC.cmake) + include (${CMAKE_CURRENT_SOURCE_DIR}/contrib/poco/cmake/FindODBC.cmake) # for poco +else () + include (cmake/find_odbc.cmake) +endif () +message (STATUS "Using odbc: ${ODBC_INCLUDE_DIRECTORIES} : ${ODBC_LIBRARIES}") include (cmake/find_poco.cmake) include (cmake/find_lz4.cmake) include (cmake/find_sparsehash.cmake) @@ -262,6 +283,7 @@ include (cmake/find_contrib_lib.cmake) find_contrib_lib(cityhash) find_contrib_lib(farmhash) find_contrib_lib(metrohash) +find_contrib_lib(murmurhash2) find_contrib_lib(btrie) find_contrib_lib(double-conversion) diff --git a/ci/install-libraries.sh b/ci/install-libraries.sh index d7fb856dbed..4868221b342 100755 --- a/ci/install-libraries.sh +++ b/ci/install-libraries.sh @@ -3,8 +3,11 @@ set -e -x source default-config +./install-os-packages.sh libssl-dev ./install-os-packages.sh libicu-dev ./install-os-packages.sh libreadline-dev +./install-os-packages.sh libmariadbclient-dev +./install-os-packages.sh libunixodbc-dev if [[ "$ENABLE_EMBEDDED_COMPILER" == 1 && "$USE_LLVM_LIBRARIES_FROM_SYSTEM" == 1 ]]; then ./install-os-packages.sh llvm-libs-5.0 diff --git a/ci/install-os-packages.sh b/ci/install-os-packages.sh index fe5b4f84833..4aae6268aa1 100755 --- a/ci/install-os-packages.sh +++ b/ci/install-os-packages.sh @@ -43,12 +43,21 @@ case $PACKAGE_MANAGER in jq) $SUDO apt-get install -y jq ;; + libssl-dev) + $SUDO apt-get install -y libssl-dev + ;; libicu-dev) $SUDO apt-get install -y libicu-dev ;; libreadline-dev) $SUDO apt-get install -y libreadline-dev ;; + libunixodbc-dev) + $SUDO apt-get install -y unixodbc-dev + ;; + libmariadbclient-dev) + $SUDO apt-get install -y libmariadbclient-dev + ;; llvm-libs*) $SUDO apt-get install -y ${WHAT/llvm-libs/liblld}-dev ${WHAT/llvm-libs/libclang}-dev ;; @@ -88,12 +97,22 @@ case $PACKAGE_MANAGER in jq) $SUDO yum install -y jq ;; + libssl-dev) + $SUDO yum install -y openssl-devel + ;; libicu-dev) $SUDO yum install -y libicu-devel ;; libreadline-dev) $SUDO yum install -y readline-devel ;; + libunixodbc-dev) + $SUDO yum install -y unixODBC-devel libtool-ltdl-devel + ;; + libmariadbclient-dev) + echo "There is no package with static mysqlclient library"; echo 1; + #$SUDO yum install -y mariadb-connector-c-devel + ;; *) echo "Unknown package"; exit 1; ;; @@ -127,12 +146,21 @@ case $PACKAGE_MANAGER in jq) $SUDO pkg install -y jq ;; + libssl-dev) + $SUDO pkg install -y openssl + ;; libicu-dev) $SUDO pkg install -y icu ;; libreadline-dev) $SUDO pkg install -y readline ;; + libunixodbc-dev) + $SUDO pkg install -y unixODBC libltdl + ;; + libmariadbclient-dev) + $SUDO pkg install -y mariadb102-client + ;; *) echo "Unknown package"; exit 1; ;; diff --git a/ci/jobs/quick-build/run.sh b/ci/jobs/quick-build/run.sh index 6a948c560ee..5fe57457645 100755 --- a/ci/jobs/quick-build/run.sh +++ b/ci/jobs/quick-build/run.sh @@ -21,7 +21,7 @@ BUILD_TARGETS=clickhouse BUILD_TYPE=Debug ENABLE_EMBEDDED_COMPILER=0 -CMAKE_FLAGS="-D CMAKE_C_FLAGS_ADD=-g0 -D CMAKE_CXX_FLAGS_ADD=-g0 -D ENABLE_JEMALLOC=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_UNWIND=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_POCO_ODBC=0 -D ENABLE_ODBC=0 -D ENABLE_MYSQL=0" +CMAKE_FLAGS="-D CMAKE_C_FLAGS_ADD=-g0 -D CMAKE_CXX_FLAGS_ADD=-g0 -D ENABLE_TCMALLOC=0 -D ENABLE_CAPNP=0 -D ENABLE_RDKAFKA=0 -D ENABLE_UNWIND=0 -D ENABLE_ICU=0 -D ENABLE_POCO_MONGODB=0 -D ENABLE_POCO_NETSSL=0 -D ENABLE_POCO_ODBC=0 -D ENABLE_MYSQL=0" [[ $(uname) == "FreeBSD" ]] && COMPILER_PACKAGE_VERSION=devel && export COMPILER_PATH=/usr/local/bin diff --git a/cmake/Modules/FindODBC.cmake b/cmake/Modules/FindODBC.cmake deleted file mode 100644 index 66d43e93d2d..00000000000 --- a/cmake/Modules/FindODBC.cmake +++ /dev/null @@ -1,88 +0,0 @@ -# This file copied from contrib/poco/cmake/FindODBC.cmake to allow build without submodules - -# -# Find the ODBC driver manager includes and library. -# -# ODBC is an open standard for connecting to different databases in a -# semi-vendor-independent fashion. First you install the ODBC driver -# manager. Then you need a driver for each separate database you want -# to connect to (unless a generic one works). VTK includes neither -# the driver manager nor the vendor-specific drivers: you have to find -# those yourself. -# -# This module defines -# ODBC_INCLUDE_DIRECTORIES, where to find sql.h -# ODBC_LIBRARIES, the libraries to link against to use ODBC -# ODBC_FOUND. If false, you cannot build anything that requires ODBC. - -option (ENABLE_ODBC "Enable ODBC" ${OS_LINUX}) -if (OS_LINUX) - option (USE_INTERNAL_ODBC_LIBRARY "Set to FALSE to use system odbc library instead of bundled" ${NOT_UNBUNDLED}) -else () - option (USE_INTERNAL_ODBC_LIBRARY "Set to FALSE to use system odbc library instead of bundled" OFF) -endif () - -if (USE_INTERNAL_ODBC_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/unixodbc/README") - message (WARNING "submodule contrib/unixodbc is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_ODBC_LIBRARY 0) -endif () - -if (ENABLE_ODBC) - if (USE_INTERNAL_ODBC_LIBRARY) - set (ODBC_LIBRARIES unixodbc) - set (ODBC_INCLUDE_DIRECTORIES ${CMAKE_SOURCE_DIR}/contrib/unixodbc/include) - set (ODBC_FOUND 1) - set (USE_ODBC 1) - else () - find_path(ODBC_INCLUDE_DIRECTORIES - NAMES sql.h - HINTS - /usr/include - /usr/include/iodbc - /usr/include/odbc - /usr/local/include - /usr/local/include/iodbc - /usr/local/include/odbc - /usr/local/iodbc/include - /usr/local/odbc/include - "C:/Program Files/ODBC/include" - "C:/Program Files/Microsoft SDKs/Windows/v7.0/include" - "C:/Program Files/Microsoft SDKs/Windows/v6.0a/include" - "C:/ODBC/include" - DOC "Specify the directory containing sql.h." - ) - - find_library(ODBC_LIBRARIES - NAMES iodbc odbc iodbcinst odbcinst odbc32 - HINTS - /usr/lib - /usr/lib/iodbc - /usr/lib/odbc - /usr/local/lib - /usr/local/lib/iodbc - /usr/local/lib/odbc - /usr/local/iodbc/lib - /usr/local/odbc/lib - "C:/Program Files/ODBC/lib" - "C:/ODBC/lib/debug" - "C:/Program Files (x86)/Microsoft SDKs/Windows/v7.0A/Lib" - DOC "Specify the ODBC driver manager library here." - ) - - # MinGW find usually fails - if(MINGW) - set(ODBC_INCLUDE_DIRECTORIES ".") - set(ODBC_LIBRARIES odbc32) - endif() - - include(FindPackageHandleStandardArgs) - find_package_handle_standard_args(ODBC - DEFAULT_MSG - ODBC_INCLUDE_DIRECTORIES - ODBC_LIBRARIES) - - mark_as_advanced(ODBC_FOUND ODBC_LIBRARIES ODBC_INCLUDE_DIRECTORIES) - endif () -endif () - -message (STATUS "Using odbc: ${ODBC_INCLUDE_DIRECTORIES} : ${ODBC_LIBRARIES}") diff --git a/cmake/arch.cmake b/cmake/arch.cmake index abc30d99e32..ba446d95676 100644 --- a/cmake/arch.cmake +++ b/cmake/arch.cmake @@ -7,16 +7,23 @@ endif () if (CMAKE_LIBRARY_ARCHITECTURE MATCHES "i386") set (ARCH_I386 1) endif () -if ((ARCH_ARM AND NOT ARCH_AARCH64) OR ARCH_I386) +if ( ( ARCH_ARM AND NOT ARCH_AARCH64 ) OR ARCH_I386) set (ARCH_32 1) - message (FATAL_ERROR "32bit platforms are not supported") + message (WARNING "Support for 32bit platforms is highly experimental") endif () - if (CMAKE_SYSTEM MATCHES "Linux") - set (OS_LINUX 1) + set (ARCH_LINUX 1) endif () if (CMAKE_SYSTEM MATCHES "FreeBSD") - set (OS_FREEBSD 1) + set (ARCH_FREEBSD 1) +endif () + +if (NOT MSVC) + set (NOT_MSVC 1) +endif () + +if (NOT APPLE) + set (NOT_APPLE 1) endif () if (CMAKE_CXX_COMPILER_ID STREQUAL "GNU") diff --git a/cmake/find_capnp.cmake b/cmake/find_capnp.cmake index 6c064112686..03ecadda6a1 100644 --- a/cmake/find_capnp.cmake +++ b/cmake/find_capnp.cmake @@ -1,4 +1,4 @@ -option (ENABLE_CAPNP "Enable Cap'n Proto" ON) +option (ENABLE_CAPNP "Enable Cap'n Proto" ${NOT_MSVC}) if (ENABLE_CAPNP) # cmake 3.5.1 bug: diff --git a/cmake/find_cpuid.cmake b/cmake/find_cpuid.cmake index d02336021bb..6a4361dc42c 100644 --- a/cmake/find_cpuid.cmake +++ b/cmake/find_cpuid.cmake @@ -2,7 +2,7 @@ # TODO: test new libcpuid - maybe already fixed if (NOT ARCH_ARM) - if (OS_FREEBSD) + if (ARCH_FREEBSD) set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY 1) else () set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY ${NOT_UNBUNDLED}) diff --git a/cmake/find_execinfo.cmake b/cmake/find_execinfo.cmake index 650d279983c..05dd72dbb3d 100644 --- a/cmake/find_execinfo.cmake +++ b/cmake/find_execinfo.cmake @@ -1,4 +1,4 @@ -if (OS_FREEBSD) +if (ARCH_FREEBSD) find_library (EXECINFO_LIBRARY execinfo) find_library (ELF_LIBRARY elf) message (STATUS "Using execinfo: ${EXECINFO_LIBRARY}") diff --git a/cmake/find_llvm.cmake b/cmake/find_llvm.cmake index d9805b03303..b10a8cb87d4 100644 --- a/cmake/find_llvm.cmake +++ b/cmake/find_llvm.cmake @@ -43,12 +43,6 @@ if (ENABLE_EMBEDDED_COMPILER) else() set (USE_EMBEDDED_COMPILER 0) endif() - - if (LLVM_FOUND AND OS_LINUX AND USE_LIBCXX) - message(WARNING "Option USE_INTERNAL_LLVM_LIBRARY is not set but the LLVM library from OS packages in Linux is incompatible with libc++ ABI. LLVM Will be disabled.") - set (LLVM_FOUND 0) - set (USE_EMBEDDED_COMPILER 0) - endif () else() set (LLVM_FOUND 1) set (USE_EMBEDDED_COMPILER 1) diff --git a/cmake/find_ltdl.cmake b/cmake/find_ltdl.cmake index 18003618dbd..935de0d4124 100644 --- a/cmake/find_ltdl.cmake +++ b/cmake/find_ltdl.cmake @@ -1,5 +1,3 @@ -if (ENABLE_ODBC AND NOT USE_INTERNAL_ODBC_LIBRARY) - set (LTDL_PATHS "/usr/local/opt/libtool/lib") - find_library (LTDL_LIBRARY ltdl PATHS ${LTDL_PATHS}) - message (STATUS "Using ltdl: ${LTDL_LIBRARY}") -endif () +set (LTDL_PATHS "/usr/local/opt/libtool/lib") +find_library (LTDL_LIBRARY ltdl PATHS ${LTDL_PATHS}) +message (STATUS "Using ltdl: ${LTDL_LIBRARY}") diff --git a/cmake/find_odbc.cmake b/cmake/find_odbc.cmake index 95acf40b2b4..338108910bf 100644 --- a/cmake/find_odbc.cmake +++ b/cmake/find_odbc.cmake @@ -13,77 +13,54 @@ # This module defines # ODBC_INCLUDE_DIRECTORIES, where to find sql.h # ODBC_LIBRARIES, the libraries to link against to use ODBC -# ODBC_FOUND. If false, you cannot build anything that requires ODBC. +# ODBC_FOUND. If false, you cannot build anything that requires MySQL. -option (ENABLE_ODBC "Enable ODBC" ${OS_LINUX}) -if (OS_LINUX) - option (USE_INTERNAL_ODBC_LIBRARY "Set to FALSE to use system odbc library instead of bundled" ${NOT_UNBUNDLED}) -else () - option (USE_INTERNAL_ODBC_LIBRARY "Set to FALSE to use system odbc library instead of bundled" OFF) -endif () +find_path(ODBC_INCLUDE_DIRECTORIES + NAMES sql.h + HINTS + /usr/include + /usr/include/odbc + /usr/include/iodbc + /usr/local/include + /usr/local/include/odbc + /usr/local/include/iodbc + /usr/local/odbc/include + /usr/local/iodbc/include + "C:/Program Files/ODBC/include" + "C:/Program Files/Microsoft SDKs/Windows/v7.0/include" + "C:/Program Files/Microsoft SDKs/Windows/v6.0a/include" + "C:/ODBC/include" + DOC "Specify the directory containing sql.h." +) -if (USE_INTERNAL_ODBC_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/unixodbc/README") - message (WARNING "submodule contrib/unixodbc is missing. to fix try run: \n git submodule update --init --recursive") - set (USE_INTERNAL_ODBC_LIBRARY 0) -endif () +find_library(ODBC_LIBRARIES + NAMES iodbc odbc iodbcinst odbcinst odbc32 + HINTS + /usr/lib + /usr/lib/odbc + /usr/lib/iodbc + /usr/local/lib + /usr/local/lib/odbc + /usr/local/lib/iodbc + /usr/local/odbc/lib + /usr/local/iodbc/lib + "C:/Program Files/ODBC/lib" + "C:/ODBC/lib/debug" + "C:/Program Files (x86)/Microsoft SDKs/Windows/v7.0A/Lib" + DOC "Specify the ODBC driver manager library here." +) -set (ODBC_INCLUDE_DIRECTORIES ) # Include directories will be either used automatically by target_include_directories or set later. +# MinGW find usually fails +if(MINGW) + set(ODBC_INCLUDE_DIRECTORIES ".") + set(ODBC_LIBRARIES odbc32) +endif() + +include(FindPackageHandleStandardArgs) +find_package_handle_standard_args(ODBC + DEFAULT_MSG + ODBC_INCLUDE_DIRECTORIES + ODBC_LIBRARIES + ) -if (ENABLE_ODBC) - if (USE_INTERNAL_ODBC_LIBRARY) - set (ODBC_LIBRARIES unixodbc) - set (ODBC_FOUND 1) - set (USE_ODBC 1) - else () - find_path(ODBC_INCLUDE_DIRECTORIES - NAMES sql.h - HINTS - /usr/include - /usr/include/iodbc - /usr/include/odbc - /usr/local/include - /usr/local/include/iodbc - /usr/local/include/odbc - /usr/local/iodbc/include - /usr/local/odbc/include - "C:/Program Files/ODBC/include" - "C:/Program Files/Microsoft SDKs/Windows/v7.0/include" - "C:/Program Files/Microsoft SDKs/Windows/v6.0a/include" - "C:/ODBC/include" - DOC "Specify the directory containing sql.h." - ) - - find_library(ODBC_LIBRARIES - NAMES iodbc odbc iodbcinst odbcinst odbc32 - HINTS - /usr/lib - /usr/lib/iodbc - /usr/lib/odbc - /usr/local/lib - /usr/local/lib/iodbc - /usr/local/lib/odbc - /usr/local/iodbc/lib - /usr/local/odbc/lib - "C:/Program Files/ODBC/lib" - "C:/ODBC/lib/debug" - "C:/Program Files (x86)/Microsoft SDKs/Windows/v7.0A/Lib" - DOC "Specify the ODBC driver manager library here." - ) - - # MinGW find usually fails - if(MINGW) - set(ODBC_INCLUDE_DIRECTORIES ".") - set(ODBC_LIBRARIES odbc32) - endif() - - include(FindPackageHandleStandardArgs) - find_package_handle_standard_args(ODBC - DEFAULT_MSG - ODBC_INCLUDE_DIRECTORIES - ODBC_LIBRARIES) - - mark_as_advanced(ODBC_FOUND ODBC_LIBRARIES ODBC_INCLUDE_DIRECTORIES) - endif () -endif () - -message (STATUS "Using odbc: ${ODBC_INCLUDE_DIRECTORIES} : ${ODBC_LIBRARIES}") +mark_as_advanced(ODBC_FOUND ODBC_LIBRARIES ODBC_INCLUDE_DIRECTORIES) diff --git a/cmake/find_poco.cmake b/cmake/find_poco.cmake index f0bc535f614..947d31951c9 100644 --- a/cmake/find_poco.cmake +++ b/cmake/find_poco.cmake @@ -92,7 +92,8 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY) endif () endif () - if (OPENSSL_FOUND AND (NOT DEFINED ENABLE_POCO_NETSSL OR ENABLE_POCO_NETSSL)) + # TODO! fix internal ssl + if (OPENSSL_FOUND AND NOT USE_INTERNAL_SSL_LIBRARY AND (NOT DEFINED ENABLE_POCO_NETSSL OR ENABLE_POCO_NETSSL)) set (Poco_NetSSL_LIBRARY PocoNetSSL) set (Poco_Crypto_LIBRARY PocoCrypto) endif () diff --git a/cmake/find_rdkafka.cmake b/cmake/find_rdkafka.cmake index dc8e9913bc7..396be18cd1c 100644 --- a/cmake/find_rdkafka.cmake +++ b/cmake/find_rdkafka.cmake @@ -1,10 +1,8 @@ -option (ENABLE_RDKAFKA "Enable kafka" ON) +option (ENABLE_RDKAFKA "Enable kafka" ${NOT_MSVC}) if (ENABLE_RDKAFKA) -if (OS_LINUX) - option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) -endif () +option (USE_INTERNAL_RDKAFKA_LIBRARY "Set to FALSE to use system librdkafka instead of the bundled" ${NOT_UNBUNDLED}) if (USE_INTERNAL_RDKAFKA_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/librdkafka/CMakeLists.txt") message (WARNING "submodule contrib/librdkafka is missing. to fix try run: \n git submodule update --init --recursive") @@ -15,7 +13,7 @@ endif () if (NOT USE_INTERNAL_RDKAFKA_LIBRARY) find_library (RDKAFKA_LIB rdkafka) find_path (RDKAFKA_INCLUDE_DIR NAMES librdkafka/rdkafka.h PATHS ${RDKAFKA_INCLUDE_PATHS}) - if (USE_STATIC_LIBRARIES AND NOT OS_FREEBSD) + if (USE_STATIC_LIBRARIES AND NOT ARCH_FREEBSD) find_library (SASL2_LIBRARY sasl2) endif () endif () diff --git a/cmake/find_rt.cmake b/cmake/find_rt.cmake index 25614fe55eb..82ec314d195 100644 --- a/cmake/find_rt.cmake +++ b/cmake/find_rt.cmake @@ -1,7 +1,7 @@ if (APPLE) # lib from libs/libcommon set (RT_LIBRARY "apple_rt") -elseif (OS_FREEBSD) +elseif (ARCH_FREEBSD) find_library (RT_LIBRARY rt) else () set (RT_LIBRARY "") diff --git a/cmake/find_ssl.cmake b/cmake/find_ssl.cmake index 51e869f86ea..ec40e498da1 100644 --- a/cmake/find_ssl.cmake +++ b/cmake/find_ssl.cmake @@ -1,4 +1,4 @@ -option (USE_INTERNAL_SSL_LIBRARY "Set to FALSE to use system *ssl library instead of bundled" ${OS_LINUX}) +option (USE_INTERNAL_SSL_LIBRARY "Set to FALSE to use system *ssl library instead of bundled" ${MSVC}) set (OPENSSL_USE_STATIC_LIBS ${USE_STATIC_LIBRARIES}) diff --git a/cmake/find_zlib.cmake b/cmake/find_zlib.cmake index 0e198c9bb0f..17350f9fd58 100644 --- a/cmake/find_zlib.cmake +++ b/cmake/find_zlib.cmake @@ -17,7 +17,7 @@ if (NOT ZLIB_FOUND) set (USE_INTERNAL_ZLIB_LIBRARY 1) set (ZLIB_COMPAT 1) # for zlib-ng, also enables WITH_GZFILEOP set (WITH_NATIVE_INSTRUCTIONS ${ARCHNATIVE}) - if (OS_FREEBSD OR ARCH_I386) + if (ARCH_FREEBSD OR ARCH_I386) set (WITH_OPTIM 0 CACHE INTERNAL "") # Bug in assembler endif () if (ARCH_AARCH64) diff --git a/cmake/sanitize.cmake b/cmake/sanitize.cmake index a90533345e6..bac27578663 100644 --- a/cmake/sanitize.cmake +++ b/cmake/sanitize.cmake @@ -1,37 +1,27 @@ -option (SANITIZE "Enable sanitizer: address, memory, thread, undefined" "") - set (SAN_FLAGS "${SAN_FLAGS} -g -fno-omit-frame-pointer -DSANITIZER") +if (SAN_DEBUG) + set (SAN_FLAGS "${SAN_FLAGS} -O0") +else () + set (SAN_FLAGS "${SAN_FLAGS} -O3") +endif () -if (SANITIZE) - if (SANITIZE STREQUAL "address") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=address") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=address") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=address") - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libasan") - endif () - elseif (SANITIZE STREQUAL "memory") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=memory") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=memory") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=memory") - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libmsan") - endif () - elseif (SANITIZE STREQUAL "thread") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=thread") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=thread") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=thread") - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libtsan") - endif () - elseif (SANITIZE STREQUAL "undefined") - set (CMAKE_CXX_FLAGS "${CMAKE_CXX_FLAGS} ${SAN_FLAGS} -fsanitize=undefined") - set (CMAKE_C_FLAGS "${CMAKE_C_FLAGS} ${SAN_FLAGS} -fsanitize=undefined") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -fsanitize=undefined") - if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") - set (CMAKE_EXE_LINKER_FLAGS "${CMAKE_EXE_LINKER_FLAGS} -static-libubsan") - endif () - else () - message (FATAL_ERROR "Unknown sanitizer type: ${SANITIZE}") - endif () -endif() +set (CMAKE_CXX_FLAGS_ASAN "${CMAKE_CXX_FLAGS_ASAN} ${SAN_FLAGS} -fsanitize=address") +set (CMAKE_C_FLAGS_ASAN "${CMAKE_C_FLAGS_ASAN} ${SAN_FLAGS} -fsanitize=address") +set (CMAKE_EXE_LINKER_FLAGS_ASAN "${CMAKE_EXE_LINKER_FLAGS_ASAN} -fsanitize=address") +set (CMAKE_CXX_FLAGS_UBSAN "${CMAKE_CXX_FLAGS_UBSAN} ${SAN_FLAGS} -fsanitize=undefined") +set (CMAKE_C_FLAGS_UBSAN "${CMAKE_C_FLAGS_UBSAN} ${SAN_FLAGS} -fsanitize=undefined") +set (CMAKE_EXE_LINKER_FLAGS_UBSAN "${CMAKE_EXE_LINKER_FLAGS_UBSAN} -fsanitize=undefined") +set (CMAKE_CXX_FLAGS_MSAN "${CMAKE_CXX_FLAGS_MSAN} ${SAN_FLAGS} -fsanitize=memory") +set (CMAKE_C_FLAGS_MSAN "${CMAKE_C_FLAGS_MSAN} ${SAN_FLAGS} -fsanitize=memory") +set (CMAKE_EXE_LINKER_FLAGS_MSAN "${CMAKE_EXE_LINKER_FLAGS_MSAN} -fsanitize=memory") +set (CMAKE_CXX_FLAGS_TSAN "${CMAKE_CXX_FLAGS_TSAN} ${SAN_FLAGS} -fsanitize=thread") +set (CMAKE_C_FLAGS_TSAN "${CMAKE_C_FLAGS_TSAN} ${SAN_FLAGS} -fsanitize=thread") +set (CMAKE_EXE_LINKER_FLAGS_TSAN "${CMAKE_EXE_LINKER_FLAGS_TSAN} -fsanitize=thread") + +# clang use static linking by default +if (MAKE_STATIC_LIBRARIES AND CMAKE_CXX_COMPILER_ID STREQUAL "GNU") + set (CMAKE_EXE_LINKER_FLAGS_ASAN "${CMAKE_EXE_LINKER_FLAGS_ASAN} -static-libasan") + set (CMAKE_EXE_LINKER_FLAGS_UBSAN "${CMAKE_EXE_LINKER_FLAGS_UBSAN} -static-libubsan") + set (CMAKE_EXE_LINKER_FLAGS_MSAN "${CMAKE_EXE_LINKER_FLAGS_MSAN} -static-libmsan") + set (CMAKE_EXE_LINKER_FLAGS_TSAN "${CMAKE_EXE_LINKER_FLAGS_TSAN} -static-libtsan") +endif () diff --git a/contrib/jemalloc b/contrib/jemalloc deleted file mode 160000 index 41b7372eade..00000000000 --- a/contrib/jemalloc +++ /dev/null @@ -1 +0,0 @@ -Subproject commit 41b7372eadee941b9164751b8d4963f915d3ceae diff --git a/contrib/jemalloc-cmake/CMakeLists.txt b/contrib/jemalloc-cmake/CMakeLists.txt deleted file mode 100644 index d60d34604a9..00000000000 --- a/contrib/jemalloc-cmake/CMakeLists.txt +++ /dev/null @@ -1,52 +0,0 @@ -set(JEMALLOC_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/jemalloc) - -set(SRCS -${JEMALLOC_SOURCE_DIR}/src/arena.c -${JEMALLOC_SOURCE_DIR}/src/background_thread.c -${JEMALLOC_SOURCE_DIR}/src/base.c -${JEMALLOC_SOURCE_DIR}/src/bin.c -${JEMALLOC_SOURCE_DIR}/src/bitmap.c -${JEMALLOC_SOURCE_DIR}/src/ckh.c -${JEMALLOC_SOURCE_DIR}/src/ctl.c -${JEMALLOC_SOURCE_DIR}/src/div.c -${JEMALLOC_SOURCE_DIR}/src/extent.c -${JEMALLOC_SOURCE_DIR}/src/extent_dss.c -${JEMALLOC_SOURCE_DIR}/src/extent_mmap.c -${JEMALLOC_SOURCE_DIR}/src/hash.c -${JEMALLOC_SOURCE_DIR}/src/hook.c -${JEMALLOC_SOURCE_DIR}/src/jemalloc.c -${JEMALLOC_SOURCE_DIR}/src/jemalloc_cpp.cpp -${JEMALLOC_SOURCE_DIR}/src/large.c -${JEMALLOC_SOURCE_DIR}/src/log.c -${JEMALLOC_SOURCE_DIR}/src/malloc_io.c -${JEMALLOC_SOURCE_DIR}/src/mutex.c -${JEMALLOC_SOURCE_DIR}/src/mutex_pool.c -${JEMALLOC_SOURCE_DIR}/src/nstime.c -${JEMALLOC_SOURCE_DIR}/src/pages.c -${JEMALLOC_SOURCE_DIR}/src/prng.c -${JEMALLOC_SOURCE_DIR}/src/prof.c -${JEMALLOC_SOURCE_DIR}/src/rtree.c -${JEMALLOC_SOURCE_DIR}/src/sc.c -${JEMALLOC_SOURCE_DIR}/src/stats.c -${JEMALLOC_SOURCE_DIR}/src/sz.c -${JEMALLOC_SOURCE_DIR}/src/tcache.c -${JEMALLOC_SOURCE_DIR}/src/test_hooks.c -${JEMALLOC_SOURCE_DIR}/src/ticker.c -${JEMALLOC_SOURCE_DIR}/src/tsd.c -${JEMALLOC_SOURCE_DIR}/src/witness.c -) - -if(CMAKE_SYSTEM_NAME MATCHES "Darwin") - list(APPEND SRCS ${JEMALLOC_SOURCE_DIR}/src/zone.c) -endif() - -add_library(jemalloc STATIC ${SRCS}) - -target_include_directories(jemalloc PUBLIC - ${CMAKE_CURRENT_SOURCE_DIR}/include - ${CMAKE_CURRENT_SOURCE_DIR}/include_linux_x86_64) # jemalloc.h - -target_include_directories(jemalloc PRIVATE - ${JEMALLOC_SOURCE_DIR}/include) - -target_compile_definitions(jemalloc PRIVATE -DJEMALLOC_NO_PRIVATE_NAMESPACE) diff --git a/contrib/jemalloc-cmake/README b/contrib/jemalloc-cmake/README deleted file mode 100644 index 0af9c4f0e45..00000000000 --- a/contrib/jemalloc-cmake/README +++ /dev/null @@ -1 +0,0 @@ -It allows to integrate JEMalloc into CMake project. diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc.h deleted file mode 100644 index d06243c5239..00000000000 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc.h +++ /dev/null @@ -1,16 +0,0 @@ -#pragma once - -#ifdef __cplusplus -extern "C" { -#endif - -#include -#include -#include -#include -#include - -#ifdef __cplusplus -} -#endif - diff --git a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h b/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h deleted file mode 100644 index a2ea2dd3533..00000000000 --- a/contrib/jemalloc-cmake/include/jemalloc/jemalloc_rename.h +++ /dev/null @@ -1,29 +0,0 @@ -/* - * Name mangling for public symbols is controlled by --with-mangling and - * --with-jemalloc-prefix. With default settings the je_ prefix is stripped by - * these macro definitions. - */ -#ifndef JEMALLOC_NO_RENAME -# define je_aligned_alloc aligned_alloc -# define je_calloc calloc -# define je_dallocx dallocx -# define je_free free -# define je_mallctl mallctl -# define je_mallctlbymib mallctlbymib -# define je_mallctlnametomib mallctlnametomib -# define je_malloc malloc -# define je_malloc_conf malloc_conf -# define je_malloc_message malloc_message -# define je_malloc_stats_print malloc_stats_print -# define je_malloc_usable_size malloc_usable_size -# define je_mallocx mallocx -# define je_nallocx nallocx -# define je_posix_memalign posix_memalign -# define je_rallocx rallocx -# define je_realloc realloc -# define je_sallocx sallocx -# define je_sdallocx sdallocx -# define je_xallocx xallocx -# define je_memalign memalign -# define je_valloc valloc -#endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/README b/contrib/jemalloc-cmake/include_linux_x86_64/README deleted file mode 100644 index bf7663bda8d..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/README +++ /dev/null @@ -1,7 +0,0 @@ -Here are pre-generated files from jemalloc on Linux x86_64. -You can obtain these files by running ./autogen.sh inside jemalloc source directory. - -Added #define GNU_SOURCE -Added JEMALLOC_OVERRIDE___POSIX_MEMALIGN because why not. -Removed JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF because it's non standard. -Removed JEMALLOC_PURGE_MADVISE_FREE because it's available only from Linux 4.5. diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h deleted file mode 100644 index 43936e8eba0..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_internal_defs.h +++ /dev/null @@ -1,373 +0,0 @@ -/* include/jemalloc/internal/jemalloc_internal_defs.h. Generated from jemalloc_internal_defs.h.in by configure. */ -#ifndef JEMALLOC_INTERNAL_DEFS_H_ -#define JEMALLOC_INTERNAL_DEFS_H_ - -#ifndef _GNU_SOURCE - #define _GNU_SOURCE -#endif - -/* - * If JEMALLOC_PREFIX is defined via --with-jemalloc-prefix, it will cause all - * public APIs to be prefixed. This makes it possible, with some care, to use - * multiple allocators simultaneously. - */ -/* #undef JEMALLOC_PREFIX */ -/* #undef JEMALLOC_CPREFIX */ - -/* - * Define overrides for non-standard allocator-related functions if they are - * present on the system. - */ -#define JEMALLOC_OVERRIDE___LIBC_CALLOC -#define JEMALLOC_OVERRIDE___LIBC_FREE -#define JEMALLOC_OVERRIDE___LIBC_MALLOC -#define JEMALLOC_OVERRIDE___LIBC_MEMALIGN -#define JEMALLOC_OVERRIDE___LIBC_REALLOC -#define JEMALLOC_OVERRIDE___LIBC_VALLOC -#define JEMALLOC_OVERRIDE___POSIX_MEMALIGN - -/* - * JEMALLOC_PRIVATE_NAMESPACE is used as a prefix for all library-private APIs. - * For shared libraries, symbol visibility mechanisms prevent these symbols - * from being exported, but for static libraries, naming collisions are a real - * possibility. - */ -#define JEMALLOC_PRIVATE_NAMESPACE je_ - -/* - * Hyper-threaded CPUs may need a special instruction inside spin loops in - * order to yield to another virtual CPU. - */ -#define CPU_SPINWAIT __asm__ volatile("pause") -/* 1 if CPU_SPINWAIT is defined, 0 otherwise. */ -#define HAVE_CPU_SPINWAIT 1 - -/* - * Number of significant bits in virtual addresses. This may be less than the - * total number of bits in a pointer, e.g. on x64, for which the uppermost 16 - * bits are the same as bit 47. - */ -#define LG_VADDR 48 - -/* Defined if C11 atomics are available. */ -#define JEMALLOC_C11_ATOMICS 1 - -/* Defined if GCC __atomic atomics are available. */ -#define JEMALLOC_GCC_ATOMIC_ATOMICS 1 - -/* Defined if GCC __sync atomics are available. */ -#define JEMALLOC_GCC_SYNC_ATOMICS 1 - -/* - * Defined if __sync_add_and_fetch(uint32_t *, uint32_t) and - * __sync_sub_and_fetch(uint32_t *, uint32_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_4 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_4 */ - -/* - * Defined if __sync_add_and_fetch(uint64_t *, uint64_t) and - * __sync_sub_and_fetch(uint64_t *, uint64_t) are available, despite - * __GCC_HAVE_SYNC_COMPARE_AND_SWAP_8 not being defined (which means the - * functions are defined in libgcc instead of being inlines). - */ -/* #undef JE_FORCE_SYNC_COMPARE_AND_SWAP_8 */ - -/* - * Defined if __builtin_clz() and __builtin_clzl() are available. - */ -#define JEMALLOC_HAVE_BUILTIN_CLZ - -/* - * Defined if os_unfair_lock_*() functions are available, as provided by Darwin. - */ -/* #undef JEMALLOC_OS_UNFAIR_LOCK */ - -/* - * Defined if OSSpin*() functions are available, as provided by Darwin, and - * documented in the spinlock(3) manual page. - */ -/* #undef JEMALLOC_OSSPIN */ - -/* Defined if syscall(2) is usable. */ -#define JEMALLOC_USE_SYSCALL - -/* - * Defined if secure_getenv(3) is available. - */ -// Don't want dependency on newer GLIBC -//#define JEMALLOC_HAVE_SECURE_GETENV - -/* - * Defined if issetugid(2) is available. - */ -/* #undef JEMALLOC_HAVE_ISSETUGID */ - -/* Defined if pthread_atfork(3) is available. */ -#define JEMALLOC_HAVE_PTHREAD_ATFORK - -/* Defined if pthread_setname_np(3) is available. */ -#define JEMALLOC_HAVE_PTHREAD_SETNAME_NP - -/* - * Defined if clock_gettime(CLOCK_MONOTONIC_COARSE, ...) is available. - */ -#define JEMALLOC_HAVE_CLOCK_MONOTONIC_COARSE 1 - -/* - * Defined if clock_gettime(CLOCK_MONOTONIC, ...) is available. - */ -#define JEMALLOC_HAVE_CLOCK_MONOTONIC 1 - -/* - * Defined if mach_absolute_time() is available. - */ -/* #undef JEMALLOC_HAVE_MACH_ABSOLUTE_TIME */ - -/* - * Defined if _malloc_thread_cleanup() exists. At least in the case of - * FreeBSD, pthread_key_create() allocates, which if used during malloc - * bootstrapping will cause recursion into the pthreads library. Therefore, if - * _malloc_thread_cleanup() exists, use it as the basis for thread cleanup in - * malloc_tsd. - */ -/* #undef JEMALLOC_MALLOC_THREAD_CLEANUP */ - -/* - * Defined if threaded initialization is known to be safe on this platform. - * Among other things, it must be possible to initialize a mutex without - * triggering allocation in order for threaded allocation to be safe. - */ -#define JEMALLOC_THREADED_INIT - -/* - * Defined if the pthreads implementation defines - * _pthread_mutex_init_calloc_cb(), in which case the function is used in order - * to avoid recursive allocation during mutex initialization. - */ -/* #undef JEMALLOC_MUTEX_INIT_CB */ - -/* Non-empty if the tls_model attribute is supported. */ -#define JEMALLOC_TLS_MODEL __attribute__((tls_model("initial-exec"))) - -/* - * JEMALLOC_DEBUG enables assertions and other sanity checks, and disables - * inline functions. - */ -/* #undef JEMALLOC_DEBUG */ - -/* JEMALLOC_STATS enables statistics calculation. */ -#define JEMALLOC_STATS - -/* JEMALLOC_PROF enables allocation profiling. */ -/* #undef JEMALLOC_PROF */ - -/* Use libunwind for profile backtracing if defined. */ -/* #undef JEMALLOC_PROF_LIBUNWIND */ - -/* Use libgcc for profile backtracing if defined. */ -/* #undef JEMALLOC_PROF_LIBGCC */ - -/* Use gcc intrinsics for profile backtracing if defined. */ -/* #undef JEMALLOC_PROF_GCC */ - -/* - * JEMALLOC_DSS enables use of sbrk(2) to allocate extents from the data storage - * segment (DSS). - */ -#define JEMALLOC_DSS - -/* Support memory filling (junk/zero). */ -#define JEMALLOC_FILL - -/* Support utrace(2)-based tracing. */ -/* #undef JEMALLOC_UTRACE */ - -/* Support optional abort() on OOM. */ -/* #undef JEMALLOC_XMALLOC */ - -/* Support lazy locking (avoid locking unless a second thread is launched). */ -/* #undef JEMALLOC_LAZY_LOCK */ - -/* - * Minimum allocation alignment is 2^LG_QUANTUM bytes (ignoring tiny size - * classes). - */ -/* #undef LG_QUANTUM */ - -/* One page is 2^LG_PAGE bytes. */ -#define LG_PAGE 12 - -/* - * One huge page is 2^LG_HUGEPAGE bytes. Note that this is defined even if the - * system does not explicitly support huge pages; system calls that require - * explicit huge page support are separately configured. - */ -#define LG_HUGEPAGE 21 - -/* - * If defined, adjacent virtual memory mappings with identical attributes - * automatically coalesce, and they fragment when changes are made to subranges. - * This is the normal order of things for mmap()/munmap(), but on Windows - * VirtualAlloc()/VirtualFree() operations must be precisely matched, i.e. - * mappings do *not* coalesce/fragment. - */ -#define JEMALLOC_MAPS_COALESCE - -/* - * If defined, retain memory for later reuse by default rather than using e.g. - * munmap() to unmap freed extents. This is enabled on 64-bit Linux because - * common sequences of mmap()/munmap() calls will cause virtual memory map - * holes. - */ -#define JEMALLOC_RETAIN - -/* TLS is used to map arenas and magazine caches to threads. */ -#define JEMALLOC_TLS - -/* - * Used to mark unreachable code to quiet "end of non-void" compiler warnings. - * Don't use this directly; instead use unreachable() from util.h - */ -#define JEMALLOC_INTERNAL_UNREACHABLE __builtin_unreachable - -/* - * ffs*() functions to use for bitmapping. Don't use these directly; instead, - * use ffs_*() from util.h. - */ -#define JEMALLOC_INTERNAL_FFSLL __builtin_ffsll -#define JEMALLOC_INTERNAL_FFSL __builtin_ffsl -#define JEMALLOC_INTERNAL_FFS __builtin_ffs - -/* - * If defined, explicitly attempt to more uniformly distribute large allocation - * pointer alignments across all cache indices. - */ -#define JEMALLOC_CACHE_OBLIVIOUS - -/* - * If defined, enable logging facilities. We make this a configure option to - * avoid taking extra branches everywhere. - */ -/* #undef JEMALLOC_LOG */ - -/* - * Darwin (OS X) uses zones to work around Mach-O symbol override shortcomings. - */ -/* #undef JEMALLOC_ZONE */ - -/* - * Methods for determining whether the OS overcommits. - * JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY: Linux's - * /proc/sys/vm.overcommit_memory file. - * JEMALLOC_SYSCTL_VM_OVERCOMMIT: FreeBSD's vm.overcommit sysctl. - */ -/* #undef JEMALLOC_SYSCTL_VM_OVERCOMMIT */ -#define JEMALLOC_PROC_SYS_VM_OVERCOMMIT_MEMORY - -/* Defined if madvise(2) is available. */ -#define JEMALLOC_HAVE_MADVISE - -/* - * Defined if transparent huge pages are supported via the MADV_[NO]HUGEPAGE - * arguments to madvise(2). - */ -#define JEMALLOC_HAVE_MADVISE_HUGE - -/* - * Methods for purging unused pages differ between operating systems. - * - * madvise(..., MADV_FREE) : This marks pages as being unused, such that they - * will be discarded rather than swapped out. - * madvise(..., MADV_DONTNEED) : If JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS is - * defined, this immediately discards pages, - * such that new pages will be demand-zeroed if - * the address region is later touched; - * otherwise this behaves similarly to - * MADV_FREE, though typically with higher - * system overhead. - */ -//#define JEMALLOC_PURGE_MADVISE_FREE -#define JEMALLOC_PURGE_MADVISE_DONTNEED -#define JEMALLOC_PURGE_MADVISE_DONTNEED_ZEROS - -/* Defined if madvise(2) is available but MADV_FREE is not (x86 Linux only). */ -/* #undef JEMALLOC_DEFINE_MADVISE_FREE */ - -/* - * Defined if MADV_DO[NT]DUMP is supported as an argument to madvise. - */ -#define JEMALLOC_MADVISE_DONTDUMP - -/* - * Defined if transparent huge pages (THPs) are supported via the - * MADV_[NO]HUGEPAGE arguments to madvise(2), and THP support is enabled. - */ -/* #undef JEMALLOC_THP */ - -/* Define if operating system has alloca.h header. */ -#define JEMALLOC_HAS_ALLOCA_H 1 - -/* C99 restrict keyword supported. */ -#define JEMALLOC_HAS_RESTRICT 1 - -/* For use by hash code. */ -/* #undef JEMALLOC_BIG_ENDIAN */ - -/* sizeof(int) == 2^LG_SIZEOF_INT. */ -#define LG_SIZEOF_INT 2 - -/* sizeof(long) == 2^LG_SIZEOF_LONG. */ -#define LG_SIZEOF_LONG 3 - -/* sizeof(long long) == 2^LG_SIZEOF_LONG_LONG. */ -#define LG_SIZEOF_LONG_LONG 3 - -/* sizeof(intmax_t) == 2^LG_SIZEOF_INTMAX_T. */ -#define LG_SIZEOF_INTMAX_T 3 - -/* glibc malloc hooks (__malloc_hook, __realloc_hook, __free_hook). */ -#define JEMALLOC_GLIBC_MALLOC_HOOK - -/* glibc memalign hook. */ -#define JEMALLOC_GLIBC_MEMALIGN_HOOK - -/* pthread support */ -#define JEMALLOC_HAVE_PTHREAD - -/* dlsym() support */ -#define JEMALLOC_HAVE_DLSYM - -/* Adaptive mutex support in pthreads. */ -#define JEMALLOC_HAVE_PTHREAD_MUTEX_ADAPTIVE_NP - -/* GNU specific sched_getcpu support */ -#define JEMALLOC_HAVE_SCHED_GETCPU - -/* GNU specific sched_setaffinity support */ -#define JEMALLOC_HAVE_SCHED_SETAFFINITY - -/* - * If defined, all the features necessary for background threads are present. - */ -#define JEMALLOC_BACKGROUND_THREAD 1 - -/* - * If defined, jemalloc symbols are not exported (doesn't work when - * JEMALLOC_PREFIX is not defined). - */ -/* #undef JEMALLOC_EXPORT */ - -/* config.malloc_conf options string. */ -#define JEMALLOC_CONFIG_MALLOC_CONF "" - -/* If defined, jemalloc takes the malloc/free/etc. symbol names. */ -#define JEMALLOC_IS_MALLOC 1 - -/* - * Defined if strerror_r returns char * if _GNU_SOURCE is defined. - */ -#define JEMALLOC_STRERROR_R_RETURNS_CHAR_WITH_GNU_SOURCE - -#endif /* JEMALLOC_INTERNAL_DEFS_H_ */ diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h deleted file mode 100644 index c150785fb4a..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/internal/jemalloc_preamble.h +++ /dev/null @@ -1,194 +0,0 @@ -#ifndef JEMALLOC_PREAMBLE_H -#define JEMALLOC_PREAMBLE_H - -#include "jemalloc_internal_defs.h" -#include "jemalloc/internal/jemalloc_internal_decls.h" - -#ifdef JEMALLOC_UTRACE -#include -#endif - -#define JEMALLOC_NO_DEMANGLE -#ifdef JEMALLOC_JET -# undef JEMALLOC_IS_MALLOC -# define JEMALLOC_N(n) jet_##n -# include "jemalloc/internal/public_namespace.h" -# define JEMALLOC_NO_RENAME -# include "jemalloc/jemalloc.h" -# undef JEMALLOC_NO_RENAME -#else -# define JEMALLOC_N(n) je_##n -# include "jemalloc/jemalloc.h" -#endif - -#if (defined(JEMALLOC_OSATOMIC) || defined(JEMALLOC_OSSPIN)) -#include -#endif - -#ifdef JEMALLOC_ZONE -#include -#include -#include -#endif - -#include "jemalloc/internal/jemalloc_internal_macros.h" - -/* - * Note that the ordering matters here; the hook itself is name-mangled. We - * want the inclusion of hooks to happen early, so that we hook as much as - * possible. - */ -#ifndef JEMALLOC_NO_PRIVATE_NAMESPACE -# ifndef JEMALLOC_JET -# include "jemalloc/internal/private_namespace.h" -# else -# include "jemalloc/internal/private_namespace_jet.h" -# endif -#endif -#include "jemalloc/internal/test_hooks.h" - -#ifdef JEMALLOC_DEFINE_MADVISE_FREE -# define JEMALLOC_MADV_FREE 8 -#endif - -static const bool config_debug = -#ifdef JEMALLOC_DEBUG - true -#else - false -#endif - ; -static const bool have_dss = -#ifdef JEMALLOC_DSS - true -#else - false -#endif - ; -static const bool have_madvise_huge = -#ifdef JEMALLOC_HAVE_MADVISE_HUGE - true -#else - false -#endif - ; -static const bool config_fill = -#ifdef JEMALLOC_FILL - true -#else - false -#endif - ; -static const bool config_lazy_lock = -#ifdef JEMALLOC_LAZY_LOCK - true -#else - false -#endif - ; -static const char * const config_malloc_conf = JEMALLOC_CONFIG_MALLOC_CONF; -static const bool config_prof = -#ifdef JEMALLOC_PROF - true -#else - false -#endif - ; -static const bool config_prof_libgcc = -#ifdef JEMALLOC_PROF_LIBGCC - true -#else - false -#endif - ; -static const bool config_prof_libunwind = -#ifdef JEMALLOC_PROF_LIBUNWIND - true -#else - false -#endif - ; -static const bool maps_coalesce = -#ifdef JEMALLOC_MAPS_COALESCE - true -#else - false -#endif - ; -static const bool config_stats = -#ifdef JEMALLOC_STATS - true -#else - false -#endif - ; -static const bool config_tls = -#ifdef JEMALLOC_TLS - true -#else - false -#endif - ; -static const bool config_utrace = -#ifdef JEMALLOC_UTRACE - true -#else - false -#endif - ; -static const bool config_xmalloc = -#ifdef JEMALLOC_XMALLOC - true -#else - false -#endif - ; -static const bool config_cache_oblivious = -#ifdef JEMALLOC_CACHE_OBLIVIOUS - true -#else - false -#endif - ; -/* - * Undocumented, for jemalloc development use only at the moment. See the note - * in jemalloc/internal/log.h. - */ -static const bool config_log = -#ifdef JEMALLOC_LOG - true -#else - false -#endif - ; -#ifdef JEMALLOC_HAVE_SCHED_GETCPU -/* Currently percpu_arena depends on sched_getcpu. */ -#define JEMALLOC_PERCPU_ARENA -#endif -static const bool have_percpu_arena = -#ifdef JEMALLOC_PERCPU_ARENA - true -#else - false -#endif - ; -/* - * Undocumented, and not recommended; the application should take full - * responsibility for tracking provenance. - */ -static const bool force_ivsalloc = -#ifdef JEMALLOC_FORCE_IVSALLOC - true -#else - false -#endif - ; -static const bool have_background_thread = -#ifdef JEMALLOC_BACKGROUND_THREAD - true -#else - false -#endif - ; - -#endif /* JEMALLOC_PREAMBLE_H */ diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h deleted file mode 100644 index d1389237a77..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_defs.h +++ /dev/null @@ -1,43 +0,0 @@ -/* include/jemalloc/jemalloc_defs.h. Generated from jemalloc_defs.h.in by configure. */ -/* Defined if __attribute__((...)) syntax is supported. */ -#define JEMALLOC_HAVE_ATTR - -/* Defined if alloc_size attribute is supported. */ -#define JEMALLOC_HAVE_ATTR_ALLOC_SIZE - -/* Defined if format(printf, ...) attribute is supported. */ -#define JEMALLOC_HAVE_ATTR_FORMAT_PRINTF - -/* - * Define overrides for non-standard allocator-related functions if they are - * present on the system. - */ -#define JEMALLOC_OVERRIDE_MEMALIGN -#define JEMALLOC_OVERRIDE_VALLOC - -/* - * At least Linux omits the "const" in: - * - * size_t malloc_usable_size(const void *ptr); - * - * Match the operating system's prototype. - */ -#define JEMALLOC_USABLE_SIZE_CONST - -/* - * If defined, specify throw() for the public function prototypes when compiling - * with C++. The only justification for this is to match the prototypes that - * glibc defines. - */ -#define JEMALLOC_USE_CXX_THROW - -#ifdef _MSC_VER -# ifdef _WIN64 -# define LG_SIZEOF_PTR_WIN 3 -# else -# define LG_SIZEOF_PTR_WIN 2 -# endif -#endif - -/* sizeof(void *) == 2^LG_SIZEOF_PTR. */ -#define LG_SIZEOF_PTR 3 diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h deleted file mode 100644 index 7432f1cda53..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_macros.h +++ /dev/null @@ -1,122 +0,0 @@ -#include -#include -#include -#include -#include - -#define JEMALLOC_VERSION "5.1.0-56-g41b7372eadee941b9164751b8d4963f915d3ceae" -#define JEMALLOC_VERSION_MAJOR 5 -#define JEMALLOC_VERSION_MINOR 1 -#define JEMALLOC_VERSION_BUGFIX 0 -#define JEMALLOC_VERSION_NREV 56 -#define JEMALLOC_VERSION_GID "41b7372eadee941b9164751b8d4963f915d3ceae" - -#define MALLOCX_LG_ALIGN(la) ((int)(la)) -#if LG_SIZEOF_PTR == 2 -# define MALLOCX_ALIGN(a) ((int)(ffs((int)(a))-1)) -#else -# define MALLOCX_ALIGN(a) \ - ((int)(((size_t)(a) < (size_t)INT_MAX) ? ffs((int)(a))-1 : \ - ffs((int)(((size_t)(a))>>32))+31)) -#endif -#define MALLOCX_ZERO ((int)0x40) -/* - * Bias tcache index bits so that 0 encodes "automatic tcache management", and 1 - * encodes MALLOCX_TCACHE_NONE. - */ -#define MALLOCX_TCACHE(tc) ((int)(((tc)+2) << 8)) -#define MALLOCX_TCACHE_NONE MALLOCX_TCACHE(-1) -/* - * Bias arena index bits so that 0 encodes "use an automatically chosen arena". - */ -#define MALLOCX_ARENA(a) ((((int)(a))+1) << 20) - -/* - * Use as arena index in "arena..{purge,decay,dss}" and - * "stats.arenas..*" mallctl interfaces to select all arenas. This - * definition is intentionally specified in raw decimal format to support - * cpp-based string concatenation, e.g. - * - * #define STRINGIFY_HELPER(x) #x - * #define STRINGIFY(x) STRINGIFY_HELPER(x) - * - * mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", NULL, NULL, NULL, - * 0); - */ -#define MALLCTL_ARENAS_ALL 4096 -/* - * Use as arena index in "stats.arenas..*" mallctl interfaces to select - * destroyed arenas. - */ -#define MALLCTL_ARENAS_DESTROYED 4097 - -#if defined(__cplusplus) && defined(JEMALLOC_USE_CXX_THROW) -# define JEMALLOC_CXX_THROW throw() -#else -# define JEMALLOC_CXX_THROW -#endif - -#if defined(_MSC_VER) -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) __declspec(align(s)) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# ifndef JEMALLOC_EXPORT -# ifdef DLLEXPORT -# define JEMALLOC_EXPORT __declspec(dllexport) -# else -# define JEMALLOC_EXPORT __declspec(dllimport) -# endif -# endif -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE __declspec(noinline) -# ifdef __cplusplus -# define JEMALLOC_NOTHROW __declspec(nothrow) -# else -# define JEMALLOC_NOTHROW -# endif -# define JEMALLOC_SECTION(s) __declspec(allocate(s)) -# define JEMALLOC_RESTRICT_RETURN __declspec(restrict) -# if _MSC_VER >= 1900 && !defined(__EDG__) -# define JEMALLOC_ALLOCATOR __declspec(allocator) -# else -# define JEMALLOC_ALLOCATOR -# endif -#elif defined(JEMALLOC_HAVE_ATTR) -# define JEMALLOC_ATTR(s) __attribute__((s)) -# define JEMALLOC_ALIGNED(s) JEMALLOC_ATTR(aligned(s)) -# ifdef JEMALLOC_HAVE_ATTR_ALLOC_SIZE -# define JEMALLOC_ALLOC_SIZE(s) JEMALLOC_ATTR(alloc_size(s)) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) JEMALLOC_ATTR(alloc_size(s1, s2)) -# else -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# endif -# ifndef JEMALLOC_EXPORT -# define JEMALLOC_EXPORT JEMALLOC_ATTR(visibility("default")) -# endif -# ifdef JEMALLOC_HAVE_ATTR_FORMAT_GNU_PRINTF -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(gnu_printf, s, i)) -# elif defined(JEMALLOC_HAVE_ATTR_FORMAT_PRINTF) -# define JEMALLOC_FORMAT_PRINTF(s, i) JEMALLOC_ATTR(format(printf, s, i)) -# else -# define JEMALLOC_FORMAT_PRINTF(s, i) -# endif -# define JEMALLOC_NOINLINE JEMALLOC_ATTR(noinline) -# define JEMALLOC_NOTHROW JEMALLOC_ATTR(nothrow) -# define JEMALLOC_SECTION(s) JEMALLOC_ATTR(section(s)) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#else -# define JEMALLOC_ATTR(s) -# define JEMALLOC_ALIGNED(s) -# define JEMALLOC_ALLOC_SIZE(s) -# define JEMALLOC_ALLOC_SIZE2(s1, s2) -# define JEMALLOC_EXPORT -# define JEMALLOC_FORMAT_PRINTF(s, i) -# define JEMALLOC_NOINLINE -# define JEMALLOC_NOTHROW -# define JEMALLOC_SECTION(s) -# define JEMALLOC_RESTRICT_RETURN -# define JEMALLOC_ALLOCATOR -#endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h deleted file mode 100644 index ff025e30fa7..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_protos.h +++ /dev/null @@ -1,66 +0,0 @@ -/* - * The je_ prefix on the following public symbol declarations is an artifact - * of namespace management, and should be omitted in application code unless - * JEMALLOC_NO_DEMANGLE is defined (see jemalloc_mangle.h). - */ -extern JEMALLOC_EXPORT const char *je_malloc_conf; -extern JEMALLOC_EXPORT void (*je_malloc_message)(void *cbopaque, - const char *s); - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_malloc(size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_calloc(size_t num, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE2(1, 2); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_posix_memalign(void **memptr, - size_t alignment, size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(nonnull(1)); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_aligned_alloc(size_t alignment, - size_t size) JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc) - JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_realloc(void *ptr, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_free(void *ptr) - JEMALLOC_CXX_THROW; - -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_mallocx(size_t size, int flags) - JEMALLOC_ATTR(malloc) JEMALLOC_ALLOC_SIZE(1); -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_rallocx(void *ptr, size_t size, - int flags) JEMALLOC_ALLOC_SIZE(2); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_xallocx(void *ptr, size_t size, - size_t extra, int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_sallocx(const void *ptr, - int flags) JEMALLOC_ATTR(pure); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_dallocx(void *ptr, int flags); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_sdallocx(void *ptr, size_t size, - int flags); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_nallocx(size_t size, int flags) - JEMALLOC_ATTR(pure); - -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctl(const char *name, - void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlnametomib(const char *name, - size_t *mibp, size_t *miblenp); -JEMALLOC_EXPORT int JEMALLOC_NOTHROW je_mallctlbymib(const size_t *mib, - size_t miblen, void *oldp, size_t *oldlenp, void *newp, size_t newlen); -JEMALLOC_EXPORT void JEMALLOC_NOTHROW je_malloc_stats_print( - void (*write_cb)(void *, const char *), void *je_cbopaque, - const char *opts); -JEMALLOC_EXPORT size_t JEMALLOC_NOTHROW je_malloc_usable_size( - JEMALLOC_USABLE_SIZE_CONST void *ptr) JEMALLOC_CXX_THROW; - -#ifdef JEMALLOC_OVERRIDE_MEMALIGN -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_memalign(size_t alignment, size_t size) - JEMALLOC_CXX_THROW JEMALLOC_ATTR(malloc); -#endif - -#ifdef JEMALLOC_OVERRIDE_VALLOC -JEMALLOC_EXPORT JEMALLOC_ALLOCATOR JEMALLOC_RESTRICT_RETURN - void JEMALLOC_NOTHROW *je_valloc(size_t size) JEMALLOC_CXX_THROW - JEMALLOC_ATTR(malloc); -#endif diff --git a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h b/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h deleted file mode 100644 index 1a58874306e..00000000000 --- a/contrib/jemalloc-cmake/include_linux_x86_64/jemalloc/jemalloc_typedefs.h +++ /dev/null @@ -1,77 +0,0 @@ -typedef struct extent_hooks_s extent_hooks_t; - -/* - * void * - * extent_alloc(extent_hooks_t *extent_hooks, void *new_addr, size_t size, - * size_t alignment, bool *zero, bool *commit, unsigned arena_ind); - */ -typedef void *(extent_alloc_t)(extent_hooks_t *, void *, size_t, size_t, bool *, - bool *, unsigned); - -/* - * bool - * extent_dalloc(extent_hooks_t *extent_hooks, void *addr, size_t size, - * bool committed, unsigned arena_ind); - */ -typedef bool (extent_dalloc_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); - -/* - * void - * extent_destroy(extent_hooks_t *extent_hooks, void *addr, size_t size, - * bool committed, unsigned arena_ind); - */ -typedef void (extent_destroy_t)(extent_hooks_t *, void *, size_t, bool, - unsigned); - -/* - * bool - * extent_commit(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_commit_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); - -/* - * bool - * extent_decommit(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_decommit_t)(extent_hooks_t *, void *, size_t, size_t, - size_t, unsigned); - -/* - * bool - * extent_purge(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t offset, size_t length, unsigned arena_ind); - */ -typedef bool (extent_purge_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - unsigned); - -/* - * bool - * extent_split(extent_hooks_t *extent_hooks, void *addr, size_t size, - * size_t size_a, size_t size_b, bool committed, unsigned arena_ind); - */ -typedef bool (extent_split_t)(extent_hooks_t *, void *, size_t, size_t, size_t, - bool, unsigned); - -/* - * bool - * extent_merge(extent_hooks_t *extent_hooks, void *addr_a, size_t size_a, - * void *addr_b, size_t size_b, bool committed, unsigned arena_ind); - */ -typedef bool (extent_merge_t)(extent_hooks_t *, void *, size_t, void *, size_t, - bool, unsigned); - -struct extent_hooks_s { - extent_alloc_t *alloc; - extent_dalloc_t *dalloc; - extent_destroy_t *destroy; - extent_commit_t *commit; - extent_decommit_t *decommit; - extent_purge_t *purge_lazy; - extent_purge_t *purge_forced; - extent_split_t *split; - extent_merge_t *merge; -}; diff --git a/contrib/libcpuid/CMakeLists.txt b/contrib/libcpuid/CMakeLists.txt index cd3e7fa06fe..c04acf99f36 100644 --- a/contrib/libcpuid/CMakeLists.txt +++ b/contrib/libcpuid/CMakeLists.txt @@ -17,4 +17,4 @@ include/libcpuid/recog_amd.h include/libcpuid/recog_intel.h ) -target_include_directories (cpuid SYSTEM PUBLIC include) +target_include_directories (cpuid PUBLIC include) diff --git a/contrib/libmurmurhash2/CMakeLists.txt b/contrib/libmurmurhash2/CMakeLists.txt new file mode 100644 index 00000000000..fede35faa50 --- /dev/null +++ b/contrib/libmurmurhash2/CMakeLists.txt @@ -0,0 +1,6 @@ +add_library(murmurhash2 + src/murmurhash2.cpp + include/murmurhash2.h) + +target_include_directories (murmurhash2 PUBLIC include) +target_include_directories (murmurhash2 PUBLIC src) diff --git a/contrib/libmurmurhash2/LICENSE b/contrib/libmurmurhash2/LICENSE new file mode 100644 index 00000000000..e4f5d0c2f40 --- /dev/null +++ b/contrib/libmurmurhash2/LICENSE @@ -0,0 +1 @@ +MurmurHash2 was written by Austin Appleby, and is placed in the publicdomain. The author hereby disclaims copyright to this source code. diff --git a/contrib/murmurhash/README b/contrib/libmurmurhash2/README similarity index 100% rename from contrib/murmurhash/README rename to contrib/libmurmurhash2/README diff --git a/contrib/murmurhash/include/murmurhash2.h b/contrib/libmurmurhash2/include/murmurhash2.h similarity index 100% rename from contrib/murmurhash/include/murmurhash2.h rename to contrib/libmurmurhash2/include/murmurhash2.h diff --git a/contrib/murmurhash/src/murmurhash2.cpp b/contrib/libmurmurhash2/src/murmurhash2.cpp similarity index 100% rename from contrib/murmurhash/src/murmurhash2.cpp rename to contrib/libmurmurhash2/src/murmurhash2.cpp diff --git a/contrib/librdkafka-cmake/config.h b/contrib/librdkafka-cmake/config.h deleted file mode 100644 index 68e93a10ff1..00000000000 --- a/contrib/librdkafka-cmake/config.h +++ /dev/null @@ -1,74 +0,0 @@ -// Automatically generated by ./configure -#ifndef _CONFIG_H_ -#define _CONFIG_H_ -#define ARCH "x86_64" -#define CPU "generic" -#define WITHOUT_OPTIMIZATION 0 -#define ENABLE_DEVEL 0 -#define ENABLE_VALGRIND 0 -#define ENABLE_REFCNT_DEBUG 0 -#define ENABLE_SHAREDPTR_DEBUG 0 -#define ENABLE_LZ4_EXT 1 -#define ENABLE_SSL 1 -//#define ENABLE_SASL 1 -#define MKL_APP_NAME "librdkafka" -#define MKL_APP_DESC_ONELINE "The Apache Kafka C/C++ library" -// distro -//#define SOLIB_EXT ".so" -// gcc -//#define WITH_GCC 1 -// gxx -//#define WITH_GXX 1 -// pkgconfig -//#define WITH_PKGCONFIG 1 -// install -//#define WITH_INSTALL 1 -// PIC -//#define HAVE_PIC 1 -// gnulib -//#define WITH_GNULD 1 -// __atomic_32 -#define HAVE_ATOMICS_32 1 -// __atomic_32 -#define HAVE_ATOMICS_32_ATOMIC 1 -// atomic_32 -#define ATOMIC_OP32(OP1,OP2,PTR,VAL) __atomic_ ## OP1 ## _ ## OP2(PTR, VAL, __ATOMIC_SEQ_CST) -// __atomic_64 -#define HAVE_ATOMICS_64 1 -// __atomic_64 -#define HAVE_ATOMICS_64_ATOMIC 1 -// atomic_64 -#define ATOMIC_OP64(OP1,OP2,PTR,VAL) __atomic_ ## OP1 ## _ ## OP2(PTR, VAL, __ATOMIC_SEQ_CST) -// atomic_64 -#define ATOMIC_OP(OP1,OP2,PTR,VAL) __atomic_ ## OP1 ## _ ## OP2(PTR, VAL, __ATOMIC_SEQ_CST) -// parseversion -#define RDKAFKA_VERSION_STR "0.11.4" -// parseversion -#define MKL_APP_VERSION "0.11.4" -// libdl -//#define WITH_LIBDL 1 -// WITH_PLUGINS -//#define WITH_PLUGINS 1 -// zlib -#define WITH_ZLIB 1 -// WITH_SNAPPY -#define WITH_SNAPPY 1 -// WITH_SOCKEM -#define WITH_SOCKEM 1 -// libssl -#define WITH_SSL 1 -// WITH_SASL_SCRAM -//#define WITH_SASL_SCRAM 1 -// crc32chw -#define WITH_CRC32C_HW 1 -// regex -#define HAVE_REGEX 1 -// strndup -#define HAVE_STRNDUP 1 -// strerror_r -#define HAVE_STRERROR_R 1 -// pthread_setname_gnu -#define HAVE_PTHREAD_SETNAME_GNU 1 -// python -//#define HAVE_PYTHON 1 -#endif /* _CONFIG_H_ */ diff --git a/contrib/librdkafka-cmake/include/README b/contrib/librdkafka-cmake/include/README deleted file mode 100644 index 58fa024e68a..00000000000 --- a/contrib/librdkafka-cmake/include/README +++ /dev/null @@ -1 +0,0 @@ -This directory is needed because rdkafka files have #include "../config.h" diff --git a/contrib/mariadb-connector-c b/contrib/mariadb-connector-c deleted file mode 160000 index a0fd36cc5a5..00000000000 --- a/contrib/mariadb-connector-c +++ /dev/null @@ -1 +0,0 @@ -Subproject commit a0fd36cc5a5313414a5a2ebe9322577a29b4782a diff --git a/contrib/mariadb-connector-c-cmake/CMakeLists.txt b/contrib/mariadb-connector-c-cmake/CMakeLists.txt deleted file mode 100644 index 4c1184b3edb..00000000000 --- a/contrib/mariadb-connector-c-cmake/CMakeLists.txt +++ /dev/null @@ -1,66 +0,0 @@ -set(MARIADB_CLIENT_SOURCE_DIR ${CMAKE_SOURCE_DIR}/contrib/mariadb-connector-c) -set(MARIADB_CLIENT_BINARY_DIR ${CMAKE_BINARY_DIR}/contrib/mariadb-connector-c) - -set(SRCS -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/bmove_upp.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/get_password.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_alloc.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_array.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_charset.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_compress.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_context.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_default.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_dtoa.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_errmsg.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_hash.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_init.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_io.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_list.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_ll2str.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_loaddata.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_net.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_password.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_pvio.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/mariadb_async.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/mariadb_charset.c -#${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/mariadb_dyncol.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/mariadb_lib.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/mariadb_stmt.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_sha1.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_stmt_codec.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_string.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_time.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/ma_tls.c -#${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/gnutls.c -#${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/ma_schannel.c -${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/openssl.c -#${MARIADB_CLIENT_SOURCE_DIR}/libmariadb/secure/schannel.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/auth_gssapi_client.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/dialog.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/gssapi_client.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/gssapi_errmsg.c -${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/mariadb_cleartext.c -${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/my_auth.c -${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/old_password.c -${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sha256_pw.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sspi_client.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/auth/sspi_errmsg.c -${MARIADB_CLIENT_SOURCE_DIR}/plugins/connection/aurora.c -${MARIADB_CLIENT_SOURCE_DIR}/plugins/connection/replication.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/io/remote_io.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/pvio/pvio_npipe.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/pvio/pvio_shmem.c -${MARIADB_CLIENT_SOURCE_DIR}/plugins/pvio/pvio_socket.c -#${MARIADB_CLIENT_SOURCE_DIR}/plugins/trace/trace_example.c -${CMAKE_CURRENT_SOURCE_DIR}/linux_x86_64/libmariadb/ma_client_plugin.c -) - -add_library(mysqlclient STATIC ${SRCS}) - -target_link_libraries(mysqlclient ${OPENSSL_LIBRARIES}) - -target_include_directories(mysqlclient PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}/linux_x86_64/include) -target_include_directories(mysqlclient PUBLIC ${CMAKE_CURRENT_SOURCE_DIR}/common/include) -target_include_directories(mysqlclient PUBLIC ${MARIADB_CLIENT_SOURCE_DIR}/include) - -target_compile_definitions(mysqlclient PRIVATE -D THREAD -D HAVE_OPENSSL -D HAVE_TLS) diff --git a/contrib/mariadb-connector-c-cmake/common/include/mysql/mysql.h b/contrib/mariadb-connector-c-cmake/common/include/mysql/mysql.h deleted file mode 100644 index 741c7ba03c9..00000000000 --- a/contrib/mariadb-connector-c-cmake/common/include/mysql/mysql.h +++ /dev/null @@ -1 +0,0 @@ -#include diff --git a/contrib/mariadb-connector-c-cmake/common/include/mysql/mysqld_error.h b/contrib/mariadb-connector-c-cmake/common/include/mysql/mysqld_error.h deleted file mode 100644 index 95d26eef163..00000000000 --- a/contrib/mariadb-connector-c-cmake/common/include/mysql/mysqld_error.h +++ /dev/null @@ -1 +0,0 @@ -#include diff --git a/contrib/mariadb-connector-c-cmake/linux_x86_64/include/config.h b/contrib/mariadb-connector-c-cmake/linux_x86_64/include/config.h deleted file mode 100644 index 90c42c97df6..00000000000 --- a/contrib/mariadb-connector-c-cmake/linux_x86_64/include/config.h +++ /dev/null @@ -1,269 +0,0 @@ - -/* - * Include file constants (processed in LibmysqlIncludeFiles.txt 1 - */ -#define HAVE_ALLOCA_H 1 -/* #undef HAVE_BIGENDIAN */ -#define HAVE_SETLOCALE 1 -#define HAVE_NL_LANGINFO 1 -#define HAVE_ARPA_INET_H 1 -#define HAVE_CRYPT_H 1 -#define HAVE_DIRENT_H 1 -#define HAVE_DLFCN_H 1 -#define HAVE_EXECINFO_H 1 -#define HAVE_FCNTL_H 1 -#define HAVE_FENV_H 1 -#define HAVE_FLOAT_H 1 -/* #undef HAVE_FPU_CONTROL_H */ -#define HAVE_GRP_H 1 -/* #undef HAVE_IEEEFP_H */ -#define HAVE_LIMITS_H 1 -#define HAVE_MALLOC_H 1 -#define HAVE_MEMORY_H 1 -#define HAVE_NETINET_IN_H 1 -#define HAVE_PATHS_H 1 -#define HAVE_PWD_H 1 -#define HAVE_SCHED_H 1 -/* #undef HAVE_SELECT_H */ -#define HAVE_STDDEF_H 1 -#define HAVE_STDINT_H 1 -#define HAVE_STDLIB_H 1 -#define HAVE_STRING_H 1 -#define HAVE_STRINGS_H 1 -/* #undef HAVE_SYNCH_H */ -/* #undef HAVE_SYS_FPU_H */ -#define HAVE_SYS_IOCTL_H 1 -#define HAVE_SYS_IPC_H 1 -#define HAVE_SYS_MMAN_H 1 -#define HAVE_SYS_PRCTL_H 1 -#define HAVE_SYS_SELECT_H 1 -#define HAVE_SYS_SHM_H 1 -#define HAVE_SYS_SOCKET_H 1 -#define HAVE_SYS_STAT_H 1 -/* #undef HAVE_SYS_STREAM_H */ -#define HAVE_SYS_TIMEB_H 1 -#define HAVE_SYS_TYPES_H 1 -#define HAVE_SYS_UN_H 1 -/* #undef HAVE_SYSENT_H */ -#define HAVE_TERMIO_H 1 -#define HAVE_TERMIOS_H 1 -#define HAVE_UNISTD_H 1 -#define HAVE_UTIME_H 1 -#define HAVE_UCONTEXT_H 1 - -/* - * function definitions - processed in LibmysqlFunctions.txt - */ -#define HAVE_ACCESS 1 -/* #undef HAVE_AIOWAIT */ -#define HAVE_ALARM 1 -/* #undef HAVE_ALLOCA */ -#define HAVE_BCMP 1 -/* #undef HAVE_BFILL */ -/* #undef HAVE_BMOVE */ -#define HAVE_BZERO 1 -#define HAVE_CLOCK_GETTIME 1 -/* #undef HAVE_COMPRESS */ -/* #undef HAVE_CRYPT */ -#define HAVE_DLERROR 1 -#define HAVE_DLOPEN 1 -#define HAVE_FCHMOD 1 -#define HAVE_FCNTL 1 -/* #undef HAVE_FCONVERT */ -#define HAVE_FDATASYNC 1 -#define HAVE_FESETROUND 1 -#define HAVE_FINITE 1 -#define HAVE_FSEEKO 1 -#define HAVE_FSYNC 1 -#define HAVE_GETADDRINFO 1 -#define HAVE_GETCWD 1 -#define HAVE_GETHOSTBYADDR_R 1 -#define HAVE_GETHOSTBYNAME_R 1 -/* #undef HAVE_GETHRTIME */ -#define HAVE_GETNAMEINFO 1 -#define HAVE_GETPAGESIZE 1 -#define HAVE_GETPASS 1 -/* #undef HAVE_GETPASSPHRASE */ -#define HAVE_GETPWNAM 1 -#define HAVE_GETPWUID 1 -#define HAVE_GETRLIMIT 1 -#define HAVE_GETRUSAGE 1 -#define HAVE_GETWD 1 -#define HAVE_GMTIME_R 1 -#define HAVE_INITGROUPS 1 -#define HAVE_LDIV 1 -#define HAVE_LOCALTIME_R 1 -#define HAVE_LOG2 1 -#define HAVE_LONGJMP 1 -#define HAVE_LSTAT 1 -#define HAVE_MADVISE 1 -#define HAVE_MALLINFO 1 -#define HAVE_MEMALIGN 1 -#define HAVE_MEMCPY 1 -#define HAVE_MEMMOVE 1 -#define HAVE_MKSTEMP 1 -#define HAVE_MLOCK 1 -#define HAVE_MLOCKALL 1 -#define HAVE_MMAP 1 -#define HAVE_MMAP64 1 -#define HAVE_PERROR 1 -#define HAVE_POLL 1 -#define HAVE_PREAD 1 -/* #undef HAVE_PTHREAD_ATTR_CREATE */ -#define HAVE_PTHREAD_ATTR_GETSTACKSIZE 1 -/* #undef HAVE_PTHREAD_ATTR_SETPRIO */ -#define HAVE_PTHREAD_ATTR_SETSCHEDPARAM 1 -#define HAVE_PTHREAD_ATTR_SETSCOPE 1 -#define HAVE_PTHREAD_ATTR_SETSTACKSIZE 1 -/* #undef HAVE_PTHREAD_CONDATTR_CREATE */ -/* #undef HAVE_PTHREAD_INIT */ -#define HAVE_PTHREAD_KEY_DELETE 1 -#define HAVE_PTHREAD_KILL 1 -#define HAVE_PTHREAD_RWLOCK_RDLOCK 1 -/* #undef HAVE_PTHREAD_SETPRIO_NP */ -#define HAVE_PTHREAD_SETSCHEDPARAM 1 -#define HAVE_PTHREAD_SIGMASK 1 -/* #undef HAVE_PTHREAD_THREADMASK */ -/* #undef HAVE_PTHREAD_YIELD_NP */ -#define HAVE_READDIR_R 1 -#define HAVE_READLINK 1 -#define HAVE_REALPATH 1 -#define HAVE_RENAME 1 -#define HAVE_SCHED_YIELD 1 -#define HAVE_SELECT 1 -/* #undef HAVE_SETFD */ -/* #undef HAVE_SETFILEPOINTER */ -#define HAVE_SIGNAL 1 -#define HAVE_SIGACTION 1 -/* #undef HAVE_SIGTHREADMASK */ -#define HAVE_SIGWAIT 1 -#define HAVE_SLEEP 1 -#define HAVE_SNPRINTF 1 -/* #undef HAVE_SQLITE */ -#define HAVE_STPCPY 1 -#define HAVE_STRERROR 1 -/* #undef HAVE_STRLCPY */ -#define HAVE_STRNLEN 1 -#define HAVE_STRPBRK 1 -#define HAVE_STRSEP 1 -#define HAVE_STRSTR 1 -#define HAVE_STRTOK_R 1 -#define HAVE_STRTOL 1 -#define HAVE_STRTOLL 1 -#define HAVE_STRTOUL 1 -#define HAVE_STRTOULL 1 -/* #undef HAVE_TELL */ -/* #undef HAVE_THR_SETCONCURRENCY */ -/* #undef HAVE_THR_YIELD */ -#define HAVE_VASPRINTF 1 -#define HAVE_VSNPRINTF 1 - -/* - * types and sizes - */ -/* Types we may use */ -#define SIZEOF_CHAR 1 -#if defined(SIZEOF_CHAR) -# define HAVE_CHAR 1 -#endif - -#define SIZEOF_CHARP 8 -#if defined(SIZEOF_CHARP) -# define HAVE_CHARP 1 -#endif - -#define SIZEOF_SHORT 2 -#if defined(SIZEOF_SHORT) -# define HAVE_SHORT 1 -#endif - -#define SIZEOF_INT 4 -#if defined(SIZEOF_INT) -# define HAVE_INT 1 -#endif - -#define SIZEOF_LONG 8 -#if defined(SIZEOF_LONG) -# define HAVE_LONG 1 -#endif - -#define SIZEOF_LONG_LONG 8 -#if defined(SIZEOF_LONG_LONG) -# define HAVE_LONG_LONG 1 -#endif - - -#define SIZEOF_SIGSET_T 128 -#if defined(SIZEOF_SIGSET_T) -# define HAVE_SIGSET_T 1 -#endif - -#define SIZEOF_SIZE_T 8 -#if defined(SIZEOF_SIZE_T) -# define HAVE_SIZE_T 1 -#endif - -/* #undef SIZEOF_UCHAR */ -#if defined(SIZEOF_UCHAR) -# define HAVE_UCHAR 1 -#endif - -#define SIZEOF_UINT 4 -#if defined(SIZEOF_UINT) -# define HAVE_UINT 1 -#endif - -#define SIZEOF_ULONG 8 -#if defined(SIZEOF_ULONG) -# define HAVE_ULONG 1 -#endif - -/* #undef SIZEOF_INT8 */ -#if defined(SIZEOF_INT8) -# define HAVE_INT8 1 -#endif -/* #undef SIZEOF_UINT8 */ -#if defined(SIZEOF_UINT8) -# define HAVE_UINT8 1 -#endif - -/* #undef SIZEOF_INT16 */ -#if defined(SIZEOF_INT16) -# define HAVE_INT16 1 -#endif -/* #undef SIZEOF_UINT16 */ -#if defined(SIZEOF_UINT16) -# define HAVE_UINT16 1 -#endif - -/* #undef SIZEOF_INT32 */ -#if defined(SIZEOF_INT32) -# define HAVE_INT32 1 -#endif -/* #undef SIZEOF_UINT32 */ -#if defined(SIZEOF_UINT32) -# define HAVE_UINT32 1 -#endif -/* #undef SIZEOF_U_INT32_T */ -#if defined(SIZEOF_U_INT32_T) -# define HAVE_U_INT32_T 1 -#endif - -/* #undef SIZEOF_INT64 */ -#if defined(SIZEOF_INT64) -# define HAVE_INT64 1 -#endif -/* #undef SIZEOF_UINT64 */ -#if defined(SIZEOF_UINT64) -# define HAVE_UINT64 1 -#endif - -/* #undef SIZEOF_SOCKLEN_T */ -#if defined(SIZEOF_SOCKLEN_T) -# define HAVE_SOCKLEN_T 1 -#endif - -#define SOCKET_SIZE_TYPE socklen_t - -#define MARIADB_DEFAULT_CHARSET "latin1" - diff --git a/contrib/mariadb-connector-c-cmake/linux_x86_64/include/ma_config.h b/contrib/mariadb-connector-c-cmake/linux_x86_64/include/ma_config.h deleted file mode 100644 index 90c42c97df6..00000000000 --- a/contrib/mariadb-connector-c-cmake/linux_x86_64/include/ma_config.h +++ /dev/null @@ -1,269 +0,0 @@ - -/* - * Include file constants (processed in LibmysqlIncludeFiles.txt 1 - */ -#define HAVE_ALLOCA_H 1 -/* #undef HAVE_BIGENDIAN */ -#define HAVE_SETLOCALE 1 -#define HAVE_NL_LANGINFO 1 -#define HAVE_ARPA_INET_H 1 -#define HAVE_CRYPT_H 1 -#define HAVE_DIRENT_H 1 -#define HAVE_DLFCN_H 1 -#define HAVE_EXECINFO_H 1 -#define HAVE_FCNTL_H 1 -#define HAVE_FENV_H 1 -#define HAVE_FLOAT_H 1 -/* #undef HAVE_FPU_CONTROL_H */ -#define HAVE_GRP_H 1 -/* #undef HAVE_IEEEFP_H */ -#define HAVE_LIMITS_H 1 -#define HAVE_MALLOC_H 1 -#define HAVE_MEMORY_H 1 -#define HAVE_NETINET_IN_H 1 -#define HAVE_PATHS_H 1 -#define HAVE_PWD_H 1 -#define HAVE_SCHED_H 1 -/* #undef HAVE_SELECT_H */ -#define HAVE_STDDEF_H 1 -#define HAVE_STDINT_H 1 -#define HAVE_STDLIB_H 1 -#define HAVE_STRING_H 1 -#define HAVE_STRINGS_H 1 -/* #undef HAVE_SYNCH_H */ -/* #undef HAVE_SYS_FPU_H */ -#define HAVE_SYS_IOCTL_H 1 -#define HAVE_SYS_IPC_H 1 -#define HAVE_SYS_MMAN_H 1 -#define HAVE_SYS_PRCTL_H 1 -#define HAVE_SYS_SELECT_H 1 -#define HAVE_SYS_SHM_H 1 -#define HAVE_SYS_SOCKET_H 1 -#define HAVE_SYS_STAT_H 1 -/* #undef HAVE_SYS_STREAM_H */ -#define HAVE_SYS_TIMEB_H 1 -#define HAVE_SYS_TYPES_H 1 -#define HAVE_SYS_UN_H 1 -/* #undef HAVE_SYSENT_H */ -#define HAVE_TERMIO_H 1 -#define HAVE_TERMIOS_H 1 -#define HAVE_UNISTD_H 1 -#define HAVE_UTIME_H 1 -#define HAVE_UCONTEXT_H 1 - -/* - * function definitions - processed in LibmysqlFunctions.txt - */ -#define HAVE_ACCESS 1 -/* #undef HAVE_AIOWAIT */ -#define HAVE_ALARM 1 -/* #undef HAVE_ALLOCA */ -#define HAVE_BCMP 1 -/* #undef HAVE_BFILL */ -/* #undef HAVE_BMOVE */ -#define HAVE_BZERO 1 -#define HAVE_CLOCK_GETTIME 1 -/* #undef HAVE_COMPRESS */ -/* #undef HAVE_CRYPT */ -#define HAVE_DLERROR 1 -#define HAVE_DLOPEN 1 -#define HAVE_FCHMOD 1 -#define HAVE_FCNTL 1 -/* #undef HAVE_FCONVERT */ -#define HAVE_FDATASYNC 1 -#define HAVE_FESETROUND 1 -#define HAVE_FINITE 1 -#define HAVE_FSEEKO 1 -#define HAVE_FSYNC 1 -#define HAVE_GETADDRINFO 1 -#define HAVE_GETCWD 1 -#define HAVE_GETHOSTBYADDR_R 1 -#define HAVE_GETHOSTBYNAME_R 1 -/* #undef HAVE_GETHRTIME */ -#define HAVE_GETNAMEINFO 1 -#define HAVE_GETPAGESIZE 1 -#define HAVE_GETPASS 1 -/* #undef HAVE_GETPASSPHRASE */ -#define HAVE_GETPWNAM 1 -#define HAVE_GETPWUID 1 -#define HAVE_GETRLIMIT 1 -#define HAVE_GETRUSAGE 1 -#define HAVE_GETWD 1 -#define HAVE_GMTIME_R 1 -#define HAVE_INITGROUPS 1 -#define HAVE_LDIV 1 -#define HAVE_LOCALTIME_R 1 -#define HAVE_LOG2 1 -#define HAVE_LONGJMP 1 -#define HAVE_LSTAT 1 -#define HAVE_MADVISE 1 -#define HAVE_MALLINFO 1 -#define HAVE_MEMALIGN 1 -#define HAVE_MEMCPY 1 -#define HAVE_MEMMOVE 1 -#define HAVE_MKSTEMP 1 -#define HAVE_MLOCK 1 -#define HAVE_MLOCKALL 1 -#define HAVE_MMAP 1 -#define HAVE_MMAP64 1 -#define HAVE_PERROR 1 -#define HAVE_POLL 1 -#define HAVE_PREAD 1 -/* #undef HAVE_PTHREAD_ATTR_CREATE */ -#define HAVE_PTHREAD_ATTR_GETSTACKSIZE 1 -/* #undef HAVE_PTHREAD_ATTR_SETPRIO */ -#define HAVE_PTHREAD_ATTR_SETSCHEDPARAM 1 -#define HAVE_PTHREAD_ATTR_SETSCOPE 1 -#define HAVE_PTHREAD_ATTR_SETSTACKSIZE 1 -/* #undef HAVE_PTHREAD_CONDATTR_CREATE */ -/* #undef HAVE_PTHREAD_INIT */ -#define HAVE_PTHREAD_KEY_DELETE 1 -#define HAVE_PTHREAD_KILL 1 -#define HAVE_PTHREAD_RWLOCK_RDLOCK 1 -/* #undef HAVE_PTHREAD_SETPRIO_NP */ -#define HAVE_PTHREAD_SETSCHEDPARAM 1 -#define HAVE_PTHREAD_SIGMASK 1 -/* #undef HAVE_PTHREAD_THREADMASK */ -/* #undef HAVE_PTHREAD_YIELD_NP */ -#define HAVE_READDIR_R 1 -#define HAVE_READLINK 1 -#define HAVE_REALPATH 1 -#define HAVE_RENAME 1 -#define HAVE_SCHED_YIELD 1 -#define HAVE_SELECT 1 -/* #undef HAVE_SETFD */ -/* #undef HAVE_SETFILEPOINTER */ -#define HAVE_SIGNAL 1 -#define HAVE_SIGACTION 1 -/* #undef HAVE_SIGTHREADMASK */ -#define HAVE_SIGWAIT 1 -#define HAVE_SLEEP 1 -#define HAVE_SNPRINTF 1 -/* #undef HAVE_SQLITE */ -#define HAVE_STPCPY 1 -#define HAVE_STRERROR 1 -/* #undef HAVE_STRLCPY */ -#define HAVE_STRNLEN 1 -#define HAVE_STRPBRK 1 -#define HAVE_STRSEP 1 -#define HAVE_STRSTR 1 -#define HAVE_STRTOK_R 1 -#define HAVE_STRTOL 1 -#define HAVE_STRTOLL 1 -#define HAVE_STRTOUL 1 -#define HAVE_STRTOULL 1 -/* #undef HAVE_TELL */ -/* #undef HAVE_THR_SETCONCURRENCY */ -/* #undef HAVE_THR_YIELD */ -#define HAVE_VASPRINTF 1 -#define HAVE_VSNPRINTF 1 - -/* - * types and sizes - */ -/* Types we may use */ -#define SIZEOF_CHAR 1 -#if defined(SIZEOF_CHAR) -# define HAVE_CHAR 1 -#endif - -#define SIZEOF_CHARP 8 -#if defined(SIZEOF_CHARP) -# define HAVE_CHARP 1 -#endif - -#define SIZEOF_SHORT 2 -#if defined(SIZEOF_SHORT) -# define HAVE_SHORT 1 -#endif - -#define SIZEOF_INT 4 -#if defined(SIZEOF_INT) -# define HAVE_INT 1 -#endif - -#define SIZEOF_LONG 8 -#if defined(SIZEOF_LONG) -# define HAVE_LONG 1 -#endif - -#define SIZEOF_LONG_LONG 8 -#if defined(SIZEOF_LONG_LONG) -# define HAVE_LONG_LONG 1 -#endif - - -#define SIZEOF_SIGSET_T 128 -#if defined(SIZEOF_SIGSET_T) -# define HAVE_SIGSET_T 1 -#endif - -#define SIZEOF_SIZE_T 8 -#if defined(SIZEOF_SIZE_T) -# define HAVE_SIZE_T 1 -#endif - -/* #undef SIZEOF_UCHAR */ -#if defined(SIZEOF_UCHAR) -# define HAVE_UCHAR 1 -#endif - -#define SIZEOF_UINT 4 -#if defined(SIZEOF_UINT) -# define HAVE_UINT 1 -#endif - -#define SIZEOF_ULONG 8 -#if defined(SIZEOF_ULONG) -# define HAVE_ULONG 1 -#endif - -/* #undef SIZEOF_INT8 */ -#if defined(SIZEOF_INT8) -# define HAVE_INT8 1 -#endif -/* #undef SIZEOF_UINT8 */ -#if defined(SIZEOF_UINT8) -# define HAVE_UINT8 1 -#endif - -/* #undef SIZEOF_INT16 */ -#if defined(SIZEOF_INT16) -# define HAVE_INT16 1 -#endif -/* #undef SIZEOF_UINT16 */ -#if defined(SIZEOF_UINT16) -# define HAVE_UINT16 1 -#endif - -/* #undef SIZEOF_INT32 */ -#if defined(SIZEOF_INT32) -# define HAVE_INT32 1 -#endif -/* #undef SIZEOF_UINT32 */ -#if defined(SIZEOF_UINT32) -# define HAVE_UINT32 1 -#endif -/* #undef SIZEOF_U_INT32_T */ -#if defined(SIZEOF_U_INT32_T) -# define HAVE_U_INT32_T 1 -#endif - -/* #undef SIZEOF_INT64 */ -#if defined(SIZEOF_INT64) -# define HAVE_INT64 1 -#endif -/* #undef SIZEOF_UINT64 */ -#if defined(SIZEOF_UINT64) -# define HAVE_UINT64 1 -#endif - -/* #undef SIZEOF_SOCKLEN_T */ -#if defined(SIZEOF_SOCKLEN_T) -# define HAVE_SOCKLEN_T 1 -#endif - -#define SOCKET_SIZE_TYPE socklen_t - -#define MARIADB_DEFAULT_CHARSET "latin1" - diff --git a/contrib/mariadb-connector-c-cmake/linux_x86_64/include/mariadb_version.h b/contrib/mariadb-connector-c-cmake/linux_x86_64/include/mariadb_version.h deleted file mode 100644 index 821a7f8add2..00000000000 --- a/contrib/mariadb-connector-c-cmake/linux_x86_64/include/mariadb_version.h +++ /dev/null @@ -1,36 +0,0 @@ -/* Copyright Abandoned 1996, 1999, 2001 MySQL AB - This file is public domain and comes with NO WARRANTY of any kind */ - -/* Version numbers for protocol & mysqld */ - -#ifndef _mariadb_version_h_ -#define _mariadb_version_h_ - -#ifdef _CUSTOMCONFIG_ -#include -#else -#define PROTOCOL_VERSION 10 -#define MARIADB_CLIENT_VERSION_STR "10.3.6" -#define MARIADB_BASE_VERSION "mariadb-10.3" -#define MARIADB_VERSION_ID 100306 -#define MYSQL_VERSION_ID 100306 -#define MARIADB_PORT 3306 -#define MARIADB_UNIX_ADDR "/var/run/mysqld/mysqld.sock" -#define MYSQL_CONFIG_NAME "my" - -#define MARIADB_PACKAGE_VERSION "3.0.6" -#define MARIADB_PACKAGE_VERSION_ID 30006 -#define MARIADB_SYSTEM_TYPE "Linux" -#define MARIADB_MACHINE_TYPE "x86_64" -#define MARIADB_PLUGINDIR "lib/mariadb/plugin" - -/* mysqld compile time options */ -#ifndef MYSQL_CHARSET -#define MYSQL_CHARSET "" -#endif -#endif - -/* Source information */ -#define CC_SOURCE_REVISION "a0fd36cc5a5313414a5a2ebe9322577a29b4782a" - -#endif /* _mariadb_version_h_ */ diff --git a/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c b/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c deleted file mode 100644 index b7fdcdbcb85..00000000000 --- a/contrib/mariadb-connector-c-cmake/linux_x86_64/libmariadb/ma_client_plugin.c +++ /dev/null @@ -1,499 +0,0 @@ -/* Copyright (C) 2010 - 2012 Sergei Golubchik and Monty Program Ab - 2015-2016 MariaDB Corporation AB - - This library is free software; you can redistribute it and/or - modify it under the terms of the GNU Library General Public - License as published by the Free Software Foundation; either - version 2 of the License, or (at your option) any later version. - - This library is distributed in the hope that it will be useful, - but WITHOUT ANY WARRANTY; without even the implied warranty of - MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU - Library General Public License for more details. - - You should have received a copy of the GNU Library General Public - License along with this library; if not see - or write to the Free Software Foundation, Inc., - 51 Franklin St., Fifth Floor, Boston, MA 02110, USA */ - -/** - @file - - Support code for the client side (libmariadb) plugins - - Client plugins are somewhat different from server plugins, they are simpler. - - They do not need to be installed or in any way explicitly loaded on the - client, they are loaded automatically on demand. - One client plugin per shared object, soname *must* match the plugin name. - - There is no reference counting and no unloading either. -*/ - -#if _MSC_VER -/* Silence warnings about variable 'unused' being used. */ -#define FORCE_INIT_OF_VARS 1 -#endif - -#include -#include -#include -#include -#include - -#include "errmsg.h" -#include - -struct st_client_plugin_int { - struct st_client_plugin_int *next; - void *dlhandle; - struct st_mysql_client_plugin *plugin; -}; - -static my_bool initialized= 0; -static MA_MEM_ROOT mem_root; - -static uint valid_plugins[][2]= { - {MYSQL_CLIENT_AUTHENTICATION_PLUGIN, MYSQL_CLIENT_AUTHENTICATION_PLUGIN_INTERFACE_VERSION}, - {MARIADB_CLIENT_PVIO_PLUGIN, MARIADB_CLIENT_PVIO_PLUGIN_INTERFACE_VERSION}, - {MARIADB_CLIENT_TRACE_PLUGIN, MARIADB_CLIENT_TRACE_PLUGIN_INTERFACE_VERSION}, - {MARIADB_CLIENT_CONNECTION_PLUGIN, MARIADB_CLIENT_CONNECTION_PLUGIN_INTERFACE_VERSION}, - {0, 0} -}; - -/* - Loaded plugins are stored in a linked list. - The list is append-only, the elements are added to the head (like in a stack). - The elements are added under a mutex, but the list can be read and traversed - without any mutex because once an element is added to the list, it stays - there. The main purpose of a mutex is to prevent two threads from - loading the same plugin twice in parallel. -*/ - - -struct st_client_plugin_int *plugin_list[MYSQL_CLIENT_MAX_PLUGINS + MARIADB_CLIENT_MAX_PLUGINS]; -#ifdef THREAD -static pthread_mutex_t LOCK_load_client_plugin; -#endif - - extern struct st_mysql_client_plugin mysql_native_password_client_plugin; - extern struct st_mysql_client_plugin mysql_old_password_client_plugin; - extern struct st_mysql_client_plugin pvio_socket_client_plugin; - - -struct st_mysql_client_plugin *mysql_client_builtins[]= -{ - (struct st_mysql_client_plugin *)&mysql_native_password_client_plugin, - (struct st_mysql_client_plugin *)&mysql_old_password_client_plugin, - (struct st_mysql_client_plugin *)&pvio_socket_client_plugin, - - 0 -}; - - -static int is_not_initialized(MYSQL *mysql, const char *name) -{ - if (initialized) - return 0; - - my_set_error(mysql, CR_AUTH_PLUGIN_CANNOT_LOAD, - SQLSTATE_UNKNOWN, ER(CR_AUTH_PLUGIN_CANNOT_LOAD), - name, "not initialized"); - return 1; -} - -static int get_plugin_nr(uint type) -{ - uint i= 0; - for(; valid_plugins[i][1]; i++) - if (valid_plugins[i][0] == type) - return i; - return -1; -} - -static const char *check_plugin_version(struct st_mysql_client_plugin *plugin, unsigned int version) -{ - if (plugin->interface_version < version || - (plugin->interface_version >> 8) > (version >> 8)) - return "Incompatible client plugin interface"; - return 0; -} - -/** - finds a plugin in the list - - @param name plugin name to search for - @param type plugin type - - @note this does NOT necessarily need a mutex, take care! - - @retval a pointer to a found plugin or 0 -*/ -static struct st_mysql_client_plugin *find_plugin(const char *name, int type) -{ - struct st_client_plugin_int *p; - int plugin_nr= get_plugin_nr(type); - - DBUG_ASSERT(initialized); - if (plugin_nr == -1) - return 0; - - if (!name) - return plugin_list[plugin_nr]->plugin; - - for (p= plugin_list[plugin_nr]; p; p= p->next) - { - if (strcmp(p->plugin->name, name) == 0) - return p->plugin; - } - return NULL; -} - - -/** - verifies the plugin and adds it to the list - - @param mysql MYSQL structure (for error reporting) - @param plugin plugin to install - @param dlhandle a handle to the shared object (returned by dlopen) - or 0 if the plugin was not dynamically loaded - @param argc number of arguments in the 'va_list args' - @param args arguments passed to the plugin initialization function - - @retval a pointer to an installed plugin or 0 -*/ - -static struct st_mysql_client_plugin * -add_plugin(MYSQL *mysql, struct st_mysql_client_plugin *plugin, void *dlhandle, - int argc, va_list args) -{ - const char *errmsg; - struct st_client_plugin_int plugin_int, *p; - char errbuf[1024]; - int plugin_nr; - - DBUG_ASSERT(initialized); - - plugin_int.plugin= plugin; - plugin_int.dlhandle= dlhandle; - - if ((plugin_nr= get_plugin_nr(plugin->type)) == -1) - { - errmsg= "Unknown client plugin type"; - goto err1; - } - if ((errmsg= check_plugin_version(plugin, valid_plugins[plugin_nr][1]))) - goto err1; - - /* Call the plugin initialization function, if any */ - if (plugin->init && plugin->init(errbuf, sizeof(errbuf), argc, args)) - { - errmsg= errbuf; - goto err1; - } - - p= (struct st_client_plugin_int *) - ma_memdup_root(&mem_root, (char *)&plugin_int, sizeof(plugin_int)); - - if (!p) - { - errmsg= "Out of memory"; - goto err2; - } - -#ifdef THREAD - safe_mutex_assert_owner(&LOCK_load_client_plugin); -#endif - - p->next= plugin_list[plugin_nr]; - plugin_list[plugin_nr]= p; - - return plugin; - -err2: - if (plugin->deinit) - plugin->deinit(); -err1: - my_set_error(mysql, CR_AUTH_PLUGIN_CANNOT_LOAD, SQLSTATE_UNKNOWN, - ER(CR_AUTH_PLUGIN_CANNOT_LOAD), plugin->name, errmsg); - if (dlhandle) - (void)dlclose(dlhandle); - return NULL; -} - - -/** - Loads plugins which are specified in the environment variable - LIBMYSQL_PLUGINS. - - Multiple plugins must be separated by semicolon. This function doesn't - return or log an error. - - The function is be called by mysql_client_plugin_init - - @todo - Support extended syntax, passing parameters to plugins, for example - LIBMYSQL_PLUGINS="plugin1(param1,param2);plugin2;..." - or - LIBMYSQL_PLUGINS="plugin1=int:param1,str:param2;plugin2;..." -*/ - -static void load_env_plugins(MYSQL *mysql) -{ - char *plugs, *free_env, *s= getenv("LIBMYSQL_PLUGINS"); - - if (ma_check_env_str(s)) - return; - - free_env= strdup(s); - plugs= s= free_env; - - do { - if ((s= strchr(plugs, ';'))) - *s= '\0'; - mysql_load_plugin(mysql, plugs, -1, 0); - plugs= s + 1; - } while (s); - - free(free_env); -} - -/********** extern functions to be used by libmariadb *********************/ - -/** - Initializes the client plugin layer. - - This function must be called before any other client plugin function. - - @retval 0 successful - @retval != 0 error occurred -*/ - -int mysql_client_plugin_init() -{ - MYSQL mysql; - struct st_mysql_client_plugin **builtin; - va_list unused; - LINT_INIT_STRUCT(unused); - - if (initialized) - return 0; - - memset(&mysql, 0, sizeof(mysql)); /* dummy mysql for set_mysql_extended_error */ - - pthread_mutex_init(&LOCK_load_client_plugin, MY_MUTEX_INIT_SLOW); - ma_init_alloc_root(&mem_root, 128, 128); - - memset(&plugin_list, 0, sizeof(plugin_list)); - - initialized= 1; - - pthread_mutex_lock(&LOCK_load_client_plugin); - for (builtin= mysql_client_builtins; *builtin; builtin++) - add_plugin(&mysql, *builtin, 0, 0, unused); - - pthread_mutex_unlock(&LOCK_load_client_plugin); - - load_env_plugins(&mysql); - - return 0; -} - - -/** - Deinitializes the client plugin layer. - - Unloades all client plugins and frees any associated resources. -*/ - -void mysql_client_plugin_deinit() -{ - int i; - struct st_client_plugin_int *p; - - if (!initialized) - return; - - for (i=0; i < MYSQL_CLIENT_MAX_PLUGINS; i++) - for (p= plugin_list[i]; p; p= p->next) - { - if (p->plugin->deinit) - p->plugin->deinit(); - if (p->dlhandle) - (void)dlclose(p->dlhandle); - } - - memset(&plugin_list, 0, sizeof(plugin_list)); - initialized= 0; - ma_free_root(&mem_root, MYF(0)); - pthread_mutex_destroy(&LOCK_load_client_plugin); -} - -/************* public facing functions, for client consumption *********/ - -/* see for a full description */ -struct st_mysql_client_plugin * STDCALL -mysql_client_register_plugin(MYSQL *mysql, - struct st_mysql_client_plugin *plugin) -{ - va_list unused; - LINT_INIT_STRUCT(unused); - - if (is_not_initialized(mysql, plugin->name)) - return NULL; - - pthread_mutex_lock(&LOCK_load_client_plugin); - - /* make sure the plugin wasn't loaded meanwhile */ - if (find_plugin(plugin->name, plugin->type)) - { - my_set_error(mysql, CR_AUTH_PLUGIN_CANNOT_LOAD, - SQLSTATE_UNKNOWN, ER(CR_AUTH_PLUGIN_CANNOT_LOAD), - plugin->name, "it is already loaded"); - plugin= NULL; - } - else - plugin= add_plugin(mysql, plugin, 0, 0, unused); - - pthread_mutex_unlock(&LOCK_load_client_plugin); - return plugin; -} - - -/* see for a full description */ -struct st_mysql_client_plugin * STDCALL -mysql_load_plugin_v(MYSQL *mysql, const char *name, int type, - int argc, va_list args) -{ - const char *errmsg; -#ifdef _WIN32 - char errbuf[1024]; -#endif - char dlpath[FN_REFLEN+1]; - void *sym, *dlhandle = NULL; - struct st_mysql_client_plugin *plugin; - char *env_plugin_dir= getenv("MARIADB_PLUGIN_DIR"); - - CLEAR_CLIENT_ERROR(mysql); - if (is_not_initialized(mysql, name)) - return NULL; - - pthread_mutex_lock(&LOCK_load_client_plugin); - - /* make sure the plugin wasn't loaded meanwhile */ - if (type >= 0 && find_plugin(name, type)) - { - errmsg= "it is already loaded"; - goto err; - } - - /* Compile dll path */ - snprintf(dlpath, sizeof(dlpath) - 1, "%s/%s%s", - mysql->options.extension && mysql->options.extension->plugin_dir ? - mysql->options.extension->plugin_dir : (env_plugin_dir) ? env_plugin_dir : - MARIADB_PLUGINDIR, name, SO_EXT); - - /* Open new dll handle */ - if (!(dlhandle= dlopen((const char *)dlpath, RTLD_NOW))) - { -#ifdef _WIN32 - char winmsg[255]; - size_t len; - winmsg[0] = 0; - FormatMessage(FORMAT_MESSAGE_FROM_SYSTEM, - NULL, - GetLastError(), - MAKELANGID(LANG_NEUTRAL, SUBLANG_DEFAULT), - winmsg, 255, NULL); - len= strlen(winmsg); - while (len > 0 && (winmsg[len - 1] == '\n' || winmsg[len - 1] == '\r')) - len--; - if (len) - winmsg[len] = 0; - snprintf(errbuf, sizeof(errbuf), "%s Library path is '%s'", winmsg, dlpath); - errmsg= errbuf; -#else - errmsg= dlerror(); -#endif - goto err; - } - - - if (!(sym= dlsym(dlhandle, plugin_declarations_sym))) - { - errmsg= "not a plugin"; - (void)dlclose(dlhandle); - goto err; - } - - plugin= (struct st_mysql_client_plugin*)sym; - - if (type >=0 && type != plugin->type) - { - errmsg= "type mismatch"; - goto err; - } - - if (strcmp(name, plugin->name)) - { - errmsg= "name mismatch"; - goto err; - } - - if (type < 0 && find_plugin(name, plugin->type)) - { - errmsg= "it is already loaded"; - goto err; - } - - plugin= add_plugin(mysql, plugin, dlhandle, argc, args); - - pthread_mutex_unlock(&LOCK_load_client_plugin); - - return plugin; - -err: - if (dlhandle) - dlclose(dlhandle); - pthread_mutex_unlock(&LOCK_load_client_plugin); - my_set_error(mysql, CR_AUTH_PLUGIN_CANNOT_LOAD, SQLSTATE_UNKNOWN, - ER(CR_AUTH_PLUGIN_CANNOT_LOAD), name, errmsg); - return NULL; -} - - -/* see for a full description */ -struct st_mysql_client_plugin * STDCALL -mysql_load_plugin(MYSQL *mysql, const char *name, int type, int argc, ...) -{ - struct st_mysql_client_plugin *p; - va_list args; - va_start(args, argc); - p= mysql_load_plugin_v(mysql, name, type, argc, args); - va_end(args); - return p; -} - -/* see for a full description */ -struct st_mysql_client_plugin * STDCALL -mysql_client_find_plugin(MYSQL *mysql, const char *name, int type) -{ - struct st_mysql_client_plugin *p; - int plugin_nr= get_plugin_nr(type); - - if (is_not_initialized(mysql, name)) - return NULL; - - if (plugin_nr == -1) - { - my_set_error(mysql, CR_AUTH_PLUGIN_CANNOT_LOAD, SQLSTATE_UNKNOWN, - ER(CR_AUTH_PLUGIN_CANNOT_LOAD), name, "invalid type"); - } - - if ((p= find_plugin(name, type))) - return p; - - /* not found, load it */ - return mysql_load_plugin(mysql, name, type, 0); -} - diff --git a/contrib/murmurhash/CMakeLists.txt b/contrib/murmurhash/CMakeLists.txt deleted file mode 100644 index c5e467a2d6d..00000000000 --- a/contrib/murmurhash/CMakeLists.txt +++ /dev/null @@ -1,7 +0,0 @@ -add_library(murmurhash - src/murmurhash2.cpp - src/murmurhash3.cpp - include/murmurhash2.h - include/murmurhash3.h) - -target_include_directories (murmurhash PUBLIC include) diff --git a/contrib/murmurhash/LICENSE b/contrib/murmurhash/LICENSE deleted file mode 100644 index f6cdede60b8..00000000000 --- a/contrib/murmurhash/LICENSE +++ /dev/null @@ -1 +0,0 @@ -MurmurHash was written by Austin Appleby, and is placed in the publicdomain. The author hereby disclaims copyright to this source code. diff --git a/contrib/murmurhash/include/murmurhash3.h b/contrib/murmurhash/include/murmurhash3.h deleted file mode 100644 index e1c6d34976c..00000000000 --- a/contrib/murmurhash/include/murmurhash3.h +++ /dev/null @@ -1,37 +0,0 @@ -//----------------------------------------------------------------------------- -// MurmurHash3 was written by Austin Appleby, and is placed in the public -// domain. The author hereby disclaims copyright to this source code. - -#ifndef _MURMURHASH3_H_ -#define _MURMURHASH3_H_ - -//----------------------------------------------------------------------------- -// Platform-specific functions and macros - -// Microsoft Visual Studio - -#if defined(_MSC_VER) && (_MSC_VER < 1600) - -typedef unsigned char uint8_t; -typedef unsigned int uint32_t; -typedef unsigned __int64 uint64_t; - -// Other compilers - -#else // defined(_MSC_VER) - -#include - -#endif // !defined(_MSC_VER) - -//----------------------------------------------------------------------------- - -void MurmurHash3_x86_32 ( const void * key, int len, uint32_t seed, void * out ); - -void MurmurHash3_x86_128 ( const void * key, int len, uint32_t seed, void * out ); - -void MurmurHash3_x64_128 ( const void * key, int len, uint32_t seed, void * out ); - -//----------------------------------------------------------------------------- - -#endif // _MURMURHASH3_H_ diff --git a/contrib/murmurhash/src/murmurhash3.cpp b/contrib/murmurhash/src/murmurhash3.cpp deleted file mode 100644 index 2831bf5c73b..00000000000 --- a/contrib/murmurhash/src/murmurhash3.cpp +++ /dev/null @@ -1,331 +0,0 @@ -// MurmurHash3 was written by Austin Appleby, and is placed in the public -// domain. The author hereby disclaims copyright to this source code. - -// Note - The x86 and x64 versions do _not_ produce the same results, as the -// algorithms are optimized for their respective platforms. You can still -// compile and run any of them on any platform, but your performance with the -// non-native version will be less than optimal. - -#include "murmurhash3.h" - -//----------------------------------------------------------------------------- -// Platform-specific functions and macros - -// Microsoft Visual Studio - -#if defined(_MSC_VER) - -#define FORCE_INLINE __forceinline - -#include - -#define ROTL32(x,y) _rotl(x,y) -#define ROTL64(x,y) _rotl64(x,y) - -#define BIG_CONSTANT(x) (x) - -// Other compilers - -#else // defined(_MSC_VER) - -#define FORCE_INLINE inline __attribute__((always_inline)) - -inline uint32_t rotl32 ( uint32_t x, int8_t r ) -{ - return (x << r) | (x >> (32 - r)); -} - -inline uint64_t rotl64 ( uint64_t x, int8_t r ) -{ - return (x << r) | (x >> (64 - r)); -} - -#define ROTL32(x,y) rotl32(x,y) -#define ROTL64(x,y) rotl64(x,y) - -#define BIG_CONSTANT(x) (x##LLU) - -#endif // !defined(_MSC_VER) - -//----------------------------------------------------------------------------- -// Block read - if your platform needs to do endian-swapping or can only -// handle aligned reads, do the conversion here - -FORCE_INLINE uint32_t getblock32 ( const uint32_t * p, int i ) -{ - return p[i]; -} - -FORCE_INLINE uint64_t getblock64 ( const uint64_t * p, int i ) -{ - return p[i]; -} - -//----------------------------------------------------------------------------- -// Finalization mix - force all bits of a hash block to avalanche - -FORCE_INLINE uint32_t fmix32 ( uint32_t h ) -{ - h ^= h >> 16; - h *= 0x85ebca6b; - h ^= h >> 13; - h *= 0xc2b2ae35; - h ^= h >> 16; - - return h; -} - -//---------- - -FORCE_INLINE uint64_t fmix64 ( uint64_t k ) -{ - k ^= k >> 33; - k *= BIG_CONSTANT(0xff51afd7ed558ccd); - k ^= k >> 33; - k *= BIG_CONSTANT(0xc4ceb9fe1a85ec53); - k ^= k >> 33; - - return k; -} - -//----------------------------------------------------------------------------- - -void MurmurHash3_x86_32 ( const void * key, int len, - uint32_t seed, void * out ) -{ - const uint8_t * data = (const uint8_t*)key; - const int nblocks = len / 4; - - uint32_t h1 = seed; - - const uint32_t c1 = 0xcc9e2d51; - const uint32_t c2 = 0x1b873593; - - //---------- - // body - - const uint32_t * blocks = (const uint32_t *)(data + nblocks*4); - - for(int i = -nblocks; i; i++) - { - uint32_t k1 = getblock32(blocks,i); - - k1 *= c1; - k1 = ROTL32(k1,15); - k1 *= c2; - - h1 ^= k1; - h1 = ROTL32(h1,13); - h1 = h1*5+0xe6546b64; - } - - //---------- - // tail - - const uint8_t * tail = (const uint8_t*)(data + nblocks*4); - - uint32_t k1 = 0; - - switch(len & 3) - { - case 3: k1 ^= tail[2] << 16; - case 2: k1 ^= tail[1] << 8; - case 1: k1 ^= tail[0]; - k1 *= c1; k1 = ROTL32(k1,15); k1 *= c2; h1 ^= k1; - }; - - //---------- - // finalization - - h1 ^= len; - - h1 = fmix32(h1); - - *(uint32_t*)out = h1; -} - -//----------------------------------------------------------------------------- - -void MurmurHash3_x86_128 ( const void * key, const int len, - uint32_t seed, void * out ) -{ - const uint8_t * data = (const uint8_t*)key; - const int nblocks = len / 16; - - uint32_t h1 = seed; - uint32_t h2 = seed; - uint32_t h3 = seed; - uint32_t h4 = seed; - - const uint32_t c1 = 0x239b961b; - const uint32_t c2 = 0xab0e9789; - const uint32_t c3 = 0x38b34ae5; - const uint32_t c4 = 0xa1e38b93; - - //---------- - // body - - const uint32_t * blocks = (const uint32_t *)(data + nblocks*16); - - for(int i = -nblocks; i; i++) - { - uint32_t k1 = getblock32(blocks,i*4+0); - uint32_t k2 = getblock32(blocks,i*4+1); - uint32_t k3 = getblock32(blocks,i*4+2); - uint32_t k4 = getblock32(blocks,i*4+3); - - k1 *= c1; k1 = ROTL32(k1,15); k1 *= c2; h1 ^= k1; - - h1 = ROTL32(h1,19); h1 += h2; h1 = h1*5+0x561ccd1b; - - k2 *= c2; k2 = ROTL32(k2,16); k2 *= c3; h2 ^= k2; - - h2 = ROTL32(h2,17); h2 += h3; h2 = h2*5+0x0bcaa747; - - k3 *= c3; k3 = ROTL32(k3,17); k3 *= c4; h3 ^= k3; - - h3 = ROTL32(h3,15); h3 += h4; h3 = h3*5+0x96cd1c35; - - k4 *= c4; k4 = ROTL32(k4,18); k4 *= c1; h4 ^= k4; - - h4 = ROTL32(h4,13); h4 += h1; h4 = h4*5+0x32ac3b17; - } - - //---------- - // tail - - const uint8_t * tail = (const uint8_t*)(data + nblocks*16); - - uint32_t k1 = 0; - uint32_t k2 = 0; - uint32_t k3 = 0; - uint32_t k4 = 0; - - switch(len & 15) - { - case 15: k4 ^= tail[14] << 16; - case 14: k4 ^= tail[13] << 8; - case 13: k4 ^= tail[12] << 0; - k4 *= c4; k4 = ROTL32(k4,18); k4 *= c1; h4 ^= k4; - - case 12: k3 ^= tail[11] << 24; - case 11: k3 ^= tail[10] << 16; - case 10: k3 ^= tail[ 9] << 8; - case 9: k3 ^= tail[ 8] << 0; - k3 *= c3; k3 = ROTL32(k3,17); k3 *= c4; h3 ^= k3; - - case 8: k2 ^= tail[ 7] << 24; - case 7: k2 ^= tail[ 6] << 16; - case 6: k2 ^= tail[ 5] << 8; - case 5: k2 ^= tail[ 4] << 0; - k2 *= c2; k2 = ROTL32(k2,16); k2 *= c3; h2 ^= k2; - - case 4: k1 ^= tail[ 3] << 24; - case 3: k1 ^= tail[ 2] << 16; - case 2: k1 ^= tail[ 1] << 8; - case 1: k1 ^= tail[ 0] << 0; - k1 *= c1; k1 = ROTL32(k1,15); k1 *= c2; h1 ^= k1; - }; - - //---------- - // finalization - - h1 ^= len; h2 ^= len; h3 ^= len; h4 ^= len; - - h1 += h2; h1 += h3; h1 += h4; - h2 += h1; h3 += h1; h4 += h1; - - h1 = fmix32(h1); - h2 = fmix32(h2); - h3 = fmix32(h3); - h4 = fmix32(h4); - - h1 += h2; h1 += h3; h1 += h4; - h2 += h1; h3 += h1; h4 += h1; - - ((uint32_t*)out)[0] = h1; - ((uint32_t*)out)[1] = h2; - ((uint32_t*)out)[2] = h3; - ((uint32_t*)out)[3] = h4; -} - -//----------------------------------------------------------------------------- - -void MurmurHash3_x64_128 ( const void * key, const int len, - const uint32_t seed, void * out ) -{ - const uint8_t * data = (const uint8_t*)key; - const int nblocks = len / 16; - - uint64_t h1 = seed; - uint64_t h2 = seed; - - const uint64_t c1 = BIG_CONSTANT(0x87c37b91114253d5); - const uint64_t c2 = BIG_CONSTANT(0x4cf5ad432745937f); - - //---------- - // body - - const uint64_t * blocks = (const uint64_t *)(data); - - for(int i = 0; i < nblocks; i++) - { - uint64_t k1 = getblock64(blocks,i*2+0); - uint64_t k2 = getblock64(blocks,i*2+1); - - k1 *= c1; k1 = ROTL64(k1,31); k1 *= c2; h1 ^= k1; - - h1 = ROTL64(h1,27); h1 += h2; h1 = h1*5+0x52dce729; - - k2 *= c2; k2 = ROTL64(k2,33); k2 *= c1; h2 ^= k2; - - h2 = ROTL64(h2,31); h2 += h1; h2 = h2*5+0x38495ab5; - } - - //---------- - // tail - - const uint8_t * tail = (const uint8_t*)(data + nblocks*16); - - uint64_t k1 = 0; - uint64_t k2 = 0; - - switch(len & 15) - { - case 15: k2 ^= ((uint64_t)tail[14]) << 48; - case 14: k2 ^= ((uint64_t)tail[13]) << 40; - case 13: k2 ^= ((uint64_t)tail[12]) << 32; - case 12: k2 ^= ((uint64_t)tail[11]) << 24; - case 11: k2 ^= ((uint64_t)tail[10]) << 16; - case 10: k2 ^= ((uint64_t)tail[ 9]) << 8; - case 9: k2 ^= ((uint64_t)tail[ 8]) << 0; - k2 *= c2; k2 = ROTL64(k2,33); k2 *= c1; h2 ^= k2; - - case 8: k1 ^= ((uint64_t)tail[ 7]) << 56; - case 7: k1 ^= ((uint64_t)tail[ 6]) << 48; - case 6: k1 ^= ((uint64_t)tail[ 5]) << 40; - case 5: k1 ^= ((uint64_t)tail[ 4]) << 32; - case 4: k1 ^= ((uint64_t)tail[ 3]) << 24; - case 3: k1 ^= ((uint64_t)tail[ 2]) << 16; - case 2: k1 ^= ((uint64_t)tail[ 1]) << 8; - case 1: k1 ^= ((uint64_t)tail[ 0]) << 0; - k1 *= c1; k1 = ROTL64(k1,31); k1 *= c2; h1 ^= k1; - }; - - //---------- - // finalization - - h1 ^= len; h2 ^= len; - - h1 += h2; - h2 += h1; - - h1 = fmix64(h1); - h2 = fmix64(h2); - - h1 += h2; - h2 += h1; - - ((uint64_t*)out)[0] = h1; - ((uint64_t*)out)[1] = h2; -} diff --git a/contrib/re2_st/CMakeLists.txt b/contrib/re2_st/CMakeLists.txt index 6bc7fd8f343..79362f4bb56 100644 --- a/contrib/re2_st/CMakeLists.txt +++ b/contrib/re2_st/CMakeLists.txt @@ -12,8 +12,7 @@ endforeach () add_library (re2_st ${RE2_ST_SOURCES}) target_compile_definitions (re2_st PRIVATE NDEBUG NO_THREADS re2=re2_st) -target_include_directories (re2_st PRIVATE .) -target_include_directories (re2_st SYSTEM PUBLIC ${CMAKE_CURRENT_BINARY_DIR} ${RE2_SOURCE_DIR}) +target_include_directories (re2_st PRIVATE . PUBLIC ${CMAKE_CURRENT_BINARY_DIR} ${RE2_SOURCE_DIR}) file (MAKE_DIRECTORY ${CMAKE_CURRENT_BINARY_DIR}/re2_st) foreach (FILENAME filtered_re2.h re2.h set.h stringpiece.h) diff --git a/contrib/unixodbc b/contrib/unixodbc deleted file mode 160000 index b0ad30f7f62..00000000000 --- a/contrib/unixodbc +++ /dev/null @@ -1 +0,0 @@ -Subproject commit b0ad30f7f6289c12b76f04bfb9d466374bb32168 diff --git a/contrib/unixodbc-cmake/linux_x86_64/libltdl/config.h b/contrib/unixodbc-cmake/linux_x86_64/libltdl/config.h deleted file mode 100644 index 194779b2b98..00000000000 --- a/contrib/unixodbc-cmake/linux_x86_64/libltdl/config.h +++ /dev/null @@ -1,181 +0,0 @@ -/* config.h. Generated from config-h.in by configure. */ -/* config-h.in. Generated from configure.ac by autoheader. */ - -/* Define to 1 if you have the `argz_add' function. */ -#define HAVE_ARGZ_ADD 1 - -/* Define to 1 if you have the `argz_append' function. */ -#define HAVE_ARGZ_APPEND 1 - -/* Define to 1 if you have the `argz_count' function. */ -#define HAVE_ARGZ_COUNT 1 - -/* Define to 1 if you have the `argz_create_sep' function. */ -#define HAVE_ARGZ_CREATE_SEP 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_ARGZ_H 1 - -/* Define to 1 if you have the `argz_insert' function. */ -#define HAVE_ARGZ_INSERT 1 - -/* Define to 1 if you have the `argz_next' function. */ -#define HAVE_ARGZ_NEXT 1 - -/* Define to 1 if you have the `argz_stringify' function. */ -#define HAVE_ARGZ_STRINGIFY 1 - -/* Define to 1 if you have the `closedir' function. */ -#define HAVE_CLOSEDIR 1 - -/* Define to 1 if you have the declaration of `cygwin_conv_path', and to 0 if - you don't. */ -/* #undef HAVE_DECL_CYGWIN_CONV_PATH */ - -/* Define to 1 if you have the header file. */ -#define HAVE_DIRENT_H 1 - -/* Define if you have the GNU dld library. */ -/* #undef HAVE_DLD */ - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_DLD_H */ - -/* Define to 1 if you have the `dlerror' function. */ -#define HAVE_DLERROR 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_DLFCN_H 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_DL_H */ - -/* Define if you have the _dyld_func_lookup function. */ -/* #undef HAVE_DYLD */ - -/* Define to 1 if the system has the type `error_t'. */ -#define HAVE_ERROR_T 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_INTTYPES_H 1 - -/* Define if you have the libdl library or equivalent. */ -#define HAVE_LIBDL 1 - -/* Define if libdlloader will be built on this platform */ -#define HAVE_LIBDLLOADER 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_MACH_O_DYLD_H */ - -/* Define to 1 if you have the header file. */ -#define HAVE_MEMORY_H 1 - -/* Define to 1 if you have the `opendir' function. */ -#define HAVE_OPENDIR 1 - -/* Define if libtool can extract symbol lists from object files. */ -#define HAVE_PRELOADED_SYMBOLS 1 - -/* Define to 1 if you have the `readdir' function. */ -#define HAVE_READDIR 1 - -/* Define if you have the shl_load function. */ -/* #undef HAVE_SHL_LOAD */ - -/* Define to 1 if you have the header file. */ -#define HAVE_STDINT_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STDLIB_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STRINGS_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STRING_H 1 - -/* Define to 1 if you have the `strlcat' function. */ -/* #undef HAVE_STRLCAT */ - -/* Define to 1 if you have the `strlcpy' function. */ -/* #undef HAVE_STRLCPY */ - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_SYS_DL_H */ - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_STAT_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_TYPES_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_UNISTD_H 1 - -/* This value is set to 1 to indicate that the system argz facility works */ -#define HAVE_WORKING_ARGZ 1 - -/* Define if the OS needs help to load dependent libraries for dlopen(). */ -/* #undef LTDL_DLOPEN_DEPLIBS */ - -/* Define to the system default library search path. */ -#define LT_DLSEARCH_PATH "/lib:/usr/lib:/usr/lib/x86_64-linux-gnu/libfakeroot:/usr/local/lib:/usr/local/lib/x86_64-linux-gnu:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/lib32:/usr/lib32" - -/* The archive extension */ -#define LT_LIBEXT "a" - -/* The archive prefix */ -#define LT_LIBPREFIX "lib" - -/* Define to the extension used for runtime loadable modules, say, ".so". */ -#define LT_MODULE_EXT ".so" - -/* Define to the name of the environment variable that determines the run-time - module search path. */ -#define LT_MODULE_PATH_VAR "LD_LIBRARY_PATH" - -/* Define to the sub-directory where libtool stores uninstalled libraries. */ -#define LT_OBJDIR ".libs/" - -/* Define to the shared library suffix, say, ".dylib". */ -/* #undef LT_SHARED_EXT */ - -/* Define to the shared archive member specification, say "(shr.o)". */ -/* #undef LT_SHARED_LIB_MEMBER */ - -/* Define if dlsym() requires a leading underscore in symbol names. */ -/* #undef NEED_USCORE */ - -/* Name of package */ -#define PACKAGE "libltdl" - -/* Define to the address where bug reports for this package should be sent. */ -#define PACKAGE_BUGREPORT "bug-libtool@gnu.org" - -/* Define to the full name of this package. */ -#define PACKAGE_NAME "libltdl" - -/* Define to the full name and version of this package. */ -#define PACKAGE_STRING "libltdl 2.4.3a" - -/* Define to the one symbol short name of this package. */ -#define PACKAGE_TARNAME "libltdl" - -/* Define to the home page for this package. */ -#define PACKAGE_URL "" - -/* Define to the version of this package. */ -#define PACKAGE_VERSION "2.4.3a" - -/* Define to 1 if you have the ANSI C header files. */ -#define STDC_HEADERS 1 - -/* Version number of package */ -#define VERSION "2.4.3a" - -/* Define so that glibc/gnulib argp.h does not typedef error_t. */ -/* #undef __error_t_defined */ - -/* Define to a type to use for 'error_t' if it is not otherwise available. */ -/* #undef error_t */ diff --git a/contrib/unixodbc-cmake/linux_x86_64/libltdl/libltdlcS.c b/contrib/unixodbc-cmake/linux_x86_64/libltdl/libltdlcS.c deleted file mode 100644 index ca866eb5986..00000000000 --- a/contrib/unixodbc-cmake/linux_x86_64/libltdl/libltdlcS.c +++ /dev/null @@ -1,53 +0,0 @@ -/* libltdlcS.c - symbol resolution table for 'libltdlc' dlsym emulation. */ -/* Generated by libtool (GNU libtool) 2.4.6 */ - -#ifdef __cplusplus -extern "C" { -#endif - -#if defined __GNUC__ && (((__GNUC__ == 4) && (__GNUC_MINOR__ >= 4)) || (__GNUC__ > 4)) -#pragma GCC diagnostic ignored "-Wstrict-prototypes" -#endif - -/* Keep this code in sync between libtool.m4, ltmain, lt_system.h, and tests. */ -#if defined _WIN32 || defined __CYGWIN__ || defined _WIN32_WCE -/* DATA imports from DLLs on WIN32 can't be const, because runtime - relocations are performed -- see ld's documentation on pseudo-relocs. */ -# define LT_DLSYM_CONST -#elif defined __osf__ -/* This system does not cope well with relocations in const data. */ -# define LT_DLSYM_CONST -#else -# define LT_DLSYM_CONST const -#endif - -#define STREQ(s1, s2) (strcmp ((s1), (s2)) == 0) - -/* External symbol declarations for the compiler. */ -extern int dlopen_LTX_get_vtable(); - -/* The mapping between symbol names and symbols. */ -typedef struct { - const char *name; - void *address; -} lt_dlsymlist; -extern LT_DLSYM_CONST lt_dlsymlist -lt_libltdlc_LTX_preloaded_symbols[]; -LT_DLSYM_CONST lt_dlsymlist -lt_libltdlc_LTX_preloaded_symbols[] = -{ {"libltdlc", (void *) 0}, - {"dlopen.a", (void *) 0}, - {"dlopen_LTX_get_vtable", (void *) &dlopen_LTX_get_vtable}, - {0, (void *) 0} -}; - -/* This works around a problem in FreeBSD linker */ -#ifdef FREEBSD_WORKAROUND -static const void *lt_preloaded_setup() { - return lt_libltdlc_LTX_preloaded_symbols; -} -#endif - -#ifdef __cplusplus -} -#endif diff --git a/contrib/unixodbc-cmake/linux_x86_64/private/config.h b/contrib/unixodbc-cmake/linux_x86_64/private/config.h deleted file mode 100644 index d80a4da4665..00000000000 --- a/contrib/unixodbc-cmake/linux_x86_64/private/config.h +++ /dev/null @@ -1,496 +0,0 @@ -/* config.h. Generated from config.h.in by configure. */ -/* config.h.in. Generated from configure.ac by autoheader. */ - -/* Encoding to use for CHAR */ -#define ASCII_ENCODING "auto-search" - -/* Install bindir */ -#define BIN_PREFIX "/usr/local/bin" - -/* Use a semaphore to allow ODBCConfig to display running counts */ -/* #undef COLLECT_STATS */ - -/* Define to one of `_getb67', `GETB67', `getb67' for Cray-2 and Cray-YMP - systems. This function is required for `alloca.c' support on those systems. - */ -/* #undef CRAY_STACKSEG_END */ - -/* Define to 1 if using `alloca.c'. */ -/* #undef C_ALLOCA */ - -/* Lib directory */ -#define DEFLIB_PATH "/usr/local/lib" - -/* Using ini cacheing */ -#define ENABLE_INI_CACHING /**/ - -/* Install exec_prefix */ -#define EXEC_PREFIX "/usr/local" - -/* Disable the precise but slow checking of the validity of handles */ -/* #undef FAST_HANDLE_VALIDATE */ - -/* Define to 1 if you have `alloca', as a function or macro. */ -#define HAVE_ALLOCA 1 - -/* Define to 1 if you have and it should be used (not on Ultrix). - */ -#define HAVE_ALLOCA_H 1 - -/* Define to 1 if you have the `argz_add' function. */ -#define HAVE_ARGZ_ADD 1 - -/* Define to 1 if you have the `argz_append' function. */ -#define HAVE_ARGZ_APPEND 1 - -/* Define to 1 if you have the `argz_count' function. */ -#define HAVE_ARGZ_COUNT 1 - -/* Define to 1 if you have the `argz_create_sep' function. */ -#define HAVE_ARGZ_CREATE_SEP 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_ARGZ_H 1 - -/* Define to 1 if you have the `argz_insert' function. */ -#define HAVE_ARGZ_INSERT 1 - -/* Define to 1 if you have the `argz_next' function. */ -#define HAVE_ARGZ_NEXT 1 - -/* Define to 1 if you have the `argz_stringify' function. */ -#define HAVE_ARGZ_STRINGIFY 1 - -/* Define to 1 if you have the `atoll' function. */ -#define HAVE_ATOLL 1 - -/* Define to 1 if you have the `closedir' function. */ -#define HAVE_CLOSEDIR 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_CRYPT_H 1 - -/* Define to 1 if you have the declaration of `cygwin_conv_path', and to 0 if - you don't. */ -/* #undef HAVE_DECL_CYGWIN_CONV_PATH */ - -/* Define to 1 if you have the header file, and it defines `DIR'. - */ -#define HAVE_DIRENT_H 1 - -/* Define if you have the GNU dld library. */ -/* #undef HAVE_DLD */ - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_DLD_H */ - -/* Define to 1 if you have the `dlerror' function. */ -#define HAVE_DLERROR 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_DLFCN_H 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_DL_H */ - -/* Define to 1 if you don't have `vprintf' but do have `_doprnt.' */ -/* #undef HAVE_DOPRNT */ - -/* Define if you have the _dyld_func_lookup function. */ -/* #undef HAVE_DYLD */ - -/* Define to 1 if you have the `endpwent' function. */ -#define HAVE_ENDPWENT 1 - -/* Define to 1 if the system has the type `error_t'. */ -#define HAVE_ERROR_T 1 - -/* Define to 1 if you have the `ftime' function. */ -#define HAVE_FTIME 1 - -/* Define to 1 if you have the `ftok' function. */ -/* #undef HAVE_FTOK */ - -/* Define to 1 if you have the `getpwuid' function. */ -#define HAVE_GETPWUID 1 - -/* Define to 1 if you have the `gettimeofday' function. */ -#define HAVE_GETTIMEOFDAY 1 - -/* Define to 1 if you have the `getuid' function. */ -#define HAVE_GETUID 1 - -/* Define if you have the iconv() function. */ -#define HAVE_ICONV 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_INTTYPES_H 1 - -/* Define if you have and nl_langinfo(CODESET). */ -#define HAVE_LANGINFO_CODESET 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_LANGINFO_H 1 - -/* Add -lcrypt to lib list */ -#define HAVE_LIBCRYPT /**/ - -/* Define if you have the libdl library or equivalent. */ -#define HAVE_LIBDL 1 - -/* Define if libdlloader will be built on this platform */ -#define HAVE_LIBDLLOADER 1 - -/* Use the -lpth thread library */ -/* #undef HAVE_LIBPTH */ - -/* Use -lpthread threading lib */ -#define HAVE_LIBPTHREAD 1 - -/* Use the -lthread threading lib */ -/* #undef HAVE_LIBTHREAD */ - -/* Define to 1 if you have the header file. */ -#define HAVE_LIMITS_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_LOCALE_H 1 - -/* Use rentrant version of localtime */ -#define HAVE_LOCALTIME_R 1 - -/* Define if you have long long */ -#define HAVE_LONG_LONG 1 - -/* Define this if a modern libltdl is already installed */ -#define HAVE_LTDL 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_MACH_O_DYLD_H */ - -/* Define to 1 if you have the header file. */ -#define HAVE_MALLOC_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_MEMORY_H 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_MSQL_H */ - -/* Define to 1 if you have the header file, and it defines `DIR'. */ -/* #undef HAVE_NDIR_H */ - -/* Define to 1 if you have the `nl_langinfo' function. */ -#define HAVE_NL_LANGINFO 1 - -/* Define to 1 if you have the `opendir' function. */ -#define HAVE_OPENDIR 1 - -/* Define if libtool can extract symbol lists from object files. */ -#define HAVE_PRELOADED_SYMBOLS 1 - -/* Define to 1 if the system has the type `ptrdiff_t'. */ -#define HAVE_PTRDIFF_T 1 - -/* Define to 1 if you have the `putenv' function. */ -#define HAVE_PUTENV 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_PWD_H 1 - -/* Define to 1 if you have the `readdir' function. */ -#define HAVE_READDIR 1 - -/* Add readline support */ -#define HAVE_READLINE 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_READLINE_HISTORY_H 1 - -/* Use the scandir lib */ -/* #undef HAVE_SCANDIR */ - -/* Define to 1 if you have the `semget' function. */ -/* #undef HAVE_SEMGET */ - -/* Define to 1 if you have the `semop' function. */ -/* #undef HAVE_SEMOP */ - -/* Define to 1 if you have the `setenv' function. */ -#define HAVE_SETENV 1 - -/* Define to 1 if you have the `setlocale' function. */ -#define HAVE_SETLOCALE 1 - -/* Define if you have the shl_load function. */ -/* #undef HAVE_SHL_LOAD */ - -/* Define to 1 if you have the `shmget' function. */ -/* #undef HAVE_SHMGET */ - -/* Define to 1 if you have the `snprintf' function. */ -/* #undef HAVE_SNPRINTF */ - -/* Define to 1 if you have the `socket' function. */ -#define HAVE_SOCKET 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STDARG_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STDDEF_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STDINT_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STDLIB_H 1 - -/* Define to 1 if you have the `strcasecmp' function. */ -#define HAVE_STRCASECMP 1 - -/* Define to 1 if you have the `strchr' function. */ -#define HAVE_STRCHR 1 - -/* Define to 1 if you have the `strdup' function. */ -#define HAVE_STRDUP 1 - -/* Define to 1 if you have the `stricmp' function. */ -/* #undef HAVE_STRICMP */ - -/* Define to 1 if you have the header file. */ -#define HAVE_STRINGS_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_STRING_H 1 - -/* Define to 1 if you have the `strlcat' function. */ -/* #undef HAVE_STRLCAT */ - -/* Define to 1 if you have the `strlcpy' function. */ -/* #undef HAVE_STRLCPY */ - -/* Define to 1 if you have the `strncasecmp' function. */ -#define HAVE_STRNCASECMP 1 - -/* Define to 1 if you have the `strnicmp' function. */ -/* #undef HAVE_STRNICMP */ - -/* Define to 1 if you have the `strstr' function. */ -#define HAVE_STRSTR 1 - -/* Define to 1 if you have the `strtol' function. */ -#define HAVE_STRTOL 1 - -/* Define to 1 if you have the `strtoll' function. */ -#define HAVE_STRTOLL 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_SYNCH_H */ - -/* Define to 1 if you have the header file, and it defines `DIR'. - */ -/* #undef HAVE_SYS_DIR_H */ - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_SYS_DL_H */ - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_SYS_MALLOC_H */ - -/* Define to 1 if you have the header file, and it defines `DIR'. - */ -/* #undef HAVE_SYS_NDIR_H */ - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_SEM_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_STAT_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_TIMEB_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_TIME_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_SYS_TYPES_H 1 - -/* Define to 1 if you have the `time' function. */ -#define HAVE_TIME 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_TIME_H 1 - -/* Define to 1 if you have the header file. */ -#define HAVE_UNISTD_H 1 - -/* Define to 1 if you have the header file. */ -/* #undef HAVE_VARARGS_H */ - -/* Define to 1 if you have the `vprintf' function. */ -#define HAVE_VPRINTF 1 - -/* Define to 1 if you have the `vsnprintf' function. */ -#define HAVE_VSNPRINTF 1 - -/* This value is set to 1 to indicate that the system argz facility works */ -#define HAVE_WORKING_ARGZ 1 - -/* Define as const if the declaration of iconv() needs const. */ -#define ICONV_CONST - -/* Install includedir */ -#define INCLUDE_PREFIX "/usr/local/include" - -/* Lib directory */ -#define LIB_PREFIX "/usr/local/lib" - -/* Define if the OS needs help to load dependent libraries for dlopen(). */ -/* #undef LTDL_DLOPEN_DEPLIBS */ - -/* Define to the system default library search path. */ -#define LT_DLSEARCH_PATH "/lib:/usr/lib:/usr/lib/x86_64-linux-gnu/libfakeroot:/usr/local/lib:/usr/local/lib/x86_64-linux-gnu:/lib/x86_64-linux-gnu:/usr/lib/x86_64-linux-gnu:/lib32:/usr/lib32" - -/* The archive extension */ -#define LT_LIBEXT "a" - -/* The archive prefix */ -#define LT_LIBPREFIX "lib" - -/* Define to the extension used for runtime loadable modules, say, ".so". */ -#define LT_MODULE_EXT ".so" - -/* Define to the name of the environment variable that determines the run-time - module search path. */ -#define LT_MODULE_PATH_VAR "LD_LIBRARY_PATH" - -/* Define to the sub-directory where libtool stores uninstalled libraries. */ -#define LT_OBJDIR ".libs/" - -/* Define to the shared library suffix, say, ".dylib". */ -/* #undef LT_SHARED_EXT */ - -/* Define to the shared archive member specification, say "(shr.o)". */ -/* #undef LT_SHARED_LIB_MEMBER */ - -/* Define if you need semundo union */ -/* #undef NEED_SEMUNDO_UNION */ - -/* Define if dlsym() requires a leading underscore in symbol names. */ -/* #undef NEED_USCORE */ - -/* Using OSX */ -/* #undef OSXHEADER */ - -/* Name of package */ -#define PACKAGE "unixODBC" - -/* Define to the address where bug reports for this package should be sent. */ -#define PACKAGE_BUGREPORT "nick@unixodbc.org" - -/* Define to the full name of this package. */ -#define PACKAGE_NAME "unixODBC" - -/* Define to the full name and version of this package. */ -#define PACKAGE_STRING "unixODBC 2.3.6" - -/* Define to the one symbol short name of this package. */ -#define PACKAGE_TARNAME "unixODBC" - -/* Define to the home page for this package. */ -#define PACKAGE_URL "" - -/* Define to the version of this package. */ -#define PACKAGE_VERSION "2.3.6" - -/* Platform is 64 bit */ -#define PLATFORM64 /**/ - -/* Install prefix */ -#define PREFIX "/usr/local" - -/* Using QNX */ -/* #undef QNX_LIBLTDL */ - -/* Shared lib extension */ -#define SHLIBEXT ".so" - -/* The size of `long', as computed by sizeof. */ -#define SIZEOF_LONG 8 - -/* The size of `long int', as computed by sizeof. */ -#define SIZEOF_LONG_INT 8 - -/* If using the C implementation of alloca, define if you know the - direction of stack growth for your system; otherwise it will be - automatically deduced at runtime. - STACK_DIRECTION > 0 => grows toward higher addresses - STACK_DIRECTION < 0 => grows toward lower addresses - STACK_DIRECTION = 0 => direction of growth unknown */ -/* #undef STACK_DIRECTION */ - -/* Define to 1 if you have the ANSI C header files. */ -#define STDC_HEADERS 1 - -/* don't include unixODBC prefix in driver error messages */ -#define STRICT_ODBC_ERROR /**/ - -/* System file path */ -#define SYSTEM_FILE_PATH "/etc" - -/* Lib path */ -#define SYSTEM_LIB_PATH "/usr/local/lib" - -/* Define to 1 if you can safely include both and . */ -#define TIME_WITH_SYS_TIME 1 - -/* Define to 1 if your declares `struct tm'. */ -/* #undef TM_IN_SYS_TIME */ - -/* Encoding to use for UNICODE */ -#define UNICODE_ENCODING "auto-search" - -/* Flag that we are not using another DM */ -#define UNIXODBC /**/ - -/* We are building inside the unixODBC source tree */ -#define UNIXODBC_SOURCE /**/ - -/* Version number of package */ -#define VERSION "2.3.6" - -/* Work with IBM drivers that use 32 bit handles on 64 bit platforms */ -/* #undef WITH_HANDLE_REDIRECT */ - -/* Define to 1 if `lex' declares `yytext' as a `char *' by default, not a - `char[]'. */ -/* #undef YYTEXT_POINTER */ - -/* Build flag for AIX */ -/* #undef _ALL_SOURCE */ - -/* Build flag for AIX */ -/* #undef _LONG_LONG */ - -/* Build flag for AIX */ -/* #undef _THREAD_SAFE */ - -/* Define so that glibc/gnulib argp.h does not typedef error_t. */ -/* #undef __error_t_defined */ - -/* Define to empty if `const' does not conform to ANSI C. */ -/* #undef const */ - -/* Define to a type to use for 'error_t' if it is not otherwise available. */ -/* #undef error_t */ - -/* Define to `int' if doesn't define. */ -/* #undef gid_t */ - -/* Define to `unsigned int' if does not define. */ -/* #undef size_t */ - -/* Define to `int' if doesn't define. */ -/* #undef uid_t */ diff --git a/contrib/unixodbc-cmake/linux_x86_64/unixodbc_conf.h b/contrib/unixodbc-cmake/linux_x86_64/unixodbc_conf.h deleted file mode 100644 index 6597c85cea6..00000000000 --- a/contrib/unixodbc-cmake/linux_x86_64/unixodbc_conf.h +++ /dev/null @@ -1,60 +0,0 @@ -/* unixodbc_conf.h. Generated from unixodbc_conf.h.in by configure. */ -#ifndef HAVE_UNISTD_H -#define HAVE_UNISTD_H 1 -#endif - -#ifndef HAVE_PWD_H -#define HAVE_PWD_H 1 -#endif - -#ifndef HAVE_SYS_TIME_H -#define HAVE_SYS_TIME_H 1 -#endif - -#ifndef ODBC_STD -/* #undef ODBC_STD */ -#endif - -#ifndef UNICODE -/* #undef UNICODE */ -#endif - -#ifndef GUID_DEFINED -/* #undef GUID_DEFINED */ -#endif - -#ifndef SQL_WCHART_CONVERT -/* #undef SQL_WCHART_CONVERT */ -#endif - -#ifndef HAVE_LONG_LONG -#define HAVE_LONG_LONG 1 -#endif - -#ifndef ODBCINT64_TYPEA -/* #undef ODBCINT64_TYPEA */ -#endif - -#ifndef UODBCINT64_TYPE -/* #undef UODBCINT64_TYPE */ -#endif - -#ifndef DISABLE_INI_CACHING -/* #undef DISABLE_INI_CACHING */ -#endif - -#ifndef SIZEOF_LONG_INT -#define SIZEOF_LONG_INT 8 -#endif - -#ifndef ALLREADY_HAVE_WINDOWS_TYPE -/* #undef ALLREADY_HAVE_WINDOWS_TYPE */ -#endif - -#ifndef DONT_TD_VOID -/* #undef DONT_TD_VOID */ -#endif - -#ifndef DO_YOU_KNOW_WHAT_YOUR_ARE_DOING -/* #undef DO_YOU_KNOW_WHAT_YOUR_ARE_DOING */ -#endif diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 573b0e4ea0f..ba014767469 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -53,7 +53,6 @@ add_headers_and_sources(dbms src/Interpreters/ClusterProxy) add_headers_and_sources(dbms src/Columns) add_headers_and_sources(dbms src/Storages) add_headers_and_sources(dbms src/Storages/Distributed) -add_headers_and_sources(dbms src/Storages/Kafka) add_headers_and_sources(dbms src/Storages/MergeTree) add_headers_and_sources(dbms src/Client) add_headers_and_sources(dbms src/Formats) @@ -85,7 +84,7 @@ list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctions add_library(clickhouse_common_io ${SPLIT_SHARED} ${clickhouse_common_io_headers} ${clickhouse_common_io_sources}) -if (OS_FREEBSD) +if (ARCH_FREEBSD) target_compile_definitions (clickhouse_common_io PUBLIC CLOCK_MONOTONIC_COARSE=CLOCK_MONOTONIC_FAST) endif () @@ -156,8 +155,8 @@ target_link_libraries (dbms ${MYSQLXX_LIBRARY} ${RE2_LIBRARY} ${RE2_ST_LIBRARY} + ${OPENSSL_CRYPTO_LIBRARY} ${BTRIE_LIBRARIES} - ${Boost_PROGRAM_OPTIONS_LIBRARY} ) if (NOT USE_INTERNAL_RE2_LIBRARY) @@ -219,8 +218,6 @@ if (USE_RDKAFKA) endif () endif () -target_link_libraries(dbms ${OPENSSL_CRYPTO_LIBRARY}) - target_link_libraries (dbms Threads::Threads ) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 38c7dbc1022..0a20652d038 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -1,11 +1,11 @@ # This strings autochanged from release_lib.sh: -set(VERSION_REVISION 54406 CACHE STRING "") +set(VERSION_REVISION 54400 CACHE STRING "") set(VERSION_MAJOR 18 CACHE STRING "") -set(VERSION_MINOR 11 CACHE STRING "") -set(VERSION_PATCH 0 CACHE STRING "") -set(VERSION_GITHASH 76af46ed5d223b3a7af92e31eae291174da16355 CACHE STRING "") -set(VERSION_DESCRIBE v18.11.0-testing CACHE STRING "") -set(VERSION_STRING 18.11.0 CACHE STRING "") +set(VERSION_MINOR 5 CACHE STRING "") +set(VERSION_PATCH 1 CACHE STRING "") +set(VERSION_GITHASH 3caadfadc6919484e6246cb2feaf3bfdb41585b7 CACHE STRING "") +set(VERSION_DESCRIBE v18.5.1-testing CACHE STRING "") +set(VERSION_STRING 18.5.1 CACHE STRING "") # end of autochange set(VERSION_EXTRA "" CACHE STRING "") @@ -14,11 +14,18 @@ set(VERSION_TWEAK "" CACHE STRING "") if (VERSION_TWEAK) string(CONCAT VERSION_STRING ${VERSION_STRING} "." ${VERSION_TWEAK}) endif () - if (VERSION_EXTRA) string(CONCAT VERSION_STRING ${VERSION_STRING} "." ${VERSION_EXTRA}) endif () -set (VERSION_NAME "${PROJECT_NAME}" CACHE STRING "") -set (VERSION_FULL "${VERSION_NAME} ${VERSION_STRING}" CACHE STRING "") -set (VERSION_SO "${VERSION_STRING}" CACHE STRING "") +set (VERSION_NAME "${PROJECT_NAME}") +set (VERSION_FULL "${VERSION_NAME} ${VERSION_STRING}") + +if (APPLE) + # dirty hack: ld: malformed 64-bit a.b.c.d.e version number: 1.1.54160 + math (EXPR VERSION_SO1 "${VERSION_REVISION}/255") + math (EXPR VERSION_SO2 "${VERSION_REVISION}%255") + set (VERSION_SO "${VERSION_MAJOR}.${VERSION_MINOR}.${VERSION_SO1}.${VERSION_SO2}") +else () + set (VERSION_SO "${VERSION_STRING}") +endif () diff --git a/dbms/programs/CMakeLists.txt b/dbms/programs/CMakeLists.txt index 136616ca44b..20baa6b039c 100644 --- a/dbms/programs/CMakeLists.txt +++ b/dbms/programs/CMakeLists.txt @@ -13,7 +13,6 @@ option (ENABLE_CLICKHOUSE_COMPRESSOR "Enable clickhouse-compressor" ${ENABLE_CLI option (ENABLE_CLICKHOUSE_COPIER "Enable clickhouse-copier" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_FORMAT "Enable clickhouse-format" ${ENABLE_CLICKHOUSE_ALL}) option (ENABLE_CLICKHOUSE_OBFUSCATOR "Enable clickhouse-obfuscator" ${ENABLE_CLICKHOUSE_ALL}) -option (ENABLE_CLICKHOUSE_ODBC_BRIDGE "Enable clickhouse-odbc-bridge" ${ENABLE_CLICKHOUSE_ALL}) configure_file (config_tools.h.in ${CMAKE_CURRENT_BINARY_DIR}/config_tools.h) @@ -28,11 +27,10 @@ add_subdirectory (copier) add_subdirectory (format) add_subdirectory (clang) add_subdirectory (obfuscator) -add_subdirectory (odbc-bridge) if (CLICKHOUSE_SPLIT_BINARY) set (CLICKHOUSE_ALL_TARGETS clickhouse-server clickhouse-client clickhouse-local clickhouse-benchmark clickhouse-performance-test - clickhouse-extract-from-config clickhouse-compressor clickhouse-format clickhouse-copier clickhouse-odbc-bridge) + clickhouse-extract-from-config clickhouse-format clickhouse-copier) if (USE_EMBEDDED_COMPILER) list (APPEND CLICKHOUSE_ALL_TARGETS clickhouse-clang clickhouse-lld) @@ -85,9 +83,6 @@ else () if (USE_EMBEDDED_COMPILER) target_link_libraries (clickhouse clickhouse-compiler-lib) endif () - if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) - target_link_libraries (clickhouse clickhouse-odbc-bridge-lib) - endif() set (CLICKHOUSE_BUNDLE) if (ENABLE_CLICKHOUSE_SERVER) @@ -140,12 +135,6 @@ else () install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-obfuscator DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) list(APPEND CLICKHOUSE_BUNDLE clickhouse-obfuscator) endif () - if (ENABLE_CLICKHOUSE_ODBC_BRIDGE) - add_custom_target (clickhouse-odbc-bridge ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-odbc-bridge DEPENDS clickhouse) - install (FILES ${CMAKE_CURRENT_BINARY_DIR}/clickhouse-odbc-bridge DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) - list(APPEND CLICKHOUSE_BUNDLE clickhouse-odbc-bridge) - endif () - # install always because depian package want this files: add_custom_target (clickhouse-clang ALL COMMAND ${CMAKE_COMMAND} -E create_symlink clickhouse clickhouse-clang DEPENDS clickhouse) @@ -163,6 +152,6 @@ else () endif () -if (TARGET clickhouse-server AND TARGET copy-headers) +if (USE_EMBEDDED_COMPILER AND ENABLE_CLICKHOUSE_SERVER) add_dependencies(clickhouse-server copy-headers) endif () diff --git a/dbms/programs/clang/CMakeLists.txt b/dbms/programs/clang/CMakeLists.txt index 4844cb37c93..88862dcf269 100644 --- a/dbms/programs/clang/CMakeLists.txt +++ b/dbms/programs/clang/CMakeLists.txt @@ -13,10 +13,10 @@ if (CLICKHOUSE_SPLIT_BINARY) endif () endif () -set(TMP_HEADERS_DIR "${CMAKE_CURRENT_BINARY_DIR}/${INTERNAL_COMPILER_HEADERS_RELATIVE}") +set(TMP_HEADERS_DIR "${CMAKE_CURRENT_BINARY_DIR}/headers") # Make and install empty dir for debian package if compiler disabled add_custom_target(make-headers-directory ALL COMMAND ${CMAKE_COMMAND} -E make_directory ${TMP_HEADERS_DIR}) -install(DIRECTORY ${TMP_HEADERS_DIR} DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse/${INTERNAL_COMPILER_HEADERS_DIR} COMPONENT clickhouse) +install(DIRECTORY ${TMP_HEADERS_DIR} DESTINATION ${CMAKE_INSTALL_DATAROOTDIR}/clickhouse COMPONENT clickhouse) # TODO: fix on macos copy_headers.sh: sed --posix if (USE_EMBEDDED_COMPILER AND NOT APPLE) add_custom_target(copy-headers ALL env CLANG=${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-clang BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS clickhouse-clang WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh) diff --git a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt index d02d266d5a5..e07a570a434 100644 --- a/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-5.0.0/CMakeLists.txt @@ -43,7 +43,4 @@ LLVMSupport #PollyPPCG PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads -${MALLOC_LIBRARIES} -${GLIBC_COMPATIBILITY_LIBRARIES} -${MEMCPY_LIBRARIES} ) diff --git a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt index 701b99d08e3..4f5e703bd63 100644 --- a/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-6.0.0/CMakeLists.txt @@ -43,7 +43,4 @@ ${REQUIRED_LLVM_LIBRARIES} #PollyPPCG PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads -${MALLOC_LIBRARIES} -${GLIBC_COMPATIBILITY_LIBRARIES} -${MEMCPY_LIBRARIES} ) diff --git a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt index 081037cdeed..f46e8ef0dc1 100644 --- a/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt +++ b/dbms/programs/clang/Compiler-7.0.0/CMakeLists.txt @@ -39,7 +39,4 @@ lldCore ${REQUIRED_LLVM_LIBRARIES} PUBLIC ${ZLIB_LIBRARIES} ${EXECINFO_LIBRARY} Threads::Threads -${MALLOC_LIBRARIES} -${GLIBC_COMPATIBILITY_LIBRARIES} -${MEMCPY_LIBRARIES} ) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index f5a66e4dd15..f2f8317f756 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -1,5 +1,3 @@ -#include "TestHint.h" - #include #include #include @@ -22,6 +20,7 @@ #include #include #include +#include #include #include #include @@ -32,7 +31,6 @@ #include #include #include -#include #include #include #include @@ -41,7 +39,6 @@ #include #include #include -#include #include #include #include @@ -90,11 +87,105 @@ namespace ErrorCodes extern const int UNEXPECTED_PACKET_FROM_SERVER; extern const int CLIENT_OUTPUT_FORMAT_SPECIFIED; extern const int LOGICAL_ERROR; - extern const int CANNOT_SET_SIGNAL_HANDLER; - extern const int CANNOT_READLINE; } +/// Checks expected server and client error codes in testmode. +/// To enable it add special comment after the query: "-- { serverError 60 }" or "-- { clientError 20 }". +class TestHint +{ +public: + TestHint(bool enabled_, const String & query) + : enabled(enabled_), + server_error(0), + client_error(0) + { + if (!enabled_) + return; + + size_t pos = query.find("--"); + if (pos != String::npos && query.find("--", pos + 2) != String::npos) + return; /// It's not last comment. Hint belongs to commented query. + + if (pos != String::npos) + { + pos = query.find('{', pos + 2); + if (pos != String::npos) + { + String hint = query.substr(pos + 1); + pos = hint.find('}'); + hint.resize(pos); + parse(hint); + } + } + } + + /// @returns true if it's possible to continue without reconnect + bool checkActual(int & actual_server_error, int & actual_client_error, + bool & got_exception, std::unique_ptr & last_exception) const + { + if (!enabled) + return true; + + if (allErrorsExpected(actual_server_error, actual_client_error)) + { + got_exception = false; + last_exception.reset(); + actual_server_error = 0; + actual_client_error = 0; + return false; + } + + if (lostExpectedError(actual_server_error, actual_client_error)) + { + std::cerr << "Success when error expected. It expects server error " + << server_error << ", client error " << client_error << "." << std::endl; + got_exception = true; + last_exception = std::make_unique("Success when error expected", ErrorCodes::LOGICAL_ERROR); /// return error to OS + return false; + } + + return true; + } + + int serverError() const { return server_error; } + int clientError() const { return client_error; } + +private: + bool enabled; + int server_error; + int client_error; + + void parse(const String & hint) + { + std::stringstream ss; + ss << hint; + while (!ss.eof()) + { + String item; + ss >> item; + if (item.empty()) + break; + + if (item == "serverError") + ss >> server_error; + else if (item == "clientError") + ss >> client_error; + } + } + + bool allErrorsExpected(int actual_server_error, int actual_client_error) const + { + return (server_error || client_error) && (server_error == actual_server_error) && (client_error == actual_client_error); + } + + bool lostExpectedError(int actual_server_error, int actual_client_error) const + { + return (server_error && !actual_server_error) || (client_error && !actual_client_error); + } +}; + + class Client : public Poco::Util::Application { public: @@ -113,7 +204,6 @@ private: bool is_interactive = true; /// Use either readline interface or batch mode. bool need_render_progress = true; /// Render query execution progress. bool echo_queries = false; /// Print queries before execution in batch mode. - bool ignore_error = false; /// In case of errors, don't print error message, continue to next query. Only applicable for non-interactive mode. bool print_time_to_stderr = false; /// Output execution time to stderr in batch mode. bool stdin_is_not_tty = false; /// stdin is not a terminal. @@ -144,11 +234,6 @@ private: std::optional out_file_buf; BlockOutputStreamPtr block_out_stream; - /// The user could specify special file for server logs (stderr by default) - std::unique_ptr out_logs_buf; - String server_logs_file; - BlockOutputStreamPtr logs_out_stream; - String home_path; String current_profile; @@ -322,10 +407,20 @@ private: /// If exception code isn't zero, we should return non-zero return code anyway. return e.code() ? e.code() : -1; } + catch (const Poco::Exception & e) + { + std::cerr << "Poco::Exception: " << e.displayText() << std::endl; + return ErrorCodes::POCO_EXCEPTION; + } + catch (const std::exception & e) + { + std::cerr << "std::exception: " << e.what() << std::endl; + return ErrorCodes::STD_EXCEPTION; + } catch (...) { - std::cerr << getCurrentExceptionMessage(false) << std::endl; - return getCurrentExceptionCode(); + std::cerr << "Unknown exception" << std::endl; + return ErrorCodes::UNKNOWN_EXCEPTION; } } @@ -373,18 +468,12 @@ private: format_max_block_size = config().getInt("format_max_block_size", context.getSettingsRef().max_block_size); insert_format = "Values"; - - /// Setting value from cmd arg overrides one from config - if (context.getSettingsRef().max_insert_block_size.changed) - insert_format_max_block_size = context.getSettingsRef().max_insert_block_size; - else - insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); + insert_format_max_block_size = config().getInt("insert_format_max_block_size", context.getSettingsRef().max_insert_block_size); if (!is_interactive) { need_render_progress = config().getBool("progress", false); echo_queries = config().getBool("echo", false); - ignore_error = config().getBool("ignore-error", false); } connect(); @@ -558,7 +647,6 @@ private: String server_name; UInt64 server_version_major = 0; UInt64 server_version_minor = 0; - UInt64 server_version_patch = 0; UInt64 server_revision = 0; if (max_client_network_bandwidth) @@ -567,9 +655,9 @@ private: connection->setThrottler(throttler); } - connection->getServerVersion(server_name, server_version_major, server_version_minor, server_version_patch, server_revision); + connection->getServerVersion(server_name, server_version_major, server_version_minor, server_revision); - server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_version_patch); + server_version = toString(server_version_major) + "." + toString(server_version_minor) + "." + toString(server_revision); if (server_display_name = connection->getServerDisplayName(); server_display_name.length() == 0) { @@ -580,7 +668,6 @@ private: { std::cout << "Connected to " << server_name << " server version " << server_version - << " revision " << server_revision << "." << std::endl << std::endl; } } @@ -707,6 +794,7 @@ private: bool process(const String & text) { + const auto ignore_error = config().getBool("ignore-error", false); const bool test_mode = config().has("testmode"); if (config().has("multiquery")) { @@ -722,7 +810,6 @@ private: { const char * pos = begin; ASTPtr ast = parseQuery(pos, end, true); - if (!ast) { if (ignore_error) @@ -738,7 +825,7 @@ private: return true; } - ASTInsertQuery * insert = typeid_cast(ast.get()); + ASTInsertQuery * insert = typeid_cast(&*ast); if (insert && insert->data) { @@ -763,10 +850,9 @@ private: } catch (...) { - last_exception = std::make_unique(getCurrentExceptionMessage(true), getCurrentExceptionCode()); - actual_client_error = last_exception->code(); - if (!ignore_error && (!actual_client_error || actual_client_error != expected_client_error)) - std::cerr << "Error on processing query: " << query << std::endl << last_exception->message(); + actual_client_error = getCurrentExceptionCode(); + if (!actual_client_error || actual_client_error != expected_client_error) + std::cerr << "Error on processing query: " << query << std::endl << getCurrentExceptionMessage(true); got_exception = true; } @@ -931,7 +1017,7 @@ private: /// If structure was received (thus, server has not thrown an exception), /// send our data with that structure. sendData(sample); - receiveEndOfQuery(); + receivePacket(); } } @@ -1013,11 +1099,6 @@ private: connection->sendData(block); processed_rows += block.rows(); - /// Check if server send Log packet - auto packet_type = connection->checkPacket(); - if (packet_type && *packet_type == Protocol::Server::Log) - receiveAndProcessPacket(); - if (!block) break; } @@ -1029,28 +1110,18 @@ private: /// Flush all buffers. void resetOutput() { - block_out_stream.reset(); - logs_out_stream.reset(); - + block_out_stream = nullptr; if (pager_cmd) { pager_cmd->in.close(); pager_cmd->wait(); } pager_cmd = nullptr; - if (out_file_buf) { out_file_buf->next(); out_file_buf.reset(); } - - if (out_logs_buf) - { - out_logs_buf->next(); - out_logs_buf.reset(); - } - std_out.next(); } @@ -1083,7 +1154,7 @@ private: continue; /// If there is no new data, continue checking whether the query was cancelled after a timeout. } - if (!receiveAndProcessPacket()) + if (!receivePacket()) break; } @@ -1094,7 +1165,7 @@ private: /// Receive a part of the result, or progress info or an exception and process it. /// Returns true if one should continue receiving packets. - bool receiveAndProcessPacket() + bool receivePacket() { Connection::Packet packet = connection->receivePacket(); @@ -1125,10 +1196,6 @@ private: last_exception = std::move(packet.exception); return false; - case Protocol::Server::Log: - onLogData(packet.block); - return true; - case Protocol::Server::EndOfStream: onEndOfStream(); return false; @@ -1142,59 +1209,22 @@ private: /// Receive the block that serves as an example of the structure of table where data will be inserted. bool receiveSampleBlock(Block & out) { - while (true) + Connection::Packet packet = connection->receivePacket(); + + switch (packet.type) { - Connection::Packet packet = connection->receivePacket(); + case Protocol::Server::Data: + out = packet.block; + return true; - switch (packet.type) - { - case Protocol::Server::Data: - out = packet.block; - return true; + case Protocol::Server::Exception: + onException(*packet.exception); + last_exception = std::move(packet.exception); + return false; - case Protocol::Server::Exception: - onException(*packet.exception); - last_exception = std::move(packet.exception); - return false; - - case Protocol::Server::Log: - onLogData(packet.block); - break; - - default: - throw NetException("Unexpected packet from server (expected Data, Exception or Log, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); - } - } - } - - - /// Process Log packets, exit when recieve Exception or EndOfStream - bool receiveEndOfQuery() - { - while (true) - { - Connection::Packet packet = connection->receivePacket(); - - switch (packet.type) - { - case Protocol::Server::EndOfStream: - onEndOfStream(); - return true; - - case Protocol::Server::Exception: - onException(*packet.exception); - last_exception = std::move(packet.exception); - return false; - - case Protocol::Server::Log: - onLogData(packet.block); - break; - - default: - throw NetException("Unexpected packet from server (expected Exception, EndOfStream or Log, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); - } + default: + throw NetException("Unexpected packet from server (expected Data, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } } @@ -1250,38 +1280,6 @@ private: } - void initLogsOutputStream() - { - if (!logs_out_stream) - { - WriteBuffer * wb = out_logs_buf.get(); - - if (!out_logs_buf) - { - if (server_logs_file.empty()) - { - /// Use stderr by default - out_logs_buf = std::make_unique(STDERR_FILENO); - wb = out_logs_buf.get(); - } - else if (server_logs_file == "-") - { - /// Use stdout if --server_logs_file=- specified - wb = &std_out; - } - else - { - out_logs_buf = std::make_unique(server_logs_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); - wb = out_logs_buf.get(); - } - } - - logs_out_stream = std::make_shared(*wb); - logs_out_stream->writePrefix(); - } - } - - void onData(Block & block) { if (written_progress_chars) @@ -1305,14 +1303,6 @@ private: } - void onLogData(Block & block) - { - initLogsOutputStream(); - logs_out_stream->write(block); - logs_out_stream->flush(); - } - - void onTotals(Block & block) { initBlockOutputStream(block); @@ -1473,9 +1463,6 @@ private: if (block_out_stream) block_out_stream->writeSuffix(); - if (logs_out_stream) - logs_out_stream->writeSuffix(); - resetOutput(); if (is_interactive && !written_first_block) @@ -1551,9 +1538,7 @@ public: ioctl(0, TIOCGWINSZ, &terminal_size); - namespace po = boost::program_options; - - unsigned line_length = po::options_description::m_default_line_length; + unsigned line_length = boost::program_options::options_description::m_default_line_length; unsigned min_description_length = line_length / 2; if (!stdin_is_not_tty) { @@ -1561,58 +1546,55 @@ public: min_description_length = std::min(min_description_length, line_length - 2); } -#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, po::value (), DESCRIPTION) +#define DECLARE_SETTING(TYPE, NAME, DEFAULT, DESCRIPTION) (#NAME, boost::program_options::value (), DESCRIPTION) /// Main commandline options related to client functionality and all parameters from Settings. - po::options_description main_description("Main options", line_length, min_description_length); + boost::program_options::options_description main_description("Main options", line_length, min_description_length); main_description.add_options() ("help", "produce help message") - ("config-file,c", po::value(), "config-file path") - ("host,h", po::value()->default_value("localhost"), "server host") - ("port", po::value()->default_value(9000), "server port") + ("config-file,c", boost::program_options::value(), "config-file path") + ("host,h", boost::program_options::value()->default_value("localhost"), "server host") + ("port", boost::program_options::value()->default_value(9000), "server port") ("secure,s", "secure") - ("user,u", po::value()->default_value("default"), "user") - ("password", po::value(), "password") + ("user,u", boost::program_options::value()->default_value("default"), "user") + ("password", boost::program_options::value(), "password") ("ask-password", "ask-password") - ("query_id", po::value(), "query_id") - ("query,q", po::value(), "query") - ("database,d", po::value(), "database") - ("pager", po::value(), "pager") + ("query_id", boost::program_options::value(), "query_id") + ("query,q", boost::program_options::value(), "query") + ("database,d", boost::program_options::value(), "database") + ("pager", boost::program_options::value(), "pager") ("multiline,m", "multiline") ("multiquery,n", "multiquery") - ("format,f", po::value(), "default output format") ("testmode,T", "enable test hints in comments") ("ignore-error", "do not stop processing in multiquery mode") + ("format,f", boost::program_options::value(), "default output format") ("vertical,E", "vertical output format, same as --format=Vertical or FORMAT Vertical or \\G at end of command") ("time,t", "print query execution time to stderr in non-interactive mode (for benchmarks)") ("stacktrace", "print stack traces of exceptions") ("progress", "print progress even in non-interactive mode") ("version,V", "print version information and exit") - ("version-clean", "print version in machine-readable format and exit") ("echo", "in batch mode, print query before execution") - ("max_client_network_bandwidth", po::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") - ("compression", po::value(), "enable or disable compression") - ("log-level", po::value(), "client log level") - ("server_logs_file", po::value(), "put server logs into specified file") + ("max_client_network_bandwidth", boost::program_options::value(), "the maximum speed of data exchange over the network for the client in bytes per second.") + ("compression", boost::program_options::value(), "enable or disable compression") APPLY_FOR_SETTINGS(DECLARE_SETTING) ; #undef DECLARE_SETTING /// Commandline options related to external tables. - po::options_description external_description("External tables options"); + boost::program_options::options_description external_description("External tables options"); external_description.add_options() - ("file", po::value(), "data file or - for stdin") - ("name", po::value()->default_value("_data"), "name of the table") - ("format", po::value()->default_value("TabSeparated"), "data format") - ("structure", po::value(), "structure") - ("types", po::value(), "types") + ("file", boost::program_options::value(), "data file or - for stdin") + ("name", boost::program_options::value()->default_value("_data"), "name of the table") + ("format", boost::program_options::value()->default_value("TabSeparated"), "data format") + ("structure", boost::program_options::value(), "structure") + ("types", boost::program_options::value(), "types") ; /// Parse main commandline options. - po::parsed_options parsed = po::command_line_parser( + boost::program_options::parsed_options parsed = boost::program_options::command_line_parser( common_arguments.size(), common_arguments.data()).options(main_description).run(); - po::variables_map options; - po::store(parsed, options); + boost::program_options::variables_map options; + boost::program_options::store(parsed, options); if (options.count("version") || options.count("V")) { @@ -1620,12 +1602,6 @@ public: exit(0); } - if (options.count("version-clean")) - { - std::cout << VERSION_STRING; - exit(0); - } - /// Output of help message. if (options.count("help") || (options.count("host") && options["host"].as() == "elp")) /// If user writes -help instead of --help. @@ -1635,17 +1611,14 @@ public: exit(0); } - if (options.count("log-level")) - Poco::Logger::root().setLevel(options["log-level"].as()); - size_t number_of_external_tables_with_stdin_source = 0; for (size_t i = 0; i < external_tables_arguments.size(); ++i) { /// Parse commandline options related to external tables. - po::parsed_options parsed = po::command_line_parser( + boost::program_options::parsed_options parsed = boost::program_options::command_line_parser( external_tables_arguments[i].size(), external_tables_arguments[i].data()).options(external_description).run(); - po::variables_map external_options; - po::store(parsed, external_options); + boost::program_options::variables_map external_options; + boost::program_options::store(parsed, external_options); try { @@ -1719,8 +1692,6 @@ public: max_client_network_bandwidth = options["max_client_network_bandwidth"].as(); if (options.count("compression")) config().setBool("compression", options["compression"].as()); - if (options.count("server_logs_file")) - server_logs_file = options["server_logs_file"].as(); } }; @@ -1740,11 +1711,6 @@ int mainEntryClickHouseClient(int argc, char ** argv) std::cerr << "Bad arguments: " << e.what() << std::endl; return 1; } - catch (...) - { - std::cerr << DB::getCurrentExceptionMessage(true) << std::endl; - return 1; - } return client.run(); } diff --git a/dbms/programs/client/TestHint.h b/dbms/programs/client/TestHint.h deleted file mode 100644 index 790e58ee7fe..00000000000 --- a/dbms/programs/client/TestHint.h +++ /dev/null @@ -1,118 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - -/// Checks expected server and client error codes in testmode. -/// To enable it add special comment after the query: "-- { serverError 60 }" or "-- { clientError 20 }". -class TestHint -{ -public: - TestHint(bool enabled_, const String & query) - : enabled(enabled_) - { - if (!enabled_) - return; - - /// TODO: This is absolutely wrong. Fragment may be contained inside string literal. - size_t pos = query.find("--"); - - if (pos != String::npos && query.find("--", pos + 2) != String::npos) - return; /// It's not last comment. Hint belongs to commented query. /// TODO Absolutely wrong: there maybe the following comment for the next query. - - if (pos != String::npos) - { - /// TODO: This is also wrong. Comment may already have ended by line break. - pos = query.find('{', pos + 2); - - if (pos != String::npos) - { - String hint = query.substr(pos + 1); - - /// TODO: And this is wrong for the same reason. - pos = hint.find('}'); - hint.resize(pos); - parse(hint); - } - } - } - - /// @returns true if it's possible to continue without reconnect - bool checkActual(int & actual_server_error, int & actual_client_error, - bool & got_exception, std::unique_ptr & last_exception) const - { - if (!enabled) - return true; - - if (allErrorsExpected(actual_server_error, actual_client_error)) - { - got_exception = false; - last_exception.reset(); - actual_server_error = 0; - actual_client_error = 0; - return false; - } - - if (lostExpectedError(actual_server_error, actual_client_error)) - { - std::cerr << "Success when error expected. It expects server error " - << server_error << ", client error " << client_error << "." << std::endl; - got_exception = true; - last_exception = std::make_unique("Success when error expected", ErrorCodes::LOGICAL_ERROR); /// return error to OS - return false; - } - - return true; - } - - int serverError() const { return server_error; } - int clientError() const { return client_error; } - -private: - bool enabled = false; - int server_error = 0; - int client_error = 0; - - void parse(const String & hint) - { - std::stringstream ss; - ss << hint; - while (!ss.eof()) - { - String item; - ss >> item; - if (item.empty()) - break; - - if (item == "serverError") - ss >> server_error; - else if (item == "clientError") - ss >> client_error; - } - } - - bool allErrorsExpected(int actual_server_error, int actual_client_error) const - { - return (server_error || client_error) && (server_error == actual_server_error) && (client_error == actual_client_error); - } - - bool lostExpectedError(int actual_server_error, int actual_client_error) const - { - return (server_error && !actual_server_error) || (client_error && !actual_client_error); - } -}; - -} diff --git a/dbms/programs/copier/ClusterCopier.cpp b/dbms/programs/copier/ClusterCopier.cpp index fab1d5af4c5..a000401f02c 100644 --- a/dbms/programs/copier/ClusterCopier.cpp +++ b/dbms/programs/copier/ClusterCopier.cpp @@ -30,7 +30,6 @@ #include #include #include -#include #include #include #include @@ -2144,9 +2143,6 @@ void ClusterCopierApp::mainImpl() context->addDatabase(default_database, std::make_shared(default_database)); context->setCurrentDatabase(default_database); - /// Initialize query scope just in case. - CurrentThread::QueryScope query_scope(*context); - auto copier = std::make_unique(task_path, host_id, default_database, *context); copier->setSafeMode(is_safe_mode); copier->setCopyFaultProbability(copy_fault_probability); diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 3bf1fa1d6ed..aba03a87a83 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -56,10 +56,6 @@ int mainEntryClickHouseClusterCopier(int argc, char ** argv); #if ENABLE_CLICKHOUSE_OBFUSCATOR int mainEntryClickHouseObfuscator(int argc, char ** argv); #endif -#if ENABLE_CLICKHOUSE_ODBC_BRIDGE || !defined(ENABLE_CLICKHOUSE_ODBC_BRIDGE) -int mainEntryClickHouseODBCBridge(int argc, char ** argv); -#endif - #if USE_EMBEDDED_COMPILER int mainEntryClickHouseClang(int argc, char ** argv); @@ -105,10 +101,6 @@ std::pair clickhouse_applications[] = #if ENABLE_CLICKHOUSE_OBFUSCATOR {"obfuscator", mainEntryClickHouseObfuscator}, #endif -#if ENABLE_CLICKHOUSE_ODBC_BRIDGE || !defined(ENABLE_CLICKHOUSE_ODBC_BRIDGE) - {"odbc-bridge", mainEntryClickHouseODBCBridge}, -#endif - #if USE_EMBEDDED_COMPILER {"clang", mainEntryClickHouseClang}, {"clang++", mainEntryClickHouseClang}, diff --git a/dbms/programs/odbc-bridge/CMakeLists.txt b/dbms/programs/odbc-bridge/CMakeLists.txt deleted file mode 100644 index df68a8c546c..00000000000 --- a/dbms/programs/odbc-bridge/CMakeLists.txt +++ /dev/null @@ -1,31 +0,0 @@ -add_library (clickhouse-odbc-bridge-lib - PingHandler.cpp - MainHandler.cpp - ColumnInfoHandler.cpp - HandlerFactory.cpp - ODBCBridge.cpp - validateODBCConnectionString.cpp -) - -target_link_libraries (clickhouse-odbc-bridge-lib clickhouse_common_io daemon dbms) -target_include_directories (clickhouse-odbc-bridge-lib PUBLIC ${ClickHouse_SOURCE_DIR}/libs/libdaemon/include) - -if (USE_POCO_SQLODBC) - target_link_libraries (clickhouse-odbc-bridge-lib ${Poco_SQLODBC_LIBRARY}) - target_include_directories (clickhouse-odbc-bridge-lib SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_SQLODBC_INCLUDE_DIRS}) -endif () - -if (USE_POCO_DATAODBC) - target_link_libraries (clickhouse-odbc-bridge-lib ${Poco_DataODBC_LIBRARY}) - target_include_directories (clickhouse-odbc-bridge-lib SYSTEM PRIVATE ${ODBC_INCLUDE_DIRECTORIES} ${Poco_DataODBC_INCLUDE_DIRS}) -endif() - - -if (ENABLE_TESTS) - add_subdirectory (tests) -endif () - -if (CLICKHOUSE_SPLIT_BINARY) - add_executable (clickhouse-odbc-bridge odbc-bridge.cpp) - target_link_libraries (clickhouse-odbc-bridge clickhouse-odbc-bridge-lib) -endif () diff --git a/dbms/programs/odbc-bridge/ColumnInfoHandler.h b/dbms/programs/odbc-bridge/ColumnInfoHandler.h deleted file mode 100644 index 426cea15b34..00000000000 --- a/dbms/programs/odbc-bridge/ColumnInfoHandler.h +++ /dev/null @@ -1,30 +0,0 @@ -#pragma once -#include -#include -#include -#include - -#if USE_POCO_SQLODBC || USE_POCO_DATAODBC -/** The structure of the table is taken from the query "SELECT * FROM table WHERE 1=0". - * TODO: It would be much better to utilize ODBC methods dedicated for columns description. - * If there is no such table, an exception is thrown. - */ -namespace DB -{ -class ODBCColumnsInfoHandler : public Poco::Net::HTTPRequestHandler -{ -public: - ODBCColumnsInfoHandler(size_t keep_alive_timeout_, std::shared_ptr context_) - : log(&Poco::Logger::get("ODBCColumnsInfoHandler")), keep_alive_timeout(keep_alive_timeout_), context(context_) - { - } - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - Poco::Logger * log; - size_t keep_alive_timeout; - std::shared_ptr context; -}; -} -#endif diff --git a/dbms/programs/odbc-bridge/HandlerFactory.cpp b/dbms/programs/odbc-bridge/HandlerFactory.cpp deleted file mode 100644 index f552203174e..00000000000 --- a/dbms/programs/odbc-bridge/HandlerFactory.cpp +++ /dev/null @@ -1,34 +0,0 @@ -#include "HandlerFactory.h" -#include "PingHandler.h" -#include "ColumnInfoHandler.h" -#include - -#include -#include -#include - -namespace DB -{ -Poco::Net::HTTPRequestHandler * HandlerFactory::createRequestHandler(const Poco::Net::HTTPServerRequest & request) -{ - Poco::URI uri{request.getURI()}; - LOG_TRACE(log, "Request URI: " + uri.toString()); - - if (uri.getPath() == "/ping" && request.getMethod() == Poco::Net::HTTPRequest::HTTP_GET) - return new PingHandler(keep_alive_timeout); - - if (request.getMethod() == Poco::Net::HTTPRequest::HTTP_POST) - { - - if (uri.getPath() == "/columns_info") -#if USE_POCO_SQLODBC || USE_POCO_DATAODBC - return new ODBCColumnsInfoHandler(keep_alive_timeout, context); -#else - return nullptr; -#endif - else - return new ODBCHandler(pool_map, keep_alive_timeout, context); - } - return nullptr; -} -} diff --git a/dbms/programs/odbc-bridge/HandlerFactory.h b/dbms/programs/odbc-bridge/HandlerFactory.h deleted file mode 100644 index 4fe00ffca98..00000000000 --- a/dbms/programs/odbc-bridge/HandlerFactory.h +++ /dev/null @@ -1,38 +0,0 @@ -#pragma once -#include -#include -#include -#include -#include "MainHandler.h" -#include "ColumnInfoHandler.h" - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" - #include -#pragma GCC diagnostic pop - - -namespace DB -{ -/** Factory for '/ping', '/' and '/columns_info' handlers. - * Also stores Session pools for ODBC connections - */ -class HandlerFactory : public Poco::Net::HTTPRequestHandlerFactory -{ -public: - HandlerFactory(const std::string & name_, size_t keep_alive_timeout_, std::shared_ptr context_) - : log(&Poco::Logger::get(name_)), name(name_), keep_alive_timeout(keep_alive_timeout_), context(context_) - { - pool_map = std::make_shared(); - } - - Poco::Net::HTTPRequestHandler * createRequestHandler(const Poco::Net::HTTPServerRequest & request) override; - -private: - Poco::Logger * log; - std::string name; - size_t keep_alive_timeout; - std::shared_ptr context; - std::shared_ptr pool_map; -}; -} diff --git a/dbms/programs/odbc-bridge/MainHandler.h b/dbms/programs/odbc-bridge/MainHandler.h deleted file mode 100644 index ae139f393f8..00000000000 --- a/dbms/programs/odbc-bridge/MainHandler.h +++ /dev/null @@ -1,49 +0,0 @@ -#pragma once - -#include -#include -#include - -#pragma GCC diagnostic push -#pragma GCC diagnostic ignored "-Wunused-parameter" - #include -#pragma GCC diagnostic pop - -namespace DB -{ -/** Main handler for requests to ODBC driver - * requires connection_string and columns in request params - * and also query in request body - * response in RowBinary format - */ -class ODBCHandler : public Poco::Net::HTTPRequestHandler -{ -public: - using PoolPtr = std::shared_ptr; - using PoolMap = std::unordered_map; - - ODBCHandler(std::shared_ptr pool_map_, - size_t keep_alive_timeout_, - std::shared_ptr context_) - : log(&Poco::Logger::get("ODBCHandler")) - , pool_map(pool_map_) - , keep_alive_timeout(keep_alive_timeout_) - , context(context_) - { - } - - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - Poco::Logger * log; - - std::shared_ptr pool_map; - size_t keep_alive_timeout; - std::shared_ptr context; - - static inline std::mutex mutex; - - PoolPtr getPool(const std::string & connection_str); -}; - -} diff --git a/dbms/programs/odbc-bridge/ODBCBridge.cpp b/dbms/programs/odbc-bridge/ODBCBridge.cpp deleted file mode 100644 index bab58250fa4..00000000000 --- a/dbms/programs/odbc-bridge/ODBCBridge.cpp +++ /dev/null @@ -1,205 +0,0 @@ -#include "ODBCBridge.h" -#include "HandlerFactory.h" - -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int ARGUMENT_OUT_OF_BOUND; -} - -namespace -{ - Poco::Net::SocketAddress makeSocketAddress(const std::string & host, UInt16 port, Poco::Logger * log) - { - Poco::Net::SocketAddress socket_address; - try - { - socket_address = Poco::Net::SocketAddress(host, port); - } - catch (const Poco::Net::DNSException & e) - { - const auto code = e.code(); - if (code == EAI_FAMILY -#if defined(EAI_ADDRFAMILY) - || code == EAI_ADDRFAMILY -#endif - ) - { - LOG_ERROR(log, - "Cannot resolve listen_host (" << host << "), error " << e.code() << ": " << e.message() - << ". " - "If it is an IPv6 address and your host has disabled IPv6, then consider to " - "specify IPv4 address to listen in element of configuration " - "file. Example: 0.0.0.0"); - } - - throw; - } - return socket_address; - } - - Poco::Net::SocketAddress socketBindListen(Poco::Net::ServerSocket & socket, const std::string & host, UInt16 port, Poco::Logger * log) - { - auto address = makeSocketAddress(host, port, log); -#if POCO_VERSION < 0x01080000 - socket.bind(address, /* reuseAddress = */ true); -#else - socket.bind(address, /* reuseAddress = */ true, /* reusePort = */ false); -#endif - - socket.listen(/* backlog = */ 64); - - return address; - }; -} - -void ODBCBridge::handleHelp(const std::string &, const std::string &) -{ - Poco::Util::HelpFormatter helpFormatter(options()); - helpFormatter.setCommand(commandName()); - helpFormatter.setHeader("HTTP-proxy for odbc requests"); - helpFormatter.setUsage("--http-port "); - helpFormatter.format(std::cerr); - - stopOptionsProcessing(); -} - - -void ODBCBridge::defineOptions(Poco::Util::OptionSet & options) -{ - options.addOption(Poco::Util::Option("http-port", "", "port to listen").argument("http-port", true).binding("http-port")); - options.addOption( - Poco::Util::Option("listen-host", "", "hostname to listen, default localhost").argument("listen-host").binding("listen-host")); - options.addOption( - Poco::Util::Option("http-timeout", "", "http timout for socket, default 1800").argument("http-timeout").binding("http-timeout")); - - options.addOption(Poco::Util::Option("max-server-connections", "", "max connections to server, default 1024") - .argument("max-server-connections") - .binding("max-server-connections")); - options.addOption(Poco::Util::Option("keep-alive-timeout", "", "keepalive timeout, default 10") - .argument("keep-alive-timeout") - .binding("keep-alive-timeout")); - - options.addOption(Poco::Util::Option("log-level", "", "sets log level, default info").argument("log-level").binding("logger.level")); - - options.addOption( - Poco::Util::Option("log-path", "", "log path for all logs, default console").argument("log-path").binding("logger.log")); - - options.addOption(Poco::Util::Option("err-log-path", "", "err log path for all logs, default no") - .argument("err-log-path") - .binding("logger.errorlog")); - - using Me = std::decay_t; - options.addOption(Poco::Util::Option("help", "", "produce this help message") - .binding("help") - .callback(Poco::Util::OptionCallback(this, &Me::handleHelp))); - - ServerApplication::defineOptions(options); /// Don't need complex BaseDaemon's .xml config -} - -void ODBCBridge::initialize(Application & self) -{ - BaseDaemon::closeFDs(); - is_help = config().has("help"); - - if (is_help) - return; - - if (!config().has("logger.log")) - config().setBool("logger.console", true); - - config().setString("logger", "ODBCBridge"); - - buildLoggers(config()); - log = &logger(); - hostname = config().getString("listen-host", "localhost"); - port = config().getUInt("http-port"); - if (port > 0xFFFF) - throw Exception("Out of range 'http-port': " + std::to_string(port), ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - http_timeout = config().getUInt("http-timeout", DEFAULT_HTTP_READ_BUFFER_TIMEOUT); - max_server_connections = config().getUInt("max-server-connections", 1024); - keep_alive_timeout = config().getUInt("keep-alive-timeout", 10); - - initializeTerminationAndSignalProcessing(); - - ServerApplication::initialize(self); -} - -void ODBCBridge::uninitialize() -{ - BaseDaemon::uninitialize(); -} - -int ODBCBridge::main(const std::vector & /*args*/) -{ - if (is_help) - return Application::EXIT_OK; - - LOG_INFO(log, "Starting up"); - Poco::Net::ServerSocket socket; - auto address = socketBindListen(socket, hostname, port, log); - socket.setReceiveTimeout(http_timeout); - socket.setSendTimeout(http_timeout); - Poco::ThreadPool server_pool(3, max_server_connections); - Poco::Net::HTTPServerParams::Ptr http_params = new Poco::Net::HTTPServerParams; - http_params->setTimeout(http_timeout); - http_params->setKeepAliveTimeout(keep_alive_timeout); - - context = std::make_shared(Context::createGlobal()); - context->setGlobalContext(*context); - - auto server = Poco::Net::HTTPServer( - new HandlerFactory("ODBCRequestHandlerFactory-factory", keep_alive_timeout, context), server_pool, socket, http_params); - server.start(); - - LOG_INFO(log, "Listening http://" + address.toString()); - - SCOPE_EXIT({ - LOG_DEBUG(log, "Received termination signal."); - LOG_DEBUG(log, "Waiting for current connections to close."); - server.stop(); - for (size_t count : ext::range(1, 6)) - { - if (server.currentConnections() == 0) - break; - LOG_DEBUG(log, "Waiting for " << server.currentConnections() << " connections, try " << count); - std::this_thread::sleep_for(std::chrono::milliseconds(1000)); - } - }); - - waitForTerminationRequest(); - return Application::EXIT_OK; -} -} - -int mainEntryClickHouseODBCBridge(int argc, char ** argv) -{ - DB::ODBCBridge app; - try - { - return app.run(argc, argv); - } - catch (...) - { - std::cerr << DB::getCurrentExceptionMessage(true) << "\n"; - auto code = DB::getCurrentExceptionCode(); - return code ? code : 1; - } -} diff --git a/dbms/programs/odbc-bridge/ODBCBridge.h b/dbms/programs/odbc-bridge/ODBCBridge.h deleted file mode 100644 index 4ae11ad7301..00000000000 --- a/dbms/programs/odbc-bridge/ODBCBridge.h +++ /dev/null @@ -1,41 +0,0 @@ -#pragma once - -#include -#include -#include - -namespace DB -{ -/** Class represents clickhouse-odbc-bridge server, which listen - * incoming HTTP POST and GET requests on specified port and host. - * Has two handlers '/' for all incoming POST requests to ODBC driver - * and /ping for GET request about service status - */ -class ODBCBridge : public BaseDaemon -{ -public: - void defineOptions(Poco::Util::OptionSet & options) override; - -protected: - void initialize(Application & self) override; - - void uninitialize() override; - - int main(const std::vector & args) override; - -private: - void handleHelp(const std::string &, const std::string &); - - bool is_help; - std::string hostname; - size_t port; - size_t http_timeout; - std::string log_level; - size_t max_server_connections; - size_t keep_alive_timeout; - - Poco::Logger * log; - - std::shared_ptr context; /// need for settings only -}; -} diff --git a/dbms/programs/odbc-bridge/PingHandler.cpp b/dbms/programs/odbc-bridge/PingHandler.cpp deleted file mode 100644 index b0313e46bf3..00000000000 --- a/dbms/programs/odbc-bridge/PingHandler.cpp +++ /dev/null @@ -1,22 +0,0 @@ -#include "PingHandler.h" -#include -#include -#include -#include - -namespace DB -{ -void PingHandler::handleRequest(Poco::Net::HTTPServerRequest & /*request*/, Poco::Net::HTTPServerResponse & response) -{ - try - { - setResponseDefaultHeaders(response, keep_alive_timeout); - const char * data = "Ok.\n"; - response.sendBuffer(data, strlen(data)); - } - catch (...) - { - tryLogCurrentException("PingHandler"); - } -} -} diff --git a/dbms/programs/odbc-bridge/PingHandler.h b/dbms/programs/odbc-bridge/PingHandler.h deleted file mode 100644 index d8109a50bb6..00000000000 --- a/dbms/programs/odbc-bridge/PingHandler.h +++ /dev/null @@ -1,17 +0,0 @@ -#pragma once -#include - -namespace DB -{ -/** Simple ping handler, answers "Ok." to GET request - */ -class PingHandler : public Poco::Net::HTTPRequestHandler -{ -public: - PingHandler(size_t keep_alive_timeout_) : keep_alive_timeout(keep_alive_timeout_) {} - void handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Net::HTTPServerResponse & response) override; - -private: - size_t keep_alive_timeout; -}; -} diff --git a/dbms/programs/odbc-bridge/README.md b/dbms/programs/odbc-bridge/README.md deleted file mode 100644 index 91a6e476733..00000000000 --- a/dbms/programs/odbc-bridge/README.md +++ /dev/null @@ -1,38 +0,0 @@ -# clickhouse-odbc-bridge - -Simple HTTP-server which works like a proxy for ODBC driver. The main motivation -was possible segfaults or another faults in ODBC implementations, which can -crash whole clickhouse-server process. - -This tool works via HTTP, not via pipes, shared memory, or TCP because: -- It's simplier to implement -- It's simplier to debug -- jdbc-bridge can be implemented in the same way - -## Usage - -`clickhouse-server` use this tool inside odbc table function and StorageODBC. -However it can be used as standalone tool from command line with the following -parameters in POST-request URL: -- `connection_string` -- ODBC connection string. -- `columns` -- columns in ClickHouse NamesAndTypesList format, name in backticks, - type as string. Name and type are space separated, rows separated with - newline. -- `max_block_size` -- optional parameter, sets maximum size of single block. -Query is send in post body. Response is returned in RowBinary format. - -## Example: - -```bash -$ clickhouse-odbc-bridge --http-port 9018 --daemon - -$ curl -d "query=SELECT PageID, ImpID, AdType FROM Keys ORDER BY PageID, ImpID" --data-urlencode "connection_string=DSN=ClickHouse;DATABASE=stat" --data-urlencode "columns=columns format version: 1 -3 columns: -\`PageID\` String -\`ImpID\` String -\`AdType\` String -" "http://localhost:9018/" > result.txt - -$ cat result.txt -12246623837185725195925621517 -``` diff --git a/dbms/programs/odbc-bridge/odbc-bridge.cpp b/dbms/programs/odbc-bridge/odbc-bridge.cpp deleted file mode 100644 index af42eef8647..00000000000 --- a/dbms/programs/odbc-bridge/odbc-bridge.cpp +++ /dev/null @@ -1,2 +0,0 @@ -int mainEntryClickHouseODBCBridge(int argc, char ** argv); -int main(int argc_, char ** argv_) { return mainEntryClickHouseODBCBridge(argc_, argv_); } diff --git a/dbms/programs/odbc-bridge/tests/CMakeLists.txt b/dbms/programs/odbc-bridge/tests/CMakeLists.txt deleted file mode 100644 index 5240a917429..00000000000 --- a/dbms/programs/odbc-bridge/tests/CMakeLists.txt +++ /dev/null @@ -1,2 +0,0 @@ -add_executable (validate-odbc-connection-string validate-odbc-connection-string.cpp) -target_link_libraries (validate-odbc-connection-string clickhouse-odbc-bridge-lib) diff --git a/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.cpp b/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.cpp deleted file mode 100644 index c4558811f77..00000000000 --- a/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.cpp +++ /dev/null @@ -1,24 +0,0 @@ -#include -#include -#include "../validateODBCConnectionString.h" - - -using namespace DB; - -int main(int argc, char ** argv) -try -{ - if (argc < 2) - { - std::cerr << "Usage: validate-odbc-connection-string 'ConnectionString'\n"; - return 1; - } - - std::cout << validateODBCConnectionString(argv[1]) << '\n'; - return 0; -} -catch (...) -{ - std::cerr << getCurrentExceptionMessage(false) << "\n"; - return 2; -} diff --git a/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.reference b/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.reference deleted file mode 100644 index 2f1a0eb5bcd..00000000000 --- a/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.reference +++ /dev/null @@ -1,39 +0,0 @@ -Code: 404, e.displayText() = DB::Exception: ODBC connection string cannot be empty, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter doesn't have value, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: DSN parameter is mandatory for ODBC connection string, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter doesn't have value, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: DSN parameter is mandatory for ODBC connection string, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter value is unescaped and contains illegal character, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: DSN parameter is mandatory for ODBC connection string, e.what() = DB::Exception -DSN={hello};ABC={de[f}; -DSN={hello};ABC={de}}f}; -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter value is unescaped and contains illegal character, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter is escaped but there is no closing curly brace, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: Unexpected character found after parameter value in ODBC connection string, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: Unexpected character found after parameter value in ODBC connection string, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: Unexpected character found after parameter value in ODBC connection string, e.what() = DB::Exception -DSN={hello};ABC={de}}f}; -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter is escaped but there is no closing curly brace, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter is escaped but there is no closing curly brace, e.what() = DB::Exception -DSN={hello};ABC={ }; -DSN={hello};ABC={ }; -Code: 404, e.displayText() = DB::Exception: Unexpected character found after parameter value in ODBC connection string, e.what() = DB::Exception -DSN={hello world};ABC={ }; -Code: 404, e.displayText() = DB::Exception: Unexpected character found after parameter value in ODBC connection string, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter name doesn't begin with valid identifier character, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter name doesn't begin with valid identifier character, e.what() = DB::Exception -DSN={hello world};ABC={ };_={}; -DSN={hello world};ABC={ };_={}; -DSN={hello world};ABC={ };_={}; -DSN={hello world};ABC={ };_={}}}; -DSN={hello world};ABC={ };_={...................................................................}; -DSN={hello world};ABC={ };_={....................................................................................}; -Code: 404, e.displayText() = DB::Exception: ODBC connection string has too long keyword or value, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string has forbidden parameter, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string has forbidden parameter, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string has forbidden parameter, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: Duplicate parameter found in ODBC connection string, e.what() = DB::Exception -Code: 404, e.displayText() = DB::Exception: ODBC connection string parameter name doesn't begin with valid identifier character, e.what() = DB::Exception -DSN={myconnection}; -DSN={myconnection};DATABASE={my_db};HOST={127.0.0.1};PORT={5432};PWD={password};UID={username}; -DSN={MSSQL};PWD={test};UID={test}; diff --git a/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.sh b/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.sh deleted file mode 100755 index 39789a8ab44..00000000000 --- a/dbms/programs/odbc-bridge/tests/validate-odbc-connection-string.sh +++ /dev/null @@ -1,41 +0,0 @@ -#!/bin/sh - -./validate-odbc-connection-string '' 2>&1 -./validate-odbc-connection-string 'abc' 2>&1 -./validate-odbc-connection-string 'abc=' 2>&1 -./validate-odbc-connection-string 'ab"c=' 2>&1 -./validate-odbc-connection-string 'abc=def' 2>&1 -./validate-odbc-connection-string 'abc=de[f' 2>&1 -./validate-odbc-connection-string 'abc={de[f}' 2>&1 -./validate-odbc-connection-string 'abc={de[f};dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}}f};dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc=de}}f};dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}}f;dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}f;dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}f;dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}f};dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}}f};dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}}f;dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={de}} ;dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={ } ;dsn=hello' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn=hello ' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn=hello world ' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ...' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;...' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;=' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_=' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= ' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {}' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {}}}' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {...................................................................}' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {....................................................................................}' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {.....................................................................................................}' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {...}; FILEDSN=x' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {...}; FileDsn = x' 2>&1 -./validate-odbc-connection-string 'abc={ } ; dsn = {hello world} ;_= {...}; Driver=x' 2>&1 -./validate-odbc-connection-string 'abc={}; abc=def' 2>&1 -./validate-odbc-connection-string 'abc={};;' 2>&1 -./validate-odbc-connection-string 'DSN=myconnection' 2>&1 -./validate-odbc-connection-string 'DSN=myconnection;UID=username;PWD=password;HOST=127.0.0.1;PORT=5432;DATABASE=my_db' 2>&1 -./validate-odbc-connection-string 'DSN=MSSQL;UID=test;PWD=test' 2>&1 diff --git a/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp b/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp deleted file mode 100644 index a817a01c288..00000000000 --- a/dbms/programs/odbc-bridge/validateODBCConnectionString.cpp +++ /dev/null @@ -1,241 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include "validateODBCConnectionString.h" - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ODBC_CONNECTION_STRING; -} - - -std::string validateODBCConnectionString(const std::string & connection_string) -{ - /// Connection string is a list of name, value pairs. - /// name and value are separated by '='. - /// names are case insensitive. - /// name=value pairs are sepated by ';'. - /// ASCII whitespace characters are skipped before and after delimiters. - /// value may be optionally enclosed by {} - /// in enclosed value, } is escaped as }}. - /// - /// Example: PWD={a}}b} means that password is a}b - /// - /// https://docs.microsoft.com/en-us/sql/odbc/reference/syntax/sqldriverconnect-function?view=sql-server-2017#comments - - /// unixODBC has fixed size buffers on stack and has buffer overflow bugs. - /// We will limit string sizes to small values. - - static constexpr size_t MAX_ELEMENT_SIZE = 100; - static constexpr size_t MAX_CONNECTION_STRING_SIZE = 1000; - - if (connection_string.empty()) - throw Exception("ODBC connection string cannot be empty", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - if (connection_string.size() >= MAX_CONNECTION_STRING_SIZE) - throw Exception("ODBC connection string is too long", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - const char * pos = connection_string.data(); - const char * end = pos + connection_string.size(); - - auto skip_whitespaces = [&] - { - while (pos < end && isWhitespaceASCII(*pos)) - { - if (*pos != ' ') - throw Exception("ODBC connection string parameter contains unusual whitespace character", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - ++pos; - } - }; - - auto read_name = [&] - { - const char * begin = pos; - - if (pos < end && isValidIdentifierBegin(*pos)) - ++pos; - else - throw Exception("ODBC connection string parameter name doesn't begin with valid identifier character", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - while (pos < end && isWordCharASCII(*pos)) - ++pos; - - return std::string(begin, pos); - }; - - auto read_plain_value = [&] - { - const char * begin = pos; - - while (pos < end && *pos != ';' && !isWhitespaceASCII(*pos)) - { - signed char c = *pos; - if (c < 32 || strchr("[]{}(),;?*=!@'\"", c) != nullptr) - throw Exception("ODBC connection string parameter value is unescaped and contains illegal character", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - ++pos; - } - - return std::string(begin, pos); - }; - - auto read_escaped_value = [&] - { - std::string res; - - if (pos < end && *pos == '{') - ++pos; - else - throw Exception("ODBC connection string parameter value doesn't begin with opening curly brace", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - while (pos < end) - { - if (*pos == '}') - { - ++pos; - if (pos >= end || *pos != '}') - return res; - } - - if (*pos == 0) - throw Exception("ODBC connection string parameter value contains ASCII NUL character", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - res += *pos; - ++pos; - } - - throw Exception("ODBC connection string parameter is escaped but there is no closing curly brace", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - }; - - auto read_value = [&] - { - if (pos >= end) - return std::string{}; - - if (*pos == '{') - return read_escaped_value(); - else - return read_plain_value(); - }; - - std::map parameters; - - while (pos < end) - { - skip_whitespaces(); - std::string name = read_name(); - skip_whitespaces(); - - Poco::toUpperInPlace(name); - if (name == "FILEDSN" || name == "SAVEFILE" || name == "DRIVER") - throw Exception("ODBC connection string has forbidden parameter", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - if (pos >= end) - throw Exception("ODBC connection string parameter doesn't have value", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - if (*pos == '=') - ++pos; - else - throw Exception("ODBC connection string parameter doesn't have value", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - skip_whitespaces(); - std::string value = read_value(); - skip_whitespaces(); - - if (name.size() > MAX_ELEMENT_SIZE || value.size() > MAX_ELEMENT_SIZE) - throw Exception("ODBC connection string has too long keyword or value", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - if (!parameters.emplace(name, value).second) - throw Exception("Duplicate parameter found in ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - if (pos >= end) - break; - - if (*pos == ';') - ++pos; - else - throw Exception("Unexpected character found after parameter value in ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - } - - /// Reconstruct the connection string. - - auto it = parameters.find("DSN"); - - if (parameters.end() == it) - throw Exception("DSN parameter is mandatory for ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - std::string dsn = it->second; - - if (dsn.empty()) - throw Exception("DSN parameter cannot be empty in ODBC connection string", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - parameters.erase(it); - - std::string reconstructed_connection_string; - - auto write_plain_value = [&](const std::string & value) - { - reconstructed_connection_string += value; - }; - - auto write_escaped_value = [&](const std::string & value) - { - reconstructed_connection_string += '{'; - - const char * value_pos = value.data(); - const char * value_end = value_pos + value.size(); - while (true) - { - const char * next_pos = find_first_symbols<'}'>(value_pos, value_end); - - if (next_pos == value_end) - { - reconstructed_connection_string.append(value_pos, next_pos - value_pos); - break; - } - else - { - reconstructed_connection_string.append(value_pos, next_pos - value_pos); - reconstructed_connection_string.append("}}"); - value_pos = next_pos + 1; - } - } - - reconstructed_connection_string += '}'; - }; - - auto write_value = [&](const std::string & value) - { - if (std::all_of(value.begin(), value.end(), isWordCharASCII)) - write_plain_value(value); - else - write_escaped_value(value); - }; - - auto write_element = [&](const std::string & name, const std::string & value) - { - reconstructed_connection_string.append(name); - reconstructed_connection_string += '='; - write_value(value); - reconstructed_connection_string += ';'; - }; - - /// Place DSN first because that's more safe. - write_element("DSN", dsn); - for (const auto & elem : parameters) - write_element(elem.first, elem.second); - - if (reconstructed_connection_string.size() >= MAX_CONNECTION_STRING_SIZE) - throw Exception("ODBC connection string is too long", ErrorCodes::BAD_ODBC_CONNECTION_STRING); - - return reconstructed_connection_string; -} - -} diff --git a/dbms/programs/odbc-bridge/validateODBCConnectionString.h b/dbms/programs/odbc-bridge/validateODBCConnectionString.h deleted file mode 100644 index f0f93b1de65..00000000000 --- a/dbms/programs/odbc-bridge/validateODBCConnectionString.h +++ /dev/null @@ -1,21 +0,0 @@ -#pragma once - -#include - - -namespace DB -{ - -/** Passing arbitary connection string to ODBC Driver Manager is insecure, for the following reasons: - * 1. Driver Manager like unixODBC has multiple bugs like buffer overflow. - * 2. Driver Manager can interpret some parameters as a path to library for dlopen or a file to read, - * thus allows arbitary remote code execution. - * - * This function will throw exception if connection string has insecure parameters. - * It may also modify connection string to harden it. - * - * Note that it is intended for ANSI (not multibyte) variant of connection string. - */ -std::string validateODBCConnectionString(const std::string & connection_string); - -} diff --git a/dbms/programs/performance-test/PerformanceTest.cpp b/dbms/programs/performance-test/PerformanceTest.cpp index cf55173ad3a..1f7421566a4 100644 --- a/dbms/programs/performance-test/PerformanceTest.cpp +++ b/dbms/programs/performance-test/PerformanceTest.cpp @@ -521,12 +521,11 @@ public: std::string name; UInt64 version_major; UInt64 version_minor; - UInt64 version_patch; UInt64 version_revision; - connection.getServerVersion(name, version_major, version_minor, version_patch, version_revision); + connection.getServerVersion(name, version_major, version_minor, version_revision); std::stringstream ss; - ss << version_major << "." << version_minor << "." << version_patch; + ss << version_major << "." << version_minor << "." << version_revision; server_version = ss.str(); processTestsConfigurations(input_files); diff --git a/dbms/programs/server/CMakeLists.txt b/dbms/programs/server/CMakeLists.txt index c146f40d281..74297d29864 100644 --- a/dbms/programs/server/CMakeLists.txt +++ b/dbms/programs/server/CMakeLists.txt @@ -19,7 +19,7 @@ if (CLICKHOUSE_SPLIT_BINARY) install (TARGETS clickhouse-server ${CLICKHOUSE_ALL_TARGETS} RUNTIME DESTINATION ${CMAKE_INSTALL_BINDIR} COMPONENT clickhouse) endif () -if (OS_LINUX) +if (NOT APPLE AND NOT ARCH_FREEBSD) set (GLIBC_MAX_REQUIRED 2.4) add_test(NAME GLIBC_required_version COMMAND bash -c "readelf -s ${CMAKE_CURRENT_BINARY_DIR}/../clickhouse-server | grep '@GLIBC' | grep -oP 'GLIBC_[\\d\\.]+' | sort | uniq | sort -r | perl -lnE 'exit 1 if $_ gt q{GLIBC_${GLIBC_MAX_REQUIRED}}'") endif () diff --git a/dbms/programs/server/HTTPHandler.cpp b/dbms/programs/server/HTTPHandler.cpp index 0db338e0fbf..f2a114c4702 100644 --- a/dbms/programs/server/HTTPHandler.cpp +++ b/dbms/programs/server/HTTPHandler.cpp @@ -9,11 +9,10 @@ #include -#include +#include #include #include #include -#include #include #include #include @@ -209,13 +208,6 @@ void HTTPHandler::processQuery( Poco::Net::HTTPServerResponse & response, Output & used_output) { - Context context = server.context(); - context.setGlobalContext(server.context()); - - /// It will forcibly detach query even if unexpected error ocurred and detachQuery() was not called - /// Normal detaching is happen in BlockIO callbacks - CurrentThread::QueryScope query_scope_holder(context); - LOG_TRACE(log, "Request URI: " << request.getURI()); std::istream & istr = request.stream(); @@ -265,9 +257,14 @@ void HTTPHandler::processQuery( } std::string query_id = params.get("query_id", ""); + + const auto & config = server.config(); + + Context context = server.context(); + context.setGlobalContext(server.context()); + context.setUser(user, password, request.clientAddress(), quota_key); context.setCurrentQueryId(query_id); - CurrentThread::attachQueryContext(context); /// The user could specify session identifier and session timeout. /// It allows to modify settings, create temporary tables and reuse them in subsequent requests. @@ -276,7 +273,6 @@ void HTTPHandler::processQuery( String session_id; std::chrono::steady_clock::duration session_timeout; bool session_is_set = params.has("session_id"); - const auto & config = server.config(); if (session_is_set) { @@ -425,45 +421,34 @@ void HTTPHandler::processQuery( std::unique_ptr in; - static const NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", - "buffer_size", "wait_end_of_query", "session_id", "session_timeout", "session_check"}; - - Names reserved_param_suffixes; - - auto param_could_be_skipped = [&] (const String & name) - { - if (reserved_param_names.count(name)) - return true; - - for (const String & suffix : reserved_param_suffixes) - { - if (endsWith(name, suffix)) - return true; - } - - return false; - }; - - /// Used in case of POST request with form-data, but it isn't expected to be deleted after that scope. + // Used in case of POST request with form-data, but it not to be expectd to be deleted after that scope std::string full_query; /// Support for "external data for query processing". if (startsWith(request.getContentType().data(), "multipart/form-data")) { ExternalTablesHandler handler(context, params); - params.load(request, istr, handler); - - /// Skip unneeded parameters to avoid confusing them later with context settings or query parameters. - reserved_param_suffixes.emplace_back("_format"); - reserved_param_suffixes.emplace_back("_types"); - reserved_param_suffixes.emplace_back("_structure"); /// Params are of both form params POST and uri (GET params) - for (const auto & it : params) - if (it.first == "query") - full_query += it.second; + params.load(request, istr, handler); + for (const auto & it : params) + { + if (it.first == "query") + { + full_query += it.second; + } + } in = std::make_unique(full_query); + + /// Erase unneeded parameters to avoid confusing them later with context settings or query + /// parameters. + for (const auto & it : handler.names) + { + params.erase(it + "_format"); + params.erase(it + "_types"); + params.erase(it + "_structure"); + } } else in = std::make_unique(*in_param, *in_post_maybe_compressed); @@ -490,6 +475,11 @@ void HTTPHandler::processQuery( auto readonly_before_query = settings.readonly; + NameSet reserved_param_names{"query", "compress", "decompress", "user", "password", "quota_key", "query_id", "stacktrace", + "buffer_size", "wait_end_of_query", + "session_id", "session_timeout", "session_check" + }; + for (auto it = params.begin(); it != params.end(); ++it) { if (it->first == "database") @@ -500,7 +490,7 @@ void HTTPHandler::processQuery( { context.setDefaultFormat(it->second); } - else if (param_could_be_skipped(it->first)) + else if (reserved_param_names.find(it->first) != reserved_param_names.end()) { } else diff --git a/dbms/programs/server/MetricsTransmitter.cpp b/dbms/programs/server/MetricsTransmitter.cpp index ed87eb5cf9f..278347a6774 100644 --- a/dbms/programs/server/MetricsTransmitter.cpp +++ b/dbms/programs/server/MetricsTransmitter.cpp @@ -81,7 +81,7 @@ void MetricsTransmitter::transmit(std::vector & prev_count { for (size_t i = 0, end = ProfileEvents::end(); i < end; ++i) { - const auto counter = ProfileEvents::global_counters[i].load(std::memory_order_relaxed); + const auto counter = ProfileEvents::counters[i].load(std::memory_order_relaxed); const auto counter_increment = counter - prev_counters[i]; prev_counters[i] = counter; diff --git a/dbms/programs/server/config.d/listen.xml b/dbms/programs/server/config.d/listen.xml deleted file mode 100644 index 24c64bbb60a..00000000000 --- a/dbms/programs/server/config.d/listen.xml +++ /dev/null @@ -1 +0,0 @@ -0.0.0.0 \ No newline at end of file diff --git a/dbms/programs/server/config.xml b/dbms/programs/server/config.xml index e461d49d522..7dd7a00517e 100644 --- a/dbms/programs/server/config.xml +++ b/dbms/programs/server/config.xml @@ -322,12 +322,10 @@ - diff --git a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp index 7876f0dcffb..353b5a213b3 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp +++ b/dbms/src/AggregateFunctions/AggregateFunctionFactory.cpp @@ -13,7 +13,6 @@ #include #include -#include namespace DB @@ -42,20 +41,6 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre ErrorCodes::LOGICAL_ERROR); } -static DataTypes convertTypesWithDictionaryToNested(const DataTypes & types) -{ - DataTypes res_types; - res_types.reserve(types.size()); - for (const auto & type : types) - { - if (auto * type_with_dict = typeid_cast(type.get())) - res_types.push_back(type_with_dict->getDictionaryType()); - else - res_types.push_back(type); - } - - return res_types; -} AggregateFunctionPtr AggregateFunctionFactory::get( const String & name, @@ -63,8 +48,6 @@ AggregateFunctionPtr AggregateFunctionFactory::get( const Array & parameters, int recursion_level) const { - auto type_without_dictionary = convertTypesWithDictionaryToNested(argument_types); - /// If one of types is Nullable, we apply aggregate function combinator "Null". if (std::any_of(argument_types.begin(), argument_types.end(), @@ -74,7 +57,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( if (!combinator) throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR); - DataTypes nested_types = combinator->transformArguments(type_without_dictionary); + DataTypes nested_types = combinator->transformArguments(argument_types); AggregateFunctionPtr nested_function; @@ -87,7 +70,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get( return combinator->transformAggregateFunction(nested_function, argument_types, parameters); } - auto res = getImpl(name, type_without_dictionary, parameters, recursion_level); + auto res = getImpl(name, argument_types, parameters, recursion_level); if (!res) throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR); return res; diff --git a/dbms/src/AggregateFunctions/QuantileExact.h b/dbms/src/AggregateFunctions/QuantileExact.h index 86be2f84921..568ad8d0950 100644 --- a/dbms/src/AggregateFunctions/QuantileExact.h +++ b/dbms/src/AggregateFunctions/QuantileExact.h @@ -77,7 +77,7 @@ struct QuantileExact return array[n]; } - return std::numeric_limits::quiet_NaN(); + return Value(); } /// Get the `size` values of `levels` quantiles. Write `size` results starting with `result` address. diff --git a/dbms/src/AggregateFunctions/QuantileExactWeighted.h b/dbms/src/AggregateFunctions/QuantileExactWeighted.h index eca89f30aa9..76e65f07dac 100644 --- a/dbms/src/AggregateFunctions/QuantileExactWeighted.h +++ b/dbms/src/AggregateFunctions/QuantileExactWeighted.h @@ -72,7 +72,7 @@ struct QuantileExactWeighted size_t size = map.size(); if (0 == size) - return std::numeric_limits::quiet_NaN(); + return Value(); /// Copy the data to a temporary array to get the element you need in order. using Pair = typename Map::value_type; diff --git a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp index 3517ad57a73..5efb1e5bf1b 100644 --- a/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp +++ b/dbms/src/AggregateFunctions/registerAggregateFunctions.cpp @@ -34,7 +34,6 @@ void registerAggregateFunctionCombinatorMerge(AggregateFunctionCombinatorFactory void registerAggregateFunctionCombinatorNull(AggregateFunctionCombinatorFactory &); void registerAggregateFunctionHistogram(AggregateFunctionFactory & factory); -void registerAggregateFunctionRetention(AggregateFunctionFactory & factory); void registerAggregateFunctions() { @@ -60,7 +59,6 @@ void registerAggregateFunctions() registerAggregateFunctionsBitwise(factory); registerAggregateFunctionsMaxIntersections(factory); registerAggregateFunctionHistogram(factory); - registerAggregateFunctionRetention(factory); } { diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 1ed186085f7..b847d905054 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -89,7 +89,7 @@ void Connection::connect() LOG_TRACE(log_wrapper.get(), "Connected to " << server_name << " server version " << server_version_major << "." << server_version_minor - << "." << server_version_patch + << "." << server_revision << "."); } catch (Poco::Net::NetException & e) @@ -114,7 +114,6 @@ void Connection::disconnect() //LOG_TRACE(log_wrapper.get(), "Disconnecting"); in = nullptr; - last_input_packet_type.reset(); out = nullptr; // can write to socket if (socket) socket->close(); @@ -151,7 +150,6 @@ void Connection::sendHello() writeStringBinary((DBMS_NAME " ") + client_name, *out); writeVarUInt(DBMS_VERSION_MAJOR, *out); writeVarUInt(DBMS_VERSION_MINOR, *out); - // NOTE For backward compatibility of the protocol, client cannot send its version_patch. writeVarUInt(ClickHouseRevision::get(), *out); writeStringBinary(default_database, *out); writeStringBinary(user, *out); @@ -176,13 +174,13 @@ void Connection::receiveHello() readVarUInt(server_version_minor, *in); readVarUInt(server_revision, *in); if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_TIMEZONE) + { readStringBinary(server_timezone, *in); + } if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME) + { readStringBinary(server_display_name, *in); - if (server_revision >= DBMS_MIN_REVISION_WITH_VERSION_PATCH) - readVarUInt(server_version_patch, *in); - else - server_version_patch = server_revision; + } } else if (packet_type == Protocol::Server::Exception) receiveException()->rethrow(); @@ -219,7 +217,7 @@ UInt16 Connection::getPort() const return port; } -void Connection::getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & version_patch, UInt64 & revision) +void Connection::getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & revision) { if (!connected) connect(); @@ -227,7 +225,6 @@ void Connection::getServerVersion(String & name, UInt64 & version_major, UInt64 name = server_name; version_major = server_version_major; version_minor = server_version_minor; - version_patch = server_version_patch; revision = server_revision; } @@ -380,7 +377,6 @@ void Connection::sendQuery( maybe_compressed_in.reset(); maybe_compressed_out.reset(); block_in.reset(); - block_logs_in.reset(); block_out.reset(); /// Send empty block which means end of data. @@ -508,50 +504,20 @@ bool Connection::poll(size_t timeout_microseconds) } -bool Connection::hasReadPendingData() const +bool Connection::hasReadBufferPendingData() const { - return last_input_packet_type.has_value() || static_cast(*in).hasPendingData(); -} - - -std::optional Connection::checkPacket(size_t timeout_microseconds) -{ - if (last_input_packet_type.has_value()) - return last_input_packet_type; - - if (hasReadPendingData() || poll(timeout_microseconds)) - { - // LOG_TRACE(log_wrapper.get(), "Receiving packet type"); - UInt64 packet_type; - readVarUInt(packet_type, *in); - - last_input_packet_type.emplace(packet_type); - return last_input_packet_type; - } - - return {}; + return static_cast(*in).hasPendingData(); } Connection::Packet Connection::receivePacket() { + //LOG_TRACE(log_wrapper.get(), "Receiving packet"); + try { Packet res; - - /// Have we already read packet type? - if (last_input_packet_type) - { - res.type = *last_input_packet_type; - last_input_packet_type.reset(); - } - else - { - //LOG_TRACE(log_wrapper.get(), "Receiving packet type"); - readVarUInt(res.type, *in); - } - - //LOG_TRACE(log_wrapper.get(), "Receiving packet " << res.type << " " << Protocol::Server::toString(res.type)); + readVarUInt(res.type, *in); switch (res.type) { @@ -581,10 +547,6 @@ Connection::Packet Connection::receivePacket() res.block = receiveData(); return res; - case Protocol::Server::Log: - res.block = receiveLogData(); - return res; - case Protocol::Server::EndOfStream: return res; @@ -612,26 +574,14 @@ Block Connection::receiveData() //LOG_TRACE(log_wrapper.get(), "Receiving data"); initBlockInput(); - return receiveDataImpl(block_in); -} - -Block Connection::receiveLogData() -{ - initBlockLogsInput(); - return receiveDataImpl(block_logs_in); -} - - -Block Connection::receiveDataImpl(BlockInputStreamPtr & stream) -{ String external_table_name; readStringBinary(external_table_name, *in); size_t prev_bytes = in->count(); /// Read one block from network. - Block res = stream->read(); + Block res = block_in->read(); if (throttler) throttler->add(in->count() - prev_bytes); @@ -640,39 +590,20 @@ Block Connection::receiveDataImpl(BlockInputStreamPtr & stream) } -void Connection::initInputBuffers() -{ - -} - - void Connection::initBlockInput() { if (!block_in) { - if (!maybe_compressed_in) - { - if (compression == Protocol::Compression::Enable) - maybe_compressed_in = std::make_shared(*in); - else - maybe_compressed_in = in; - } + if (compression == Protocol::Compression::Enable) + maybe_compressed_in = std::make_shared(*in); + else + maybe_compressed_in = in; block_in = std::make_shared(*maybe_compressed_in, server_revision); } } -void Connection::initBlockLogsInput() -{ - if (!block_logs_in) - { - /// Have to return superset of SystemLogsQueue::getSampleBlock() columns - block_logs_in = std::make_shared(*in, server_revision); - } -} - - void Connection::setDescription() { auto resolved_address = getResolvedAddress(); diff --git a/dbms/src/Client/Connection.h b/dbms/src/Client/Connection.h index ad98df3cc8f..2da794b8434 100644 --- a/dbms/src/Client/Connection.h +++ b/dbms/src/Client/Connection.h @@ -23,7 +23,6 @@ #include #include -#include namespace DB @@ -105,7 +104,7 @@ public: /// Change default database. Changes will take effect on next reconnect. void setDefaultDatabase(const String & database); - void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & version_patch, UInt64 & revision); + void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & revision); const String & getServerTimezone(); const String & getServerDisplayName(); @@ -139,10 +138,7 @@ public: bool poll(size_t timeout_microseconds = 0); /// Check, if has data in read buffer. - bool hasReadPendingData() const; - - /// Checks if there is input data in connection and reads packet ID. - std::optional checkPacket(size_t timeout_microseconds = 0); + bool hasReadBufferPendingData() const; /// Receive packet from server. Packet receivePacket(); @@ -191,7 +187,6 @@ private: String server_name; UInt64 server_version_major = 0; UInt64 server_version_minor = 0; - UInt64 server_version_patch = 0; UInt64 server_revision = 0; String server_timezone; String server_display_name; @@ -199,7 +194,6 @@ private: std::unique_ptr socket; std::shared_ptr in; std::shared_ptr out; - std::optional last_input_packet_type; String query_id; Protocol::Compression compression; /// Enable data compression for communication. @@ -219,7 +213,6 @@ private: /// From where to read query execution result. std::shared_ptr maybe_compressed_in; BlockInputStreamPtr block_in; - BlockInputStreamPtr block_logs_in; /// Where to write data for INSERT. std::shared_ptr maybe_compressed_out; @@ -255,16 +248,11 @@ private: bool ping(); Block receiveData(); - Block receiveLogData(); - Block receiveDataImpl(BlockInputStreamPtr & stream); - std::unique_ptr receiveException(); Progress receiveProgress(); BlockStreamProfileInfo receiveProfileInfo(); - void initInputBuffers(); void initBlockInput(); - void initBlockLogsInput(); void throwUnexpectedPacket(UInt64 packet_type, const char * expected) const; }; diff --git a/dbms/src/Client/ConnectionPoolWithFailover.cpp b/dbms/src/Client/ConnectionPoolWithFailover.cpp index 8c2f1a07bc1..a311dac95b1 100644 --- a/dbms/src/Client/ConnectionPoolWithFailover.cpp +++ b/dbms/src/Client/ConnectionPoolWithFailover.cpp @@ -156,10 +156,9 @@ ConnectionPoolWithFailover::tryGetEntry( String server_name; UInt64 server_version_major; UInt64 server_version_minor; - UInt64 server_version_patch; UInt64 server_revision; if (table_to_check) - result.entry->getServerVersion(server_name, server_version_major, server_version_minor, server_version_patch, server_revision); + result.entry->getServerVersion(server_name, server_version_major, server_version_minor, server_revision); if (!table_to_check || server_revision < DBMS_MIN_REVISION_WITH_TABLES_STATUS) { @@ -226,6 +225,6 @@ ConnectionPoolWithFailover::tryGetEntry( } } return result; -} +}; } diff --git a/dbms/src/Client/MultiplexedConnections.cpp b/dbms/src/Client/MultiplexedConnections.cpp index 3e88a20caa3..8fe27ecf7fa 100644 --- a/dbms/src/Client/MultiplexedConnections.cpp +++ b/dbms/src/Client/MultiplexedConnections.cpp @@ -247,7 +247,6 @@ Connection::Packet MultiplexedConnections::receivePacketUnlocked() case Protocol::Server::ProfileInfo: case Protocol::Server::Totals: case Protocol::Server::Extremes: - case Protocol::Server::Log: break; case Protocol::Server::EndOfStream: @@ -277,7 +276,7 @@ MultiplexedConnections::ReplicaState & MultiplexedConnections::getReplicaForRead for (const ReplicaState & state : replica_states) { Connection * connection = state.connection; - if ((connection != nullptr) && connection->hasReadPendingData()) + if ((connection != nullptr) && connection->hasReadBufferPendingData()) read_list.push_back(*connection->socket); } diff --git a/dbms/src/Columns/Collator.cpp b/dbms/src/Columns/Collator.cpp index 507ae9f54c9..aaf917fb93d 100644 --- a/dbms/src/Columns/Collator.cpp +++ b/dbms/src/Columns/Collator.cpp @@ -3,7 +3,10 @@ #include #if USE_ICU + #pragma GCC diagnostic push + #pragma GCC diagnostic ignored "-Wold-style-cast" #include + #pragma GCC diagnostic pop #else #ifdef __clang__ #pragma clang diagnostic push diff --git a/dbms/src/Columns/ColumnAggregateFunction.cpp b/dbms/src/Columns/ColumnAggregateFunction.cpp index 80b4b95fd4a..b989c007e56 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.cpp +++ b/dbms/src/Columns/ColumnAggregateFunction.cpp @@ -4,7 +4,6 @@ #include #include #include -#include namespace DB { @@ -162,25 +161,6 @@ ColumnPtr ColumnAggregateFunction::permute(const Permutation & perm, size_t limi return std::move(res); } -ColumnPtr ColumnAggregateFunction::index(const IColumn & indexes, size_t limit) const -{ - return selectIndexImpl(*this, indexes, limit); -} - -template -ColumnPtr ColumnAggregateFunction::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - auto res = createView(); - - res->getData().resize(limit); - for (size_t i = 0; i < limit; ++i) - res->getData()[i] = getData()[indexes[i]]; - - return std::move(res); -} - -INSTANTIATE_INDEX_IMPL(ColumnAggregateFunction); - /// Is required to support operations with Set void ColumnAggregateFunction::updateHashWithValue(size_t n, SipHash & hash) const { diff --git a/dbms/src/Columns/ColumnAggregateFunction.h b/dbms/src/Columns/ColumnAggregateFunction.h index 05c3698d611..27b6537a485 100644 --- a/dbms/src/Columns/ColumnAggregateFunction.h +++ b/dbms/src/Columns/ColumnAggregateFunction.h @@ -156,11 +156,6 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; - - template - ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; - ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; diff --git a/dbms/src/Columns/ColumnArray.cpp b/dbms/src/Columns/ColumnArray.cpp index e3808a73d55..010569e40ad 100644 --- a/dbms/src/Columns/ColumnArray.cpp +++ b/dbms/src/Columns/ColumnArray.cpp @@ -626,44 +626,6 @@ ColumnPtr ColumnArray::permute(const Permutation & perm, size_t limit) const return std::move(res); } -ColumnPtr ColumnArray::index(const IColumn & indexes, size_t limit) const -{ - return selectIndexImpl(*this, indexes, limit); -} - -template -ColumnPtr ColumnArray::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - if (limit == 0) - return ColumnArray::create(data); - - /// Convert indexes to UInt64 in case of overflow. - auto nested_indexes_column = ColumnUInt64::create(); - PaddedPODArray & nested_indexes = nested_indexes_column->getData(); - nested_indexes.reserve(getOffsets().back()); - - auto res = ColumnArray::create(data->cloneEmpty()); - - Offsets & res_offsets = res->getOffsets(); - res_offsets.resize(limit); - size_t current_offset = 0; - - for (size_t i = 0; i < limit; ++i) - { - for (size_t j = 0; j < sizeAt(indexes[i]); ++j) - nested_indexes.push_back(offsetAt(indexes[i]) + j); - current_offset += sizeAt(indexes[i]); - res_offsets[i] = current_offset; - } - - if (current_offset != 0) - res->data = data->index(*nested_indexes_column, current_offset); - - return std::move(res); -} - -INSTANTIATE_INDEX_IMPL(ColumnArray); - void ColumnArray::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const { size_t s = size(); diff --git a/dbms/src/Columns/ColumnArray.h b/dbms/src/Columns/ColumnArray.h index 642d3da2327..ce7744b1fc1 100644 --- a/dbms/src/Columns/ColumnArray.h +++ b/dbms/src/Columns/ColumnArray.h @@ -71,8 +71,6 @@ public: void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; - template ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; int compareAt(size_t n, size_t m, const IColumn & rhs_, int nan_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; void reserve(size_t n) override; diff --git a/dbms/src/Columns/ColumnConst.cpp b/dbms/src/Columns/ColumnConst.cpp index 95c21786484..325160e5fc5 100644 --- a/dbms/src/Columns/ColumnConst.cpp +++ b/dbms/src/Columns/ColumnConst.cpp @@ -30,11 +30,6 @@ ColumnPtr ColumnConst::convertToFullColumn() const return data->replicate(Offsets(1, s)); } -ColumnPtr ColumnConst::removeLowCardinality() const -{ - return ColumnConst::create(data->convertToFullColumnIfWithDictionary(), s); -} - ColumnPtr ColumnConst::filter(const Filter & filt, ssize_t /*result_size_hint*/) const { if (s != filt.size()) @@ -68,18 +63,6 @@ ColumnPtr ColumnConst::permute(const Permutation & perm, size_t limit) const return ColumnConst::create(data, limit); } -ColumnPtr ColumnConst::index(const IColumn & indexes, size_t limit) const -{ - if (limit == 0) - limit = indexes.size(); - - if (indexes.size() < limit) - throw Exception("Size of indexes (" + toString(indexes.size()) + ") is less than required (" + toString(limit) + ")", - ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - return ColumnConst::create(data, limit); -} - MutableColumns ColumnConst::scatter(ColumnIndex num_columns, const Selector & selector) const { if (s != selector.size()) diff --git a/dbms/src/Columns/ColumnConst.h b/dbms/src/Columns/ColumnConst.h index bd40246b1e5..d0cdaf97018 100644 --- a/dbms/src/Columns/ColumnConst.h +++ b/dbms/src/Columns/ColumnConst.h @@ -36,8 +36,6 @@ public: return convertToFullColumn(); } - ColumnPtr removeLowCardinality() const; - std::string getName() const override { return "Const(" + data->getName() + ")"; @@ -155,7 +153,6 @@ public: ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; void getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const override; size_t byteSize() const override diff --git a/dbms/src/Columns/ColumnFixedString.cpp b/dbms/src/Columns/ColumnFixedString.cpp index d110ef10a84..2da3c28e50a 100644 --- a/dbms/src/Columns/ColumnFixedString.cpp +++ b/dbms/src/Columns/ColumnFixedString.cpp @@ -1,5 +1,4 @@ #include -#include #include #include @@ -259,32 +258,6 @@ ColumnPtr ColumnFixedString::permute(const Permutation & perm, size_t limit) con return std::move(res); } - -ColumnPtr ColumnFixedString::index(const IColumn & indexes, size_t limit) const -{ - return selectIndexImpl(*this, indexes, limit); -} - - -template -ColumnPtr ColumnFixedString::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - if (limit == 0) - return ColumnFixedString::create(n); - - auto res = ColumnFixedString::create(n); - - Chars_t & res_chars = res->chars; - - res_chars.resize(n * limit); - - size_t offset = 0; - for (size_t i = 0; i < limit; ++i, offset += n) - memcpySmallAllowReadWriteOverflow15(&res_chars[offset], &chars[indexes[i] * n], n); - - return std::move(res); -} - ColumnPtr ColumnFixedString::replicate(const Offsets & offsets) const { size_t col_size = size(); diff --git a/dbms/src/Columns/ColumnFixedString.h b/dbms/src/Columns/ColumnFixedString.h index e8ce153e1a1..40de17dfcce 100644 --- a/dbms/src/Columns/ColumnFixedString.h +++ b/dbms/src/Columns/ColumnFixedString.h @@ -108,11 +108,6 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; - - template - ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; - ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override diff --git a/dbms/src/Columns/ColumnFunction.cpp b/dbms/src/Columns/ColumnFunction.cpp index 852f6f03adf..f03b62dc7ed 100644 --- a/dbms/src/Columns/ColumnFunction.cpp +++ b/dbms/src/Columns/ColumnFunction.cpp @@ -88,15 +88,6 @@ ColumnPtr ColumnFunction::permute(const Permutation & perm, size_t limit) const return ColumnFunction::create(limit, function, capture); } -ColumnPtr ColumnFunction::index(const IColumn & indexes, size_t limit) const -{ - ColumnsWithTypeAndName capture = captured_columns; - for (auto & column : capture) - column.column = column.column->index(indexes, limit); - - return ColumnFunction::create(limit, function, capture); -} - std::vector ColumnFunction::scatter(IColumn::ColumnIndex num_columns, const IColumn::Selector & selector) const { diff --git a/dbms/src/Columns/ColumnFunction.h b/dbms/src/Columns/ColumnFunction.h index 458167ffefe..d10459175cc 100644 --- a/dbms/src/Columns/ColumnFunction.h +++ b/dbms/src/Columns/ColumnFunction.h @@ -33,7 +33,6 @@ public: ColumnPtr replicate(const Offsets & offsets) const override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; void insertDefault() override; void popBack(size_t n) override; std::vector scatter(IColumn::ColumnIndex num_columns, diff --git a/dbms/src/Columns/ColumnNullable.cpp b/dbms/src/Columns/ColumnNullable.cpp index 5235bd23d04..557a57b3c6d 100644 --- a/dbms/src/Columns/ColumnNullable.cpp +++ b/dbms/src/Columns/ColumnNullable.cpp @@ -166,13 +166,6 @@ ColumnPtr ColumnNullable::permute(const Permutation & perm, size_t limit) const return ColumnNullable::create(permuted_data, permuted_null_map); } -ColumnPtr ColumnNullable::index(const IColumn & indexes, size_t limit) const -{ - ColumnPtr indexed_data = getNestedColumn().index(indexes, limit); - ColumnPtr indexed_null_map = getNullMapColumn().index(indexes, limit); - return ColumnNullable::create(indexed_data, indexed_null_map); -} - int ColumnNullable::compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const { /// NULL values share the properties of NaN values. diff --git a/dbms/src/Columns/ColumnNullable.h b/dbms/src/Columns/ColumnNullable.h index c8453a29689..8716b2272ff 100644 --- a/dbms/src/Columns/ColumnNullable.h +++ b/dbms/src/Columns/ColumnNullable.h @@ -65,7 +65,6 @@ public: void popBack(size_t n) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; int compareAt(size_t n, size_t m, const IColumn & rhs_, int null_direction_hint) const override; void getPermutation(bool reverse, size_t limit, int null_direction_hint, Permutation & res) const override; void reserve(size_t n) override; diff --git a/dbms/src/Columns/ColumnString.cpp b/dbms/src/Columns/ColumnString.cpp index 488c8dd52bc..5ba5d49ed81 100644 --- a/dbms/src/Columns/ColumnString.cpp +++ b/dbms/src/Columns/ColumnString.cpp @@ -159,47 +159,6 @@ ColumnPtr ColumnString::permute(const Permutation & perm, size_t limit) const } -ColumnPtr ColumnString::index(const IColumn & indexes, size_t limit) const -{ - return selectIndexImpl(*this, indexes, limit); -} - -template -ColumnPtr ColumnString::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - if (limit == 0) - return ColumnString::create(); - - auto res = ColumnString::create(); - - Chars_t & res_chars = res->chars; - Offsets & res_offsets = res->offsets; - - size_t new_chars_size = 0; - for (size_t i = 0; i < limit; ++i) - new_chars_size += sizeAt(indexes[i]); - res_chars.resize(new_chars_size); - - res_offsets.resize(limit); - - Offset current_new_offset = 0; - - for (size_t i = 0; i < limit; ++i) - { - size_t j = indexes[i]; - size_t string_offset = j == 0 ? 0 : offsets[j - 1]; - size_t string_size = offsets[j] - string_offset; - - memcpySmallAllowReadWriteOverflow15(&res_chars[current_new_offset], &chars[string_offset], string_size); - - current_new_offset += string_size; - res_offsets[i] = current_new_offset; - } - - return std::move(res); -} - - template struct ColumnString::less { diff --git a/dbms/src/Columns/ColumnString.h b/dbms/src/Columns/ColumnString.h index 212104a8f9a..33f5360710d 100644 --- a/dbms/src/Columns/ColumnString.h +++ b/dbms/src/Columns/ColumnString.h @@ -220,11 +220,6 @@ public: ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; - - template - ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; - void insertDefault() override { chars.push_back(0); diff --git a/dbms/src/Columns/ColumnTuple.cpp b/dbms/src/Columns/ColumnTuple.cpp index 21f98949802..fab6704ae26 100644 --- a/dbms/src/Columns/ColumnTuple.cpp +++ b/dbms/src/Columns/ColumnTuple.cpp @@ -181,17 +181,6 @@ ColumnPtr ColumnTuple::permute(const Permutation & perm, size_t limit) const return ColumnTuple::create(new_columns); } -ColumnPtr ColumnTuple::index(const IColumn & indexes, size_t limit) const -{ - const size_t tuple_size = columns.size(); - Columns new_columns(tuple_size); - - for (size_t i = 0; i < tuple_size; ++i) - new_columns[i] = columns[i]->index(indexes, limit); - - return ColumnTuple::create(new_columns); -} - ColumnPtr ColumnTuple::replicate(const Offsets & offsets) const { const size_t tuple_size = columns.size(); diff --git a/dbms/src/Columns/ColumnTuple.h b/dbms/src/Columns/ColumnTuple.h index b9fefd26db4..a91cb210212 100644 --- a/dbms/src/Columns/ColumnTuple.h +++ b/dbms/src/Columns/ColumnTuple.h @@ -60,7 +60,6 @@ public: void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; MutableColumns scatter(ColumnIndex num_columns, const Selector & selector) const override; void gather(ColumnGathererStream & gatherer_stream) override; diff --git a/dbms/src/Columns/ColumnVector.cpp b/dbms/src/Columns/ColumnVector.cpp index 5f86623c49a..31f0a0a06dd 100644 --- a/dbms/src/Columns/ColumnVector.cpp +++ b/dbms/src/Columns/ColumnVector.cpp @@ -17,8 +17,6 @@ #if __SSE2__ #include -#include - #endif @@ -118,7 +116,7 @@ MutableColumnPtr ColumnVector::cloneResized(size_t size) const memcpy(&new_col.data[0], &data[0], count * sizeof(data[0])); if (size > count) - memset(static_cast(&new_col.data[count]), static_cast(value_type()), (size - count) * sizeof(value_type)); + memset(&new_col.data[count], static_cast(value_type()), (size - count) * sizeof(value_type)); } return std::move(res); @@ -232,12 +230,6 @@ ColumnPtr ColumnVector::permute(const IColumn::Permutation & perm, size_t lim return std::move(res); } -template -ColumnPtr ColumnVector::index(const IColumn & indexes, size_t limit) const -{ - return selectIndexImpl(*this, indexes, limit); -} - template ColumnPtr ColumnVector::replicate(const IColumn::Offsets & offsets) const { @@ -327,11 +319,6 @@ template class ColumnVector; template class ColumnVector; template class ColumnVector; template class ColumnVector; -template class ColumnVector; template class ColumnVector; template class ColumnVector; - -template class ColumnVector; -template class ColumnVector; -template class ColumnVector; } diff --git a/dbms/src/Columns/ColumnVector.h b/dbms/src/Columns/ColumnVector.h index 7240a9627ac..ddeef0a1390 100644 --- a/dbms/src/Columns/ColumnVector.h +++ b/dbms/src/Columns/ColumnVector.h @@ -8,12 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - - /** Stuff for comparing numbers. * Integer values are compared as usual. * Floating-point numbers are compared this way that NaNs always end up at the end @@ -123,55 +117,22 @@ template <> inline UInt64 unionCastToUInt64(Float32 x) } -/// PaddedPODArray extended by Decimal scale -template -class DecimalPaddedPODArray : public PaddedPODArray -{ -public: - using Base = PaddedPODArray; - using Base::operator[]; - using Base::Base; - - DecimalPaddedPODArray(std::initializer_list il) - : DecimalPaddedPODArray(std::begin(il), std::end(il)) - {} - - DecimalPaddedPODArray(DecimalPaddedPODArray && other) - { - this->swap(other); - std::swap(scale, other.scale); - } - - DecimalPaddedPODArray & operator=(DecimalPaddedPODArray && other) - { - this->swap(other); - std::swap(scale, other.scale); - return *this; - } - - void setScale(UInt32 s) { scale = s; } - UInt32 getScale() const { return scale; } - -private: - UInt32 scale = DecimalField::wrongScale(); -}; - - /** A template for columns that use a simple array to store. - */ + */ template class ColumnVector final : public COWPtrHelper> { private: + friend class COWPtrHelper>; + using Self = ColumnVector; - friend class COWPtrHelper; struct less; struct greater; public: using value_type = T; - using Container = std::conditional_t, DecimalPaddedPODArray, PaddedPODArray>; + using Container = PaddedPODArray; private: ColumnVector() {} @@ -255,20 +216,12 @@ public: Field operator[](size_t n) const override { - if constexpr (IsDecimalNumber) - { - UInt32 scale = data.getScale(); - if (scale == DecimalField::wrongScale()) - throw Exception("Extracting Decimal field with unknown scale. Scale is lost.", ErrorCodes::LOGICAL_ERROR); - return DecimalField(data[n], scale); - } - else - return typename NearestFieldType::Type(data[n]); + return typename NearestFieldType::Type(data[n]); } void get(size_t n, Field & res) const override { - res = (*this)[n]; + res = typename NearestFieldType::Type(data[n]); } UInt64 get64(size_t n) const override; @@ -299,11 +252,6 @@ public: ColumnPtr permute(const IColumn::Permutation & perm, size_t limit) const override; - ColumnPtr index(const IColumn & indexes, size_t limit) const override; - - template - ColumnPtr indexImpl(const PaddedPODArray & indexes, size_t limit) const; - ColumnPtr replicate(const IColumn::Offsets & offsets) const override; void getExtremes(Field & min, Field & max) const override; @@ -347,23 +295,5 @@ protected: Container data; }; -template -template -ColumnPtr ColumnVector::indexImpl(const PaddedPODArray & indexes, size_t limit) const -{ - size_t size = indexes.size(); - - if (limit == 0) - limit = size; - else - limit = std::min(size, limit); - - auto res = this->create(limit); - typename Self::Container & res_data = res->getData(); - for (size_t i = 0; i < limit; ++i) - res_data[i] = data[indexes[i]]; - - return std::move(res); -} } diff --git a/dbms/src/Columns/ColumnWithDictionary.cpp b/dbms/src/Columns/ColumnWithDictionary.cpp deleted file mode 100644 index 59bca74ebad..00000000000 --- a/dbms/src/Columns/ColumnWithDictionary.cpp +++ /dev/null @@ -1,619 +0,0 @@ -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace -{ - template - PaddedPODArray * getIndexesData(IColumn & indexes) - { - auto * column = typeid_cast *>(&indexes); - if (column) - return &column->getData(); - - return nullptr; - } - - template - MutableColumnPtr mapUniqueIndexImplRef(PaddedPODArray & index) - { - PaddedPODArray copy(index.cbegin(), index.cend()); - - HashMap hash_map; - for (auto val : index) - hash_map.insert({val, hash_map.size()}); - - auto res_col = ColumnVector::create(); - auto & data = res_col->getData(); - - data.resize(hash_map.size()); - for (auto val : hash_map) - data[val.second] = val.first; - - for (auto & ind : index) - ind = hash_map[ind]; - - for (size_t i = 0; i < index.size(); ++i) - if (data[index[i]] != copy[i]) - throw Exception("Expected " + toString(data[index[i]]) + ", but got " + toString(copy[i]), ErrorCodes::LOGICAL_ERROR); - - return std::move(res_col); - } - - template - MutableColumnPtr mapUniqueIndexImpl(PaddedPODArray & index) - { - if (index.empty()) - return ColumnVector::create(); - - auto size = index.size(); - - T max_val = index[0]; - for (size_t i = 1; i < size; ++i) - max_val = std::max(max_val, index[i]); - - /// May happen when dictionary is shared. - if (max_val > size) - return mapUniqueIndexImplRef(index); - - auto map_size = UInt64(max_val) + 1; - PaddedPODArray map(map_size, 0); - T zero_pos_value = index[0]; - index[0] = 0; - T cur_pos = 0; - for (size_t i = 1; i < size; ++i) - { - T val = index[i]; - if (val != zero_pos_value && map[val] == 0) - { - ++cur_pos; - map[val] = cur_pos; - } - - index[i] = map[val]; - } - - auto res_col = ColumnVector::create(UInt64(cur_pos) + 1); - auto & data = res_col->getData(); - data[0] = zero_pos_value; - for (size_t i = 0; i < map_size; ++i) - { - auto val = map[i]; - if (val) - data[val] = static_cast(i); - } - - return std::move(res_col); - } - - /// Returns unique values of column. Write new index to column. - MutableColumnPtr mapUniqueIndex(IColumn & column) - { - if (auto * data_uint8 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint8); - else if (auto * data_uint16 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint16); - else if (auto * data_uint32 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint32); - else if (auto * data_uint64 = getIndexesData(column)) - return mapUniqueIndexImpl(*data_uint64); - else - throw Exception("Indexes column for getUniqueIndex must be ColumnUInt, got" + column.getName(), - ErrorCodes::LOGICAL_ERROR); - } -} - - -ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_) - : dictionary(std::move(column_unique_)), idx(std::move(indexes_)) -{ - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insert(const Field & x) -{ - compactIfSharedDictionary(); - idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x)); - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insertDefault() -{ - idx.insertPosition(getDictionary().getDefaultValueIndex()); -} - -void ColumnWithDictionary::insertFrom(const IColumn & src, size_t n) -{ - auto * src_with_dict = typeid_cast(&src); - - if (!src_with_dict) - throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); - - size_t position = src_with_dict->getIndexes().getUInt(n); - - if (&src_with_dict->getDictionary() == &getDictionary()) - { - /// Dictionary is shared with src column. Insert only index. - idx.insertPosition(position); - } - else - { - compactIfSharedDictionary(); - const auto & nested = *src_with_dict->getDictionary().getNestedColumn(); - idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position)); - } - - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insertFromFullColumn(const IColumn & src, size_t n) -{ - compactIfSharedDictionary(); - idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n)); - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, size_t length) -{ - auto * src_with_dict = typeid_cast(&src); - - if (!src_with_dict) - throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN); - - if (&src_with_dict->getDictionary() == &getDictionary()) - { - /// Dictionary is shared with src column. Insert only indexes. - idx.insertPositionsRange(src_with_dict->getIndexes(), start, length); - } - else - { - compactIfSharedDictionary(); - - /// TODO: Support native insertion from other unique column. It will help to avoid null map creation. - - auto sub_idx = (*src_with_dict->getIndexes().cut(start, length)).mutate(); - auto idx_map = mapUniqueIndex(*sub_idx); - - auto src_nested = src_with_dict->getDictionary().getNestedColumn(); - auto used_keys = src_nested->index(*idx_map, 0); - - auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(*used_keys, 0, used_keys->size()); - idx.insertPositionsRange(*inserted_indexes->index(*sub_idx, 0), 0, length); - } - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length) -{ - compactIfSharedDictionary(); - auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length); - idx.insertPositionsRange(*inserted_indexes, 0, length); - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions) -{ - Index(positions.getPtr()).check(keys.size()); - compactIfSharedDictionary(); - auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(keys, 0, keys.size()); - idx.insertPositionsRange(*inserted_indexes->index(positions, 0), 0, positions.size()); - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insertData(const char * pos, size_t length) -{ - compactIfSharedDictionary(); - idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length)); - idx.check(getDictionary().size()); -} - -void ColumnWithDictionary::insertDataWithTerminatingZero(const char * pos, size_t length) -{ - compactIfSharedDictionary(); - idx.insertPosition(dictionary.getColumnUnique().uniqueInsertDataWithTerminatingZero(pos, length)); - idx.check(getDictionary().size()); -} - -StringRef ColumnWithDictionary::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const -{ - return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin); -} - -const char * ColumnWithDictionary::deserializeAndInsertFromArena(const char * pos) -{ - compactIfSharedDictionary(); - - const char * new_pos; - idx.insertPosition(dictionary.getColumnUnique().uniqueDeserializeAndInsertFromArena(pos, new_pos)); - - idx.check(getDictionary().size()); - return new_pos; -} - -void ColumnWithDictionary::gather(ColumnGathererStream & gatherer) -{ - gatherer.gather(*this); -} - -MutableColumnPtr ColumnWithDictionary::cloneResized(size_t size) const -{ - auto unique_ptr = dictionary.getColumnUniquePtr(); - return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size)); -} - -int ColumnWithDictionary::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const -{ - const auto & column_with_dictionary = static_cast(rhs); - size_t n_index = getIndexes().getUInt(n); - size_t m_index = column_with_dictionary.getIndexes().getUInt(m); - return getDictionary().compareAt(n_index, m_index, column_with_dictionary.getDictionary(), nan_direction_hint); -} - -void ColumnWithDictionary::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const -{ - if (limit == 0) - limit = size(); - - size_t unique_limit = std::min(limit, getDictionary().size()); - Permutation unique_perm; - getDictionary().getNestedColumn()->getPermutation(reverse, unique_limit, nan_direction_hint, unique_perm); - - /// TODO: optimize with sse. - - /// Get indexes per row in column_unique. - std::vector> indexes_per_row(getDictionary().size()); - size_t indexes_size = getIndexes().size(); - for (size_t row = 0; row < indexes_size; ++row) - indexes_per_row[getIndexes().getUInt(row)].push_back(row); - - /// Replicate permutation. - size_t perm_size = std::min(indexes_size, limit); - res.resize(perm_size); - size_t perm_index = 0; - for (size_t row = 0; row < indexes_size && perm_index < perm_size; ++row) - { - const auto & row_indexes = indexes_per_row[unique_perm[row]]; - for (auto row_index : row_indexes) - { - res[perm_index] = row_index; - ++perm_index; - - if (perm_index == perm_size) - break; - } - } -} - -std::vector ColumnWithDictionary::scatter(ColumnIndex num_columns, const Selector & selector) const -{ - auto columns = getIndexes().scatter(num_columns, selector); - for (auto & column : columns) - { - auto unique_ptr = dictionary.getColumnUniquePtr(); - column = ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), std::move(column)); - } - - return columns; -} - -void ColumnWithDictionary::setSharedDictionary(const ColumnPtr & column_unique) -{ - if (!empty()) - throw Exception("Can't set ColumnUnique for ColumnWithDictionary because is't not empty.", - ErrorCodes::LOGICAL_ERROR); - - dictionary.setShared(column_unique); -} - -ColumnWithDictionary::MutablePtr ColumnWithDictionary::compact() -{ - auto positions = idx.getPositions(); - /// Create column with new indexes and old dictionary. - auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), (*std::move(positions)).mutate()); - /// Will create new dictionary. - column->compactInplace(); - - return column; -} - -ColumnWithDictionary::MutablePtr ColumnWithDictionary::cutAndCompact(size_t start, size_t length) const -{ - auto sub_positions = (*idx.getPositions()->cut(start, length)).mutate(); - /// Create column with new indexes and old dictionary. - auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), std::move(sub_positions)); - /// Will create new dictionary. - column->compactInplace(); - - return column; -} - -void ColumnWithDictionary::compactInplace() -{ - auto positions = idx.detachPositions(); - dictionary.compact(positions); - idx.attachPositions(std::move(positions)); -} - -void ColumnWithDictionary::compactIfSharedDictionary() -{ - if (dictionary.isShared()) - compactInplace(); -} - - -ColumnWithDictionary::DictionaryEncodedColumn -ColumnWithDictionary::getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const -{ - MutableColumnPtr sub_indexes = (*std::move(idx.getPositions()->cut(offset, limit))).mutate(); - auto indexes_map = mapUniqueIndex(*sub_indexes); - auto sub_keys = getDictionary().getNestedColumn()->index(*indexes_map, 0); - - return {std::move(sub_keys), std::move(sub_indexes)}; -} - - -ColumnWithDictionary::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {} - -ColumnWithDictionary::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions)) -{ - updateSizeOfType(); -} - -ColumnWithDictionary::Index::Index(ColumnPtr positions) : positions(std::move(positions)) -{ - updateSizeOfType(); -} - -template -void ColumnWithDictionary::Index::callForType(Callback && callback, size_t size_of_type) -{ - switch (size_of_type) - { - case sizeof(UInt8): { callback(UInt8()); break; } - case sizeof(UInt16): { callback(UInt16()); break; } - case sizeof(UInt32): { callback(UInt32()); break; } - case sizeof(UInt64): { callback(UInt64()); break; } - default: { - throw Exception("Unexpected size of index type for ColumnWithDictionary: " + toString(size_of_type), - ErrorCodes::LOGICAL_ERROR); - } - } -} - -size_t ColumnWithDictionary::Index::getSizeOfIndexType(const IColumn & column, size_t hint) -{ - auto checkFor = [&](auto type) { return typeid_cast *>(&column) != nullptr; }; - auto tryGetSizeFor = [&](auto type) -> size_t { return checkFor(type) ? sizeof(decltype(type)) : 0; }; - - if (hint) - { - size_t size = 0; - callForType([&](auto type) { size = tryGetSizeFor(type); }, hint); - - if (size) - return size; - } - - if (auto size = tryGetSizeFor(UInt8())) - return size; - if (auto size = tryGetSizeFor(UInt16())) - return size; - if (auto size = tryGetSizeFor(UInt32())) - return size; - if (auto size = tryGetSizeFor(UInt64())) - return size; - - throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected UInt, got " + column.getName(), - ErrorCodes::ILLEGAL_COLUMN); -} - -void ColumnWithDictionary::Index::attachPositions(ColumnPtr positions_) -{ - positions = std::move(positions_); - updateSizeOfType(); -} - -template -typename ColumnVector::Container & ColumnWithDictionary::Index::getPositionsData() -{ - auto * positions_ptr = typeid_cast *>(positions->assumeMutable().get()); - if (!positions_ptr) - throw Exception("Invalid indexes type for ColumnWithDictionary." - " Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(), - ErrorCodes::LOGICAL_ERROR); - - return positions_ptr->getData(); -} - -template -void ColumnWithDictionary::Index::convertPositions() -{ - auto convert = [&](auto x) - { - using CurIndexType = decltype(x); - auto & data = getPositionsData(); - - if (sizeof(CurIndexType) > sizeof(IndexType)) - throw Exception("Converting indexes to smaller type: from " + toString(sizeof(CurIndexType)) + - " to " + toString(sizeof(IndexType)), ErrorCodes::LOGICAL_ERROR); - - if (sizeof(CurIndexType) != sizeof(IndexType)) - { - size_t size = data.size(); - auto new_positions = ColumnVector::create(size); - auto & new_data = new_positions->getData(); - - /// TODO: Optimize with SSE? - for (size_t i = 0; i < size; ++i) - new_data[i] = data[i]; - - positions = std::move(new_positions); - size_of_type = sizeof(IndexType); - } - }; - - callForType(std::move(convert), size_of_type); - - checkSizeOfType(); -} - -void ColumnWithDictionary::Index::expandType() -{ - auto expand = [&](auto type) - { - using CurIndexType = decltype(type); - constexpr auto next_size = NumberTraits::nextSize(sizeof(CurIndexType)); - if (next_size == sizeof(CurIndexType)) - throw Exception("Can't expand indexes type for ColumnWithDictionary from type: " - + demangle(typeid(CurIndexType).name()), ErrorCodes::LOGICAL_ERROR); - - using NewIndexType = typename NumberTraits::Construct::Type; - convertPositions(); - }; - - callForType(std::move(expand), size_of_type); -} - -UInt64 ColumnWithDictionary::Index::getMaxPositionForCurrentType() const -{ - UInt64 value = 0; - callForType([&](auto type) { value = std::numeric_limits::max(); }, size_of_type); - return value; -} - -void ColumnWithDictionary::Index::insertPosition(UInt64 position) -{ - while (position > getMaxPositionForCurrentType()) - expandType(); - - positions->assumeMutableRef().insert(UInt64(position)); - checkSizeOfType(); -} - -void ColumnWithDictionary::Index::insertPositionsRange(const IColumn & column, size_t offset, size_t limit) -{ - auto insertForType = [&](auto type) - { - using ColumnType = decltype(type); - const auto * column_ptr = typeid_cast *>(&column); - - if (!column_ptr) - return false; - - if (size_of_type < sizeof(ColumnType)) - convertPositions(); - - if (size_of_type == sizeof(ColumnType)) - positions->assumeMutableRef().insertRangeFrom(column, offset, limit); - else - { - auto copy = [&](auto cur_type) - { - using CurIndexType = decltype(cur_type); - auto & positions_data = getPositionsData(); - const auto & column_data = column_ptr->getData(); - - size_t size = positions_data.size(); - positions_data.resize(size + limit); - - for (size_t i = 0; i < limit; ++i) - positions_data[size + i] = column_data[offset + i]; - }; - - callForType(std::move(copy), size_of_type); - } - - return true; - }; - - if (!insertForType(UInt8()) && - !insertForType(UInt16()) && - !insertForType(UInt32()) && - !insertForType(UInt64())) - throw Exception("Invalid column for ColumnWithDictionary index. Expected UInt, got " + column.getName(), - ErrorCodes::ILLEGAL_COLUMN); - - checkSizeOfType(); -} - -void ColumnWithDictionary::Index::check(size_t /*max_dictionary_size*/) -{ - /// TODO: remove - /* - auto check = [&](auto cur_type) - { - using CurIndexType = decltype(cur_type); - auto & positions_data = getPositionsData(); - - for (size_t i = 0; i < positions_data.size(); ++i) - { - if (positions_data[i] >= max_dictionary_size) - { - throw Exception("Found index " + toString(positions_data[i]) + " at position " + toString(i) - + " which is grated or equal than dictionary size " + toString(max_dictionary_size), - ErrorCodes::LOGICAL_ERROR); - } - } - }; - - callForType(std::move(check), size_of_type); - */ -} - -void ColumnWithDictionary::Index::checkSizeOfType() -{ - if (size_of_type != getSizeOfIndexType(*positions, size_of_type)) - throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) + - ", but positions are " + positions->getName(), ErrorCodes::LOGICAL_ERROR); -} - - -ColumnWithDictionary::Dictionary::Dictionary(MutableColumnPtr && column_unique_) - : column_unique(std::move(column_unique_)) -{ - checkColumn(*column_unique); -} -ColumnWithDictionary::Dictionary::Dictionary(ColumnPtr column_unique_) - : column_unique(std::move(column_unique_)) -{ - checkColumn(*column_unique); -} - -void ColumnWithDictionary::Dictionary::checkColumn(const IColumn & column) -{ - - if (!dynamic_cast(&column)) - throw Exception("ColumnUnique expected as an argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN); -} - -void ColumnWithDictionary::Dictionary::setShared(const ColumnPtr & dictionary) -{ - checkColumn(*dictionary); - - column_unique = dictionary; - shared = true; -} - -void ColumnWithDictionary::Dictionary::compact(ColumnPtr & positions) -{ - auto new_column_unique = column_unique->cloneEmpty(); - - auto & unique = getColumnUnique(); - auto & new_unique = static_cast(*new_column_unique); - - auto indexes = mapUniqueIndex(positions->assumeMutableRef()); - auto sub_keys = unique.getNestedColumn()->index(*indexes, 0); - auto new_indexes = new_unique.uniqueInsertRangeFrom(*sub_keys, 0, sub_keys->size()); - - positions = (*new_indexes->index(*positions, 0)).mutate(); - column_unique = std::move(new_column_unique); - - shared = false; -} - -} diff --git a/dbms/src/Columns/ColumnsCommon.cpp b/dbms/src/Columns/ColumnsCommon.cpp index ed62028fa2e..d4763697fe5 100644 --- a/dbms/src/Columns/ColumnsCommon.cpp +++ b/dbms/src/Columns/ColumnsCommon.cpp @@ -3,10 +3,6 @@ #endif #include -#include -#include -#include -#include namespace DB @@ -312,22 +308,4 @@ INSTANTIATE(Float64) #undef INSTANTIATE -namespace detail -{ - template - const PaddedPODArray * getIndexesData(const IColumn & indexes) - { - auto * column = typeid_cast *>(&indexes); - if (column) - return &column->getData(); - - return nullptr; - } - - template const PaddedPODArray * getIndexesData(const IColumn & indexes); - template const PaddedPODArray * getIndexesData(const IColumn & indexes); - template const PaddedPODArray * getIndexesData(const IColumn & indexes); - template const PaddedPODArray * getIndexesData(const IColumn & indexes); -} - } diff --git a/dbms/src/Columns/ColumnsCommon.h b/dbms/src/Columns/ColumnsCommon.h index 1902774529b..a0048dee6c2 100644 --- a/dbms/src/Columns/ColumnsCommon.h +++ b/dbms/src/Columns/ColumnsCommon.h @@ -8,11 +8,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - /// Counts how many bytes of `filt` are greater than zero. size_t countBytesInFilter(const IColumn::Filter & filt); @@ -38,38 +33,4 @@ void filterArraysImplOnlyData( PaddedPODArray & res_elems, const IColumn::Filter & filt, ssize_t result_size_hint); -namespace detail -{ - template - const PaddedPODArray * getIndexesData(const IColumn & indexes); -} - -/// Check limit <= indexes->size() and call column.indexImpl(const PaddedPodArray & indexes, size_t limit). -template -ColumnPtr selectIndexImpl(const Column & column, const IColumn & indexes, size_t limit) -{ - if (limit == 0) - limit = indexes.size(); - - if (indexes.size() < limit) - throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - if (auto * data_uint8 = detail::getIndexesData(indexes)) - return column.template indexImpl(*data_uint8, limit); - else if (auto * data_uint16 = detail::getIndexesData(indexes)) - return column.template indexImpl(*data_uint16, limit); - else if (auto * data_uint32 = detail::getIndexesData(indexes)) - return column.template indexImpl(*data_uint32, limit); - else if (auto * data_uint64 = detail::getIndexesData(indexes)) - return column.template indexImpl(*data_uint64, limit); - else - throw Exception("Indexes column for IColumn::select must be ColumnUInt, got" + indexes.getName(), - ErrorCodes::LOGICAL_ERROR); -} - -#define INSTANTIATE_INDEX_IMPL(Column) \ - template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ - template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ - template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; \ - template ColumnPtr Column::indexImpl(const PaddedPODArray & indexes, size_t limit) const; } diff --git a/dbms/src/Columns/FilterDescription.cpp b/dbms/src/Columns/FilterDescription.cpp index 92c2765feda..b4779648b82 100644 --- a/dbms/src/Columns/FilterDescription.cpp +++ b/dbms/src/Columns/FilterDescription.cpp @@ -27,14 +27,14 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) if (column.isColumnConst()) { const ColumnConst & column_const = static_cast(column); - ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfWithDictionary(); + const IColumn & column_nested = column_const.getDataColumn(); - if (!typeid_cast(column_nested.get())) + if (!typeid_cast(&column_nested)) { - const ColumnNullable * column_nested_nullable = typeid_cast(column_nested.get()); + const ColumnNullable * column_nested_nullable = typeid_cast(&column_nested); if (!column_nested_nullable || !typeid_cast(&column_nested_nullable->getNestedColumn())) { - throw Exception("Illegal type " + column_nested->getName() + " of column for constant filter. Must be UInt8 or Nullable(UInt8).", + throw Exception("Illegal type " + column_nested.getName() + " of column for constant filter. Must be UInt8 or Nullable(UInt8).", ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER); } } @@ -48,13 +48,8 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column) } -FilterDescription::FilterDescription(const IColumn & column_) +FilterDescription::FilterDescription(const IColumn & column) { - if (column_.withDictionary()) - data_holder = column_.convertToFullColumnIfWithDictionary(); - - const auto & column = data_holder ? *data_holder : column_; - if (const ColumnUInt8 * concrete_column = typeid_cast(&column)) { data = &concrete_column->getData(); diff --git a/dbms/src/Columns/IColumn.h b/dbms/src/Columns/IColumn.h index d4ac8c9552a..281306fb084 100644 --- a/dbms/src/Columns/IColumn.h +++ b/dbms/src/Columns/IColumn.h @@ -47,10 +47,6 @@ public: */ virtual Ptr convertToFullColumnIfConst() const { return {}; } - /// If column isn't ColumnWithDictionary, return itself. - /// If column is ColumnWithDictionary, transforms is to full column. - virtual Ptr convertToFullColumnIfWithDictionary() const { return getPtr(); } - /// Creates empty column with the same type. virtual MutablePtr cloneEmpty() const { return cloneResized(0); } @@ -192,10 +188,6 @@ public: using Permutation = PaddedPODArray; virtual Ptr permute(const Permutation & perm, size_t limit) const = 0; - /// Creates new column with values column[indexes[:limit]]. If limit is 0, all indexes are used. - /// Indexes must be one of the ColumnUInt. For default implementation, see selectIndexImpl from ColumnsCommon.h - virtual Ptr index(const IColumn & indexes, size_t limit) const = 0; - /** Compares (*this)[n] and rhs[m]. * Returns negative number, 0, or positive number (*this)[n] is less, equal, greater than rhs[m] respectively. * Is used in sortings. @@ -333,8 +325,6 @@ public: /// Can be inside ColumnNullable. virtual bool canBeInsideNullable() const { return false; } - virtual bool withDictionary() const { return false; } - virtual ~IColumn() {} diff --git a/dbms/src/Columns/IColumnDummy.h b/dbms/src/Columns/IColumnDummy.h index 576b2d70459..262ab48024a 100644 --- a/dbms/src/Columns/IColumnDummy.h +++ b/dbms/src/Columns/IColumnDummy.h @@ -87,14 +87,6 @@ public: return cloneDummy(limit ? std::min(s, limit) : s); } - ColumnPtr index(const IColumn & indexes, size_t limit) const override - { - if (indexes.size() < limit) - throw Exception("Size of indexes is less than required.", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH); - - return cloneDummy(limit ? limit : s); - } - void getPermutation(bool /*reverse*/, size_t /*limit*/, int /*nan_direction_hint*/, Permutation & res) const override { res.resize(s); diff --git a/dbms/src/Columns/IColumnUnique.h b/dbms/src/Columns/IColumnUnique.h deleted file mode 100644 index 44d655f2aab..00000000000 --- a/dbms/src/Columns/IColumnUnique.h +++ /dev/null @@ -1,129 +0,0 @@ -#pragma once -#include - -namespace DB -{ - -class IColumnUnique : public IColumn -{ -public: - using ColumnUniquePtr = IColumn::template immutable_ptr; - using MutableColumnUniquePtr = IColumn::template mutable_ptr; - - /// Column always contains Null if it's Nullable and empty string if it's String or Nullable(String). - /// So, size may be greater than the number of inserted unique values. - virtual const ColumnPtr & getNestedColumn() const = 0; - /// The same as getNestedColumn, but removes null map if nested column is nullable. - virtual const ColumnPtr & getNestedNotNullableColumn() const = 0; - - size_t size() const override { return getNestedColumn()->size(); } - - /// Appends new value at the end of column (column's size is increased by 1). - /// Is used to transform raw strings to Blocks (for example, inside input format parsers) - virtual size_t uniqueInsert(const Field & x) = 0; - - virtual size_t uniqueInsertFrom(const IColumn & src, size_t n) = 0; - /// Appends range of elements from other column. - /// Could be used to concatenate columns. - virtual MutableColumnPtr uniqueInsertRangeFrom(const IColumn & src, size_t start, size_t length) = 0; - - struct IndexesWithOverflow - { - MutableColumnPtr indexes; - MutableColumnPtr overflowed_keys; - }; - /// Like uniqueInsertRangeFrom, but doesn't insert keys if inner dictionary has more than max_dictionary_size keys. - /// Keys that won't be inserted into dictionary will be into overflowed_keys, indexes will be calculated for - /// concatenation of nested column (which can be got from getNestedColumn() function) and overflowed_keys. - virtual IndexesWithOverflow uniqueInsertRangeWithOverflow(const IColumn & src, size_t start, - size_t length, size_t max_dictionary_size) = 0; - - /// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented). - /// Is used to optimize some computations (in aggregation, for example). - /// Parameter length could be ignored if column values have fixed size. - virtual size_t uniqueInsertData(const char * pos, size_t length) = 0; - virtual size_t uniqueInsertDataWithTerminatingZero(const char * pos, size_t length) = 0; - - virtual size_t getDefaultValueIndex() const = 0; - virtual size_t getNullValueIndex() const = 0; - virtual bool canContainNulls() const = 0; - - virtual size_t uniqueDeserializeAndInsertFromArena(const char * pos, const char *& new_pos) = 0; - - const char * getFamilyName() const override { return "ColumnUnique"; } - - void insert(const Field &) override - { - throw Exception("Method insert is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - void insertRangeFrom(const IColumn &, size_t, size_t) override - { - throw Exception("Method insertRangeFrom is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - void insertData(const char *, size_t) override - { - throw Exception("Method insertData is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - void insertDefault() override - { - throw Exception("Method insertDefault is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - void popBack(size_t) override - { - throw Exception("Method popBack is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - void gather(ColumnGathererStream &) override - { - throw Exception("Method gather is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - const char * deserializeAndInsertFromArena(const char *) override - { - throw Exception("Method deserializeAndInsertFromArena is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - ColumnPtr index(const IColumn &, size_t) const override - { - throw Exception("Method index is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - ColumnPtr cut(size_t, size_t) const override - { - throw Exception("Method cut is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - ColumnPtr filter(const IColumn::Filter &, ssize_t) const override - { - throw Exception("Method filter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - ColumnPtr permute(const IColumn::Permutation &, size_t) const override - { - throw Exception("Method permute is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - ColumnPtr replicate(const IColumn::Offsets &) const override - { - throw Exception("Method replicate is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - void getPermutation(bool, size_t, int, IColumn::Permutation &) const override - { - throw Exception("Method getPermutation is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } - - std::vector scatter(IColumn::ColumnIndex, const IColumn::Selector &) const override - { - throw Exception("Method scatter is not supported for ColumnUnique.", ErrorCodes::NOT_IMPLEMENTED); - } -}; - -using ColumnUniquePtr = IColumnUnique::ColumnUniquePtr; -using MutableColumnUniquePtr = IColumnUnique::MutableColumnUniquePtr; - -} diff --git a/dbms/src/Columns/ReverseIndex.h b/dbms/src/Columns/ReverseIndex.h deleted file mode 100644 index 0a7cc5b935b..00000000000 --- a/dbms/src/Columns/ReverseIndex.h +++ /dev/null @@ -1,412 +0,0 @@ -#pragma once - -#include -#include -#include - -#include -#include - -namespace DB -{ - -namespace -{ - template - struct ReverseIndexHashTableState; - - template - struct ReverseIndexHashTableState - { - constexpr static bool with_saved_hash = false; - constexpr static bool has_base_index = false; - - ColumnType * index_column; - }; - - template - struct ReverseIndexHashTableState - { - constexpr static bool with_saved_hash = false; - constexpr static bool has_base_index = true; - - ColumnType * index_column; - size_t base_index; - }; - - template - struct ReverseIndexHashTableState - { - constexpr static bool with_saved_hash = true; - constexpr static bool has_base_index = false; - - ColumnType * index_column; - typename ColumnVector::Container * saved_hash_column; - }; - - template - struct ReverseIndexHashTableState - { - constexpr static bool with_saved_hash = true; - constexpr static bool has_base_index = true; - - ColumnType * index_column; - typename ColumnVector::Container * saved_hash_column; - size_t base_index; - }; - - - template - struct ReverseIndexHash : public Hash - { - template - size_t operator()(T) const - { - throw Exception("operator()(key) is not implemented for ReverseIndexHash.", ErrorCodes::LOGICAL_ERROR); - } - - template - size_t operator()(const State & state, T key) const - { - auto index = key; - if constexpr (State::has_base_index) - index -= state.base_index; - - return Hash::operator()(state.index_column->getElement(index)); - } - }; - - using ReverseIndexStringHash = ReverseIndexHash; - - template - using ReverseIndexNumberHash = ReverseIndexHash>; - - - template - struct ReverseIndexHashTableCell - : public HashTableCell> - { - using Base = HashTableCell>; - using State = typename Base::State; - using Base::Base; - using Base::key; - using Base::keyEquals; - using Base::isZero; - - template - static bool isZero(const T &, const State & /*state*/) - { - static_assert(!std::is_same_v::type, typename std::decay::type>); - return false; - } - /// Special case when we want to compare with something not in index_column. - /// When we compare something inside column default keyEquals checks only that row numbers are equal. - bool keyEquals(const StringRef & object, size_t hash_ [[maybe_unused]], const State & state) const - { - auto index = key; - if constexpr (has_base_index) - index -= state.base_index; - - if constexpr (string_hash) - return hash_ == (*state.saved_hash_column)[index] && object == state.index_column->getDataAt(index); - else - return object == state.index_column->getDataAt(index); - } - - size_t getHash(const Hash & hash) const - { - auto index = key; - - /// Hack. HashTable is Hash itself. - const auto & state = static_cast(static_cast(hash)); - - if constexpr (has_base_index) - index -= state.base_index; - - if constexpr (string_hash) - return (*state.saved_hash_column)[index]; - else - return hash(state, key); - } - }; - - - template - class HashTableWithPublicState : public HashTable, HashTableAllocator> - { - using State = typename Cell::State; - using Base = HashTable, HashTableAllocator>; - - public: - using Base::Base; - State & getState() { return *this; } - }; - - template - class ReverseIndexStringHashTable : public HashTableWithPublicState< - IndexType, - ReverseIndexHashTableCell< - IndexType, - ReverseIndexStringHash, - ReverseIndexStringHashTable, - ColumnType, - true, - has_base_index>, - ReverseIndexStringHash> - { - using Base = HashTableWithPublicState< - IndexType, - ReverseIndexHashTableCell< - IndexType, - ReverseIndexStringHash, - ReverseIndexStringHashTable, - ColumnType, - true, - has_base_index>, - ReverseIndexStringHash>; - public: - using Base::Base; - friend struct ReverseIndexHashTableCell< - IndexType, - ReverseIndexStringHash, - ReverseIndexStringHashTable, - ColumnType, - true, - has_base_index>; - }; - - template - class ReverseIndexNumberHashTable : public HashTableWithPublicState< - IndexType, - ReverseIndexHashTableCell< - IndexType, - ReverseIndexNumberHash, - ReverseIndexNumberHashTable, - ColumnType, - false, - has_base_index>, - ReverseIndexNumberHash> - { - using Base = HashTableWithPublicState< - IndexType, - ReverseIndexHashTableCell< - IndexType, - ReverseIndexNumberHash, - ReverseIndexNumberHashTable, - ColumnType, - false, - has_base_index>, - ReverseIndexNumberHash>; - public: - using Base::Base; - friend struct ReverseIndexHashTableCell< - IndexType, - ReverseIndexNumberHash, - ReverseIndexNumberHashTable, - ColumnType, - false, - has_base_index>; - }; - - - template - struct SelectReverseIndexHashTable; - - template - struct SelectReverseIndexHashTable - { - using Type = ReverseIndexNumberHashTable; - }; - - template - struct SelectReverseIndexHashTable - { - using Type = ReverseIndexStringHashTable; - }; - - - template - constexpr bool isNumericColumn(const T *) { return false; } - - template - constexpr bool isNumericColumn(const ColumnVector *) { return true; } - - static_assert(isNumericColumn(static_cast *>(nullptr))); - static_assert(!isNumericColumn(static_cast(nullptr))); - - - template - using ReverseIndexHashTable = typename SelectReverseIndexHashTable(nullptr))>::Type; -} - - -template -class ReverseIndex -{ -public: - explicit ReverseIndex(UInt64 num_prefix_rows_to_skip, UInt64 base_index) - : num_prefix_rows_to_skip(num_prefix_rows_to_skip), base_index(base_index) {} - - void setColumn(ColumnType * column_); - - static constexpr bool is_numeric_column = isNumericColumn(static_cast(nullptr)); - static constexpr bool use_saved_hash = !is_numeric_column; - - UInt64 insert(UInt64 from_position); /// Insert into index column[from_position]; - UInt64 insertFromLastRow(); - UInt64 getInsertionPoint(const StringRef & data); - UInt64 lastInsertionPoint() const { return size() + base_index; } - - ColumnType * getColumn() const { return column; } - size_t size() const; - - size_t allocatedBytes() const { return index ? index->getBufferSizeInBytes() : 0; } - -private: - ColumnType * column = nullptr; - UInt64 num_prefix_rows_to_skip; /// The number prefix tows in column which won't be sored at index. - UInt64 base_index; /// This values will be added to row number which is inserted into index. - - using IndexMapType = ReverseIndexHashTable; - - /// Lazy initialized. - std::unique_ptr index; - ColumnUInt64::MutablePtr saved_hash; - - void buildIndex(); - - UInt64 getHash(const StringRef & ref) const - { - if constexpr (is_numeric_column) - { - using ValueType = typename ColumnType::value_type; - ValueType value = *reinterpret_cast(ref.data); - return DefaultHash()(value); - } - else - return StringRefHash()(ref); - } -}; - - - -template -void ReverseIndex:: setColumn(ColumnType * column_) -{ - if (column != column_) - index = nullptr; - - column = column_; -} - -template -size_t ReverseIndex::size() const -{ - if (!column) - throw Exception("ReverseIndex has not size because index column wasn't set.", ErrorCodes::LOGICAL_ERROR); - - return column->size(); -} - -template -void ReverseIndex::buildIndex() -{ - if (index) - return; - - if (!column) - throw Exception("ReverseIndex can't build index because index column wasn't set.", ErrorCodes::LOGICAL_ERROR); - - auto size = column->size(); - index = std::make_unique(size); - - if constexpr (use_saved_hash) - saved_hash = ColumnUInt64::create(size); - - auto & state = index->getState(); - state.index_column = column; - state.base_index = base_index; - if constexpr (use_saved_hash) - state.saved_hash_column = &saved_hash->getData(); - - using IteratorType = typename IndexMapType::iterator; - IteratorType iterator; - bool inserted; - - for (auto row : ext::range(num_prefix_rows_to_skip, size)) - { - auto hash = getHash(column->getDataAt(row)); - - if constexpr (use_saved_hash) - saved_hash->getElement(row) = hash; - - index->emplace(row + base_index, iterator, inserted, hash); - - if (!inserted) - throw Exception("Duplicating keys found in ReverseIndex.", ErrorCodes::LOGICAL_ERROR); - } -} - -template -UInt64 ReverseIndex::insert(UInt64 from_position) -{ - if (!index) - buildIndex(); - - using IteratorType = typename IndexMapType::iterator; - IteratorType iterator; - bool inserted; - - auto hash = getHash(column->getDataAt(from_position)); - - if constexpr (use_saved_hash) - { - auto & data = saved_hash->getData(); - if (data.size() <= from_position) - data.resize(from_position + 1); - data[from_position] = hash; - } - - index->emplace(from_position + base_index, iterator, inserted, hash); - - return *iterator; -} - -template -UInt64 ReverseIndex::insertFromLastRow() -{ - if (!column) - throw Exception("ReverseIndex can't insert row from column because index column wasn't set.", - ErrorCodes::LOGICAL_ERROR); - - UInt64 num_rows = size(); - - if (num_rows == 0) - throw Exception("ReverseIndex can't insert row from column because it is empty.", ErrorCodes::LOGICAL_ERROR); - - UInt64 position = num_rows - 1; - UInt64 inserted_pos = insert(position); - if (position + base_index != inserted_pos) - throw Exception("Can't insert into reverse index from last row (" + toString(position + base_index) - + ") because the same row is in position " + toString(inserted_pos), ErrorCodes::LOGICAL_ERROR); - - return inserted_pos; -} - -template -UInt64 ReverseIndex::getInsertionPoint(const StringRef & data) -{ - if (!index) - buildIndex(); - - using IteratorType = typename IndexMapType::iterator; - IteratorType iterator; - - auto hash = getHash(data); - iterator = index->find(data, hash); - - return iterator == index->end() ? size() + base_index : *iterator; -} - -} diff --git a/dbms/src/Common/ActionBlocker.h b/dbms/src/Common/ActionBlocker.h index 055d266c72e..2ddea2d4359 100644 --- a/dbms/src/Common/ActionBlocker.h +++ b/dbms/src/Common/ActionBlocker.h @@ -1,15 +1,14 @@ #pragma once + #include #include #include - namespace DB { -/// An atomic variable that is used to block and interrupt certain actions. -/// If it is not zero then actions related with it should be considered as interrupted. -/// Uses shared_ptr and the lock uses weak_ptr to be able to "hold" a lock when an object with blocker has already died. +/// An atomic variable that is used to block and interrupt certain actions +/// If it is not zero then actions related with it should be considered as interrupted class ActionBlocker { public: @@ -34,5 +33,4 @@ private: CounterPtr counter; }; - } diff --git a/dbms/src/Common/ActionLock.h b/dbms/src/Common/ActionLock.h index 1167a23b8dd..3d6bfc8ada7 100644 --- a/dbms/src/Common/ActionLock.h +++ b/dbms/src/Common/ActionLock.h @@ -1,7 +1,7 @@ #pragma once #include #include - +#include namespace DB { diff --git a/dbms/src/Core/BackgroundSchedulePool.cpp b/dbms/src/Common/BackgroundSchedulePool.cpp similarity index 100% rename from dbms/src/Core/BackgroundSchedulePool.cpp rename to dbms/src/Common/BackgroundSchedulePool.cpp diff --git a/dbms/src/Core/BackgroundSchedulePool.h b/dbms/src/Common/BackgroundSchedulePool.h similarity index 100% rename from dbms/src/Core/BackgroundSchedulePool.h rename to dbms/src/Common/BackgroundSchedulePool.h diff --git a/dbms/src/Common/COWPtr.h b/dbms/src/Common/COWPtr.h index 525f2372c6c..472bdb23390 100644 --- a/dbms/src/Common/COWPtr.h +++ b/dbms/src/Common/COWPtr.h @@ -1,7 +1,17 @@ #pragma once +#ifdef __clang__ + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" +#endif + #include #include + +#ifdef __clang__ + #pragma clang diagnostic pop +#endif + #include diff --git a/dbms/src/Common/ConcurrentBoundedQueue.h b/dbms/src/Common/ConcurrentBoundedQueue.h index 5ac887bf4d4..aafeefbba38 100644 --- a/dbms/src/Common/ConcurrentBoundedQueue.h +++ b/dbms/src/Common/ConcurrentBoundedQueue.h @@ -37,7 +37,7 @@ namespace detail { MoveOrCopyIfThrow()(std::forward(src), dst); } -} +}; /** A very simple thread-safe queue of limited size. * If you try to pop an item from an empty queue, the thread is blocked until the queue becomes nonempty. diff --git a/dbms/src/Common/Config/ConfigProcessor.cpp b/dbms/src/Common/Config/ConfigProcessor.cpp index 95189affce8..112ea7e5fdc 100644 --- a/dbms/src/Common/Config/ConfigProcessor.cpp +++ b/dbms/src/Common/Config/ConfigProcessor.cpp @@ -406,6 +406,9 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & /// Add path_to_config/conf.d dir merge_dir_path.setBaseName("conf"); merge_dirs.insert(merge_dir_path.toString()); + /// Add path_to_config/config.d dir + merge_dir_path.setBaseName("config"); + merge_dirs.insert(merge_dir_path.toString()); for (const std::string & merge_dir_name : merge_dirs) { diff --git a/dbms/src/Common/CurrentThread.cpp b/dbms/src/Common/CurrentThread.cpp deleted file mode 100644 index 38a42cb65c8..00000000000 --- a/dbms/src/Common/CurrentThread.cpp +++ /dev/null @@ -1,80 +0,0 @@ -#include "CurrentThread.h" -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - -void CurrentThread::updatePerformanceCounters() -{ - get()->updatePerformanceCounters(); -} - -ThreadStatusPtr CurrentThread::get() -{ -#ifndef NDEBUG - if (!current_thread || current_thread.use_count() <= 0) - throw Exception("Thread #" + std::to_string(Poco::ThreadNumber::get()) + " status was not initialized", ErrorCodes::LOGICAL_ERROR); - - if (Poco::ThreadNumber::get() != current_thread->thread_number) - throw Exception("Current thread has different thread number", ErrorCodes::LOGICAL_ERROR); -#endif - - return current_thread; -} - -ProfileEvents::Counters & CurrentThread::getProfileEvents() -{ - return current_thread->performance_counters; -} - -MemoryTracker & CurrentThread::getMemoryTracker() -{ - return current_thread->memory_tracker; -} - -void CurrentThread::updateProgressIn(const Progress & value) -{ - current_thread->progress_in.incrementPiecewiseAtomically(value); -} - -void CurrentThread::updateProgressOut(const Progress & value) -{ - current_thread->progress_out.incrementPiecewiseAtomically(value); -} - -void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr & logs_queue) -{ - get()->attachInternalTextLogsQueue(logs_queue); -} - -std::shared_ptr CurrentThread::getInternalTextLogsQueue() -{ - /// NOTE: this method could be called at early server startup stage - /// NOTE: this method could be called in ThreadStatus destructor, therefore we make use_count() check just in case - - if (!current_thread || current_thread.use_count() <= 0) - return nullptr; - - if (current_thread->getCurrentState() == ThreadStatus::ThreadState::Died) - return nullptr; - - return current_thread->getInternalTextLogsQueue(); -} - -ThreadGroupStatusPtr CurrentThread::getGroup() -{ - return get()->getThreadGroup(); -} - -} diff --git a/dbms/src/Common/CurrentThread.h b/dbms/src/Common/CurrentThread.h deleted file mode 100644 index afde7ad8bf2..00000000000 --- a/dbms/src/Common/CurrentThread.h +++ /dev/null @@ -1,83 +0,0 @@ -#pragma once -#include -#include - - -namespace ProfileEvents -{ -class Counters; -} - -class MemoryTracker; - - -namespace DB -{ - -class Context; -class QueryStatus; -class ThreadStatus; -struct Progress; -using ThreadStatusPtr = std::shared_ptr; -class InternalTextLogsQueue; -class ThreadGroupStatus; -using ThreadGroupStatusPtr = std::shared_ptr; - - -class CurrentThread -{ -public: - - /// Handler to current thread - static ThreadStatusPtr get(); - /// Group to which belongs current thread - static ThreadGroupStatusPtr getGroup(); - - /// A logs queue used by TCPHandler to pass logs to a client - static void attachInternalTextLogsQueue(const std::shared_ptr & logs_queue); - static std::shared_ptr getInternalTextLogsQueue(); - - /// Makes system calls to update ProfileEvents that contain info from rusage and taskstats - static void updatePerformanceCounters(); - - static ProfileEvents::Counters & getProfileEvents(); - static MemoryTracker & getMemoryTracker(); - - /// Update read and write rows (bytes) statistics (used in system.query_thread_log) - static void updateProgressIn(const Progress & value); - static void updateProgressOut(const Progress & value); - - /// Query management: - - /// Call from master thread as soon as possible (e.g. when thread accepted connection) - static void initializeQuery(); - - /// Sets query_context for current thread group - static void attachQueryContext(Context & query_context); - - /// You must call one of these methods when create a query child thread: - /// Add current thread to a group associated with the thread group - static void attachTo(const ThreadGroupStatusPtr & thread_group); - /// Is useful for a ThreadPool tasks - static void attachToIfDetached(const ThreadGroupStatusPtr & thread_group); - - /// Update ProfileEvents and dumps info to system.query_thread_log - static void finalizePerformanceCounters(); - - /// Returns a non-empty string if the thread is attached to a query - static std::string getCurrentQueryID(); - - /// Non-master threads call this method in destructor automatically - static void detachQuery(); - static void detachQueryIfNotDetached(); - - /// Initializes query with current thread as master thread in constructor, and detaches it in desstructor - struct QueryScope - { - explicit QueryScope(Context & query_context); - ~QueryScope(); - }; -}; - -} - diff --git a/dbms/src/Common/DNSResolver.cpp b/dbms/src/Common/DNSResolver.cpp index c82f302d7b2..e3d9442deea 100644 --- a/dbms/src/Common/DNSResolver.cpp +++ b/dbms/src/Common/DNSResolver.cpp @@ -7,7 +7,6 @@ #include #include #include -#include namespace DB @@ -80,10 +79,6 @@ struct DNSResolver::Impl { SimpleCache cache_host; - /// Cached server host name - std::mutex mutex; - std::optional host_name; - /// If disabled, will not make cache lookups, will resolve addresses manually on each call std::atomic disable_cache{false}; }; @@ -113,9 +108,6 @@ Poco::Net::SocketAddress DNSResolver::resolveAddress(const std::string & host, U void DNSResolver::dropCache() { impl->cache_host.drop(); - - std::unique_lock lock(impl->mutex); - impl->host_name.reset(); } void DNSResolver::setDisableCacheFlag(bool is_disabled) @@ -123,19 +115,6 @@ void DNSResolver::setDisableCacheFlag(bool is_disabled) impl->disable_cache = is_disabled; } -String DNSResolver::getHostName() -{ - if (impl->disable_cache) - return Poco::Net::DNS::hostName(); - - std::unique_lock lock(impl->mutex); - - if (!impl->host_name.has_value()) - impl->host_name.emplace(Poco::Net::DNS::hostName()); - - return *impl->host_name; -} - DNSResolver::~DNSResolver() = default; diff --git a/dbms/src/Common/DNSResolver.h b/dbms/src/Common/DNSResolver.h index 097e646fa65..fb3892e101f 100644 --- a/dbms/src/Common/DNSResolver.h +++ b/dbms/src/Common/DNSResolver.h @@ -25,9 +25,6 @@ public: Poco::Net::SocketAddress resolveAddress(const std::string & host, UInt16 port); - /// Get this server host name - String getHostName(); - /// Disables caching void setDisableCacheFlag(bool is_disabled = true); diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 928d011fb75..a789a602751 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -378,17 +378,7 @@ namespace ErrorCodes extern const int FEATURE_IS_NOT_ENABLED_AT_BUILD_TIME = 401; extern const int CANNOT_IOSETUP = 402; extern const int INVALID_JOIN_ON_EXPRESSION = 403; - extern const int BAD_ODBC_CONNECTION_STRING = 404; - extern const int PARTITION_SIZE_EXCEEDS_MAX_DROP_SIZE_LIMIT = 405; - extern const int TOP_AND_LIMIT_TOGETHER = 406; - extern const int DECIMAL_OVERFLOW = 407; - extern const int BAD_REQUEST_PARAMETER = 408; - extern const int EXTERNAL_EXECUTABLE_NOT_FOUND = 409; - extern const int EXTERNAL_SERVER_IS_NOT_RESPONDING = 410; - extern const int PTHREAD_ERROR = 411; - extern const int NETLINK_ERROR = 412; - extern const int CANNOT_SET_SIGNAL_HANDLER = 413; - extern const int CANNOT_READLINE = 414; + extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/Exception.cpp b/dbms/src/Common/Exception.cpp index ab59780c44d..95bc2cd0660 100644 --- a/dbms/src/Common/Exception.cpp +++ b/dbms/src/Common/Exception.cpp @@ -130,11 +130,11 @@ int getCurrentExceptionCode() { return e.code(); } - catch (const Poco::Exception &) + catch (const Poco::Exception & e) { return ErrorCodes::POCO_EXCEPTION; } - catch (const std::exception &) + catch (const std::exception & e) { return ErrorCodes::STD_EXCEPTION; } diff --git a/dbms/src/Common/ExternalTable.h b/dbms/src/Common/ExternalTable.h new file mode 100644 index 00000000000..6331ecba5ca --- /dev/null +++ b/dbms/src/Common/ExternalTable.h @@ -0,0 +1,225 @@ +#pragma once + +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; +} + + +/// The base class containing the basic information about external table and +/// basic functions for extracting this information from text fields. +class BaseExternalTable +{ +public: + std::string file; /// File with data or '-' if stdin + std::string name; /// The name of the table + std::string format; /// Name of the data storage format + + /// Description of the table structure: (column name, data type name) + std::vector> structure; + + std::unique_ptr read_buffer; + Block sample_block; + + virtual ~BaseExternalTable() {} + + /// Initialize read_buffer, depending on the data source. By default, does nothing. + virtual void initReadBuffer() {} + + /// Get the table data - a pair (a thread with the contents of the table, the name of the table) + ExternalTableData getData(const Context & context) + { + initReadBuffer(); + initSampleBlock(); + ExternalTableData res = std::make_pair(std::make_shared(context.getInputFormat( + format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE)), name); + return res; + } + +protected: + /// Clear all accumulated information + void clean() + { + name = ""; + file = ""; + format = ""; + structure.clear(); + sample_block = Block(); + read_buffer.reset(); + } + + /// Function for debugging information output + void write() + { + std::cerr << "file " << file << std::endl; + std::cerr << "name " << name << std::endl; + std::cerr << "format " << format << std::endl; + std::cerr << "structure: \n"; + for (size_t i = 0; i < structure.size(); ++i) + std::cerr << "\t" << structure[i].first << " " << structure[i].second << std::endl; + } + + static std::vector split(const std::string & s, const std::string & d) + { + std::vector res; + boost::split(res, s, boost::algorithm::is_any_of(d), boost::algorithm::token_compress_on); + return res; + } + + /// Construct the `structure` vector from the text field `structure` + virtual void parseStructureFromStructureField(const std::string & argument) + { + std::vector vals = split(argument, " ,"); + + if (vals.size() & 1) + throw Exception("Odd number of attributes in section structure", ErrorCodes::BAD_ARGUMENTS); + + for (size_t i = 0; i < vals.size(); i += 2) + structure.emplace_back(vals[i], vals[i + 1]); + } + + /// Construct the `structure` vector from the text field `types` + virtual void parseStructureFromTypesField(const std::string & argument) + { + std::vector vals = split(argument, " ,"); + + for (size_t i = 0; i < vals.size(); ++i) + structure.emplace_back("_" + toString(i + 1), vals[i]); + } + +private: + /// Initialize sample_block according to the structure of the table stored in the `structure` + void initSampleBlock() + { + const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); + + for (size_t i = 0; i < structure.size(); ++i) + { + ColumnWithTypeAndName column; + column.name = structure[i].first; + column.type = data_type_factory.get(structure[i].second); + column.column = column.type->createColumn(); + sample_block.insert(std::move(column)); + } + } +}; + + +/// Parsing of external table used in the tcp client. +class ExternalTable : public BaseExternalTable +{ +public: + void initReadBuffer() override + { + if (file == "-") + read_buffer = std::make_unique(STDIN_FILENO); + else + read_buffer = std::make_unique(file); + } + + /// Extract parameters from variables_map, which is built on the client command line + ExternalTable(const boost::program_options::variables_map & external_options) + { + if (external_options.count("file")) + file = external_options["file"].as(); + else + throw Exception("--file field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + + if (external_options.count("name")) + name = external_options["name"].as(); + else + throw Exception("--name field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + + if (external_options.count("format")) + format = external_options["format"].as(); + else + throw Exception("--format field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + + if (external_options.count("structure")) + parseStructureFromStructureField(external_options["structure"].as()); + else if (external_options.count("types")) + parseStructureFromTypesField(external_options["types"].as()); + else + throw Exception("Neither --structure nor --types have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); + } +}; + +/// Parsing of external table used when sending tables via http +/// The `handlePart` function will be called for each table passed, +/// so it's also necessary to call `clean` at the end of the `handlePart`. +class ExternalTablesHandler : public Poco::Net::PartHandler, BaseExternalTable +{ +public: + std::vector names; + + ExternalTablesHandler(Context & context_, Poco::Net::NameValueCollection params_) : context(context_), params(params_) { } + + void handlePart(const Poco::Net::MessageHeader & header, std::istream & stream) + { + /// The buffer is initialized here, not in the virtual function initReadBuffer + read_buffer = std::make_unique(stream); + + /// Retrieve a collection of parameters from MessageHeader + Poco::Net::NameValueCollection content; + std::string label; + Poco::Net::MessageHeader::splitParameters(header.get("Content-Disposition"), label, content); + + /// Get parameters + name = content.get("name", "_data"); + format = params.get(name + "_format", "TabSeparated"); + + if (params.has(name + "_structure")) + parseStructureFromStructureField(params.get(name + "_structure")); + else if (params.has(name + "_types")) + parseStructureFromTypesField(params.get(name + "_types")); + else + throw Exception("Neither structure nor types have not been provided for external table " + name + ". Use fields " + name + "_structure or " + name + "_types to do so.", ErrorCodes::BAD_ARGUMENTS); + + ExternalTableData data = getData(context); + + /// Create table + NamesAndTypesList columns = sample_block.getNamesAndTypesList(); + StoragePtr storage = StorageMemory::create(data.second, ColumnsDescription{columns}); + storage->startup(); + context.addExternalTable(data.second, storage); + BlockOutputStreamPtr output = storage->write(ASTPtr(), context.getSettingsRef()); + + /// Write data + data.first->readPrefix(); + output->writePrefix(); + while(Block block = data.first->read()) + output->write(block); + data.first->readSuffix(); + output->writeSuffix(); + + names.push_back(name); + /// We are ready to receive the next file, for this we clear all the information received + clean(); + } + +private: + Context & context; + Poco::Net::NameValueCollection params; +}; + + +} diff --git a/dbms/src/Common/FieldVisitors.cpp b/dbms/src/Common/FieldVisitors.cpp index 62b7667d936..3132a7412ca 100644 --- a/dbms/src/Common/FieldVisitors.cpp +++ b/dbms/src/Common/FieldVisitors.cpp @@ -4,7 +4,6 @@ #include #include #include -#include #include #include @@ -36,13 +35,6 @@ String FieldVisitorDump::operator() (const UInt64 & x) const { return formatQuot String FieldVisitorDump::operator() (const Int64 & x) const { return formatQuotedWithPrefix(x, "Int64_"); } String FieldVisitorDump::operator() (const Float64 & x) const { return formatQuotedWithPrefix(x, "Float64_"); } -String FieldVisitorDump::operator() (const UInt128 & x) const -{ - WriteBufferFromOwnString wb; - wb << "UInt128_" << x.low << "_" << x.high; - return wb.str(); - -} String FieldVisitorDump::operator() (const String & x) const { @@ -55,14 +47,14 @@ String FieldVisitorDump::operator() (const Array & x) const { WriteBufferFromOwnString wb; - wb << "Array_["; + wb.write("Array_[", 7); for (auto it = x.begin(); it != x.end(); ++it) { if (it != x.begin()) - wb << ", "; - wb << applyVisitor(*this, *it); + wb.write(", ", 2); + writeString(applyVisitor(*this, *it), wb); } - wb << ']'; + writeChar(']', wb); return wb.str(); } @@ -72,14 +64,14 @@ String FieldVisitorDump::operator() (const Tuple & x_def) const auto & x = x_def.toUnderType(); WriteBufferFromOwnString wb; - wb << "Tuple_("; + wb.write("Tuple_(", 7); for (auto it = x.begin(); it != x.end(); ++it) { if (it != x.begin()) - wb << ", "; - wb << applyVisitor(*this, *it); + wb.write(", ", 2); + writeString(applyVisitor(*this, *it), wb); } - wb << ')'; + writeChar(')', wb); return wb.str(); } @@ -113,24 +105,19 @@ String FieldVisitorToString::operator() (const Int64 & x) const { return formatQ String FieldVisitorToString::operator() (const Float64 & x) const { return formatFloat(x); } String FieldVisitorToString::operator() (const String & x) const { return formatQuoted(x); } -String FieldVisitorToString::operator() (const UInt128 & x) const -{ - /// Dummy implementation. There is no UInt128 literals in SQL. - return FieldVisitorDump()(x); -} String FieldVisitorToString::operator() (const Array & x) const { WriteBufferFromOwnString wb; - wb << '['; + writeChar('[', wb); for (Array::const_iterator it = x.begin(); it != x.end(); ++it) { if (it != x.begin()) wb.write(", ", 2); - wb << applyVisitor(*this, *it); + writeString(applyVisitor(*this, *it), wb); } - wb << ']'; + writeChar(']', wb); return wb.str(); } @@ -140,14 +127,14 @@ String FieldVisitorToString::operator() (const Tuple & x_def) const auto & x = x_def.toUnderType(); WriteBufferFromOwnString wb; - wb << '('; + writeChar('(', wb); for (auto it = x.begin(); it != x.end(); ++it) { if (it != x.begin()) - wb << ", "; - wb << applyVisitor(*this, *it); + wb.write(", ", 2); + writeString(applyVisitor(*this, *it), wb); } - wb << ')'; + writeChar(')', wb); return wb.str(); } @@ -168,13 +155,6 @@ void FieldVisitorHash::operator() (const UInt64 & x) const hash.update(x); } -void FieldVisitorHash::operator() (const UInt128 & x) const -{ - UInt8 type = Field::Types::UInt128; - hash.update(type); - hash.update(x); -} - void FieldVisitorHash::operator() (const Int64 & x) const { UInt8 type = Field::Types::Int64; diff --git a/dbms/src/Common/FieldVisitors.h b/dbms/src/Common/FieldVisitors.h index 8abf75dbc64..b59c6a47aa7 100644 --- a/dbms/src/Common/FieldVisitors.h +++ b/dbms/src/Common/FieldVisitors.h @@ -38,7 +38,6 @@ typename std::decay_t::ResultType applyVisitor(Visitor && visitor, F && { case Field::Types::Null: return visitor(field.template get()); case Field::Types::UInt64: return visitor(field.template get()); - case Field::Types::UInt128: return visitor(field.template get()); case Field::Types::Int64: return visitor(field.template get()); case Field::Types::Float64: return visitor(field.template get()); case Field::Types::String: return visitor(field.template get()); @@ -58,7 +57,6 @@ static typename std::decay_t::ResultType applyBinaryVisitorImpl(Visitor { case Field::Types::Null: return visitor(field1, field2.template get()); case Field::Types::UInt64: return visitor(field1, field2.template get()); - case Field::Types::UInt128: return visitor(field1, field2.template get()); case Field::Types::Int64: return visitor(field1, field2.template get()); case Field::Types::Float64: return visitor(field1, field2.template get()); case Field::Types::String: return visitor(field1, field2.template get()); @@ -81,9 +79,6 @@ typename std::decay_t::ResultType applyVisitor(Visitor && visitor, F1 & case Field::Types::UInt64: return applyBinaryVisitorImpl( std::forward(visitor), field1.template get(), std::forward(field2)); - case Field::Types::UInt128: - return applyBinaryVisitorImpl( - std::forward(visitor), field1.template get(), std::forward(field2)); case Field::Types::Int64: return applyBinaryVisitorImpl( std::forward(visitor), field1.template get(), std::forward(field2)); @@ -112,7 +107,6 @@ class FieldVisitorToString : public StaticVisitor public: String operator() (const Null & x) const; String operator() (const UInt64 & x) const; - String operator() (const UInt128 & x) const; String operator() (const Int64 & x) const; String operator() (const Float64 & x) const; String operator() (const String & x) const; @@ -127,7 +121,6 @@ class FieldVisitorDump : public StaticVisitor public: String operator() (const Null & x) const; String operator() (const UInt64 & x) const; - String operator() (const UInt128 & x) const; String operator() (const Int64 & x) const; String operator() (const Float64 & x) const; String operator() (const String & x) const; @@ -164,11 +157,6 @@ public: T operator() (const UInt64 & x) const { return x; } T operator() (const Int64 & x) const { return x; } T operator() (const Float64 & x) const { return x; } - - T operator() (const UInt128 &) const - { - throw Exception("Cannot convert UInt128 to " + demangle(typeid(T).name()), ErrorCodes::CANNOT_CONVERT_TYPE); - } }; @@ -182,7 +170,6 @@ public: void operator() (const Null & x) const; void operator() (const UInt64 & x) const; - void operator() (const UInt128 & x) const; void operator() (const Int64 & x) const; void operator() (const Float64 & x) const; void operator() (const String & x) const; @@ -193,60 +180,44 @@ public: /** More precise comparison, used for index. * Differs from Field::operator< and Field::operator== in that it also compares values of different types. * Comparison rules are same as in FunctionsComparison (to be consistent with expression evaluation in query). - * - * TODO Comparisons of UInt128 with different type are incorrect. */ class FieldVisitorAccurateEquals : public StaticVisitor { public: bool operator() (const Null &, const Null &) const { return true; } bool operator() (const Null &, const UInt64 &) const { return false; } - bool operator() (const Null &, const UInt128 &) const { return false; } bool operator() (const Null &, const Int64 &) const { return false; } bool operator() (const Null &, const Float64 &) const { return false; } bool operator() (const Null &, const String &) const { return false; } bool operator() (const Null &, const Array &) const { return false; } bool operator() (const Null &, const Tuple &) const { return false; } - bool operator() (const UInt64 &, const Null &) const { return false; } + bool operator() (const UInt64 &, const Null &) const { return false; } bool operator() (const UInt64 & l, const UInt64 & r) const { return l == r; } - bool operator() (const UInt64 &, const UInt128) const { return true; } bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); } bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const UInt64 &, const String &) const { return false; } - bool operator() (const UInt64 &, const Array &) const { return false; } - bool operator() (const UInt64 &, const Tuple &) const { return false; } + bool operator() (const UInt64 &, const String &) const { return false; } + bool operator() (const UInt64 &, const Array &) const { return false; } + bool operator() (const UInt64 &, const Tuple &) const { return false; } - bool operator() (const UInt128 &, const Null &) const { return false; } - bool operator() (const UInt128 &, const UInt64) const { return false; } - bool operator() (const UInt128 & l, const UInt128 & r) const { return l == r; } - bool operator() (const UInt128 &, const Int64) const { return false; } - bool operator() (const UInt128 &, const Float64) const { return false; } - bool operator() (const UInt128 &, const String &) const { return false; } - bool operator() (const UInt128 &, const Array &) const { return false; } - bool operator() (const UInt128 &, const Tuple &) const { return false; } - - bool operator() (const Int64 &, const Null &) const { return false; } + bool operator() (const Int64 &, const Null &) const { return false; } bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const Int64 &, const UInt128) const { return false; } bool operator() (const Int64 & l, const Int64 & r) const { return l == r; } bool operator() (const Int64 & l, const Float64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const Int64 &, const String &) const { return false; } - bool operator() (const Int64 &, const Array &) const { return false; } - bool operator() (const Int64 &, const Tuple &) const { return false; } + bool operator() (const Int64 &, const String &) const { return false; } + bool operator() (const Int64 &, const Array &) const { return false; } + bool operator() (const Int64 &, const Tuple &) const { return false; } - bool operator() (const Float64 &, const Null &) const { return false; } + bool operator() (const Float64 &, const Null &) const { return false; } bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::equalsOp(l, r); } - bool operator() (const Float64 &, const UInt128) const { return false; } bool operator() (const Float64 & l, const Int64 & r) const { return accurate::equalsOp(l, r); } bool operator() (const Float64 & l, const Float64 & r) const { return l == r; } - bool operator() (const Float64 &, const String &) const { return false; } - bool operator() (const Float64 &, const Array &) const { return false; } - bool operator() (const Float64 &, const Tuple &) const { return false; } + bool operator() (const Float64 &, const String &) const { return false; } + bool operator() (const Float64 &, const Array &) const { return false; } + bool operator() (const Float64 &, const Tuple &) const { return false; } bool operator() (const String &, const Null &) const { return false; } bool operator() (const String &, const UInt64 &) const { return false; } - bool operator() (const String &, const UInt128 &) const { return false; } bool operator() (const String &, const Int64 &) const { return false; } bool operator() (const String &, const Float64 &) const { return false; } bool operator() (const String & l, const String & r) const { return l == r; } @@ -255,7 +226,6 @@ public: bool operator() (const Array &, const Null &) const { return false; } bool operator() (const Array &, const UInt64 &) const { return false; } - bool operator() (const Array &, const UInt128 &) const { return false; } bool operator() (const Array &, const Int64 &) const { return false; } bool operator() (const Array &, const Float64 &) const { return false; } bool operator() (const Array &, const String &) const { return false; } @@ -264,7 +234,6 @@ public: bool operator() (const Tuple &, const Null &) const { return false; } bool operator() (const Tuple &, const UInt64 &) const { return false; } - bool operator() (const Tuple &, const UInt128 &) const { return false; } bool operator() (const Tuple &, const Int64 &) const { return false; } bool operator() (const Tuple &, const Float64 &) const { return false; } bool operator() (const Tuple &, const String &) const { return false; } @@ -278,60 +247,45 @@ public: bool operator() (const Null &, const Null &) const { return false; } bool operator() (const Null &, const UInt64 &) const { return true; } bool operator() (const Null &, const Int64 &) const { return true; } - bool operator() (const Null &, const UInt128 &) const { return true; } bool operator() (const Null &, const Float64 &) const { return true; } bool operator() (const Null &, const String &) const { return true; } bool operator() (const Null &, const Array &) const { return true; } bool operator() (const Null &, const Tuple &) const { return true; } - bool operator() (const UInt64 &, const Null &) const { return false; } + bool operator() (const UInt64 &, const Null &) const { return false; } bool operator() (const UInt64 & l, const UInt64 & r) const { return l < r; } - bool operator() (const UInt64 &, const UInt128 &) const { return true; } bool operator() (const UInt64 & l, const Int64 & r) const { return accurate::lessOp(l, r); } bool operator() (const UInt64 & l, const Float64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const UInt64 &, const String &) const { return true; } - bool operator() (const UInt64 &, const Array &) const { return true; } - bool operator() (const UInt64 &, const Tuple &) const { return true; } + bool operator() (const UInt64 &, const String &) const { return true; } + bool operator() (const UInt64 &, const Array &) const { return true; } + bool operator() (const UInt64 &, const Tuple &) const { return true; } - bool operator() (const UInt128 &, const Null &) const { return false; } - bool operator() (const UInt128 &, const UInt64) const { return false; } - bool operator() (const UInt128 & l, const UInt128 & r) const { return l < r; } - bool operator() (const UInt128 &, const Int64) const { return false; } - bool operator() (const UInt128 &, const Float64) const { return false; } - bool operator() (const UInt128 &, const String &) const { return false; } - bool operator() (const UInt128 &, const Array &) const { return false; } - bool operator() (const UInt128 &, const Tuple &) const { return false; } - - bool operator() (const Int64 &, const Null &) const { return false; } + bool operator() (const Int64 &, const Null &) const { return false; } bool operator() (const Int64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const Int64 &, const UInt128 &) const { return false; } bool operator() (const Int64 & l, const Int64 & r) const { return l < r; } bool operator() (const Int64 & l, const Float64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const Int64 &, const String &) const { return true; } - bool operator() (const Int64 &, const Array &) const { return true; } - bool operator() (const Int64 &, const Tuple &) const { return true; } + bool operator() (const Int64 &, const String &) const { return true; } + bool operator() (const Int64 &, const Array &) const { return true; } + bool operator() (const Int64 &, const Tuple &) const { return true; } - bool operator() (const Float64 &, const Null &) const { return false; } + bool operator() (const Float64 &, const Null &) const { return false; } bool operator() (const Float64 & l, const UInt64 & r) const { return accurate::lessOp(l, r); } - bool operator() (const Float64, const UInt128 &) const { return false; } bool operator() (const Float64 & l, const Int64 & r) const { return accurate::lessOp(l, r); } bool operator() (const Float64 & l, const Float64 & r) const { return l < r; } - bool operator() (const Float64 &, const String &) const { return true; } - bool operator() (const Float64 &, const Array &) const { return true; } - bool operator() (const Float64 &, const Tuple &) const { return true; } + bool operator() (const Float64 &, const String &) const { return true; } + bool operator() (const Float64 &, const Array &) const { return true; } + bool operator() (const Float64 &, const Tuple &) const { return true; } - bool operator() (const String &, const Null &) const { return false; } - bool operator() (const String &, const UInt64 &) const { return false; } - bool operator() (const String &, const UInt128 &) const { return false; } - bool operator() (const String &, const Int64 &) const { return false; } - bool operator() (const String &, const Float64 &) const { return false; } + bool operator() (const String &, const Null &) const { return false; } + bool operator() (const String &, const UInt64 &) const { return false; } + bool operator() (const String &, const Int64 &) const { return false; } + bool operator() (const String &, const Float64 &) const { return false; } bool operator() (const String & l, const String & r) const { return l < r; } - bool operator() (const String &, const Array &) const { return true; } - bool operator() (const String &, const Tuple &) const { return true; } + bool operator() (const String &, const Array &) const { return true; } + bool operator() (const String &, const Tuple &) const { return true; } bool operator() (const Array &, const Null &) const { return false; } bool operator() (const Array &, const UInt64 &) const { return false; } - bool operator() (const Array &, const UInt128 &) const { return false; } bool operator() (const Array &, const Int64 &) const { return false; } bool operator() (const Array &, const Float64 &) const { return false; } bool operator() (const Array &, const String &) const { return false; } @@ -340,7 +294,6 @@ public: bool operator() (const Tuple &, const Null &) const { return false; } bool operator() (const Tuple &, const UInt64 &) const { return false; } - bool operator() (const Tuple &, const UInt128 &) const { return false; } bool operator() (const Tuple &, const Int64 &) const { return false; } bool operator() (const Tuple &, const Float64 &) const { return false; } bool operator() (const Tuple &, const String &) const { return false; } @@ -365,7 +318,6 @@ public: bool operator() (Null &) const { throw Exception("Cannot sum Nulls", ErrorCodes::LOGICAL_ERROR); } bool operator() (String &) const { throw Exception("Cannot sum Strings", ErrorCodes::LOGICAL_ERROR); } bool operator() (Array &) const { throw Exception("Cannot sum Arrays", ErrorCodes::LOGICAL_ERROR); } - bool operator() (UInt128 &) const { throw Exception("Cannot sum UUIDs", ErrorCodes::LOGICAL_ERROR); } }; } diff --git a/dbms/src/Common/FileChecker.cpp b/dbms/src/Common/FileChecker.cpp index bd8e00e38c0..e3b1db745ca 100644 --- a/dbms/src/Common/FileChecker.cpp +++ b/dbms/src/Common/FileChecker.cpp @@ -95,14 +95,13 @@ void FileChecker::save() const /// So complex JSON structure - for compatibility with the old format. writeCString("{\"yandex\":{", out); - auto settings = FormatSettings(); for (auto it = map.begin(); it != map.end(); ++it) { if (it != map.begin()) writeString(",", out); /// `escapeForFileName` is not really needed. But it is left for compatibility with the old code. - writeJSONString(escapeForFileName(it->first), out, settings); + writeJSONString(escapeForFileName(it->first), out); writeString(":{\"size\":\"", out); writeIntText(it->second, out); writeString("\"}", out); diff --git a/dbms/src/Common/HashTable/HashMap.h b/dbms/src/Common/HashTable/HashMap.h index c799444622d..62923616100 100644 --- a/dbms/src/Common/HashTable/HashMap.h +++ b/dbms/src/Common/HashTable/HashMap.h @@ -53,7 +53,6 @@ struct HashMapCell bool keyEquals(const Key & key_) const { return value.first == key_; } bool keyEquals(const Key & key_, size_t /*hash_*/) const { return value.first == key_; } - bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return value.first == key_; } void setHash(size_t /*hash_value*/) {} size_t getHash(const Hash & hash) const { return hash(value.first); } @@ -113,7 +112,6 @@ struct HashMapCellWithSavedHash : public HashMapCell bool keyEquals(const Key & key_) const { return this->value.first == key_; } bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->value.first == key_; } - bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); } void setHash(size_t hash_value) { saved_hash = hash_value; } size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; } diff --git a/dbms/src/Common/HashTable/HashSet.h b/dbms/src/Common/HashTable/HashSet.h index 7f941aeefde..1112cad8b68 100644 --- a/dbms/src/Common/HashTable/HashSet.h +++ b/dbms/src/Common/HashTable/HashSet.h @@ -75,7 +75,6 @@ struct HashSetCellWithSavedHash : public HashTableCell bool keyEquals(const Key & key_) const { return this->key == key_; } bool keyEquals(const Key & key_, size_t hash_) const { return saved_hash == hash_ && this->key == key_; } - bool keyEquals(const Key & key_, size_t hash_, const typename Base::State &) const { return keyEquals(key_, hash_); } void setHash(size_t hash_value) { saved_hash = hash_value; } size_t getHash(const Hash & /*hash_function*/) const { return saved_hash; } diff --git a/dbms/src/Common/HashTable/HashTable.h b/dbms/src/Common/HashTable/HashTable.h index bb731083e4d..9fe446ccb8f 100644 --- a/dbms/src/Common/HashTable/HashTable.h +++ b/dbms/src/Common/HashTable/HashTable.h @@ -74,7 +74,7 @@ bool check(const T x) { return x == 0; } template void set(T & x) { x = 0; } -} +}; /** Compile-time interface for cell of the hash table. @@ -108,7 +108,6 @@ struct HashTableCell /// Are the keys at the cells equal? bool keyEquals(const Key & key_) const { return key == key_; } bool keyEquals(const Key & key_, size_t /*hash_*/) const { return key == key_; } - bool keyEquals(const Key & key_, size_t /*hash_*/, const State & /*state*/) const { return key == key_; } /// If the cell can remember the value of the hash function, then remember it. void setHash(size_t /*hash_value*/) {} @@ -281,10 +280,9 @@ protected: #endif /// Find a cell with the same key or an empty cell, starting from the specified position and further along the collision resolution chain. - template - size_t ALWAYS_INLINE findCell(const ObjectToCompareWith & x, size_t hash_value, size_t place_value) const + size_t ALWAYS_INLINE findCell(const Key & x, size_t hash_value, size_t place_value) const { - while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value, *this)) + while (!buf[place_value].isZero(*this) && !buf[place_value].keyEquals(x, hash_value)) { place_value = grower.next(place_value); #ifdef DBMS_HASH_MAP_COUNT_COLLISIONS @@ -736,8 +734,7 @@ public: } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x) + iterator ALWAYS_INLINE find(Key x) { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -748,8 +745,7 @@ public: } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x) const + const_iterator ALWAYS_INLINE find(Key x) const { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -760,8 +756,7 @@ public: } - template - iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) + iterator ALWAYS_INLINE find(Key x, size_t hash_value) { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); @@ -771,8 +766,7 @@ public: } - template - const_iterator ALWAYS_INLINE find(ObjectToCompareWith x, size_t hash_value) const + const_iterator ALWAYS_INLINE find(Key x, size_t hash_value) const { if (Cell::isZero(x, *this)) return this->hasZero() ? iteratorToZero() : end(); diff --git a/dbms/src/Common/IFactoryWithAliases.h b/dbms/src/Common/IFactoryWithAliases.h index c66782af798..9006a3c7cfd 100644 --- a/dbms/src/Common/IFactoryWithAliases.h +++ b/dbms/src/Common/IFactoryWithAliases.h @@ -94,8 +94,8 @@ public: { if (auto it = aliases.find(name); it != aliases.end()) return it->second; - else if (auto jt = case_insensitive_aliases.find(Poco::toLower(name)); jt != case_insensitive_aliases.end()) - return jt->second; + else if (auto it = case_insensitive_aliases.find(Poco::toLower(name)); it != case_insensitive_aliases.end()) + return it->second; throw Exception(getFactoryName() + ": name '" + name + "' is not alias", ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/src/Common/MemoryTracker.cpp b/dbms/src/Common/MemoryTracker.cpp index 7e957ae1ae4..2c4d781056d 100644 --- a/dbms/src/Common/MemoryTracker.cpp +++ b/dbms/src/Common/MemoryTracker.cpp @@ -1,10 +1,11 @@ -#include "MemoryTracker.h" #include #include #include #include -#include #include +#include + +#include namespace DB @@ -18,7 +19,7 @@ namespace DB MemoryTracker::~MemoryTracker() { - if (static_cast(level) < static_cast(VariableContext::Process) && peak) + if (peak) { try { @@ -55,16 +56,13 @@ void MemoryTracker::logPeakMemoryUsage() const void MemoryTracker::alloc(Int64 size) { - if (blocker.isCancelled()) - return; - /** Using memory_order_relaxed means that if allocations are done simultaneously, * we allow exception about memory limit exceeded to be thrown only on next allocation. * So, we allow over-allocations. */ Int64 will_be = size + amount.fetch_add(size, std::memory_order_relaxed); - if (!parent.load(std::memory_order_relaxed)) + if (!next.load(std::memory_order_relaxed)) CurrentMetrics::add(metric, size); Int64 current_limit = limit.load(std::memory_order_relaxed); @@ -104,62 +102,45 @@ void MemoryTracker::alloc(Int64 size) if (will_be > peak.load(std::memory_order_relaxed)) /// Races doesn't matter. Could rewrite with CAS, but not worth. peak.store(will_be, std::memory_order_relaxed); - if (auto loaded_next = parent.load(std::memory_order_relaxed)) + if (auto loaded_next = next.load(std::memory_order_relaxed)) loaded_next->alloc(size); } void MemoryTracker::free(Int64 size) { - if (blocker.isCancelled()) - return; + Int64 new_amount = amount.fetch_sub(size, std::memory_order_relaxed) - size; - if (level == VariableContext::Thread) + /** Sometimes, query could free some data, that was allocated outside of query context. + * Example: cache eviction. + * To avoid negative memory usage, we "saturate" amount. + * Memory usage will be calculated with some error. + * NOTE The code is not atomic. Not worth to fix. + */ + if (new_amount < 0) { - /// Could become negative if memory allocated in this thread is freed in another one - amount.fetch_sub(size, std::memory_order_relaxed); - } - else - { - Int64 new_amount = amount.fetch_sub(size, std::memory_order_relaxed) - size; - - /** Sometimes, query could free some data, that was allocated outside of query context. - * Example: cache eviction. - * To avoid negative memory usage, we "saturate" amount. - * Memory usage will be calculated with some error. - * NOTE: The code is not atomic. Not worth to fix. - */ - if (unlikely(new_amount < 0)) - { - amount.fetch_sub(new_amount); - size += new_amount; - } + amount.fetch_sub(new_amount); + size += new_amount; } - if (auto loaded_next = parent.load(std::memory_order_relaxed)) + if (auto loaded_next = next.load(std::memory_order_relaxed)) loaded_next->free(size); else CurrentMetrics::sub(metric, size); } -void MemoryTracker::resetCounters() +void MemoryTracker::reset() { + if (!next.load(std::memory_order_relaxed)) + CurrentMetrics::sub(metric, amount.load(std::memory_order_relaxed)); + amount.store(0, std::memory_order_relaxed); peak.store(0, std::memory_order_relaxed); limit.store(0, std::memory_order_relaxed); } -void MemoryTracker::reset() -{ - if (!parent.load(std::memory_order_relaxed)) - CurrentMetrics::sub(metric, amount.load(std::memory_order_relaxed)); - - resetCounters(); -} - - void MemoryTracker::setOrRaiseLimit(Int64 value) { /// This is just atomic set to maximum. @@ -168,26 +149,29 @@ void MemoryTracker::setOrRaiseLimit(Int64 value) ; } +#if __APPLE__ && __clang__ +__thread MemoryTracker * current_memory_tracker = nullptr; +#else +thread_local MemoryTracker * current_memory_tracker = nullptr; +#endif namespace CurrentMemoryTracker { void alloc(Int64 size) { - DB::CurrentThread::getMemoryTracker().alloc(size); + if (current_memory_tracker) + current_memory_tracker->alloc(size); } void realloc(Int64 old_size, Int64 new_size) { - DB::CurrentThread::getMemoryTracker().alloc(new_size - old_size); + if (current_memory_tracker) + current_memory_tracker->alloc(new_size - old_size); } void free(Int64 size) { - DB::CurrentThread::getMemoryTracker().free(size); + if (current_memory_tracker) + current_memory_tracker->free(size); } } - -DB::SimpleActionLock getCurrentMemoryTrackerActionLock() -{ - return DB::CurrentThread::getMemoryTracker().blocker.cancel(); -} diff --git a/dbms/src/Common/MemoryTracker.h b/dbms/src/Common/MemoryTracker.h index 68c145393fe..32a93978fcd 100644 --- a/dbms/src/Common/MemoryTracker.h +++ b/dbms/src/Common/MemoryTracker.h @@ -3,8 +3,6 @@ #include #include #include -#include -#include namespace CurrentMetrics @@ -28,7 +26,7 @@ class MemoryTracker /// Singly-linked list. All information will be passed to subsequent memory trackers also (it allows to implement trackers hierarchy). /// In terms of tree nodes it is the list of parents. Lifetime of these trackers should "include" lifetime of current tracker. - std::atomic parent {}; + std::atomic next {}; /// You could specify custom metric to track memory usage. CurrentMetrics::Metric metric = CurrentMetrics::MemoryTracking; @@ -37,14 +35,11 @@ class MemoryTracker const char * description = nullptr; public: - MemoryTracker(VariableContext level = VariableContext::Thread) : level(level) {} - MemoryTracker(Int64 limit_, VariableContext level = VariableContext::Thread) : limit(limit_), level(level) {} - MemoryTracker(MemoryTracker * parent_, VariableContext level = VariableContext::Thread) : parent(parent_), level(level) {} + MemoryTracker() {} + MemoryTracker(Int64 limit_) : limit(limit_) {} ~MemoryTracker(); - VariableContext level; - /** Call the following functions before calling of corresponding operations with memory allocators. */ void alloc(Int64 size); @@ -84,15 +79,9 @@ public: } /// next should be changed only once: from nullptr to some value. - /// NOTE: It is not true in MergeListElement - void setParent(MemoryTracker * elem) + void setNext(MemoryTracker * elem) { - parent.store(elem, std::memory_order_relaxed); - } - - MemoryTracker * getParent() - { - return parent.load(std::memory_order_relaxed); + next.store(elem, std::memory_order_relaxed); } /// The memory consumption could be shown in realtime via CurrentMetrics counter @@ -106,21 +95,26 @@ public: description = description_; } - /// Reset the accumulated data - void resetCounters(); - - /// Reset the accumulated data and the parent. + /// Reset the accumulated data. void reset(); /// Prints info about peak memory consumption into log. void logPeakMemoryUsage() const; - - /// To be able to temporarily stop memory tracker - DB::SimpleActionBlocker blocker; }; -/// Convenience methods, that use current thread's memory_tracker if it is available. +/** The MemoryTracker object is quite difficult to pass to all places where significant amounts of memory are allocated. + * Therefore, a thread-local pointer to used MemoryTracker is set, or nullptr if MemoryTracker does not need to be used. + * This pointer is set when memory consumption is monitored in current thread. + * So, you just need to pass it to all the threads that handle one request. + */ +#if defined(__APPLE__) && defined(__clang__) +extern __thread MemoryTracker * current_memory_tracker; +#else +extern thread_local MemoryTracker * current_memory_tracker; +#endif + +/// Convenience methods, that use current_memory_tracker if it is available. namespace CurrentMemoryTracker { void alloc(Int64 size); @@ -129,4 +123,20 @@ namespace CurrentMemoryTracker } -DB::SimpleActionLock getCurrentMemoryTrackerActionLock(); +#include + +struct TemporarilyDisableMemoryTracker : private boost::noncopyable +{ + MemoryTracker * memory_tracker; + + TemporarilyDisableMemoryTracker() + { + memory_tracker = current_memory_tracker; + current_memory_tracker = nullptr; + } + + ~TemporarilyDisableMemoryTracker() + { + current_memory_tracker = memory_tracker; + } +}; diff --git a/dbms/src/Common/NaNUtils.h b/dbms/src/Common/NaNUtils.h index af0b26266a7..b30b2336c97 100644 --- a/dbms/src/Common/NaNUtils.h +++ b/dbms/src/Common/NaNUtils.h @@ -47,11 +47,3 @@ std::enable_if_t, T> NaNOrZero() { return T{}; } - -#if 1 /// __int128 -template -std::enable_if_t && !std::numeric_limits::is_integer, __int128> NaNOrZero() -{ - return __int128(0); -} -#endif diff --git a/dbms/src/Common/ODBCBridgeHelper.h b/dbms/src/Common/ODBCBridgeHelper.h deleted file mode 100644 index 807782d73eb..00000000000 --- a/dbms/src/Common/ODBCBridgeHelper.h +++ /dev/null @@ -1,52 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ -namespace ErrorCodes -{ - extern const int EXTERNAL_EXECUTABLE_NOT_FOUND; -} -/** Helper for odbc-bridge, provide utility methods, not main request - */ -class ODBCBridgeHelper -{ -private: - - using Configuration = Poco::Util::AbstractConfiguration; - - const Configuration & config; - Poco::Timespan http_timeout; - - std::string connection_string; - - Poco::URI ping_url; - - Poco::Logger * log = &Poco::Logger::get("ODBCBridgeHelper"); - -public: - static constexpr inline size_t DEFAULT_PORT = 9018; - - static constexpr inline auto DEFAULT_HOST = "localhost"; - static constexpr inline auto DEFAULT_FORMAT = "RowBinary"; - static constexpr inline auto PING_HANDLER = "/ping"; - static constexpr inline auto MAIN_HANDLER = "/"; - static constexpr inline auto COL_INFO_HANDLER = "/columns_info"; - static constexpr inline auto PING_OK_ANSWER = "Ok."; - - ODBCBridgeHelper(const Configuration & config_, const Poco::Timespan & http_timeout_, const std::string & connection_string_); - - std::vector> getURLParams(const std::string & cols, size_t max_block_size) const; - bool checkODBCBridgeIsRunning() const; - - void startODBCBridge() const; - void startODBCBridgeSync() const; - - Poco::URI getMainURI() const; - Poco::URI getColumnsInfoURI() const; -}; -} diff --git a/dbms/src/Common/PODArray.h b/dbms/src/Common/PODArray.h index cb228f5b554..e8a0ce85be0 100644 --- a/dbms/src/Common/PODArray.h +++ b/dbms/src/Common/PODArray.h @@ -5,9 +5,24 @@ #include #include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wold-style-cast" + +#ifdef __clang__ +#pragma clang diagnostic push +#pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" +#pragma clang diagnostic ignored "-Wreserved-id-macro" +#endif + #include #include +#ifdef __clang__ +#pragma clang diagnostic pop +#endif + +#pragma GCC diagnostic pop + #include #include diff --git a/dbms/src/Common/ProfileEvents.cpp b/dbms/src/Common/ProfileEvents.cpp index b4cfd6e6852..adef2331609 100644 --- a/dbms/src/Common/ProfileEvents.cpp +++ b/dbms/src/Common/ProfileEvents.cpp @@ -1,7 +1,5 @@ #include -#include -#include -#include + /// Available events. Add something here as you wish. #define APPLY_FOR_EVENTS(M) \ @@ -39,11 +37,6 @@ M(CreatedReadBufferAIO) \ M(CreatedWriteBufferOrdinary) \ M(CreatedWriteBufferAIO) \ - M(DiskReadElapsedMicroseconds) \ - M(DiskWriteElapsedMicroseconds) \ - M(NetworkReceiveElapsedMicroseconds) \ - M(NetworkSendElapsedMicroseconds) \ - M(ThrottlerSleepMicroseconds) \ \ M(ReplicatedPartFetches) \ M(ReplicatedPartFailedFetches) \ @@ -74,9 +67,7 @@ M(ZooKeeperCheck) \ M(ZooKeeperClose) \ M(ZooKeeperWatchResponse) \ - M(ZooKeeperUserExceptions) \ - M(ZooKeeperHardwareExceptions) \ - M(ZooKeeperOtherExceptions) \ + M(ZooKeeperExceptions) \ M(ZooKeeperWaitMicroseconds) \ M(ZooKeeperBytesSent) \ M(ZooKeeperBytesReceived) \ @@ -152,93 +143,31 @@ M(RWLockAcquiredWriteLocks) \ M(RWLockReadersWaitMilliseconds) \ M(RWLockWritersWaitMilliseconds) \ - M(NetworkErrors) \ \ - M(RealTimeMicroseconds) \ - M(UserTimeMicroseconds) \ - M(SystemTimeMicroseconds) \ - M(SoftPageFaults) \ - M(HardPageFaults) \ - M(VoluntaryContextSwitches) \ - M(InvoluntaryContextSwitches) \ - \ - M(OSIOWaitMicroseconds) \ - M(OSCPUWaitMicroseconds) \ - M(OSCPUVirtualTimeMicroseconds) \ - M(OSReadBytes) \ - M(OSWriteBytes) \ - M(OSReadChars) \ - M(OSWriteChars) \ - + M(NetworkErrors) namespace ProfileEvents { - -#define M(NAME) extern const Event NAME = __COUNTER__; - APPLY_FOR_EVENTS(M) -#undef M -constexpr Event END = __COUNTER__; - -/// Global variable, initialized by zeros. -Counter global_counters_array[END] {}; -/// Initialize global counters statically -Counters global_counters(global_counters_array); - -const Event Counters::num_counters = END; - - -Counters::Counters(VariableContext level, Counters * parent) - : counters_holder(new Counter[num_counters] {}), - parent(parent), - level(level) -{ - counters = counters_holder.get(); -} - -void Counters::resetCounters() -{ - if (counters) - { - for (Event i = 0; i < num_counters; ++i) - counters[i].store(0, std::memory_order_relaxed); - } -} - -void Counters::reset() -{ - parent = nullptr; - resetCounters(); -} - -Counters Counters::getPartiallyAtomicSnapshot() const -{ - Counters res(VariableContext::Snapshot, nullptr); - for (Event i = 0; i < num_counters; ++i) - res.counters[i].store(counters[i].load(std::memory_order_relaxed), std::memory_order_relaxed); - return res; -} - -const char * getDescription(Event event) -{ - static const char * descriptions[] = - { - #define M(NAME) #NAME, + #define M(NAME) extern const Event NAME = __COUNTER__; APPLY_FOR_EVENTS(M) #undef M - }; + constexpr Event END = __COUNTER__; - return descriptions[event]; -} + std::atomic counters[END] {}; /// Global variable, initialized by zeros. + const char * getDescription(Event event) + { + static const char * descriptions[] = + { + #define M(NAME) #NAME, + APPLY_FOR_EVENTS(M) + #undef M + }; -Event end() { return END; } - - -void increment(Event event, Count amount) -{ - DB::CurrentThread::getProfileEvents().increment(event, amount); -} + return descriptions[event]; + } + Event end() { return END; } } #undef APPLY_FOR_EVENTS diff --git a/dbms/src/Common/ProfileEvents.h b/dbms/src/Common/ProfileEvents.h index 38d8a9df7b9..0cb88f0ceed 100644 --- a/dbms/src/Common/ProfileEvents.h +++ b/dbms/src/Common/ProfileEvents.h @@ -1,9 +1,8 @@ #pragma once -#include -#include -#include #include +#include + /** Implements global counters for various events happening in the application * - for high level profiling. @@ -15,80 +14,19 @@ namespace ProfileEvents /// Event identifier (index in array). using Event = size_t; using Count = size_t; - using Counter = std::atomic; - class Counters; - - /// Counters - how many times each event happened - extern Counters global_counters; - - class Counters - { - Counter * counters = nullptr; - std::unique_ptr counters_holder; - /// Used to propagate increments - Counters * parent = nullptr; - - public: - - VariableContext level = VariableContext::Thread; - - /// By default, any instance have to increment global counters - Counters(VariableContext level = VariableContext::Thread, Counters * parent = &global_counters); - - /// Global level static initializer - Counters(Counter * allocated_counters) - : counters(allocated_counters), parent(nullptr), level(VariableContext::Global) {} - - Counter & operator[] (Event event) - { - return counters[event]; - } - - const Counter & operator[] (Event event) const - { - return counters[event]; - } - - inline void increment(Event event, Count amount = 1) - { - Counters * current = this; - do - { - current->counters[event].fetch_add(amount, std::memory_order_relaxed); - current = current->parent; - } while (current != nullptr); - } - - /// Every single value is fetched atomically, but not all values as a whole. - Counters getPartiallyAtomicSnapshot() const; - - /// Reset all counters to zero and reset parent. - void reset(); - - /// Get parent (thread unsafe) - Counters * getParent() - { - return parent; - } - - /// Set parent (thread unsafe) - void setParent(Counters * parent_) - { - parent = parent_; - } - - /// Set all counters to zero - void resetCounters(); - - static const Event num_counters; - }; - - /// Increment a counter for event. Thread-safe. - void increment(Event event, Count amount = 1); /// Get text description of event by identifier. Returns statically allocated string. const char * getDescription(Event event); + /// Counters - how many times each event happened. + extern std::atomic counters[]; + + /// Increment a counter for event. Thread-safe. + inline void increment(Event event, Count amount = 1) + { + counters[event].fetch_add(amount, std::memory_order_relaxed); + } + /// Get index just after last event identifier. Event end(); } diff --git a/dbms/src/Common/RWLockFIFO.cpp b/dbms/src/Common/RWLockFIFO.cpp index a1211a0bb9d..51a2f756475 100644 --- a/dbms/src/Common/RWLockFIFO.cpp +++ b/dbms/src/Common/RWLockFIFO.cpp @@ -86,7 +86,7 @@ RWLockFIFO::LockHandler RWLockFIFO::getLock(RWLockFIFO::Type type, RWLockFIFO::C handler_ptr->it_client->info += "; " + client.info; - return handler_ptr; + return handler_ptr; } if (type == Type::Write || queue.empty() || queue.back().type == Type::Write) diff --git a/dbms/src/Common/SimpleActionBlocker.h b/dbms/src/Common/SimpleActionBlocker.h deleted file mode 100644 index 4a96db0e09d..00000000000 --- a/dbms/src/Common/SimpleActionBlocker.h +++ /dev/null @@ -1,79 +0,0 @@ -#pragma once -#include - - -namespace DB -{ - -class SimpleActionLock; - - -/// Similar to ActionBlocker, but without weak_ptr magic -class SimpleActionBlocker -{ - using Counter = std::atomic; - Counter counter = 0; - -public: - - SimpleActionBlocker() = default; - - bool isCancelled() const { return counter > 0; } - - /// Temporarily blocks corresponding actions (while the returned object is alive) - friend class SimpleActionLock; - inline SimpleActionLock cancel(); - - /// Cancel the actions forever. - void cancelForever() { ++counter; } -}; - - -/// Blocks related action while a SimpleActionLock instance exists -class SimpleActionLock -{ - SimpleActionBlocker * block = nullptr; - -public: - - SimpleActionLock() = default; - - explicit SimpleActionLock(SimpleActionBlocker & block_) : block(&block_) - { - ++block->counter; - } - - SimpleActionLock(const SimpleActionLock &) = delete; - - SimpleActionLock(SimpleActionLock && rhs) noexcept - { - *this = std::move(rhs); - } - - SimpleActionLock & operator=(const SimpleActionLock &) = delete; - - SimpleActionLock & operator=(SimpleActionLock && rhs) noexcept - { - if (block) - --block->counter; - - block = rhs.block; - rhs.block = nullptr; - - return *this; - } - - ~SimpleActionLock() - { - if (block) - --block->counter; - } -}; - - -SimpleActionLock SimpleActionBlocker::cancel() -{ - return SimpleActionLock(*this); -} - -} diff --git a/dbms/src/Common/SpaceSaving.h b/dbms/src/Common/SpaceSaving.h index 08d392bf56d..b08fa140fef 100644 --- a/dbms/src/Common/SpaceSaving.h +++ b/dbms/src/Common/SpaceSaving.h @@ -340,4 +340,4 @@ private: size_t m_capacity; }; -} +}; diff --git a/dbms/src/Common/StatusFile.cpp b/dbms/src/Common/StatusFile.cpp index 4da9ea48aa7..463d04b62e7 100644 --- a/dbms/src/Common/StatusFile.cpp +++ b/dbms/src/Common/StatusFile.cpp @@ -30,7 +30,7 @@ StatusFile::StatusFile(const std::string & path_) std::string contents; { ReadBufferFromFile in(path, 1024); - LimitReadBuffer limit_in(in, 1024, false); + LimitReadBuffer limit_in(in, 1024); readStringUntilEOF(contents, limit_in); } diff --git a/dbms/src/Common/Stopwatch.h b/dbms/src/Common/Stopwatch.h index c1ec623e100..334e1574dde 100644 --- a/dbms/src/Common/Stopwatch.h +++ b/dbms/src/Common/Stopwatch.h @@ -32,11 +32,9 @@ public: void stop() { stop_ns = nanoseconds(); is_running = false; } void reset() { start_ns = 0; stop_ns = 0; is_running = false; } void restart() { start(); } - UInt64 elapsed() const { return elapsedNanoseconds(); } - UInt64 elapsedNanoseconds() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; } - UInt64 elapsedMicroseconds() const { return elapsedNanoseconds() / 1000U; } - UInt64 elapsedMilliseconds() const { return elapsedNanoseconds() / 1000000UL; } - double elapsedSeconds() const { return static_cast(elapsedNanoseconds()) / 1000000000ULL; } + UInt64 elapsed() const { return is_running ? nanoseconds() - start_ns : stop_ns - start_ns; } + UInt64 elapsedMilliseconds() const { return elapsed() / 1000000UL; } + double elapsedSeconds() const { return static_cast(elapsed()) / 1000000000ULL; } private: UInt64 start_ns = 0; @@ -133,59 +131,3 @@ private: /// Most significant bit is a lock. When it is set, compareAndRestartDeferred method will return false. UInt64 nanoseconds() const { return StopWatchDetail::nanoseconds(clock_type) & 0x7FFFFFFFFFFFFFFFULL; } }; - - -/// Like ordinary StopWatch, but uses getrusage() system call -struct StopwatchRUsage -{ - StopwatchRUsage() = default; - - void start() { start_ts = Timestamp::current(); is_running = true; } - void stop() { stop_ts = Timestamp::current(); is_running = false; } - void reset() { start_ts = Timestamp(); stop_ts = Timestamp(); is_running = false; } - void restart() { start(); } - - UInt64 elapsed(bool count_user = true, bool count_sys = true) const - { - return elapsedNanoseconds(count_user, count_sys); - } - - UInt64 elapsedNanoseconds(bool count_user = true, bool count_sys = true) const - { - return (is_running ? Timestamp::current() : stop_ts).nanoseconds(count_user, count_sys) - start_ts.nanoseconds(count_user, count_sys); - } - - UInt64 elapsedMicroseconds(bool count_user = true, bool count_sys = true) const - { - return elapsedNanoseconds(count_user, count_sys) / 1000UL; - } - - UInt64 elapsedMilliseconds(bool count_user = true, bool count_sys = true) const - { - return elapsedNanoseconds(count_user, count_sys) / 1000000UL; - } - - double elapsedSeconds(bool count_user = true, bool count_sys = true) const - { - return static_cast(elapsedNanoseconds(count_user, count_sys)) / 1000000000.0; - } - -private: - - struct Timestamp - { - UInt64 user_ns = 0; - UInt64 sys_ns = 0; - - static Timestamp current(); - - UInt64 nanoseconds(bool count_user = true, bool count_sys = true) const - { - return (count_user ? user_ns : 0) + (count_sys ? sys_ns : 0); - } - }; - - Timestamp start_ts; - Timestamp stop_ts; - bool is_running = false; -}; diff --git a/dbms/src/Common/Throttler.h b/dbms/src/Common/Throttler.h index 9c7b446dd96..abc87ffae50 100644 --- a/dbms/src/Common/Throttler.h +++ b/dbms/src/Common/Throttler.h @@ -5,17 +5,10 @@ #include #include #include -#include #include #include -namespace ProfileEvents -{ - extern const Event ThrottlerSleepMicroseconds; -} - - namespace DB { @@ -76,14 +69,10 @@ public: if (desired_ns > elapsed_ns) { UInt64 sleep_ns = desired_ns - elapsed_ns; - ::timespec sleep_ts; + timespec sleep_ts; sleep_ts.tv_sec = sleep_ns / 1000000000; sleep_ts.tv_nsec = sleep_ns % 1000000000; - - /// NOTE: Returns early in case of a signal. This is considered normal. - ::nanosleep(&sleep_ts, nullptr); - - ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_ns / 1000UL); + nanosleep(&sleep_ts, nullptr); /// NOTE Returns early in case of a signal. This is considered normal. } } diff --git a/dbms/src/Common/UInt128.h b/dbms/src/Common/UInt128.h index 9a4f4b570f6..8bda6e2b204 100644 --- a/dbms/src/Common/UInt128.h +++ b/dbms/src/Common/UInt128.h @@ -28,16 +28,8 @@ struct UInt128 UInt64 high; UInt128() = default; - explicit UInt128(const UInt64 low, const UInt64 high) : low(low), high(high) {} - -#if 1 - explicit UInt128(const unsigned __int128 rhs) - : low(rhs & 0xffffffffffffffffll), - high(rhs >> 64) - {} -#else explicit UInt128(const UInt64 rhs) : low(rhs), high() {} -#endif + explicit UInt128(const UInt64 low, const UInt64 high) : low(low), high(high) {} auto tuple() const { return std::tie(high, low); } @@ -73,7 +65,6 @@ template bool inline operator< (T a, const UInt128 b) { return UIn template <> constexpr bool IsNumber = true; template <> struct TypeName { static const char * get() { return "UInt128"; } }; -template <> struct TypeId { static constexpr const size_t value = 5; }; struct UInt128Hash { diff --git a/dbms/src/Common/VariableContext.h b/dbms/src/Common/VariableContext.h deleted file mode 100644 index 2fe4ffb565a..00000000000 --- a/dbms/src/Common/VariableContext.h +++ /dev/null @@ -1,12 +0,0 @@ -#pragma once - -/// Used in ProfileEvents and MemoryTracker to determine their hierarchy level -/// The less value the higher level (zero level is the root) -enum class VariableContext -{ - Global = 0, - User, /// Group of processes - Process, /// For example, a query or a merge - Thread, /// A thread of a process - Snapshot /// Does not belong to anybody -}; diff --git a/dbms/src/Common/ZooKeeper/KeeperException.h b/dbms/src/Common/ZooKeeper/KeeperException.h index 5e1c4616757..fe6c7aedeff 100644 --- a/dbms/src/Common/ZooKeeper/KeeperException.h +++ b/dbms/src/Common/ZooKeeper/KeeperException.h @@ -7,6 +7,27 @@ namespace zkutil { +/// You should reinitialize ZooKeeper session in case of these errors +inline bool isHardwareError(int32_t zk_return_code) +{ + return zk_return_code == ZooKeeperImpl::ZooKeeper::ZINVALIDSTATE + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONMOVED + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZOPERATIONTIMEOUT; +} + +/// Valid errors sent from server +inline bool isUserError(int32_t zk_return_code) +{ + return zk_return_code == ZooKeeperImpl::ZooKeeper::ZNONODE + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZBADVERSION + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS + || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY; +} + + using KeeperException = ZooKeeperImpl::Exception; @@ -29,4 +50,4 @@ private: static size_t getFailedOpIndex(int32_t code, const Responses & responses); }; -} +}; diff --git a/dbms/src/Common/ZooKeeper/LeaderElection.h b/dbms/src/Common/ZooKeeper/LeaderElection.h index b1649732db8..e99c20c8f7e 100644 --- a/dbms/src/Common/ZooKeeper/LeaderElection.h +++ b/dbms/src/Common/ZooKeeper/LeaderElection.h @@ -6,7 +6,7 @@ #include #include #include -#include +#include namespace ProfileEvents diff --git a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp index d0702e0604d..622e09c485c 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeper.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeper.cpp @@ -493,7 +493,7 @@ Responses ZooKeeper::multi(const Requests & requests) int32_t ZooKeeper::tryMulti(const Requests & requests, Responses & responses) { int32_t code = multiImpl(requests, responses); - if (code && !ZooKeeperImpl::ZooKeeper::isUserError(code)) + if (code && !isUserError(code)) throw KeeperException(code); return code; } @@ -824,7 +824,7 @@ size_t KeeperMultiException::getFailedOpIndex(int32_t code, const Responses & re if (responses[index]->error) return index; - if (!ZooKeeperImpl::ZooKeeper::isUserError(code)) + if (!isUserError(code)) throw DB::Exception("There are no failed OPs because '" + ZooKeeper::error2string(code) + "' is not valid response code for that", DB::ErrorCodes::LOGICAL_ERROR); @@ -850,7 +850,7 @@ void KeeperMultiException::check(int32_t code, const Requests & requests, const if (!code) return; - if (ZooKeeperImpl::ZooKeeper::isUserError(code)) + if (isUserError(code)) throw KeeperMultiException(code, requests, responses); else throw KeeperException(code); diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperHolder.h b/dbms/src/Common/ZooKeeper/ZooKeeperHolder.h index d5792b8fde6..49ba6596ef9 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperHolder.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperHolder.h @@ -83,4 +83,4 @@ void ZooKeeperHolder::init(Args &&... args) using ZooKeeperHolderPtr = std::shared_ptr; -} +}; diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp index 3a0978f415e..8b80e74c96d 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.cpp @@ -25,9 +25,7 @@ namespace DB namespace ProfileEvents { - extern const Event ZooKeeperUserExceptions; - extern const Event ZooKeeperHardwareExceptions; - extern const Event ZooKeeperOtherExceptions; + extern const Event ZooKeeperExceptions; extern const Event ZooKeeperInit; extern const Event ZooKeeperTransactions; extern const Event ZooKeeperCreate; @@ -269,12 +267,7 @@ namespace ZooKeeperImpl Exception::Exception(const std::string & msg, const int32_t code, int) : DB::Exception(msg, DB::ErrorCodes::KEEPER_EXCEPTION), code(code) { - if (ZooKeeper::isUserError(code)) - ProfileEvents::increment(ProfileEvents::ZooKeeperUserExceptions); - else if (ZooKeeper::isHardwareError(code)) - ProfileEvents::increment(ProfileEvents::ZooKeeperHardwareExceptions); - else - ProfileEvents::increment(ProfileEvents::ZooKeeperOtherExceptions); + ProfileEvents::increment(ProfileEvents::ZooKeeperExceptions); } Exception::Exception(const std::string & msg, const int32_t code) @@ -522,25 +515,6 @@ const char * ZooKeeper::errorMessage(int32_t code) return "unknown error"; } -bool ZooKeeper::isHardwareError(int32_t zk_return_code) -{ - return zk_return_code == ZooKeeperImpl::ZooKeeper::ZINVALIDSTATE - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONEXPIRED - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZSESSIONMOVED - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZCONNECTIONLOSS - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZMARSHALLINGERROR - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZOPERATIONTIMEOUT; -} - -bool ZooKeeper::isUserError(int32_t zk_return_code) -{ - return zk_return_code == ZooKeeperImpl::ZooKeeper::ZNONODE - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZBADVERSION - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOCHILDRENFOREPHEMERALS - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNODEEXISTS - || zk_return_code == ZooKeeperImpl::ZooKeeper::ZNOTEMPTY; -} - ZooKeeper::~ZooKeeper() { diff --git a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h index 8b6f57a8741..35d97c04025 100644 --- a/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h +++ b/dbms/src/Common/ZooKeeper/ZooKeeperImpl.h @@ -549,12 +549,6 @@ public: ZSESSIONMOVED = -118 /// Session moved to another server, so operation is ignored }; - /// Network errors and similar. You should reinitialize ZooKeeper session in case of these errors - static bool isHardwareError(int32_t code); - - /// Valid errors sent from the server about database state (like "no node"). Logical and authentication errors (like "bad arguments") are not here. - static bool isUserError(int32_t code); - static const char * errorMessage(int32_t code); /// For watches. @@ -652,4 +646,4 @@ private: CurrentMetrics::Increment active_session_metric_increment{CurrentMetrics::ZooKeeperSession}; }; -} +}; diff --git a/dbms/src/Common/formatIPv6.cpp b/dbms/src/Common/formatIPv6.cpp index e2a295cb923..a9eea719e6f 100644 --- a/dbms/src/Common/formatIPv6.cpp +++ b/dbms/src/Common/formatIPv6.cpp @@ -77,10 +77,7 @@ void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_ if (words[i] == 0) { if (cur.base == -1) - { - cur.base = i; - cur.len = 1; - } + cur.base = i, cur.len = 1; else cur.len++; } diff --git a/dbms/src/Common/localBackup.cpp b/dbms/src/Common/localBackup.cpp index aec7da1f65d..bd12d5509cc 100644 --- a/dbms/src/Common/localBackup.cpp +++ b/dbms/src/Common/localBackup.cpp @@ -108,7 +108,7 @@ void localBackup(const Poco::Path & source_path, const Poco::Path & destination_ continue; } - catch (const Poco::FileNotFoundException &) + catch (const Poco::FileNotFoundException & e) { ++try_no; if (try_no == max_tries) diff --git a/dbms/src/Common/setThreadName.h b/dbms/src/Common/setThreadName.h index cdcb6b46914..dc6af7336e0 100644 --- a/dbms/src/Common/setThreadName.h +++ b/dbms/src/Common/setThreadName.h @@ -1,10 +1,7 @@ #pragma once -#include /** Sets the thread name (maximum length is 15 bytes), * which will be visible in ps, gdb, /proc, * for convenience of observation and debugging. */ void setThreadName(const char * name); - -std::string getThreadName(); diff --git a/dbms/src/Common/tests/CMakeLists.txt b/dbms/src/Common/tests/CMakeLists.txt index f4d01e85bd2..f1605a13447 100644 --- a/dbms/src/Common/tests/CMakeLists.txt +++ b/dbms/src/Common/tests/CMakeLists.txt @@ -60,7 +60,7 @@ add_executable (space_saving space_saving.cpp) target_link_libraries (space_saving clickhouse_common_io) add_executable (integer_hash_tables_and_hashes integer_hash_tables_and_hashes.cpp) -target_include_directories (integer_hash_tables_and_hashes SYSTEM BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) +target_include_directories (integer_hash_tables_and_hashes BEFORE PRIVATE ${SPARCEHASH_INCLUDE_DIR}) target_link_libraries (integer_hash_tables_and_hashes clickhouse_common_io) add_executable (allocator allocator.cpp) diff --git a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp index 15b01367cac..9d1eda9178c 100644 --- a/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock_fifo.cpp.cpp @@ -32,7 +32,7 @@ TEST(Common, RWLockFIFO_1) auto func = [&] (size_t threads, int round) { - for (int i = 0; i < cycles; ++i) + for (int i = 0; i < cycles; ++i) { auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockFIFO::Read : RWLockFIFO::Write; auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); diff --git a/dbms/src/Core/AccurateComparison.h b/dbms/src/Core/AccurateComparison.h index 23876d8e306..c84405e2dd4 100644 --- a/dbms/src/Core/AccurateComparison.h +++ b/dbms/src/Core/AccurateComparison.h @@ -139,6 +139,7 @@ inline bool_if_double_can_be_used equalsOpTmpl(TAFloat a, TAInt return static_cast(a) == static_cast(b); } + /* Final realiztions */ @@ -332,37 +333,6 @@ inline bool equalsOp(DB::Float32 f, DB::UInt128 u) return equalsOp(static_cast(f), u); } -inline bool greaterOp(DB::Int128 i, DB::Float64 f) -{ - static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64; - static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll; - - if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) - return static_cast(i) > f; - - return (f < static_cast(min_int128)) - || (f < static_cast(max_int128) && i > static_cast(f)); -} - -inline bool greaterOp(DB::Float64 f, DB::Int128 i) -{ - static constexpr __int128 min_int128 = __int128(0x8000000000000000ll) << 64; - static constexpr __int128 max_int128 = (__int128(0x7fffffffffffffffll) << 64) + 0xffffffffffffffffll; - - if (-MAX_INT64_WITH_EXACT_FLOAT64_REPR <= i && i <= MAX_INT64_WITH_EXACT_FLOAT64_REPR) - return f > static_cast(i); - - return (f >= static_cast(max_int128)) - || (f > static_cast(min_int128) && static_cast(f) > i); -} - -inline bool greaterOp(DB::Int128 i, DB::Float32 f) { return greaterOp(i, static_cast(f)); } -inline bool greaterOp(DB::Float32 f, DB::Int128 i) { return greaterOp(static_cast(f), i); } - -inline bool equalsOp(DB::Int128 i, DB::Float64 f) { return i == static_cast(f) && static_cast(i) == f; } -inline bool equalsOp(DB::Int128 i, DB::Float32 f) { return i == static_cast(f) && static_cast(i) == f; } -inline bool equalsOp(DB::Float64 f, DB::Int128 i) { return equalsOp(i, f); } -inline bool equalsOp(DB::Float32 f, DB::Int128 i) { return equalsOp(i, f); } template inline bool_if_not_safe_conversion notEqualsOp(A a, B b) diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index c3e77c11a92..514289ee325 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -54,7 +54,7 @@ void Block::insert(size_t position, const ColumnWithTypeAndName & elem) if (name_pos.second >= position) ++name_pos.second; - index_by_name.emplace(elem.name, position); + index_by_name[elem.name] = position; data.emplace(data.begin() + position, elem); } @@ -68,20 +68,20 @@ void Block::insert(size_t position, ColumnWithTypeAndName && elem) if (name_pos.second >= position) ++name_pos.second; - index_by_name.emplace(elem.name, position); + index_by_name[elem.name] = position; data.emplace(data.begin() + position, std::move(elem)); } void Block::insert(const ColumnWithTypeAndName & elem) { - index_by_name.emplace(elem.name, data.size()); + index_by_name[elem.name] = data.size(); data.emplace_back(elem); } void Block::insert(ColumnWithTypeAndName && elem) { - index_by_name.emplace(elem.name, data.size()); + index_by_name[elem.name] = data.size(); data.emplace_back(std::move(elem)); } @@ -368,18 +368,6 @@ Block Block::cloneWithColumns(const Columns & columns) const } -Block Block::cloneWithoutColumns() const -{ - Block res; - - size_t num_columns = data.size(); - for (size_t i = 0; i < num_columns; ++i) - res.insert({ nullptr, data[i].type, data[i].name }); - - return res; -} - - Block Block::sortColumns() const { Block sorted_block; diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 23c4b513f5f..94a4147aac2 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -109,7 +109,6 @@ public: Columns getColumns() const; void setColumns(const Columns & columns); Block cloneWithColumns(const Columns & columns) const; - Block cloneWithoutColumns() const; /** Get empty columns with the same types as in block. */ MutableColumns cloneEmptyColumns() const; diff --git a/dbms/src/Core/Defines.h b/dbms/src/Core/Defines.h index d7a2ca419e3..b7036171e6e 100644 --- a/dbms/src/Core/Defines.h +++ b/dbms/src/Core/Defines.h @@ -45,8 +45,6 @@ #define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226 #define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337 #define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372 -#define DBMS_MIN_REVISION_WITH_VERSION_PATCH 54401 -#define DBMS_MIN_REVISION_WITH_SERVER_LOGS 54406 /// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change. #define DBMS_TCP_PROTOCOL_VERSION 54226 diff --git a/dbms/src/Core/ExternalTable.cpp b/dbms/src/Core/ExternalTable.cpp deleted file mode 100644 index 3d8d8077e97..00000000000 --- a/dbms/src/Core/ExternalTable.cpp +++ /dev/null @@ -1,182 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int BAD_ARGUMENTS; -} - - -ExternalTableData BaseExternalTable::getData(const Context & context) -{ - initReadBuffer(); - initSampleBlock(); - auto input = context.getInputFormat(format, *read_buffer, sample_block, DEFAULT_BLOCK_SIZE); - return std::make_pair(std::make_shared(input), name); -} - -void BaseExternalTable::clean() -{ - name = ""; - file = ""; - format = ""; - structure.clear(); - sample_block = Block(); - read_buffer.reset(); -} - -/// Function for debugging information output -void BaseExternalTable::write() -{ - std::cerr << "file " << file << std::endl; - std::cerr << "name " << name << std::endl; - std::cerr << "format " << format << std::endl; - std::cerr << "structure: \n"; - for (size_t i = 0; i < structure.size(); ++i) - std::cerr << "\t" << structure[i].first << " " << structure[i].second << std::endl; -} - -std::vector BaseExternalTable::split(const std::string & s, const std::string & d) -{ - std::vector res; - boost::split(res, s, boost::algorithm::is_any_of(d), boost::algorithm::token_compress_on); - return res; -} - -void BaseExternalTable::parseStructureFromStructureField(const std::string & argument) -{ - std::vector vals = split(argument, " ,"); - - if (vals.size() & 1) - throw Exception("Odd number of attributes in section structure", ErrorCodes::BAD_ARGUMENTS); - - for (size_t i = 0; i < vals.size(); i += 2) - structure.emplace_back(vals[i], vals[i + 1]); -} - -void BaseExternalTable::parseStructureFromTypesField(const std::string & argument) -{ - std::vector vals = split(argument, " ,"); - - for (size_t i = 0; i < vals.size(); ++i) - structure.emplace_back("_" + toString(i + 1), vals[i]); -} - -void BaseExternalTable::initSampleBlock() -{ - const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); - - for (size_t i = 0; i < structure.size(); ++i) - { - ColumnWithTypeAndName column; - column.name = structure[i].first; - column.type = data_type_factory.get(structure[i].second); - column.column = column.type->createColumn(); - sample_block.insert(std::move(column)); - } -} - - -void ExternalTable::initReadBuffer() -{ - if (file == "-") - read_buffer = std::make_unique(STDIN_FILENO); - else - read_buffer = std::make_unique(file); -} - -ExternalTable::ExternalTable(const boost::program_options::variables_map & external_options) -{ - if (external_options.count("file")) - file = external_options["file"].as(); - else - throw Exception("--file field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); - - if (external_options.count("name")) - name = external_options["name"].as(); - else - throw Exception("--name field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); - - if (external_options.count("format")) - format = external_options["format"].as(); - else - throw Exception("--format field have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); - - if (external_options.count("structure")) - parseStructureFromStructureField(external_options["structure"].as()); - else if (external_options.count("types")) - parseStructureFromTypesField(external_options["types"].as()); - else - throw Exception("Neither --structure nor --types have not been provided for external table", ErrorCodes::BAD_ARGUMENTS); -} - - -void ExternalTablesHandler::handlePart(const Poco::Net::MessageHeader & header, std::istream & stream) -{ - const Settings & settings = context.getSettingsRef(); - - /// The buffer is initialized here, not in the virtual function initReadBuffer - read_buffer_impl = std::make_unique(stream); - - if (settings.http_max_multipart_form_data_size) - read_buffer = std::make_unique( - *read_buffer_impl, settings.http_max_multipart_form_data_size, - true, "the maximum size of multipart/form-data. This limit can be tuned by 'http_max_multipart_form_data_size' setting"); - else - read_buffer = std::move(read_buffer_impl); - - /// Retrieve a collection of parameters from MessageHeader - Poco::Net::NameValueCollection content; - std::string label; - Poco::Net::MessageHeader::splitParameters(header.get("Content-Disposition"), label, content); - - /// Get parameters - name = content.get("name", "_data"); - format = params.get(name + "_format", "TabSeparated"); - - if (params.has(name + "_structure")) - parseStructureFromStructureField(params.get(name + "_structure")); - else if (params.has(name + "_types")) - parseStructureFromTypesField(params.get(name + "_types")); - else - throw Exception("Neither structure nor types have not been provided for external table " + name + ". Use fields " + name + "_structure or " + name + "_types to do so.", ErrorCodes::BAD_ARGUMENTS); - - ExternalTableData data = getData(context); - - /// Create table - NamesAndTypesList columns = sample_block.getNamesAndTypesList(); - StoragePtr storage = StorageMemory::create(data.second, ColumnsDescription{columns}); - storage->startup(); - context.addExternalTable(data.second, storage); - BlockOutputStreamPtr output = storage->write(ASTPtr(), settings); - - /// Write data - data.first->readPrefix(); - output->writePrefix(); - while(Block block = data.first->read()) - output->write(block); - data.first->readSuffix(); - output->writeSuffix(); - - /// We are ready to receive the next file, for this we clear all the information received - clean(); -} - -} diff --git a/dbms/src/Core/ExternalTable.h b/dbms/src/Core/ExternalTable.h deleted file mode 100644 index 6927f19b524..00000000000 --- a/dbms/src/Core/ExternalTable.h +++ /dev/null @@ -1,111 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -#include - -#include -#include -#include - - -namespace Poco -{ - namespace Net - { - class NameValueCollection; - class MessageHeader; - } -} - -namespace boost -{ - namespace program_options - { - class variables_map; - } -} - - -namespace DB -{ - -class Context; - - -/// The base class containing the basic information about external table and -/// basic functions for extracting this information from text fields. -class BaseExternalTable -{ -public: - std::string file; /// File with data or '-' if stdin - std::string name; /// The name of the table - std::string format; /// Name of the data storage format - - /// Description of the table structure: (column name, data type name) - std::vector> structure; - - std::unique_ptr read_buffer; - Block sample_block; - - virtual ~BaseExternalTable() {} - - /// Initialize read_buffer, depending on the data source. By default, does nothing. - virtual void initReadBuffer() {} - - /// Get the table data - a pair (a stream with the contents of the table, the name of the table) - ExternalTableData getData(const Context & context); - -protected: - /// Clear all accumulated information - void clean(); - - /// Function for debugging information output - void write(); - - static std::vector split(const std::string & s, const std::string & d); - - /// Construct the `structure` vector from the text field `structure` - virtual void parseStructureFromStructureField(const std::string & argument); - - /// Construct the `structure` vector from the text field `types` - virtual void parseStructureFromTypesField(const std::string & argument); - -private: - /// Initialize sample_block according to the structure of the table stored in the `structure` - void initSampleBlock(); -}; - - -/// Parsing of external table used in the tcp client. -class ExternalTable : public BaseExternalTable -{ -public: - void initReadBuffer() override; - - /// Extract parameters from variables_map, which is built on the client command line - ExternalTable(const boost::program_options::variables_map & external_options); -}; - - -/// Parsing of external table used when sending tables via http -/// The `handlePart` function will be called for each table passed, -/// so it's also necessary to call `clean` at the end of the `handlePart`. -class ExternalTablesHandler : public Poco::Net::PartHandler, BaseExternalTable -{ -public: - ExternalTablesHandler(Context & context_, const Poco::Net::NameValueCollection & params_) : context(context_), params(params_) {} - - void handlePart(const Poco::Net::MessageHeader & header, std::istream & stream); - -private: - Context & context; - const Poco::Net::NameValueCollection & params; - std::unique_ptr read_buffer_impl; -}; - - -} diff --git a/dbms/src/Core/Field.cpp b/dbms/src/Core/Field.cpp index 279dd90c52b..1e14689d856 100644 --- a/dbms/src/Core/Field.cpp +++ b/dbms/src/Core/Field.cpp @@ -5,7 +5,6 @@ #include #include -#include namespace DB @@ -271,23 +270,4 @@ namespace DB DB::String res = applyVisitor(DB::FieldVisitorToString(), DB::Field(x)); buf.write(res.data(), res.size()); } - - - bool DecimalField::operator < (const DecimalField & r) const - { - using Comparator = DecimalComparison; - return Comparator::compare(Decimal128(dec), Decimal128(r.dec), scale, r.scale); - } - - bool DecimalField::operator <= (const DecimalField & r) const - { - using Comparator = DecimalComparison; - return Comparator::compare(Decimal128(dec), Decimal128(r.dec), scale, r.scale); - } - - bool DecimalField::operator == (const DecimalField & r) const - { - using Comparator = DecimalComparison; - return Comparator::compare(Decimal128(dec), Decimal128(r.dec), scale, r.scale); - } } diff --git a/dbms/src/Core/Field.h b/dbms/src/Core/Field.h index a77b22a3da4..0e1a9ca6c0a 100644 --- a/dbms/src/Core/Field.h +++ b/dbms/src/Core/Field.h @@ -25,38 +25,9 @@ namespace ErrorCodes class Field; using Array = std::vector; using TupleBackend = std::vector; -STRONG_TYPEDEF(TupleBackend, Tuple) /// Array and Tuple are different types with equal representation inside Field. +STRONG_TYPEDEF(TupleBackend, Tuple); /// Array and Tuple are different types with equal representation inside Field. -class DecimalField -{ -public: - static constexpr UInt32 wrongScale() { return std::numeric_limits::max(); } - - DecimalField(Int128 value, UInt32 scale_ = wrongScale()) - : dec(value), - scale(scale_) - {} - - operator Decimal32() const { return dec; } - operator Decimal64() const { return dec; } - operator Decimal128() const { return dec; } - - UInt32 getScale() const { return scale; } - - bool operator < (const DecimalField & r) const; - bool operator <= (const DecimalField & r) const; - bool operator == (const DecimalField & r) const; - - bool operator > (const DecimalField & r) const { return r < *this; } - bool operator >= (const DecimalField & r) const { return r <= * this; } - bool operator != (const DecimalField & r) const { return !(*this == r); } - -private: - Int128 dec; - UInt32 scale; -}; - /** 32 is enough. Round number is used for alignment and for better arithmetic inside std::vector. * NOTE: Actually, sizeof(std::string) is 32 when using libc++, so Field is 40 bytes. */ @@ -84,14 +55,12 @@ public: Int64 = 2, Float64 = 3, UInt128 = 4, - Int128 = 5, /// Non-POD types. String = 16, Array = 17, Tuple = 18, - Decimal = 19, }; static const int MIN_NON_POD = 16; @@ -104,15 +73,14 @@ public: case UInt64: return "UInt64"; case UInt128: return "UInt128"; case Int64: return "Int64"; - case Int128: return "Int128"; case Float64: return "Float64"; case String: return "String"; case Array: return "Array"; case Tuple: return "Tuple"; - case Decimal: return "Decimal"; - } - throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + default: + throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + } } }; @@ -289,15 +257,14 @@ public: case Types::UInt64: return get() < rhs.get(); case Types::UInt128: return get() < rhs.get(); case Types::Int64: return get() < rhs.get(); - case Types::Int128: return get() < rhs.get(); case Types::Float64: return get() < rhs.get(); case Types::String: return get() < rhs.get(); case Types::Array: return get() < rhs.get(); case Types::Tuple: return get() < rhs.get(); - case Types::Decimal: return get() < rhs.get(); - } - throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + default: + throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + } } bool operator> (const Field & rhs) const @@ -318,15 +285,15 @@ public: case Types::UInt64: return get() <= rhs.get(); case Types::UInt128: return get() <= rhs.get(); case Types::Int64: return get() <= rhs.get(); - case Types::Int128: return get() <= rhs.get(); case Types::Float64: return get() <= rhs.get(); case Types::String: return get() <= rhs.get(); case Types::Array: return get() <= rhs.get(); case Types::Tuple: return get() <= rhs.get(); - case Types::Decimal: return get() <= rhs.get(); - } - throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + + default: + throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + } } bool operator>= (const Field & rhs) const @@ -349,11 +316,10 @@ public: case Types::Array: return get() == rhs.get(); case Types::Tuple: return get() == rhs.get(); case Types::UInt128: return get() == rhs.get(); - case Types::Int128: return get() == rhs.get(); - case Types::Decimal: return get() == rhs.get(); - } - throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + default: + throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); + } } bool operator!= (const Field & rhs) const @@ -363,7 +329,7 @@ public: private: std::aligned_union_t storage; Types::Which which; @@ -404,7 +370,6 @@ private: case Types::UInt64: f(field.template get()); return; case Types::UInt128: f(field.template get()); return; case Types::Int64: f(field.template get()); return; - case Types::Int128: f(field.template get()); return; case Types::Float64: f(field.template get()); return; #if !__clang__ #pragma GCC diagnostic pop @@ -412,7 +377,6 @@ private: case Types::String: f(field.template get()); return; case Types::Array: f(field.template get()); return; case Types::Tuple: f(field.template get()); return; - case Types::Decimal: f(field.template get()); return; default: throw Exception("Bad type of Field", ErrorCodes::BAD_TYPE_OF_FIELD); @@ -491,23 +455,19 @@ template <> struct Field::TypeToEnum { static const Types::Which value template <> struct Field::TypeToEnum { static const Types::Which value = Types::UInt64; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::UInt128; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::Int64; }; -template <> struct Field::TypeToEnum { static const Types::Which value = Types::Int128; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::Float64; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::String; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::Array; }; template <> struct Field::TypeToEnum { static const Types::Which value = Types::Tuple; }; -template <> struct Field::TypeToEnum{ static const Types::Which value = Types::Decimal; }; template <> struct Field::EnumToType { using Type = Null; }; template <> struct Field::EnumToType { using Type = UInt64; }; template <> struct Field::EnumToType { using Type = UInt128; }; template <> struct Field::EnumToType { using Type = Int64; }; -template <> struct Field::EnumToType { using Type = Int128; }; template <> struct Field::EnumToType { using Type = Float64; }; template <> struct Field::EnumToType { using Type = String; }; template <> struct Field::EnumToType { using Type = Array; }; template <> struct Field::EnumToType { using Type = Tuple; }; -template <> struct Field::EnumToType { using Type = DecimalField; }; template @@ -550,10 +510,6 @@ template <> struct NearestFieldType { using Type = Int64; }; template <> struct NearestFieldType { using Type = Int64; }; template <> struct NearestFieldType { using Type = Int64; }; template <> struct NearestFieldType { using Type = Int64; }; -template <> struct NearestFieldType { using Type = Int128; }; -template <> struct NearestFieldType { using Type = DecimalField; }; -template <> struct NearestFieldType { using Type = DecimalField; }; -template <> struct NearestFieldType { using Type = DecimalField; }; template <> struct NearestFieldType { using Type = Float64; }; template <> struct NearestFieldType { using Type = Float64; }; template <> struct NearestFieldType { using Type = String; }; diff --git a/dbms/src/Core/Names.h b/dbms/src/Core/Names.h index ff8252084ac..5c3384112ae 100644 --- a/dbms/src/Core/Names.h +++ b/dbms/src/Core/Names.h @@ -12,6 +12,5 @@ namespace DB using Names = std::vector; using NameSet = std::unordered_set; using NameToNameMap = std::unordered_map; -using NameToNameSetMap = std::unordered_map; } diff --git a/dbms/src/Core/Protocol.h b/dbms/src/Core/Protocol.h index 5451e1550f6..cd5456cca34 100644 --- a/dbms/src/Core/Protocol.h +++ b/dbms/src/Core/Protocol.h @@ -69,7 +69,6 @@ namespace Protocol Totals = 7, /// A block with totals (compressed or not). Extremes = 8, /// A block with minimums and maximums (compressed or not). TablesStatusResponse = 9, /// A response to TablesStatus request. - Log = 10 /// System logs of the query execution }; /// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10 @@ -78,8 +77,8 @@ namespace Protocol /// See https://www.securecoding.cert.org/confluence/display/cplusplus/INT36-CPP.+Do+not+use+out-of-range+enumeration+values inline const char * toString(UInt64 packet) { - static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes", "TablesStatusResponse", "Log" }; - return packet < 11 + static const char * data[] = { "Hello", "Data", "Exception", "Progress", "Pong", "EndOfStream", "ProfileInfo", "Totals", "Extremes", "TablesStatusResponse" }; + return packet < 10 ? data[packet] : "Unknown packet"; } @@ -98,7 +97,6 @@ namespace Protocol Cancel = 3, /// Cancel the query execution. Ping = 4, /// Check that connection to the server is alive. TablesStatusRequest = 5, /// Check status of tables on the server. - KeepAlive = 6 /// Keep the connection alive }; inline const char * toString(UInt64 packet) diff --git a/dbms/src/Core/Types.h b/dbms/src/Core/Types.h index d967d4f52a3..1f09dcba17a 100644 --- a/dbms/src/Core/Types.h +++ b/dbms/src/Core/Types.h @@ -2,7 +2,7 @@ #include #include -#include +#include namespace DB @@ -12,15 +12,15 @@ namespace DB struct Null {}; -using UInt8 = uint8_t; -using UInt16 = uint16_t; -using UInt32 = uint32_t; -using UInt64 = uint64_t; +using UInt8 = Poco::UInt8; +using UInt16 = Poco::UInt16; +using UInt32 = Poco::UInt32; +using UInt64 = Poco::UInt64; -using Int8 = int8_t; -using Int16 = int16_t; -using Int32 = int32_t; -using Int64 = int64_t; +using Int8 = Poco::Int8; +using Int16 = Poco::Int16; +using Int32 = Poco::Int32; +using Int64 = Poco::Int64; using Float32 = float; using Float64 = double; @@ -57,116 +57,8 @@ template <> struct TypeName { static const char * get() { return "Float template <> struct TypeName { static const char * get() { return "Float64"; } }; template <> struct TypeName { static const char * get() { return "String"; } }; -template struct TypeId; - -/// 0 reserved for types without number -template <> struct TypeId { static constexpr const size_t value = 1; }; -template <> struct TypeId { static constexpr const size_t value = 2; }; -template <> struct TypeId { static constexpr const size_t value = 3; }; -template <> struct TypeId { static constexpr const size_t value = 4; }; -/// 5 reserved for TypeId -template <> struct TypeId { static constexpr const size_t value = 7; }; -template <> struct TypeId { static constexpr const size_t value = 8; }; -template <> struct TypeId { static constexpr const size_t value = 9; }; -template <> struct TypeId { static constexpr const size_t value = 10; }; -template <> struct TypeId { static constexpr const size_t value = 11; }; -template <> struct TypeId { static constexpr const size_t value = 12; }; -/// 13 reserved for TypeId /// Not a data type in database, defined just for convenience. using Strings = std::vector; } - -#if 1 /// __int128 -namespace DB -{ - -using Int128 = __int128; -template <> constexpr bool IsNumber = true; -template <> struct TypeName { static const char * get() { return "Int128"; } }; -template <> struct TypeId { static constexpr const size_t value = 13; }; - -} - -namespace std -{ - -template <> struct is_signed<__int128> -{ - static constexpr bool value = true; -}; - -template <> struct is_unsigned<__int128> -{ - static constexpr bool value = false; -}; - -template <> struct is_integral<__int128> -{ - static constexpr bool value = true; -}; - -template <> struct is_arithmetic<__int128> -{ - static constexpr bool value = true; -}; - -} -#endif - - -namespace DB -{ - /// Own FieldType for Decimal. - /// It is only a "storage" for decimal. To perform operations, you also have to provide a scale (number of digits after point). - template - struct Decimal - { - using NativeType = T; - - Decimal() = default; - Decimal(Decimal &&) = default; - Decimal(const Decimal &) = default; - - Decimal(const T & value_) - : value(value_) - {} - - template - Decimal(const Decimal & x) - : value(x) - {} - - constexpr Decimal & operator = (Decimal &&) = default; - constexpr Decimal & operator = (const Decimal &) = default; - - operator T () const { return value; } - - const Decimal & operator += (const T & x) { value += x; return *this; } - const Decimal & operator -= (const T & x) { value -= x; return *this; } - const Decimal & operator *= (const T & x) { value *= x; return *this; } - const Decimal & operator /= (const T & x) { value /= x; return *this; } - const Decimal & operator %= (const T & x) { value %= x; return *this; } - - T value; - }; - - using Decimal32 = Decimal; - using Decimal64 = Decimal; - using Decimal128 = Decimal; - - template <> struct TypeName { static const char * get() { return "Decimal32"; } }; - template <> struct TypeName { static const char * get() { return "Decimal64"; } }; - template <> struct TypeName { static const char * get() { return "Decimal128"; } }; - - template <> struct TypeId { static constexpr const size_t value = 16; }; - template <> struct TypeId { static constexpr const size_t value = 17; }; - template <> struct TypeId { static constexpr const size_t value = 18; }; - - template - constexpr bool IsDecimalNumber = false; - template <> constexpr bool IsDecimalNumber = true; - template <> constexpr bool IsDecimalNumber = true; - template <> constexpr bool IsDecimalNumber = true; -} diff --git a/dbms/src/Core/UUID.h b/dbms/src/Core/UUID.h index 4f8fdced814..bd6d330255f 100644 --- a/dbms/src/Core/UUID.h +++ b/dbms/src/Core/UUID.h @@ -6,6 +6,6 @@ namespace DB { -STRONG_TYPEDEF(UInt128, UUID) +STRONG_TYPEDEF(UInt128, UUID); } diff --git a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp index 0697ec8167c..7b431e206e9 100644 --- a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.cpp @@ -63,7 +63,7 @@ Block AggregatingSortedBlockInputStream::readImpl() for (size_t i = 0, size = columns_to_aggregate.size(); i < size; ++i) columns_to_aggregate[i] = typeid_cast(merged_columns[column_numbers_to_aggregate[i]].get()); - merge(merged_columns, queue_without_collation); + merge(merged_columns, queue); return header.cloneWithColumns(std::move(merged_columns)); } diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp b/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp deleted file mode 100644 index ba31b45bfd2..00000000000 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.cpp +++ /dev/null @@ -1,84 +0,0 @@ -#include "AsynchronousBlockInputStream.h" -#include - - -namespace DB -{ - -Block AsynchronousBlockInputStream::readImpl() -{ - /// If there were no calculations yet, calculate the first block synchronously - if (!started) - { - calculate(); - started = true; - } - else /// If the calculations are already in progress - wait for the result - pool.wait(); - - if (exception) - std::rethrow_exception(exception); - - Block res = block; - if (!res) - return res; - - /// Start the next block calculation - block.clear(); - next(); - - return res; -} - - -void AsynchronousBlockInputStream::next() -{ - ready.reset(); - - pool.schedule([this, thread_group=CurrentThread::getGroup()] () - { - CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; - - try - { - if (first) - setThreadName("AsyncBlockInput"); - - /// AsynchronousBlockInputStream is used in Client which does not create queries and thread groups - if (thread_group) - CurrentThread::attachToIfDetached(thread_group); - } - catch (...) - { - exception = std::current_exception(); - ready.set(); - return; - } - - calculate(); - }); -} - - -void AsynchronousBlockInputStream::calculate() -{ - try - { - if (first) - { - first = false; - children.back()->readPrefix(); - } - - block = children.back()->read(); - } - catch (...) - { - exception = std::current_exception(); - } - - ready.set(); -} - -} - diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index c790deb49c2..0a80628cf2a 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -7,7 +7,6 @@ #include #include #include -#include namespace CurrentMetrics @@ -92,12 +91,64 @@ protected: Block block; std::exception_ptr exception; - Block readImpl() override; - void next(); + Block readImpl() override + { + /// If there were no calculations yet, calculate the first block synchronously + if (!started) + { + calculate(current_memory_tracker); + started = true; + } + else /// If the calculations are already in progress - wait for the result + pool.wait(); + + if (exception) + std::rethrow_exception(exception); + + Block res = block; + if (!res) + return res; + + /// Start the next block calculation + block.clear(); + next(); + + return res; + } + + + void next() + { + ready.reset(); + pool.schedule(std::bind(&AsynchronousBlockInputStream::calculate, this, current_memory_tracker)); + } + /// Calculations that can be performed in a separate thread - void calculate(); + void calculate(MemoryTracker * memory_tracker) + { + CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; + + try + { + if (first) + { + first = false; + setThreadName("AsyncBlockInput"); + current_memory_tracker = memory_tracker; + children.back()->readPrefix(); + } + + block = children.back()->read(); + } + catch (...) + { + exception = std::current_exception(); + } + + ready.set(); + } }; } diff --git a/dbms/src/DataStreams/BlockIO.h b/dbms/src/DataStreams/BlockIO.h index 9f69f834e5f..6d97e30e510 100644 --- a/dbms/src/DataStreams/BlockIO.h +++ b/dbms/src/DataStreams/BlockIO.h @@ -12,7 +12,8 @@ class ProcessListEntry; struct BlockIO { /** process_list_entry should be destroyed after in and after out, - * since in and out contain pointer to objects inside process_list_entry (query-level MemoryTracker for example), + * since in and out contain pointer to an object inside process_list_entry + * (MemoryTracker * current_memory_tracker), * which could be used before destroying of in and out. */ std::shared_ptr process_list_entry; @@ -37,17 +38,12 @@ struct BlockIO exception_callback(); } - /// We provide the correct order of destruction. - void reset() - { - out.reset(); - in.reset(); - process_list_entry.reset(); - } - BlockIO & operator= (const BlockIO & rhs) { - reset(); + /// We provide the correct order of destruction. + out = nullptr; + in = nullptr; + process_list_entry = nullptr; process_list_entry = rhs.process_list_entry; in = rhs.in; diff --git a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp index 72df102a57f..01127b5029b 100644 --- a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.cpp @@ -40,45 +40,60 @@ void CollapsingSortedBlockInputStream::reportIncorrectData() } -void CollapsingSortedBlockInputStream::insertRows(MutableColumns & merged_columns, size_t & merged_rows) +void CollapsingSortedBlockInputStream::insertRows(MutableColumns & merged_columns, size_t & merged_rows, bool last_in_stream) { if (count_positive == 0 && count_negative == 0) - { - /// No input rows have been read. return; - } if (count_positive == count_negative && !last_is_positive) { - /// Input rows exactly cancel out. - return; + /// If all the rows in the input streams was collapsed, we still want to give at least one block in the result. + if (last_in_stream && merged_rows == 0 && !blocks_written) + { + LOG_INFO(log, "All rows collapsed"); + ++merged_rows; + for (size_t i = 0; i < num_columns; ++i) + merged_columns[i]->insertFrom(*(*last_positive.columns)[i], last_positive.row_num); + ++merged_rows; + for (size_t i = 0; i < num_columns; ++i) + merged_columns[i]->insertFrom(*(*last_negative.columns)[i], last_negative.row_num); + + if (out_row_sources_buf) + { + /// true flag value means "skip row" + current_row_sources[last_positive_pos].setSkipFlag(false); + current_row_sources[last_negative_pos].setSkipFlag(false); + } + } } - - if (count_positive <= count_negative) + else { - ++merged_rows; - for (size_t i = 0; i < num_columns; ++i) - merged_columns[i]->insertFrom(*(*first_negative.columns)[i], first_negative.row_num); + if (count_positive <= count_negative) + { + ++merged_rows; + for (size_t i = 0; i < num_columns; ++i) + merged_columns[i]->insertFrom(*(*first_negative.columns)[i], first_negative.row_num); - if (out_row_sources_buf) - current_row_sources[first_negative_pos].setSkipFlag(false); - } + if (out_row_sources_buf) + current_row_sources[first_negative_pos].setSkipFlag(false); + } - if (count_positive >= count_negative) - { - ++merged_rows; - for (size_t i = 0; i < num_columns; ++i) - merged_columns[i]->insertFrom(*(*last_positive.columns)[i], last_positive.row_num); + if (count_positive >= count_negative) + { + ++merged_rows; + for (size_t i = 0; i < num_columns; ++i) + merged_columns[i]->insertFrom(*(*last_positive.columns)[i], last_positive.row_num); - if (out_row_sources_buf) - current_row_sources[last_positive_pos].setSkipFlag(false); - } + if (out_row_sources_buf) + current_row_sources[last_positive_pos].setSkipFlag(false); + } - if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1)) - { - if (count_incorrect_data < MAX_ERROR_MESSAGES) - reportIncorrectData(); - ++count_incorrect_data; + if (!(count_positive == count_negative || count_positive + 1 == count_negative || count_positive == count_negative + 1)) + { + if (count_incorrect_data < MAX_ERROR_MESSAGES) + reportIncorrectData(); + ++count_incorrect_data; + } } if (out_row_sources_buf) @@ -102,7 +117,7 @@ Block CollapsingSortedBlockInputStream::readImpl() if (merged_columns.empty()) return {}; - merge(merged_columns, queue_without_collation); + merge(merged_columns, queue); return header.cloneWithColumns(std::move(merged_columns)); } @@ -196,7 +211,7 @@ void CollapsingSortedBlockInputStream::merge(MutableColumns & merged_columns, st } /// Write data for last primary key. - insertRows(merged_columns, merged_rows); + insertRows(merged_columns, merged_rows, true); finished = true; } diff --git a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h index 776b39c76d0..e8650b4efc5 100644 --- a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h @@ -74,7 +74,7 @@ private: void merge(MutableColumns & merged_columns, std::priority_queue & queue); /// Output to result rows for the current primary key. - void insertRows(MutableColumns & merged_columns, size_t & merged_rows); + void insertRows(MutableColumns & merged_columns, size_t & merged_rows, bool last_in_stream = false); void reportIncorrectData(); }; diff --git a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h b/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h deleted file mode 100644 index 1e3ce882dab..00000000000 --- a/dbms/src/DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h +++ /dev/null @@ -1,49 +0,0 @@ -#pragma once - -#include -#include -#include -#include - -namespace DB -{ - - -/** Combines several sources into one. - * Unlike UnionBlockInputStream, it does this sequentially. - * Blocks of different sources are not interleaved with each other. - */ -class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlockInputStream -{ -public: - explicit ConvertColumnWithDictionaryToFullBlockInputStream(const BlockInputStreamPtr & input) - { - children.push_back(input); - } - - String getName() const override { return "ConvertColumnWithDictionaryToFull"; } - - Block getHeader() const override { return convert(children.at(0)->getHeader()); } - -protected: - Block readImpl() override { return convert(children.back()->read()); } - -private: - Block convert(Block && block) const - { - for (auto & column : block) - { - if (auto * column_const = typeid_cast(column.column.get())) - column.column = column_const->removeLowCardinality(); - else - column.column = column.column->convertToFullColumnIfWithDictionary(); - - if (auto * low_cardinality_type = typeid_cast(column.type.get())) - column.type = low_cardinality_type->getDictionaryType(); - } - - return std::move(block); - } -}; - -} diff --git a/dbms/src/DataStreams/ConvertingBlockInputStream.cpp b/dbms/src/DataStreams/ConvertingBlockInputStream.cpp index 4c78aeb7ce5..8313f5820e5 100644 --- a/dbms/src/DataStreams/ConvertingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ConvertingBlockInputStream.cpp @@ -15,20 +15,6 @@ namespace ErrorCodes } -static ColumnPtr castColumnWithDiagnostic(const ColumnWithTypeAndName & src_elem, const ColumnWithTypeAndName & res_elem, const Context & context) -{ - try - { - return castColumn(src_elem, res_elem.type, context); - } - catch (Exception & e) - { - e.addMessage("while converting source column " + backQuoteIfNeed(src_elem.name) + " to destination column " + backQuoteIfNeed(res_elem.name)); - throw; - } -} - - ConvertingBlockInputStream::ConvertingBlockInputStream( const Context & context_, const BlockInputStreamPtr & input, @@ -83,7 +69,7 @@ ConvertingBlockInputStream::ConvertingBlockInputStream( /// Check conversion by dry run CAST function. - castColumnWithDiagnostic(src_elem, res_elem, context); + castColumn(src_elem, res_elem.type, context); } } @@ -101,7 +87,7 @@ Block ConvertingBlockInputStream::readImpl() const auto & src_elem = src.getByPosition(conversion[res_pos]); auto & res_elem = res.getByPosition(res_pos); - ColumnPtr converted = castColumnWithDiagnostic(src_elem, res_elem, context); + ColumnPtr converted = castColumn(src_elem, res_elem.type, context); if (src_elem.column->isColumnConst() && !res_elem.column->isColumnConst()) converted = converted->convertToFullColumnIfConst(); diff --git a/dbms/src/DataStreams/CountingBlockOutputStream.h b/dbms/src/DataStreams/CountingBlockOutputStream.h index ea1b5ec037d..12732473ac3 100644 --- a/dbms/src/DataStreams/CountingBlockOutputStream.h +++ b/dbms/src/DataStreams/CountingBlockOutputStream.h @@ -20,7 +20,7 @@ public: progress_callback = callback; } - void setProcessListElement(QueryStatus * elem) + void setProcessListElement(ProcessListElement * elem) { process_elem = elem; } @@ -43,7 +43,7 @@ protected: BlockOutputStreamPtr stream; Progress progress; ProgressCallback progress_callback; - QueryStatus * process_elem = nullptr; + ProcessListElement * process_elem = nullptr; }; } diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index a72784ffd96..0a58050339a 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -125,21 +125,6 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) if (!done_with_join) { - if (subquery.joined_block_actions) - subquery.joined_block_actions->execute(block); - - for (const auto & name_with_alias : subquery.joined_block_aliases) - { - if (block.has(name_with_alias.first)) - { - auto pos = block.getPositionByName(name_with_alias.first); - auto column = block.getByPosition(pos); - block.erase(pos); - column.name = name_with_alias.second; - block.insert(std::move(column)); - } - } - if (!subquery.join->insertFromBlock(block)) done_with_join = true; } diff --git a/dbms/src/DataStreams/FilterBlockInputStream.cpp b/dbms/src/DataStreams/FilterBlockInputStream.cpp index 39d0d0c9615..bd75dda1293 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.cpp +++ b/dbms/src/DataStreams/FilterBlockInputStream.cpp @@ -17,9 +17,8 @@ namespace ErrorCodes } -FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, - const String & filter_column_name, bool remove_filter) - : remove_filter(remove_filter), expression(expression_) +FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name) + : expression(expression_) { children.push_back(input); @@ -41,9 +40,6 @@ FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input FilterDescription filter_description_check(*column_elem.column); column_elem.column = column_elem.type->createColumnConst(header.rows(), UInt64(1)); } - - if (remove_filter) - header.erase(filter_column_name); } @@ -73,7 +69,7 @@ Block FilterBlockInputStream::readImpl() Block res; if (constant_filter_description.always_false) - return removeFilterIfNeed(std::move(res)); + return res; /// Until non-empty block after filtering or end of stream. while (1) @@ -85,7 +81,7 @@ Block FilterBlockInputStream::readImpl() expression->execute(res); if (constant_filter_description.always_true) - return removeFilterIfNeed(std::move(res)); + return res; size_t columns = res.columns(); ColumnPtr column = res.safeGetByPosition(filter_column).column; @@ -104,7 +100,7 @@ Block FilterBlockInputStream::readImpl() } if (constant_filter_description.always_true) - return removeFilterIfNeed(std::move(res)); + return res; FilterDescription filter_and_holder(*column); @@ -146,7 +142,7 @@ Block FilterBlockInputStream::readImpl() /// Replace the column with the filter by a constant. res.safeGetByPosition(filter_column).column = res.safeGetByPosition(filter_column).type->createColumnConst(filtered_rows, UInt64(1)); /// No need to touch the rest of the columns. - return removeFilterIfNeed(std::move(res)); + return res; } /// Filter the rest of the columns. @@ -174,18 +170,9 @@ Block FilterBlockInputStream::readImpl() current_column.column = current_column.column->filter(*filter_and_holder.data, -1); } - return removeFilterIfNeed(std::move(res)); + return res; } } -Block FilterBlockInputStream::removeFilterIfNeed(Block && block) -{ - if (block && remove_filter) - block.erase(static_cast(filter_column)); - - return std::move(block); -} - - } diff --git a/dbms/src/DataStreams/FilterBlockInputStream.h b/dbms/src/DataStreams/FilterBlockInputStream.h index ca63b34f45c..8089cf87420 100644 --- a/dbms/src/DataStreams/FilterBlockInputStream.h +++ b/dbms/src/DataStreams/FilterBlockInputStream.h @@ -20,8 +20,7 @@ private: using ExpressionActionsPtr = std::shared_ptr; public: - FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, - const String & filter_column_name_, bool remove_filter = false); + FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_); String getName() const override; Block getTotals() override; @@ -30,16 +29,12 @@ public: protected: Block readImpl() override; - bool remove_filter; - private: ExpressionActionsPtr expression; Block header; ssize_t filter_column; ConstantFilterDescription constant_filter_description; - - Block removeFilterIfNeed(Block && block); }; } diff --git a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp index f4424b29a11..0a2273d45a9 100644 --- a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp @@ -102,7 +102,7 @@ Block GraphiteRollupSortedBlockInputStream::readImpl() if (merged_columns.empty()) return Block(); - merge(merged_columns, queue_without_collation); + merge(merged_columns, queue); return header.cloneWithColumns(std::move(merged_columns)); } diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index 998ea2b42db..19842f74f03 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -1,13 +1,6 @@ #include #include #include -#include - - -namespace ProfileEvents -{ - extern const Event ThrottlerSleepMicroseconds; -} namespace DB @@ -195,7 +188,7 @@ static bool handleOverflowMode(OverflowMode mode, const String & message, int co default: throw Exception("Logical error: unknown overflow mode", ErrorCodes::LOGICAL_ERROR); } -} +}; bool IProfilingBlockInputStream::checkTimeLimit() @@ -293,34 +286,21 @@ void IProfilingBlockInputStream::progressImpl(const Progress & value) size_t total_rows = progress.total_rows; - constexpr UInt64 profile_events_update_period_microseconds = 10 * 1000; // 10 milliseconds - UInt64 total_elapsed_microseconds = info.total_stopwatch.elapsedMicroseconds(); - - if (last_profile_events_update_time + profile_events_update_period_microseconds < total_elapsed_microseconds) + if (limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0)) { - CurrentThread::updatePerformanceCounters(); - last_profile_events_update_time = total_elapsed_microseconds; - } + double total_elapsed = info.total_stopwatch.elapsedSeconds(); - if ((limits.min_execution_speed || (total_rows && limits.timeout_before_checking_execution_speed != 0)) - && (static_cast(total_elapsed_microseconds) > limits.timeout_before_checking_execution_speed.totalMicroseconds())) - { - /// Do not count sleeps in throttlers - UInt64 throttler_sleep_microseconds = CurrentThread::getProfileEvents()[ProfileEvents::ThrottlerSleepMicroseconds]; - double elapsed_seconds = (throttler_sleep_microseconds > total_elapsed_microseconds) - ? 0.0 : (total_elapsed_microseconds - throttler_sleep_microseconds) / 1000000.0; - - if (elapsed_seconds > 0) + if (total_elapsed > limits.timeout_before_checking_execution_speed.totalMicroseconds() / 1000000.0) { - if (limits.min_execution_speed && progress.rows / elapsed_seconds < limits.min_execution_speed) - throw Exception("Query is executing too slow: " + toString(progress.rows / elapsed_seconds) + 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); /// If the predicted execution time is longer than `max_execution_time`. if (limits.max_execution_time != 0 && total_rows) { - double estimated_execution_time_seconds = elapsed_seconds * (static_cast(total_rows) / progress.rows); + double estimated_execution_time_seconds = total_elapsed * (static_cast(total_rows) / progress.rows); if (estimated_execution_time_seconds > limits.max_execution_time.totalSeconds()) throw Exception("Estimated query execution time (" + toString(estimated_execution_time_seconds) + " seconds)" @@ -383,7 +363,7 @@ void IProfilingBlockInputStream::setProgressCallback(const ProgressCallback & ca } -void IProfilingBlockInputStream::setProcessListElement(QueryStatus * elem) +void IProfilingBlockInputStream::setProcessListElement(ProcessListElement * elem) { process_list_elem = elem; diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.h b/dbms/src/DataStreams/IProfilingBlockInputStream.h index ab0db8dd99d..5febcb18c56 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.h +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.h @@ -20,7 +20,6 @@ namespace ErrorCodes } class QuotaForIntervals; -class QueryStatus; class ProcessListElement; class IProfilingBlockInputStream; @@ -104,7 +103,7 @@ public: * Based on this information, the quota and some restrictions will be checked. * This information will also be available in the SHOW PROCESSLIST request. */ - void setProcessListElement(QueryStatus * elem); + void setProcessListElement(ProcessListElement * elem); /** Set the approximate total number of rows to read. */ @@ -179,9 +178,7 @@ protected: std::atomic is_cancelled{false}; std::atomic is_killed{false}; ProgressCallback progress_callback; - QueryStatus * process_list_elem = nullptr; - /// According to total_stopwatch in microseconds - UInt64 last_profile_events_update_time = 0; + ProcessListElement * process_list_elem = nullptr; /// Additional information that can be generated during the work process. diff --git a/dbms/src/DataStreams/InternalTextLogsRowOutputStream.cpp b/dbms/src/DataStreams/InternalTextLogsRowOutputStream.cpp deleted file mode 100644 index 02d2f8cf440..00000000000 --- a/dbms/src/DataStreams/InternalTextLogsRowOutputStream.cpp +++ /dev/null @@ -1,82 +0,0 @@ -#include "InternalTextLogsRowOutputStream.h" -#include -#include -#include -#include -#include -#include -#include - - -namespace DB -{ - -Block InternalTextLogsRowOutputStream::getHeader() const -{ - return InternalTextLogsQueue::getSampleBlock(); -} - -void InternalTextLogsRowOutputStream::write(const Block & block) -{ - auto & array_event_time = typeid_cast(*block.getByName("event_time").column).getData(); - auto & array_microseconds = typeid_cast(*block.getByName("event_time_microseconds").column).getData(); - - auto & column_host_name = typeid_cast(*block.getByName("host_name").column); - auto & column_query_id = typeid_cast(*block.getByName("query_id").column); - - auto & array_thread_number = typeid_cast(*block.getByName("thread_number").column).getData(); - auto & array_priority = typeid_cast(*block.getByName("priority").column).getData(); - auto & column_source = typeid_cast(*block.getByName("source").column); - auto & column_text = typeid_cast(*block.getByName("text").column); - - for (size_t row_num = 0; row_num < block.rows(); ++row_num) - { - auto host_name = column_host_name.getDataAt(row_num); - if (host_name.size) - { - writeCString("[", wb); - writeString(host_name, wb); - writeCString("] ", wb); - } - - auto event_time = array_event_time[row_num]; - writeDateTimeText<'.', ':'>(event_time, wb); - - auto microseconds = array_microseconds[row_num]; - writeChar('.', wb); - writeChar('0' + ((microseconds / 100000) % 10), wb); - writeChar('0' + ((microseconds / 10000) % 10), wb); - writeChar('0' + ((microseconds / 1000) % 10), wb); - writeChar('0' + ((microseconds / 100) % 10), wb); - writeChar('0' + ((microseconds / 10) % 10), wb); - writeChar('0' + ((microseconds / 1) % 10), wb); - - auto query_id = column_query_id.getDataAt(row_num); - if (query_id.size) - { - writeCString(" {", wb); - writeString(query_id, wb); - writeCString("}", wb); - } - - UInt32 thread_number = array_thread_number[row_num]; - writeCString(" [ ", wb); - writeIntText(thread_number, wb); - writeCString(" ] <", wb); - - Int8 priority = array_priority[row_num]; - writeString(InternalTextLogsQueue::getPriorityName(priority), wb); - writeCString("> ", wb); - - auto source = column_source.getDataAt(row_num); - writeString(source, wb); - writeCString(": ", wb); - - auto text = column_text.getDataAt(row_num); - writeString(text, wb); - - writeChar('\n', wb); - } -} - -} diff --git a/dbms/src/DataStreams/InternalTextLogsRowOutputStream.h b/dbms/src/DataStreams/InternalTextLogsRowOutputStream.h deleted file mode 100644 index 3f54a00e633..00000000000 --- a/dbms/src/DataStreams/InternalTextLogsRowOutputStream.h +++ /dev/null @@ -1,32 +0,0 @@ -#pragma once -#include -#include - - -namespace DB -{ - -/// Prints internal server logs -/// Input blocks have to have the same structure as SystemLogsQueue::getSampleBlock() -/// NOTE: IRowOutputStream does not suite well for this case -class InternalTextLogsRowOutputStream : public IBlockOutputStream -{ -public: - - InternalTextLogsRowOutputStream(WriteBuffer & buf_out) : wb(buf_out) {} - - Block getHeader() const override; - - void write(const Block & block) override; - - void flush() override - { - wb.next(); - } - -private: - - WriteBuffer & wb; -}; - -} diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index b062c679c0a..432bb0216c7 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -183,7 +183,7 @@ MergeSortingBlocksBlockInputStream::MergeSortingBlocksBlockInputStream( if (!has_collation) { for (size_t i = 0; i < cursors.size(); ++i) - queue_without_collation.push(SortCursor(&cursors[i])); + queue.push(SortCursor(&cursors[i])); } else { @@ -206,7 +206,7 @@ Block MergeSortingBlocksBlockInputStream::readImpl() } return !has_collation - ? mergeImpl(queue_without_collation) + ? mergeImpl(queue) : mergeImpl(queue_with_collation); } diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index ad6d81984cc..c52806f1b56 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -55,7 +55,7 @@ private: bool has_collation = false; - std::priority_queue queue_without_collation; + std::priority_queue queue; std::priority_queue queue_with_collation; /** Two different cursors are supported - with and without Collation. diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index bdabd8cc1f8..837c10869cf 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -3,7 +3,6 @@ #include #include #include -#include #include #include @@ -152,7 +151,7 @@ private: std::unique_ptr parallel_merge_data; - void mergeThread(ThreadGroupStatusPtr main_thread); + void mergeThread(MemoryTracker * memory_tracker); void finalize(); }; diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp index 8dd929759ca..62b32330679 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -58,7 +58,7 @@ void MergingSortedBlockInputStream::init(MutableColumns & merged_columns) if (has_collation) initQueue(queue_with_collation); else - initQueue(queue_without_collation); + initQueue(queue); } /// Let's check that all source blocks have the same structure. @@ -105,7 +105,7 @@ Block MergingSortedBlockInputStream::readImpl() if (has_collation) merge(merged_columns, queue_with_collation); else - merge(merged_columns, queue_without_collation); + merge(merged_columns, queue); return header.cloneWithColumns(std::move(merged_columns)); } @@ -200,7 +200,7 @@ void MergingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: // std::cerr << "copied columns\n"; - merged_rows = merged_columns.at(0)->size(); + size_t merged_rows = merged_columns.at(0)->size(); if (limit && total_merged_rows + merged_rows > limit) { diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.h b/dbms/src/DataStreams/MergingSortedBlockInputStream.h index 2a3fb7f5133..1da53d21fa4 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.h @@ -2,8 +2,17 @@ #include +#ifdef __clang__ + #pragma clang diagnostic push + #pragma clang diagnostic ignored "-Wzero-as-null-pointer-constant" +#endif + #include +#ifdef __clang__ + #pragma clang diagnostic pop +#endif + #include #include @@ -152,7 +161,7 @@ protected: CursorImpls cursors; using Queue = std::priority_queue; - Queue queue_without_collation; + Queue queue; using QueueWithCollation = std::priority_queue; QueueWithCollation queue_with_collation; diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index fbb2bb6e08f..dd99c8a3cf1 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -59,14 +59,8 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server void NativeBlockInputStream::readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint) { - IDataType::DeserializeBinaryBulkSettings settings; - settings.getter = [&](IDataType::SubstreamPath) -> ReadBuffer * { return &istr; }; - settings.avg_value_size_hint = avg_value_size_hint; - settings.position_independent_encoding = false; - - IDataType::DeserializeBinaryBulkStatePtr state; - type.deserializeBinaryBulkStatePrefix(settings, state); - type.deserializeBinaryBulkWithMultipleStreams(column, rows, settings, state); + IDataType::InputStreamGetter input_stream_getter = [&] (const IDataType::SubstreamPath &) { return &istr; }; + type.deserializeBinaryBulkWithMultipleStreams(column, input_stream_getter, rows, avg_value_size_hint, false, {}); if (column.size() != rows) throw Exception("Cannot read all data in NativeBlockInputStream. Rows read: " + toString(column.size()) + ". Rows expected: " + toString(rows) + ".", diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index 8be3373fc44..b02d435b39f 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -52,15 +52,8 @@ void NativeBlockOutputStream::writeData(const IDataType & type, const ColumnPtr else full_column = column; - IDataType::SerializeBinaryBulkSettings settings; - settings.getter = [&ostr](IDataType::SubstreamPath) -> WriteBuffer * { return &ostr; }; - settings.position_independent_encoding = false; - settings.low_cardinality_max_dictionary_size = 0; - - IDataType::SerializeBinaryBulkStatePtr state; - type.serializeBinaryBulkStatePrefix(settings, state); - type.serializeBinaryBulkWithMultipleStreams(*full_column, offset, limit, settings, state); - type.serializeBinaryBulkStateSuffix(settings, state); + IDataType::OutputStreamGetter output_stream_getter = [&] (const IDataType::SubstreamPath &) { return &ostr; }; + type.serializeBinaryBulkWithMultipleStreams(*full_column, output_stream_getter, offset, limit, false, {}); } diff --git a/dbms/src/DataStreams/ParallelInputsProcessor.h b/dbms/src/DataStreams/ParallelInputsProcessor.h index a1c4e2ac480..115dc6e1a3a 100644 --- a/dbms/src/DataStreams/ParallelInputsProcessor.h +++ b/dbms/src/DataStreams/ParallelInputsProcessor.h @@ -12,7 +12,6 @@ #include #include #include -#include /** Allows to process multiple block input streams (sources) in parallel, using specified number of threads. @@ -106,9 +105,8 @@ public: { active_threads = max_threads; threads.reserve(max_threads); - auto thread_group = CurrentThread::getGroup(); for (size_t i = 0; i < max_threads; ++i) - threads.emplace_back([=] () { thread(thread_group, i); } ); + threads.emplace_back(std::bind(&ParallelInputsProcessor::thread, this, current_memory_tracker, i)); } /// Ask all sources to stop earlier than they run out. @@ -176,16 +174,16 @@ private: } } - void thread(ThreadGroupStatusPtr thread_group, size_t thread_num) + void thread(MemoryTracker * memory_tracker, size_t thread_num) { + current_memory_tracker = memory_tracker; std::exception_ptr exception; + + setThreadName("ParalInputsProc"); CurrentMetrics::Increment metric_increment{CurrentMetrics::QueryThread}; try { - setThreadName("ParalInputsProc"); - CurrentThread::attachTo(thread_group); - while (!finish) { InputData unprepared_input; diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 670a70ad6bb..0aa129e264f 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -1,10 +1,8 @@ #include #include #include -#include #include #include -#include #include @@ -60,21 +58,21 @@ RemoteBlockInputStream::RemoteBlockInputStream( create_multiplexed_connections = [this, pool, throttler]() { - const Settings & current_settings = context.getSettingsRef(); + const Settings & settings = context.getSettingsRef(); std::vector connections; if (main_table) { - auto try_results = pool->getManyChecked(¤t_settings, pool_mode, *main_table); + auto try_results = pool->getManyChecked(&settings, pool_mode, *main_table); connections.reserve(try_results.size()); for (auto & try_result : try_results) connections.emplace_back(std::move(try_result.entry)); } else - connections = pool->getMany(¤t_settings, pool_mode); + connections = pool->getMany(&settings, pool_mode); return std::make_unique( - std::move(connections), current_settings, throttler, append_extra_info); + std::move(connections), settings, throttler, append_extra_info); }; } @@ -139,7 +137,7 @@ void RemoteBlockInputStream::sendExternalTables() for (const auto & table : external_tables) { StoragePtr cur = table.second; - QueryProcessingStage::Enum read_from_table_stage = cur->getQueryProcessingStage(context); + QueryProcessingStage::Enum read_from_table_stage = QueryProcessingStage::Complete; BlockInputStreams input = cur->read(cur->getColumns().getNamesOfPhysical(), {}, context, read_from_table_stage, DEFAULT_BLOCK_SIZE, 1); if (input.size() == 0) @@ -234,12 +232,6 @@ Block RemoteBlockInputStream::readImpl() extremes = packet.block; break; - case Protocol::Server::Log: - /// Pass logs from remote server to client - if (auto log_queue = CurrentThread::getInternalTextLogsQueue()) - log_queue->pushBlock(std::move(packet.block)); - break; - default: got_unknown_packet_from_replica = true; throw Exception("Unknown packet from server", ErrorCodes::UNKNOWN_PACKET_FROM_SERVER); diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index f1e68a6a0c1..d9095ec91b9 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -4,8 +4,6 @@ #include #include -#include -#include namespace DB @@ -26,33 +24,23 @@ RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const */ connection.sendQuery(query, "", QueryProcessingStage::Complete, settings, nullptr); - while (true) + Connection::Packet packet = connection.receivePacket(); + + if (Protocol::Server::Data == packet.type) { - Connection::Packet packet = connection.receivePacket(); + header = packet.block; - if (Protocol::Server::Data == packet.type) - { - header = packet.block; - - if (!header) - throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); - break; - } - else if (Protocol::Server::Exception == packet.type) - { - packet.exception->rethrow(); - break; - } - else if (Protocol::Server::Log == packet.type) - { - /// Pass logs from remote server to client - if (auto log_queue = CurrentThread::getInternalTextLogsQueue()) - log_queue->pushBlock(std::move(packet.block)); - } - else - throw NetException("Unexpected packet from server (expected Data or Exception, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + if (!header) + throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); } + else if (Protocol::Server::Exception == packet.type) + { + packet.exception->rethrow(); + return; + } + else + throw NetException("Unexpected packet from server (expected Data or Exception, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } @@ -67,11 +55,15 @@ void RemoteBlockOutputStream::write(const Block & block) catch (const NetException &) { /// Try to get more detailed exception from server - auto packet_type = connection.checkPacket(); - if (packet_type && *packet_type == Protocol::Server::Exception) + if (connection.poll(0)) { Connection::Packet packet = connection.receivePacket(); - packet.exception->rethrow(); + + if (Protocol::Server::Exception == packet.type) + { + packet.exception->rethrow(); + return; + } } throw; @@ -91,23 +83,18 @@ void RemoteBlockOutputStream::writeSuffix() /// Empty block means end of data. connection.sendData(Block()); - /// Wait for EndOfStream or Exception packet, skip Log packets. - while (true) - { - Connection::Packet packet = connection.receivePacket(); + /// Receive EndOfStream packet. + Connection::Packet packet = connection.receivePacket(); - if (Protocol::Server::EndOfStream == packet.type) - break; - else if (Protocol::Server::Exception == packet.type) - packet.exception->rethrow(); - else if (Protocol::Server::Log == packet.type) - { - // Do nothing - } - else - throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + if (Protocol::Server::EndOfStream == packet.type) + { + /// Do nothing. } + else if (Protocol::Server::Exception == packet.type) + packet.exception->rethrow(); + else + throw NetException("Unexpected packet from server (expected EndOfStream or Exception, got " + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); finished = true; } diff --git a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp index d0298ac77fe..db8efcd162f 100644 --- a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp @@ -44,7 +44,7 @@ Block ReplacingSortedBlockInputStream::readImpl() if (merged_columns.empty()) return Block(); - merge(merged_columns, queue_without_collation); + merge(merged_columns, queue); return header.cloneWithColumns(std::move(merged_columns)); } diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp index f87b2d63b11..a56b2928fc5 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.cpp @@ -195,7 +195,7 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream( } -void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns) +void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & merged_columns, bool force_insertion) { for (auto & desc : columns_to_aggregate) { @@ -237,9 +237,9 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me desc.merged_column->insertDefault(); } - /// If it is "zero" row, then rollback the insertion - /// (at this moment we need rollback only cols from columns_to_aggregate) - if (current_row_is_zero) + /// If it is "zero" row and it is not the last row of the result block, then + /// rollback the insertion (at this moment we need rollback only cols from columns_to_aggregate) + if (!force_insertion && current_row_is_zero) { for (auto & desc : columns_to_aggregate) desc.merged_column->popBack(1); @@ -252,6 +252,7 @@ void SummingSortedBlockInputStream::insertCurrentRowIfNeeded(MutableColumns & me /// Update per-block and per-group flags ++merged_rows; + output_is_non_empty = true; } @@ -286,7 +287,7 @@ Block SummingSortedBlockInputStream::readImpl() desc.merged_column = header.safeGetByPosition(desc.column_numbers[0]).column->cloneEmpty(); } - merge(merged_columns, queue_without_collation); + merge(merged_columns, queue); Block res = header.cloneWithColumns(std::move(merged_columns)); /// Place aggregation results into block. @@ -332,7 +333,7 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: { if (!current_key.empty()) /// Write the data for the previous group. - insertCurrentRowIfNeeded(merged_columns); + insertCurrentRowIfNeeded(merged_columns, false); if (merged_rows >= max_block_size) { @@ -392,7 +393,7 @@ void SummingSortedBlockInputStream::merge(MutableColumns & merged_columns, std:: /// We will write the data for the last group, if it is non-zero. /// If it is zero, and without it the output stream will be empty, we will write it anyway. - insertCurrentRowIfNeeded(merged_columns); + insertCurrentRowIfNeeded(merged_columns, !output_is_non_empty); finished = true; } diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.h b/dbms/src/DataStreams/SummingSortedBlockInputStream.h index 52c38454598..8cbd3c7c0f2 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.h @@ -134,6 +134,7 @@ private: Row current_row; bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. + bool output_is_non_empty = false; /// Have we given out at least one row as a result. size_t merged_rows = 0; /// Number of rows merged into current result block /** We support two different cursors - with Collation and without. @@ -142,7 +143,8 @@ private: void merge(MutableColumns & merged_columns, std::priority_queue & queue); /// Insert the summed row for the current group into the result and updates some of per-block flags if the row is not "zero". - void insertCurrentRowIfNeeded(MutableColumns & merged_columns); + /// If force_insertion=true, then the row will be inserted even if it is "zero" + void insertCurrentRowIfNeeded(MutableColumns & merged_columns, bool force_insertion); /// Returns true if merge result is not empty bool mergeMap(const MapDescription & map, Row & row, SortCursor & cursor); diff --git a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp index fc24bef60bc..071752137c6 100644 --- a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp @@ -15,11 +15,11 @@ namespace ErrorCodes VersionedCollapsingSortedBlockInputStream::VersionedCollapsingSortedBlockInputStream( const BlockInputStreams & inputs_, const SortDescription & description_, - const String & sign_column_, size_t max_block_size_, + const String & sign_column_, size_t max_block_size_, bool can_collapse_all_rows_, WriteBuffer * out_row_sources_buf_) : MergingSortedBlockInputStream(inputs_, description_, max_block_size_, 0, out_row_sources_buf_) , max_rows_in_queue(std::min(std::max(3, max_block_size_), MAX_ROWS_IN_MULTIVERSION_QUEUE) - 2) - , current_keys(max_rows_in_queue + 1) + , current_keys(max_rows_in_queue + 1), can_collapse_all_rows(can_collapse_all_rows_) { sign_column_number = header.getPositionByName(sign_column_); } @@ -76,7 +76,7 @@ Block VersionedCollapsingSortedBlockInputStream::readImpl() if (merged_columns.empty()) return {}; - merge(merged_columns, queue_without_collation); + merge(merged_columns, queue); return header.cloneWithColumns(std::move(merged_columns)); } @@ -130,7 +130,10 @@ void VersionedCollapsingSortedBlockInputStream::merge(MutableColumns & merged_co { update_queue(current); - if (sign == sign_in_queue) + /// If all the rows was collapsed, we still want to give at least one block in the result. + /// If queue is empty then don't collapse two last rows. + if (sign == sign_in_queue || (!can_collapse_all_rows && blocks_written == 0 + && merged_rows == 0 && queue.empty() && current_keys.size() == 1)) current_keys.pushBack(next_key); else { diff --git a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h index 5e84284d2ee..636ee5e3833 100644 --- a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h @@ -176,7 +176,7 @@ public: /// max_rows_in_queue should be about max_block_size_ if we won't store a lot of extra blocks (RowRef holds SharedBlockPtr). VersionedCollapsingSortedBlockInputStream( const BlockInputStreams & inputs_, const SortDescription & description_, - const String & sign_column_, size_t max_block_size_, + const String & sign_column_, size_t max_block_size_, bool can_collapse_all_rows_, WriteBuffer * out_row_sources_buf_ = nullptr); String getName() const override { return "VersionedCollapsingSorted"; } @@ -203,6 +203,8 @@ private: /// Sources of rows for VERTICAL merge algorithm. Size equals to (size + number of gaps) in current_keys. std::queue current_row_sources; + const bool can_collapse_all_rows; + void merge(MutableColumns & merged_columns, std::priority_queue & queue); /// Output to result row for the current primary key. diff --git a/dbms/src/DataStreams/tests/expression_stream.cpp b/dbms/src/DataStreams/tests/expression_stream.cpp index 3556c06b026..445c2a4b0df 100644 --- a/dbms/src/DataStreams/tests/expression_stream.cpp +++ b/dbms/src/DataStreams/tests/expression_stream.cpp @@ -47,7 +47,7 @@ try Names column_names; column_names.push_back("number"); - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); + QueryProcessingStage::Enum stage; BlockInputStreamPtr in; in = table->read(column_names, {}, context, stage, 8192, 1)[0]; diff --git a/dbms/src/DataStreams/tests/filter_stream.cpp b/dbms/src/DataStreams/tests/filter_stream.cpp index f1e9494e874..5e634fcf9a8 100644 --- a/dbms/src/DataStreams/tests/filter_stream.cpp +++ b/dbms/src/DataStreams/tests/filter_stream.cpp @@ -52,7 +52,7 @@ try Names column_names; column_names.push_back("number"); - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); + QueryProcessingStage::Enum stage; BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0]; in = std::make_shared(in, expression, "equals(modulo(number, 3), 1)"); diff --git a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/src/DataStreams/tests/native_streams.cpp b/dbms/src/DataStreams/tests/native_streams.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/src/DataStreams/tests/sorting_stream.cpp b/dbms/src/DataStreams/tests/sorting_stream.cpp deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/src/DataStreams/tests/union_stream2.cpp b/dbms/src/DataStreams/tests/union_stream2.cpp index 377dd76bf2e..fe0b225c376 100644 --- a/dbms/src/DataStreams/tests/union_stream2.cpp +++ b/dbms/src/DataStreams/tests/union_stream2.cpp @@ -34,7 +34,7 @@ try StoragePtr table = context.getTable("default", "hits6"); - QueryProcessingStage::Enum stage = table->getQueryProcessingStage(context); + QueryProcessingStage::Enum stage; BlockInputStreams streams = table->read(column_names, {}, context, stage, settings.max_block_size, settings.max_threads); for (size_t i = 0, size = streams.size(); i < size; ++i) diff --git a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp index f005f2e2eea..86d96ece5f2 100644 --- a/dbms/src/DataTypes/DataTypeAggregateFunction.cpp +++ b/dbms/src/DataTypes/DataTypeAggregateFunction.cpp @@ -212,9 +212,9 @@ void DataTypeAggregateFunction::deserializeTextQuoted(IColumn & column, ReadBuff } -void DataTypeAggregateFunction::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeAggregateFunction::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeJSONString(serializeToString(function, column, row_num), ostr, settings); + writeJSONString(serializeToString(function, column, row_num), ostr); } diff --git a/dbms/src/DataTypes/DataTypeArray.cpp b/dbms/src/DataTypes/DataTypeArray.cpp index 9a533424198..bc7109f65f5 100644 --- a/dbms/src/DataTypes/DataTypeArray.cpp +++ b/dbms/src/DataTypes/DataTypeArray.cpp @@ -145,67 +145,37 @@ namespace } -void DataTypeArray::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeArray::enumerateStreams(StreamCallback callback, SubstreamPath path) const { path.push_back(Substream::ArraySizes); callback(path); path.back() = Substream::ArrayElements; nested->enumerateStreams(callback, path); - path.pop_back(); -} - - -void DataTypeArray::serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - settings.path.push_back(Substream::ArrayElements); - nested->serializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); -} - - -void DataTypeArray::serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - settings.path.push_back(Substream::ArrayElements); - nested->serializeBinaryBulkStateSuffix(settings, state); - settings.path.pop_back(); -} - - -void DataTypeArray::deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const -{ - settings.path.push_back(Substream::ArrayElements); - nested->deserializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); } void DataTypeArray::serializeBinaryBulkWithMultipleStreams( const IColumn & column, + OutputStreamGetter getter, size_t offset, size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const + bool position_independent_encoding, + SubstreamPath path) const { const ColumnArray & column_array = typeid_cast(column); /// First serialize array sizes. - settings.path.push_back(Substream::ArraySizes); - if (auto stream = settings.getter(settings.path)) + path.push_back(Substream::ArraySizes); + if (auto stream = getter(path)) { - if (settings.position_independent_encoding) + if (position_independent_encoding) serializeArraySizesPositionIndependent(column, *stream, offset, limit); else DataTypeNumber().serializeBinaryBulk(*column_array.getOffsetsPtr(), *stream, offset, limit); } /// Then serialize contents of arrays. - settings.path.back() = Substream::ArrayElements; + path.back() = Substream::ArrayElements; const ColumnArray::Offsets & offset_values = column_array.getOffsets(); if (offset > offset_values.size()) @@ -227,29 +197,30 @@ void DataTypeArray::serializeBinaryBulkWithMultipleStreams( : 0; if (limit == 0 || nested_limit) - nested->serializeBinaryBulkWithMultipleStreams(column_array.getData(), nested_offset, nested_limit, settings, state); - settings.path.pop_back(); + nested->serializeBinaryBulkWithMultipleStreams(column_array.getData(), getter, nested_offset, nested_limit, position_independent_encoding, path); } void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( IColumn & column, + InputStreamGetter getter, size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + double /*avg_value_size_hint*/, + bool position_independent_encoding, + SubstreamPath path) const { ColumnArray & column_array = typeid_cast(column); - settings.path.push_back(Substream::ArraySizes); - if (auto stream = settings.getter(settings.path)) + path.push_back(Substream::ArraySizes); + if (auto stream = getter(path)) { - if (settings.position_independent_encoding) + if (position_independent_encoding) deserializeArraySizesPositionIndependent(column, *stream, limit); else DataTypeNumber().deserializeBinaryBulk(column_array.getOffsetsColumn(), *stream, limit, 0); } - settings.path.back() = Substream::ArrayElements; + path.back() = Substream::ArrayElements; ColumnArray::Offsets & offset_values = column_array.getOffsets(); IColumn & nested_column = column_array.getData(); @@ -259,8 +230,7 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreams( if (last_offset < nested_column.size()) throw Exception("Nested column is longer than last offset", ErrorCodes::LOGICAL_ERROR); size_t nested_limit = last_offset - nested_column.size(); - nested->deserializeBinaryBulkWithMultipleStreams(nested_column, nested_limit, settings, state); - settings.path.pop_back(); + nested->deserializeBinaryBulkWithMultipleStreams(nested_column, getter, nested_limit, 0, position_independent_encoding, path); /// Check consistency between offsets and elements subcolumns. /// But if elements column is empty - it's ok for columns of Nested types that was added by ALTER. diff --git a/dbms/src/DataTypes/DataTypeArray.h b/dbms/src/DataTypes/DataTypeArray.h index cc6e437bd15..75a0f6c9c8a 100644 --- a/dbms/src/DataTypes/DataTypeArray.h +++ b/dbms/src/DataTypes/DataTypeArray.h @@ -61,32 +61,23 @@ public: * This is necessary, because when implementing nested structures, several arrays can have common sizes. */ - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; - - void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - size_t offset, - size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; + const IColumn & column, + OutputStreamGetter getter, + size_t offset, + size_t limit, + bool position_independent_encoding, + SubstreamPath path) const override; void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + IColumn & column, + InputStreamGetter getter, + size_t limit, + double avg_value_size_hint, + bool position_independent_encoding, + SubstreamPath path) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeEnum.cpp b/dbms/src/DataTypes/DataTypeEnum.cpp index edd3b797602..e9b87670928 100644 --- a/dbms/src/DataTypes/DataTypeEnum.cpp +++ b/dbms/src/DataTypes/DataTypeEnum.cpp @@ -97,7 +97,7 @@ DataTypeEnum::DataTypeEnum(const Values & values_) : values{values_} }); fillMaps(); - type_name = generateName(values); + name = generateName(values); } template @@ -145,9 +145,9 @@ template void DataTypeEnum::deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { /// NOTE It would be nice to do without creating a temporary object - at least extract std::string out. - std::string field_name; - readEscapedString(field_name, istr); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + std::string name; + readEscapedString(name, istr); + static_cast(column).getData().push_back(getValue(StringRef(name))); } template @@ -159,15 +159,15 @@ void DataTypeEnum::serializeTextQuoted(const IColumn & column, size_t row_ template void DataTypeEnum::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - std::string field_name; - readQuotedStringWithSQLStyle(field_name, istr); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + std::string name; + readQuotedStringWithSQLStyle(name, istr); + static_cast(column).getData().push_back(getValue(StringRef(name))); } template -void DataTypeEnum::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeEnum::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeJSONString(getNameForValue(static_cast(column).getData()[row_num]), ostr, settings); + writeJSONString(getNameForValue(static_cast(column).getData()[row_num]), ostr); } template @@ -179,9 +179,9 @@ void DataTypeEnum::serializeTextXML(const IColumn & column, size_t row_num template void DataTypeEnum::deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings &) const { - std::string field_name; - readJSONString(field_name, istr); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + std::string name; + readJSONString(name, istr); + static_cast(column).getData().push_back(getValue(StringRef(name))); } template @@ -193,9 +193,9 @@ void DataTypeEnum::serializeTextCSV(const IColumn & column, size_t row_num template void DataTypeEnum::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const { - std::string field_name; - readCSVString(field_name, istr, settings.csv); - static_cast(column).getData().push_back(getValue(StringRef(field_name))); + std::string name; + readCSVString(name, istr, settings.csv); + static_cast(column).getData().push_back(getValue(StringRef(name))); } template @@ -237,7 +237,7 @@ void DataTypeEnum::insertDefaultInto(IColumn & column) const template bool DataTypeEnum::equals(const IDataType & rhs) const { - return typeid(rhs) == typeid(*this) && type_name == static_cast &>(rhs).type_name; + return typeid(rhs) == typeid(*this) && name == static_cast &>(rhs).name; } @@ -346,14 +346,14 @@ static DataTypePtr create(const ASTPtr & arguments) throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer", ErrorCodes::UNEXPECTED_AST_STRUCTURE); - const String & field_name = name_literal->value.get(); + const String & name = name_literal->value.get(); const auto value = value_literal->value.get::Type>(); if (value > std::numeric_limits::max() || value < std::numeric_limits::min()) - throw Exception{"Value " + toString(value) + " for element '" + field_name + "' exceeds range of " + EnumName::value, + throw Exception{"Value " + toString(value) + " for element '" + name + "' exceeds range of " + EnumName::value, ErrorCodes::ARGUMENT_OUT_OF_BOUND}; - values.emplace_back(field_name, value); + values.emplace_back(name, value); } return std::make_shared(values); diff --git a/dbms/src/DataTypes/DataTypeEnum.h b/dbms/src/DataTypes/DataTypeEnum.h index 14be08aae10..c9abddb742c 100644 --- a/dbms/src/DataTypes/DataTypeEnum.h +++ b/dbms/src/DataTypes/DataTypeEnum.h @@ -53,7 +53,7 @@ private: Values values; NameToValueMap name_to_value_map; ValueToNameMap value_to_name_map; - std::string type_name; + std::string name; static std::string generateName(const Values & values); void fillMaps(); @@ -62,7 +62,7 @@ public: explicit DataTypeEnum(const Values & values_); const Values & getValues() const { return values; } - std::string getName() const override { return type_name; } + std::string getName() const override { return name; } const char * getFamilyName() const override; const StringRef & getNameForValue(const FieldType & value) const @@ -74,11 +74,11 @@ public: return it->second; } - FieldType getValue(StringRef field_name) const + FieldType getValue(StringRef name) const { - const auto it = name_to_value_map.find(field_name); + const auto it = name_to_value_map.find(name); if (it == std::end(name_to_value_map)) - throw Exception{"Unknown element '" + field_name.toString() + "' for type " + getName(), ErrorCodes::LOGICAL_ERROR}; + throw Exception{"Unknown element '" + name.toString() + "' for type " + getName(), ErrorCodes::LOGICAL_ERROR}; return it->second; } diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index ad123a7431b..9706ecf4944 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -6,7 +6,6 @@ #include #include #include -#include namespace DB @@ -56,23 +55,6 @@ DataTypePtr DataTypeFactory::get(const String & family_name_param, const ASTPtr { String family_name = getAliasToOrName(family_name_param); - if (endsWith(family_name, "WithDictionary")) - { - ASTPtr low_cardinality_params = std::make_shared(); - String param_name = family_name.substr(0, family_name.size() - strlen("WithDictionary")); - if (parameters) - { - auto func = std::make_shared(); - func->name = param_name; - func->arguments = parameters; - low_cardinality_params->children.push_back(func); - } - else - low_cardinality_params->children.push_back(std::make_shared(param_name)); - - return get("LowCardinality", low_cardinality_params); - } - { DataTypesDictionary::const_iterator it = data_types.find(family_name); if (data_types.end() != it) @@ -129,7 +111,6 @@ void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator } void registerDataTypeNumbers(DataTypeFactory & factory); -void registerDataTypeDecimal(DataTypeFactory & factory); void registerDataTypeDate(DataTypeFactory & factory); void registerDataTypeDateTime(DataTypeFactory & factory); void registerDataTypeString(DataTypeFactory & factory); @@ -143,13 +124,11 @@ void registerDataTypeUUID(DataTypeFactory & factory); void registerDataTypeAggregateFunction(DataTypeFactory & factory); void registerDataTypeNested(DataTypeFactory & factory); void registerDataTypeInterval(DataTypeFactory & factory); -void registerDataTypeWithDictionary(DataTypeFactory & factory); DataTypeFactory::DataTypeFactory() { registerDataTypeNumbers(*this); - registerDataTypeDecimal(*this); registerDataTypeDate(*this); registerDataTypeDateTime(*this); registerDataTypeString(*this); @@ -163,7 +142,6 @@ DataTypeFactory::DataTypeFactory() registerDataTypeAggregateFunction(*this); registerDataTypeNested(*this); registerDataTypeInterval(*this); - registerDataTypeWithDictionary(*this); } } diff --git a/dbms/src/DataTypes/DataTypeFixedString.cpp b/dbms/src/DataTypes/DataTypeFixedString.cpp index c256c5e6214..ad875c4f85e 100644 --- a/dbms/src/DataTypes/DataTypeFixedString.cpp +++ b/dbms/src/DataTypes/DataTypeFixedString.cpp @@ -168,10 +168,10 @@ void DataTypeFixedString::deserializeTextQuoted(IColumn & column, ReadBuffer & i } -void DataTypeFixedString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeFixedString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { const char * pos = reinterpret_cast(&static_cast(column).getChars()[n * row_num]); - writeJSONString(pos, pos + n, ostr, settings); + writeJSONString(pos, pos + n, ostr); } diff --git a/dbms/src/DataTypes/DataTypeNullable.cpp b/dbms/src/DataTypes/DataTypeNullable.cpp index ebd10fe9735..caaf514d231 100644 --- a/dbms/src/DataTypes/DataTypeNullable.cpp +++ b/dbms/src/DataTypes/DataTypeNullable.cpp @@ -37,83 +37,53 @@ bool DataTypeNullable::onlyNull() const } -void DataTypeNullable::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeNullable::enumerateStreams(StreamCallback callback, SubstreamPath path) const { path.push_back(Substream::NullMap); callback(path); path.back() = Substream::NullableElements; nested_data_type->enumerateStreams(callback, path); - path.pop_back(); -} - - -void DataTypeNullable::serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - settings.path.push_back(Substream::NullableElements); - nested_data_type->serializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); -} - - -void DataTypeNullable::serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - settings.path.push_back(Substream::NullableElements); - nested_data_type->serializeBinaryBulkStateSuffix(settings, state); - settings.path.pop_back(); -} - - -void DataTypeNullable::deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const -{ - settings.path.push_back(Substream::NullableElements); - nested_data_type->deserializeBinaryBulkStatePrefix(settings, state); - settings.path.pop_back(); } void DataTypeNullable::serializeBinaryBulkWithMultipleStreams( const IColumn & column, + OutputStreamGetter getter, size_t offset, size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const + bool position_independent_encoding, + SubstreamPath path) const { const ColumnNullable & col = static_cast(column); col.checkConsistency(); /// First serialize null map. - settings.path.push_back(Substream::NullMap); - if (auto stream = settings.getter(settings.path)) + path.push_back(Substream::NullMap); + if (auto stream = getter(path)) DataTypeUInt8().serializeBinaryBulk(col.getNullMapColumn(), *stream, offset, limit); /// Then serialize contents of arrays. - settings.path.back() = Substream::NullableElements; - nested_data_type->serializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), offset, limit, settings, state); - settings.path.pop_back(); + path.back() = Substream::NullableElements; + nested_data_type->serializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, offset, limit, position_independent_encoding, path); } void DataTypeNullable::deserializeBinaryBulkWithMultipleStreams( IColumn & column, + InputStreamGetter getter, size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + double avg_value_size_hint, + bool position_independent_encoding, + SubstreamPath path) const { ColumnNullable & col = static_cast(column); - settings.path.push_back(Substream::NullMap); - if (auto stream = settings.getter(settings.path)) + path.push_back(Substream::NullMap); + if (auto stream = getter(path)) DataTypeUInt8().deserializeBinaryBulk(col.getNullMapColumn(), *stream, limit, 0); - settings.path.back() = Substream::NullableElements; - nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), limit, settings, state); - settings.path.pop_back(); + path.back() = Substream::NullableElements; + nested_data_type->deserializeBinaryBulkWithMultipleStreams(col.getNestedColumn(), getter, limit, avg_value_size_hint, position_independent_encoding, path); } diff --git a/dbms/src/DataTypes/DataTypeNullable.h b/dbms/src/DataTypes/DataTypeNullable.h index a331d42738c..efca95f2d66 100644 --- a/dbms/src/DataTypes/DataTypeNullable.h +++ b/dbms/src/DataTypes/DataTypeNullable.h @@ -13,36 +13,27 @@ class DataTypeNullable final : public IDataType public: static constexpr bool is_parametric = true; - explicit DataTypeNullable(const DataTypePtr & nested_data_type_); + DataTypeNullable(const DataTypePtr & nested_data_type_); std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; } const char * getFamilyName() const override { return "Nullable"; } - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; - - void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - size_t offset, - size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; + const IColumn & column, + OutputStreamGetter getter, + size_t offset, + size_t limit, + bool position_independent_encoding, + SubstreamPath path) const override; void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + IColumn & column, + InputStreamGetter getter, + size_t limit, + double avg_value_size_hint, + bool position_independent_encoding, + SubstreamPath path) const override; void serializeBinary(const Field & field, WriteBuffer & ostr) const override { nested_data_type->serializeBinary(field, ostr); } void deserializeBinary(Field & field, ReadBuffer & istr) const override { nested_data_type->deserializeBinary(field, istr); } diff --git a/dbms/src/DataTypes/DataTypeNumberBase.h b/dbms/src/DataTypes/DataTypeNumberBase.h index ca365bfaa78..439b3ebc8db 100644 --- a/dbms/src/DataTypes/DataTypeNumberBase.h +++ b/dbms/src/DataTypes/DataTypeNumberBase.h @@ -16,7 +16,6 @@ public: using FieldType = T; const char * getFamilyName() const override { return TypeName::get(); } - size_t getTypeId() const override { return TypeId::value; } void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; diff --git a/dbms/src/DataTypes/DataTypeString.cpp b/dbms/src/DataTypes/DataTypeString.cpp index 308c3e04957..671d1b2d3a5 100644 --- a/dbms/src/DataTypes/DataTypeString.cpp +++ b/dbms/src/DataTypes/DataTypeString.cpp @@ -262,9 +262,9 @@ void DataTypeString::deserializeTextQuoted(IColumn & column, ReadBuffer & istr, } -void DataTypeString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +void DataTypeString::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const { - writeJSONString(static_cast(column).getDataAt(row_num), ostr, settings); + writeJSONString(static_cast(column).getDataAt(row_num), ostr); } diff --git a/dbms/src/DataTypes/DataTypeTuple.cpp b/dbms/src/DataTypes/DataTypeTuple.cpp index c74b7c9191f..9f671b6ddcc 100644 --- a/dbms/src/DataTypes/DataTypeTuple.cpp +++ b/dbms/src/DataTypes/DataTypeTuple.cpp @@ -283,7 +283,7 @@ void DataTypeTuple::deserializeTextCSV(IColumn & column, ReadBuffer & istr, cons }); } -void DataTypeTuple::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const +void DataTypeTuple::enumerateStreams(StreamCallback callback, SubstreamPath path) const { path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) @@ -291,139 +291,40 @@ void DataTypeTuple::enumerateStreams(const StreamCallback & callback, SubstreamP path.back().tuple_element_name = names[i]; elems[i]->enumerateStreams(callback, path); } - path.pop_back(); -} - -struct SerializeBinaryBulkStateTuple : public IDataType::SerializeBinaryBulkState -{ - std::vector states; -}; - -struct DeserializeBinaryBulkStateTuple : public IDataType::DeserializeBinaryBulkState -{ - std::vector states; -}; - -static SerializeBinaryBulkStateTuple * checkAndGetTupleSerializeState(IDataType::SerializeBinaryBulkStatePtr & state) -{ - if (!state) - throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR); - - auto * tuple_state = typeid_cast(state.get()); - if (!tuple_state) - { - auto & state_ref = *state; - throw Exception("Invalid SerializeBinaryBulkState for DataTypeTuple. Expected: " - + demangle(typeid(SerializeBinaryBulkStateTuple).name()) + ", got " - + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); - } - - return tuple_state; -} - -static DeserializeBinaryBulkStateTuple * checkAndGetTupleDeserializeState(IDataType::DeserializeBinaryBulkStatePtr & state) -{ - if (!state) - throw Exception("Got empty state for DataTypeTuple.", ErrorCodes::LOGICAL_ERROR); - - auto * tuple_state = typeid_cast(state.get()); - if (!tuple_state) - { - auto & state_ref = *state; - throw Exception("Invalid DeserializeBinaryBulkState for DataTypeTuple. Expected: " - + demangle(typeid(DeserializeBinaryBulkStateTuple).name()) + ", got " - + demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR); - } - - return tuple_state; -} - -void DataTypeTuple::serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - auto tuple_state = std::make_shared(); - tuple_state->states.resize(elems.size()); - - settings.path.push_back(Substream::TupleElement); - for (size_t i = 0; i < elems.size(); ++i) - { - settings.path.back().tuple_element_name = names[i]; - elems[i]->serializeBinaryBulkStatePrefix(settings, tuple_state->states[i]); - } - settings.path.pop_back(); - - state = std::move(tuple_state); -} - -void DataTypeTuple::serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const -{ - auto * tuple_state = checkAndGetTupleSerializeState(state); - - settings.path.push_back(Substream::TupleElement); - for (size_t i = 0; i < elems.size(); ++i) - { - settings.path.back().tuple_element_name = names[i]; - elems[i]->serializeBinaryBulkStateSuffix(settings, tuple_state->states[i]); - } - settings.path.pop_back(); -} - -void DataTypeTuple::deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const -{ - auto tuple_state = std::make_shared(); - tuple_state->states.resize(elems.size()); - - settings.path.push_back(Substream::TupleElement); - for (size_t i = 0; i < elems.size(); ++i) - { - settings.path.back().tuple_element_name = names[i]; - elems[i]->deserializeBinaryBulkStatePrefix(settings, tuple_state->states[i]); - } - settings.path.pop_back(); - - state = std::move(tuple_state); } void DataTypeTuple::serializeBinaryBulkWithMultipleStreams( const IColumn & column, + OutputStreamGetter getter, size_t offset, size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const + bool position_independent_encoding, + SubstreamPath path) const { - auto * tuple_state = checkAndGetTupleSerializeState(state); - - settings.path.push_back(Substream::TupleElement); + path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) { - settings.path.back().tuple_element_name = names[i]; - auto & element_col = extractElementColumn(column, i); - elems[i]->serializeBinaryBulkWithMultipleStreams(element_col, offset, limit, settings, tuple_state->states[i]); + path.back().tuple_element_name = names[i]; + elems[i]->serializeBinaryBulkWithMultipleStreams( + extractElementColumn(column, i), getter, offset, limit, position_independent_encoding, path); } - settings.path.pop_back(); } void DataTypeTuple::deserializeBinaryBulkWithMultipleStreams( IColumn & column, + InputStreamGetter getter, size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const + double avg_value_size_hint, + bool position_independent_encoding, + SubstreamPath path) const { - auto * tuple_state = checkAndGetTupleDeserializeState(state); - - settings.path.push_back(Substream::TupleElement); + path.push_back(Substream::TupleElement); for (const auto i : ext::range(0, ext::size(elems))) { - settings.path.back().tuple_element_name = names[i]; - auto & element_col = extractElementColumn(column, i); - elems[i]->deserializeBinaryBulkWithMultipleStreams(element_col, limit, settings, tuple_state->states[i]); + path.back().tuple_element_name = names[i]; + elems[i]->deserializeBinaryBulkWithMultipleStreams( + extractElementColumn(column, i), getter, limit, avg_value_size_hint, position_independent_encoding, path); } - settings.path.pop_back(); } MutableColumnPtr DataTypeTuple::createColumn() const diff --git a/dbms/src/DataTypes/DataTypeTuple.h b/dbms/src/DataTypes/DataTypeTuple.h index edd982fa840..74b8b9e6226 100644 --- a/dbms/src/DataTypes/DataTypeTuple.h +++ b/dbms/src/DataTypes/DataTypeTuple.h @@ -53,32 +53,23 @@ public: /** Each sub-column in a tuple is serialized in separate stream. */ - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; - - void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + void enumerateStreams(StreamCallback callback, SubstreamPath path) const override; void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - size_t offset, - size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; + const IColumn & column, + OutputStreamGetter getter, + size_t offset, + size_t limit, + bool position_independent_encoding, + SubstreamPath path) const override; void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; + IColumn & column, + InputStreamGetter getter, + size_t limit, + double avg_value_size_hint, + bool position_independent_encoding, + SubstreamPath path) const override; MutableColumnPtr createColumn() const override; diff --git a/dbms/src/DataTypes/DataTypeWithDictionary.h b/dbms/src/DataTypes/DataTypeWithDictionary.h deleted file mode 100644 index b3b8bba6abe..00000000000 --- a/dbms/src/DataTypes/DataTypeWithDictionary.h +++ /dev/null @@ -1,170 +0,0 @@ -#pragma once -#include -#include - -namespace DB -{ - -class DataTypeWithDictionary : public IDataType -{ -private: - DataTypePtr dictionary_type; - -public: - DataTypeWithDictionary(DataTypePtr dictionary_type_); - - const DataTypePtr & getDictionaryType() const { return dictionary_type; } - - String getName() const override - { - return "LowCardinality(" + dictionary_type->getName() + ")"; - } - const char * getFamilyName() const override { return "LowCardinality"; } - - void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override; - - void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; - - void serializeBinaryBulkWithMultipleStreams( - const IColumn & column, - size_t offset, - size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & state) const override; - - void deserializeBinaryBulkWithMultipleStreams( - IColumn & column, - size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & state) const override; - - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override - { - serializeImpl(column, row_num, ostr, &IDataType::serializeBinary); - } - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override - { - deserializeImpl(column, istr, &IDataType::deserializeBinary); - } - - void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextEscaped, settings); - } - - void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, istr, &IDataType::deserializeTextEscaped, settings); - } - - void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextQuoted, settings); - } - - void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, istr, &IDataType::deserializeTextQuoted, settings); - } - - void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextCSV, settings); - } - - void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, istr, &IDataType::deserializeTextCSV, settings); - } - - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, ostr, &IDataType::serializeText, settings); - } - - void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextJSON, settings); - } - void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeImpl(column, istr, &IDataType::deserializeTextJSON, settings); - } - - void serializeTextXML(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeImpl(column, row_num, ostr, &IDataType::serializeTextXML, settings); - } - - MutableColumnPtr createColumn() const override; - - Field getDefault() const override { return dictionary_type->getDefault(); } - - bool equals(const IDataType & rhs) const override; - - bool isParametric() const override { return true; } - bool haveSubtypes() const override { return true; } - bool cannotBeStoredInTables() const override { return dictionary_type->cannotBeStoredInTables(); } - bool shouldAlignRightInPrettyFormats() const override { return dictionary_type->shouldAlignRightInPrettyFormats(); } - bool textCanContainOnlyValidUTF8() const override { return dictionary_type->textCanContainOnlyValidUTF8(); } - bool isComparable() const override { return dictionary_type->isComparable(); } - bool canBeComparedWithCollation() const override { return dictionary_type->canBeComparedWithCollation(); } - bool canBeUsedAsVersion() const override { return dictionary_type->canBeUsedAsVersion(); } - bool isSummable() const override { return dictionary_type->isSummable(); }; - bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); }; - bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); }; - bool isNumber() const override { return false; } - bool isInteger() const override { return false; } - bool isUnsignedInteger() const override { return false; } - bool isDateOrDateTime() const override { return false; } - bool isValueRepresentedByNumber() const override { return dictionary_type->isValueRepresentedByNumber(); } - bool isValueRepresentedByInteger() const override { return dictionary_type->isValueRepresentedByInteger(); } - bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } - bool isString() const override { return false; } - bool isFixedString() const override { return false; } - bool haveMaximumSizeOfValue() const override { return dictionary_type->haveMaximumSizeOfValue(); } - size_t getMaximumSizeOfValueInMemory() const override { return dictionary_type->getMaximumSizeOfValueInMemory(); } - size_t getSizeOfValueInMemory() const override { return dictionary_type->getSizeOfValueInMemory(); } - bool isCategorial() const override { return false; } - bool isEnum() const override { return false; } - bool isNullable() const override { return false; } - bool onlyNull() const override { return false; } - bool withDictionary() const override { return true; } - - static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type); - static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys); - -private: - - template - using SerealizeFunctionPtr = void (IDataType::*)(const IColumn &, size_t, WriteBuffer &, Args & ...) const; - - template - void serializeImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, - SerealizeFunctionPtr func, Args & ... args) const; - - template - using DeserealizeFunctionPtr = void (IDataType::*)(IColumn &, ReadBuffer &, Args & ...) const; - - template - void deserializeImpl(IColumn & column, ReadBuffer & istr, - DeserealizeFunctionPtr func, Args & ... args) const; - - template - static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator); -}; - -} diff --git a/dbms/src/DataTypes/DataTypesDecimal.cpp b/dbms/src/DataTypes/DataTypesDecimal.cpp deleted file mode 100644 index b094619c914..00000000000 --- a/dbms/src/DataTypes/DataTypesDecimal.cpp +++ /dev/null @@ -1,220 +0,0 @@ -#include -#include -#include -#include -#include -#include -#include -#include -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; - extern const int ILLEGAL_TYPE_OF_ARGUMENT; - extern const int ARGUMENT_OUT_OF_BOUND; -} - - -// - -template -std::string DataTypeDecimal::getName() const -{ - std::stringstream ss; - ss << "Decimal(" << precision << ", " << scale << ")"; - return ss.str(); -} - -template -bool DataTypeDecimal::equals(const IDataType & rhs) const -{ - if (auto * ptype = typeid_cast *>(&rhs)) - return scale == ptype->getScale(); - return false; -} - -template -void DataTypeDecimal::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const -{ - T value = static_cast(column).getData()[row_num]; - if (value < T(0)) - { - value *= T(-1); - writeChar('-', ostr); /// avoid crop leading minus when whole part is zero - } - - writeIntText(static_cast(wholePart(value)), ostr); - if (scale) - { - writeChar('.', ostr); - String str_fractional(scale, '0'); - for (Int32 pos = scale - 1; pos >= 0; --pos, value /= T(10)) - str_fractional[pos] += value % T(10); - ostr.write(str_fractional.data(), scale); - } -} - - -template -void DataTypeDecimal::deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const -{ - T x; - UInt32 unread_scale = scale; - readDecimalText(istr, x, precision, unread_scale); - x *= getScaleMultiplier(unread_scale); - static_cast(column).getData().push_back(x); -} - - -template -T DataTypeDecimal::parseFromString(const String & str) const -{ - ReadBufferFromMemory buf(str.data(), str.size()); - T x; - UInt32 unread_scale = scale; - readDecimalText(buf, x, precision, unread_scale, true); - x *= getScaleMultiplier(unread_scale); - return x; -} - - -template -void DataTypeDecimal::serializeBinary(const Field & field, WriteBuffer & ostr) const -{ - FieldType x = get(field); - writeBinary(x, ostr); -} - -template -void DataTypeDecimal::serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const -{ - const FieldType & x = static_cast(column).getData()[row_num]; - writeBinary(x, ostr); -} - -template -void DataTypeDecimal::serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const -{ - const typename ColumnType::Container & x = typeid_cast(column).getData(); - - size_t size = x.size(); - - if (limit == 0 || offset + limit > size) - limit = size - offset; - - ostr.write(reinterpret_cast(&x[offset]), sizeof(FieldType) * limit); -} - - -template -void DataTypeDecimal::deserializeBinary(Field & field, ReadBuffer & istr) const -{ - typename FieldType::NativeType x; - readBinary(x, istr); - field = DecimalField(T(x), scale); -} - -template -void DataTypeDecimal::deserializeBinary(IColumn & column, ReadBuffer & istr) const -{ - typename FieldType::NativeType x; - readBinary(x, istr); - static_cast(column).getData().push_back(FieldType(x)); -} - -template -void DataTypeDecimal::deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double ) const -{ - typename ColumnType::Container & x = typeid_cast(column).getData(); - size_t initial_size = x.size(); - x.resize(initial_size + limit); - size_t size = istr.readBig(reinterpret_cast(&x[initial_size]), sizeof(FieldType) * limit); - x.resize(initial_size + size / sizeof(FieldType)); -} - - -template -Field DataTypeDecimal::getDefault() const -{ - return DecimalField(T(0), scale); -} - - -template -MutableColumnPtr DataTypeDecimal::createColumn() const -{ - auto column = ColumnType::create(); - column->getData().setScale(scale); - return column; -} - - -// - -static DataTypePtr create(const ASTPtr & arguments) -{ - if (!arguments || arguments->children.size() != 2) - throw Exception("Decimal data type family must have exactly two arguments: precision and scale", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - - const ASTLiteral * precision = typeid_cast(arguments->children[0].get()); - const ASTLiteral * scale = typeid_cast(arguments->children[1].get()); - - if (!precision || precision->value.getType() != Field::Types::UInt64 || - !scale || !(scale->value.getType() == Field::Types::Int64 || scale->value.getType() == Field::Types::UInt64)) - throw Exception("Decimal data type family must have a two numbers as its arguments", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - - UInt64 precision_value = precision->value.get(); - Int64 scale_value = scale->value.get(); - - if (precision_value < minDecimalPrecision() || precision_value > maxDecimalPrecision()) - throw Exception("Wrong precision", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (scale_value < 0 || static_cast(scale_value) > precision_value) - throw Exception("Negative scales and scales larger than presicion are not supported", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - - if (precision_value <= maxDecimalPrecision()) - return std::make_shared>(precision_value, scale_value); - else if (precision_value <= maxDecimalPrecision()) - return std::make_shared>(precision_value, scale_value); - return std::make_shared>(precision_value, scale_value); -} - - -void registerDataTypeDecimal(DataTypeFactory & factory) -{ - factory.registerDataType("Decimal", create, DataTypeFactory::CaseInsensitive); - factory.registerAlias("DEC", "Decimal", DataTypeFactory::CaseInsensitive); -} - - -template <> -Decimal32 DataTypeDecimal::getScaleMultiplier(UInt32 scale_) -{ - return common::exp10_i32(scale_); -} - -template <> -Decimal64 DataTypeDecimal::getScaleMultiplier(UInt32 scale_) -{ - return common::exp10_i64(scale_); -} - -template <> -Decimal128 DataTypeDecimal::getScaleMultiplier(UInt32 scale_) -{ - return common::exp10_i128(scale_); -} - - -/// Explicit template instantiations. -template class DataTypeDecimal; -template class DataTypeDecimal; -template class DataTypeDecimal; - -} diff --git a/dbms/src/DataTypes/DataTypesDecimal.h b/dbms/src/DataTypes/DataTypesDecimal.h deleted file mode 100644 index dc4ef6adedc..00000000000 --- a/dbms/src/DataTypes/DataTypesDecimal.h +++ /dev/null @@ -1,272 +0,0 @@ -#pragma once -#include -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; - extern const int ARGUMENT_OUT_OF_BOUND; -} - -/// -class DataTypeSimpleSerialization : public IDataType -{ - void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeText(column, row_num, ostr, settings); - } - - void serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeText(column, row_num, ostr, settings); - } - - void serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeText(column, row_num, ostr, settings); - } - - void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const override - { - serializeText(column, row_num, ostr, settings); - } - - void deserializeTextEscaped(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeText(column, istr, settings); - } - - void deserializeTextQuoted(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeText(column, istr, settings); - } - - void deserializeTextJSON(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeText(column, istr, settings); - } - - void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override - { - deserializeText(column, istr, settings); - } - - virtual void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const = 0; -}; - - -static constexpr size_t minDecimalPrecision() { return 1; } -template static constexpr size_t maxDecimalPrecision() { return 0; } -template <> constexpr size_t maxDecimalPrecision() { return 9; } -template <> constexpr size_t maxDecimalPrecision() { return 18; } -template <> constexpr size_t maxDecimalPrecision() { return 38; } - - -/// Implements Decimal(P, S), where P is precision, S is scale. -/// Maximum precisions for underlying types are: -/// Int32 9 -/// Int64 18 -/// Int128 38 -/// Operation between two decimals leads to Decimal(P, S), where -/// P is one of (9, 18, 38); equals to the maximum precision for the biggest underlying type of operands. -/// S is maximum scale of operands. -/// -/// NOTE: It's possible to set scale as a template parameter then most of functions become static. -template -class DataTypeDecimal final : public DataTypeSimpleSerialization -{ -public: - using FieldType = T; - using ColumnType = ColumnVector; - - static constexpr bool is_parametric = true; - - DataTypeDecimal(UInt32 precision_, UInt32 scale_) - : precision(precision_), - scale(scale_) - { - if (unlikely(precision < 1 || precision > maxDecimalPrecision())) - throw Exception("Precision is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - if (unlikely(scale < 0 || static_cast(scale) > maxDecimalPrecision())) - throw Exception("Scale is out of bounds", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - } - - const char * getFamilyName() const override { return "Decimal"; } - std::string getName() const override; - size_t getTypeId() const override { return TypeId::value; } - - void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const override; - void deserializeText(IColumn & column, ReadBuffer & istr, const FormatSettings &) const override; - - void serializeBinary(const Field & field, WriteBuffer & ostr) const override; - void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override; - void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override; - - void deserializeBinary(Field & field, ReadBuffer & istr) const override; - void deserializeBinary(IColumn & column, ReadBuffer & istr) const override; - void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override; - - Field getDefault() const override; - MutableColumnPtr createColumn() const override; - bool equals(const IDataType & rhs) const override; - - bool isParametric() const override { return true; } - bool haveSubtypes() const override { return false; } - bool shouldAlignRightInPrettyFormats() const override { return true; } - bool textCanContainOnlyValidUTF8() const override { return true; } - bool isComparable() const override { return true; } - bool isValueRepresentedByNumber() const override { return true; } - bool isValueRepresentedByInteger() const override { return true; } - bool isValueRepresentedByUnsignedInteger() const override { return false; } - bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; } - bool haveMaximumSizeOfValue() const override { return true; } - size_t getSizeOfValueInMemory() const override { return sizeof(T); } - bool isCategorial() const override { return isValueRepresentedByInteger(); } - - bool canBeUsedAsVersion() const override { return false; } - bool isSummable() const override { return true; } - bool canBeUsedInBitOperations() const override { return false; } - bool isUnsignedInteger() const override { return false; } - bool canBeUsedInBooleanContext() const override { return true; } - bool isNumber() const override { return true; } - bool isInteger() const override { return false; } - bool canBeInsideNullable() const override { return true; } - - /// Decimal specific - - UInt32 getPrecision() const { return precision; } - UInt32 getScale() const { return scale; } - T getScaleMultiplier() const { return getScaleMultiplier(scale); } - - T wholePart(T x) const - { - if (scale == 0) - return x; - return x / getScaleMultiplier(); - } - - T fractionalPart(T x) const - { - if (scale == 0) - return 0; - if (x < T(0)) - x *= T(-1); - return x % getScaleMultiplier(); - } - - T maxWholeValue() const { return getScaleMultiplier(maxDecimalPrecision() - scale) - T(1); } - - bool canStoreWhole(T x) const - { - T max = maxWholeValue(); - if (x > max || x < -max) - return false; - return true; - } - - /// @returns multiplier for U to become T with correct scale - template - T scaleFactorFor(const DataTypeDecimal & x, bool ) const - { - if (getScale() < x.getScale()) - throw Exception("Decimal result's scale is less then argiment's one", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - UInt32 scale_delta = getScale() - x.getScale(); /// scale_delta >= 0 - return getScaleMultiplier(scale_delta); - } - - template - T scaleFactorFor(const DataTypeNumber & , bool is_multiply_or_divisor) const - { - if (is_multiply_or_divisor) - return 1; - return getScaleMultiplier(); - } - - T parseFromString(const String & str) const; - - static T getScaleMultiplier(UInt32 scale); - -private: - const UInt32 precision; - const UInt32 scale; /// TODO: should we support scales out of [0, precision]? -}; - - -template -typename std::enable_if_t<(sizeof(T) >= sizeof(U)), const DataTypeDecimal> -decimalResultType(const DataTypeDecimal & tx, const DataTypeDecimal & ty, bool is_multiply, bool is_divide) -{ - UInt32 scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); - if (is_multiply) - scale = tx.getScale() + ty.getScale(); - else if (is_divide) - scale = tx.getScale(); - return DataTypeDecimal(maxDecimalPrecision(), scale); -} - -template -typename std::enable_if_t<(sizeof(T) < sizeof(U)), const DataTypeDecimal> -decimalResultType(const DataTypeDecimal & tx, const DataTypeDecimal & ty, bool is_multiply, bool is_divide) -{ - UInt32 scale = (tx.getScale() > ty.getScale() ? tx.getScale() : ty.getScale()); - if (is_multiply) - scale = tx.getScale() * ty.getScale(); - else if (is_divide) - scale = tx.getScale(); - return DataTypeDecimal(maxDecimalPrecision(), scale); -} - -template -const DataTypeDecimal decimalResultType(const DataTypeDecimal & tx, const DataTypeNumber &, bool, bool) -{ - return DataTypeDecimal(maxDecimalPrecision(), tx.getScale()); -} - -template -const DataTypeDecimal decimalResultType(const DataTypeNumber &, const DataTypeDecimal & ty, bool, bool) -{ - return DataTypeDecimal(maxDecimalPrecision(), ty.getScale()); -} - - -template -inline const DataTypeDecimal * checkDecimal(const IDataType & data_type) -{ - return typeid_cast *>(&data_type); -} - -inline bool isDecimal(const IDataType & data_type) -{ - if (typeid_cast *>(&data_type)) - return true; - if (typeid_cast *>(&data_type)) - return true; - if (typeid_cast *>(&data_type)) - return true; - return false; -} - -/// -inline bool notDecimalButComparableToDecimal(const IDataType & data_type) -{ - if (data_type.isInteger()) - return true; - return false; -} - -/// -inline bool comparableToDecimal(const IDataType & data_type) -{ - if (data_type.isInteger()) - return true; - return isDecimal(data_type); -} - -} diff --git a/dbms/src/DataTypes/FieldToDataType.cpp b/dbms/src/DataTypes/FieldToDataType.cpp index 3c2e78b4295..1b4fbd53c6b 100644 --- a/dbms/src/DataTypes/FieldToDataType.cpp +++ b/dbms/src/DataTypes/FieldToDataType.cpp @@ -18,7 +18,6 @@ namespace DB namespace ErrorCodes { extern const int EMPTY_DATA_PASSED; - extern const int NOT_IMPLEMENTED; } @@ -35,11 +34,6 @@ DataTypePtr FieldToDataType::operator() (const UInt64 & x) const return std::make_shared(); } -DataTypePtr FieldToDataType::operator() (const UInt128 &) const -{ - throw Exception("There are no UInt128 literals in SQL", ErrorCodes::NOT_IMPLEMENTED); -} - DataTypePtr FieldToDataType::operator() (const Int64 & x) const { if (x <= std::numeric_limits::max() && x >= std::numeric_limits::min()) return std::make_shared(); diff --git a/dbms/src/DataTypes/FieldToDataType.h b/dbms/src/DataTypes/FieldToDataType.h index a60c6a725d8..c6256a6f04b 100644 --- a/dbms/src/DataTypes/FieldToDataType.h +++ b/dbms/src/DataTypes/FieldToDataType.h @@ -19,7 +19,6 @@ class FieldToDataType : public StaticVisitor public: DataTypePtr operator() (const Null & x) const; DataTypePtr operator() (const UInt64 & x) const; - DataTypePtr operator() (const UInt128 & x) const; DataTypePtr operator() (const Int64 & x) const; DataTypePtr operator() (const Float64 & x) const; DataTypePtr operator() (const String & x) const; diff --git a/dbms/src/DataTypes/IDataType.cpp b/dbms/src/DataTypes/IDataType.cpp index e8c6a3cc63d..6a19e7b3e67 100644 --- a/dbms/src/DataTypes/IDataType.cpp +++ b/dbms/src/DataTypes/IDataType.cpp @@ -97,8 +97,6 @@ String IDataType::getFileNameForStream(const String & column_name, const IDataTy /// and name is encoded as a whole. stream_name += "%2E" + escapeForFileName(elem.tuple_element_name); } - else if (elem.type == Substream::DictionaryKeys) - stream_name += ".dict"; } return stream_name; } diff --git a/dbms/src/DataTypes/IDataType.h b/dbms/src/DataTypes/IDataType.h index da55762c02f..d291af2d89b 100644 --- a/dbms/src/DataTypes/IDataType.h +++ b/dbms/src/DataTypes/IDataType.h @@ -45,9 +45,6 @@ public: /// Name of data type family (example: FixedString, Array). virtual const char * getFamilyName() const = 0; - /// Unique type number or zero - virtual size_t getTypeId() const { return 0; } - /** Binary serialization for range of values in column - for writing to disk/network, etc. * * Some data types are represented in multiple streams while being serialized. @@ -82,9 +79,6 @@ public: NullMap, TupleElement, - - DictionaryKeys, - DictionaryIndexes, }; Type type; @@ -97,67 +91,14 @@ public: using SubstreamPath = std::vector; using StreamCallback = std::function; - virtual void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const + virtual void enumerateStreams(StreamCallback callback, SubstreamPath path) const { callback(path); } - void enumerateStreams(const StreamCallback & callback, SubstreamPath && path) const { enumerateStreams(callback, path); } - void enumerateStreams(const StreamCallback & callback) const { enumerateStreams(callback, {}); } using OutputStreamGetter = std::function; using InputStreamGetter = std::function; - struct SerializeBinaryBulkState - { - virtual ~SerializeBinaryBulkState() = default; - }; - struct DeserializeBinaryBulkState - { - virtual ~DeserializeBinaryBulkState() = default; - }; - - using SerializeBinaryBulkStatePtr = std::shared_ptr; - using DeserializeBinaryBulkStatePtr = std::shared_ptr; - - struct SerializeBinaryBulkSettings - { - OutputStreamGetter getter; - SubstreamPath path; - - size_t low_cardinality_max_dictionary_size = 0; - bool low_cardinality_use_single_dictionary_for_part = true; - - bool position_independent_encoding = true; - }; - - struct DeserializeBinaryBulkSettings - { - InputStreamGetter getter; - SubstreamPath path; - - /// True if continue reading from previous positions in file. False if made fseek to the start of new granule. - bool continuous_reading = true; - - bool position_independent_encoding = true; - /// If not zero, may be used to avoid reallocations while reading column of String type. - double avg_value_size_hint = 0; - }; - - /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. - virtual void serializeBinaryBulkStatePrefix( - SerializeBinaryBulkSettings & /*settings*/, - SerializeBinaryBulkStatePtr & /*state*/) const {} - - /// Call after serializeBinaryBulkWithMultipleStreams chain to finish serialization. - virtual void serializeBinaryBulkStateSuffix( - SerializeBinaryBulkSettings & /*settings*/, - SerializeBinaryBulkStatePtr & /*state*/) const {} - - /// Call before before deserializeBinaryBulkWithMultipleStreams chain to get DeserializeBinaryBulkStatePtr. - virtual void deserializeBinaryBulkStatePrefix( - DeserializeBinaryBulkSettings & /*settings*/, - DeserializeBinaryBulkStatePtr & /*state*/) const {} - /** 'offset' and 'limit' are used to specify range. * limit = 0 - means no limit. * offset must be not greater than size of column. @@ -166,24 +107,29 @@ public: */ virtual void serializeBinaryBulkWithMultipleStreams( const IColumn & column, + OutputStreamGetter getter, size_t offset, size_t limit, - SerializeBinaryBulkSettings & settings, - SerializeBinaryBulkStatePtr & /*state*/) const + bool /*position_independent_encoding*/, + SubstreamPath path) const { - if (WriteBuffer * stream = settings.getter(settings.path)) + if (WriteBuffer * stream = getter(path)) serializeBinaryBulk(column, *stream, offset, limit); } - /// Read no more than limit values and append them into column. + /** Read no more than limit values and append them into column. + * avg_value_size_hint - if not zero, may be used to avoid reallocations while reading column of String type. + */ virtual void deserializeBinaryBulkWithMultipleStreams( IColumn & column, + InputStreamGetter getter, size_t limit, - DeserializeBinaryBulkSettings & settings, - DeserializeBinaryBulkStatePtr & /*state*/) const + double avg_value_size_hint, + bool /*position_independent_encoding*/, + SubstreamPath path) const { - if (ReadBuffer * stream = settings.getter(settings.path)) - deserializeBinaryBulk(column, *stream, limit, settings.avg_value_size_hint); + if (ReadBuffer * stream = getter(path)) + deserializeBinaryBulk(column, *stream, limit, avg_value_size_hint); } /** Override these methods for data types that require just single stream (most of data types). @@ -413,8 +359,6 @@ public: */ virtual bool canBeInsideNullable() const { return false; } - virtual bool withDictionary() const { return false; } - /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); diff --git a/dbms/src/DataTypes/NestedUtils.h b/dbms/src/DataTypes/NestedUtils.h index e3513562b37..3bbc8845be2 100644 --- a/dbms/src/DataTypes/NestedUtils.h +++ b/dbms/src/DataTypes/NestedUtils.h @@ -22,6 +22,6 @@ namespace Nested /// Collect Array columns in a form of `column_name.element_name` to single Array(Tuple(...)) column. NamesAndTypesList collect(const NamesAndTypesList & names_and_types); -} +}; } diff --git a/dbms/src/DataTypes/tests/data_type_string.cpp b/dbms/src/DataTypes/tests/data_type_string.cpp index 65a15d059b4..a8cd5ad126b 100644 --- a/dbms/src/DataTypes/tests/data_type_string.cpp +++ b/dbms/src/DataTypes/tests/data_type_string.cpp @@ -22,7 +22,6 @@ try size_t size = strlen(s) + 1; DataTypeString data_type; - { auto column = ColumnString::create(); ColumnString::Chars_t & data = column->getChars(); @@ -38,14 +37,8 @@ try WriteBufferFromFile out_buf("test"); - IDataType::SerializeBinaryBulkSettings settings; - IDataType::SerializeBinaryBulkStatePtr state; - settings.getter = [&](const IDataType::SubstreamPath &){ return &out_buf; }; - stopwatch.restart(); - data_type.serializeBinaryBulkStatePrefix(settings, state); - data_type.serializeBinaryBulkWithMultipleStreams(*column, 0, 0, settings, state); - data_type.serializeBinaryBulkStateSuffix(settings, state); + data_type.serializeBinaryBulkWithMultipleStreams(*column, [&](const IDataType::SubstreamPath &){ return &out_buf; }, 0, 0, true, {}); stopwatch.stop(); std::cout << "Writing, elapsed: " << stopwatch.elapsedSeconds() << std::endl; @@ -56,13 +49,8 @@ try ReadBufferFromFile in_buf("test"); - IDataType::DeserializeBinaryBulkSettings settings; - IDataType::DeserializeBinaryBulkStatePtr state; - settings.getter = [&](const IDataType::SubstreamPath &){ return &in_buf; }; - stopwatch.restart(); - data_type.deserializeBinaryBulkStatePrefix(settings, state); - data_type.deserializeBinaryBulkWithMultipleStreams(*column, n, settings, state); + data_type.deserializeBinaryBulkWithMultipleStreams(*column, [&](const IDataType::SubstreamPath &){ return &in_buf; }, n, 0, true, {}); stopwatch.stop(); std::cout << "Reading, elapsed: " << stopwatch.elapsedSeconds() << std::endl; diff --git a/dbms/src/DataTypes/tests/data_types_number_fixed.cpp b/dbms/src/DataTypes/tests/data_types_number_fixed.cpp index 998d0f44481..a88229c2a3f 100644 --- a/dbms/src/DataTypes/tests/data_types_number_fixed.cpp +++ b/dbms/src/DataTypes/tests/data_types_number_fixed.cpp @@ -27,12 +27,7 @@ int main(int, char **) WriteBufferFromOStream out_buf(ostr); stopwatch.restart(); - IDataType::SerializeBinaryBulkSettings settings; - settings.getter = [&](const IDataType::SubstreamPath &){ return &out_buf; }; - IDataType::SerializeBinaryBulkStatePtr state; - data_type.serializeBinaryBulkStatePrefix(settings, state); - data_type.serializeBinaryBulkWithMultipleStreams(*column, 0, 0, settings, state); - data_type.serializeBinaryBulkStateSuffix(settings, state); + data_type.serializeBinaryBulkWithMultipleStreams(*column, [&](const IDataType::SubstreamPath &){ return &out_buf; }, 0, 0, true, {}); stopwatch.stop(); std::cout << "Elapsed: " << stopwatch.elapsedSeconds() << std::endl; diff --git a/dbms/src/Databases/DatabaseDictionary.cpp b/dbms/src/Databases/DatabaseDictionary.cpp index 04fbd3b24a6..4d121ddbe0f 100644 --- a/dbms/src/Databases/DatabaseDictionary.cpp +++ b/dbms/src/Databases/DatabaseDictionary.cpp @@ -39,15 +39,15 @@ Tables DatabaseDictionary::loadTables() Tables tables; for (const auto & pair : dictionaries) { - const std::string & dict_name = pair.first; - if (deleted_tables.count(dict_name)) + const std::string & name = pair.first; + if (deleted_tables.count(name)) continue; auto dict_ptr = std::static_pointer_cast(pair.second.loadable); if (dict_ptr) { const DictionaryStructure & dictionary_structure = dict_ptr->getStructure(); auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure); - tables[dict_name] = StorageDictionary::create(dict_name, ColumnsDescription{columns}, dictionary_structure, dict_name); + tables[name] = StorageDictionary::create(name, ColumnsDescription{columns}, dictionary_structure, name); } } diff --git a/dbms/src/Dictionaries/CacheDictionary.cpp b/dbms/src/Dictionaries/CacheDictionary.cpp index 6b378faf928..32587f6ed41 100644 --- a/dbms/src/Dictionaries/CacheDictionary.cpp +++ b/dbms/src/Dictionaries/CacheDictionary.cpp @@ -101,12 +101,12 @@ void CacheDictionary::isInImpl( { /// Transform all children to parents until ancestor id or null_value will be reached. - size_t out_size = out.size(); - memset(out.data(), 0xFF, out_size); /// 0xFF means "not calculated" + size_t size = out.size(); + memset(out.data(), 0xFF, size); /// 0xFF means "not calculated" const auto null_value = std::get(hierarchical_attribute->null_values); - PaddedPODArray children(out_size); + PaddedPODArray children(size); PaddedPODArray parents(child_ids.begin(), child_ids.end()); while (true) @@ -115,7 +115,7 @@ void CacheDictionary::isInImpl( size_t parents_idx = 0; size_t new_children_idx = 0; - while (out_idx < out_size) + while (out_idx < size) { /// Already calculated if (out[out_idx] != 0xFF) @@ -203,7 +203,7 @@ void CacheDictionary::isInConstantVector( } /// Assuming short hierarchy, so linear search is Ok. - for (size_t i = 0, out_size = out.size(); i < out_size; ++i) + for (size_t i = 0, size = out.size(); i < size; ++i) out[i] = std::find(ancestors.begin(), ancestors.end(), ancestor_ids[i]) != ancestors.end(); } @@ -936,12 +936,12 @@ void CacheDictionary::setAttributeValue(Attribute & attribute, const Key idx, co if (string_ref.data && string_ref.data != null_value_ref.data()) string_arena->free(const_cast(string_ref.data), string_ref.size); - const auto str_size = string.size(); - if (str_size != 0) + const auto size = string.size(); + if (size != 0) { - auto string_ptr = string_arena->alloc(str_size + 1); - std::copy(string.data(), string.data() + str_size + 1, string_ptr); - string_ref = StringRef{string_ptr, str_size}; + auto string_ptr = string_arena->alloc(size + 1); + std::copy(string.data(), string.data() + size + 1, string_ptr); + string_ref = StringRef{string_ptr, size}; } else string_ref = {}; diff --git a/dbms/src/Dictionaries/CatBoostModel.cpp b/dbms/src/Dictionaries/CatBoostModel.cpp index 6714a47e0ed..d36ec1ecae0 100644 --- a/dbms/src/Dictionaries/CatBoostModel.cpp +++ b/dbms/src/Dictionaries/CatBoostModel.cpp @@ -307,13 +307,13 @@ private: /// buffer[column_size * cat_features_count] -> char * => cat_features[column_size][cat_features_count] -> char * void fillCatFeaturesBuffer(const char *** cat_features, const char ** buffer, - size_t column_size, size_t cat_features_count_current) const + size_t column_size, size_t cat_features_count) const { for (size_t i = 0; i < column_size; ++i) { *cat_features = buffer; ++cat_features; - buffer += cat_features_count_current; + buffer += cat_features_count; } } @@ -321,7 +321,7 @@ private: /// * CalcModelPredictionFlat if no cat features /// * CalcModelPrediction if all cat features are strings /// * CalcModelPredictionWithHashedCatFeatures if has int cat features. - ColumnPtr evalImpl(const ColumnRawPtrs & columns, size_t float_features_count_current, size_t cat_features_count_current, + ColumnPtr evalImpl(const ColumnRawPtrs & columns, size_t float_features_count, size_t cat_features_count, bool cat_features_are_strings) const { std::string error_msg = "Error occurred while applying CatBoost model: "; @@ -334,12 +334,12 @@ private: PODArray float_features(column_size); auto float_features_buf = float_features.data(); /// Store all float data into single column. float_features is a list of pointers to it. - auto float_features_col = placeNumericColumns(columns, 0, float_features_count_current, float_features_buf); + auto float_features_col = placeNumericColumns(columns, 0, float_features_count, float_features_buf); - if (cat_features_count_current == 0) + if (cat_features_count == 0) { if (!api->CalcModelPredictionFlat(handle->get(), column_size, - float_features_buf, float_features_count_current, + float_features_buf, float_features_count, result_buf, column_size)) { @@ -352,18 +352,18 @@ private: if (cat_features_are_strings) { /// cat_features_holder stores pointers to ColumnString data or fixed_strings_data. - PODArray cat_features_holder(cat_features_count_current * column_size); + PODArray cat_features_holder(cat_features_count * column_size); PODArray cat_features(column_size); auto cat_features_buf = cat_features.data(); - fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count_current); + fillCatFeaturesBuffer(cat_features_buf, cat_features_holder.data(), column_size, cat_features_count); /// Fixed strings are stored without termination zero, so have to copy data into fixed_strings_data. - auto fixed_strings_data = placeStringColumns(columns, float_features_count_current, - cat_features_count_current, cat_features_holder.data()); + auto fixed_strings_data = placeStringColumns(columns, float_features_count, + cat_features_count, cat_features_holder.data()); if (!api->CalcModelPrediction(handle->get(), column_size, - float_features_buf, float_features_count_current, - cat_features_buf, cat_features_count_current, + float_features_buf, float_features_count, + cat_features_buf, cat_features_count, result_buf, column_size)) { throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); @@ -373,13 +373,13 @@ private: { PODArray cat_features(column_size); auto cat_features_buf = cat_features.data(); - auto cat_features_col = placeNumericColumns(columns, float_features_count_current, - cat_features_count_current, cat_features_buf); - calcHashes(columns, float_features_count_current, cat_features_count_current, cat_features_buf); + auto cat_features_col = placeNumericColumns(columns, float_features_count, + cat_features_count, cat_features_buf); + calcHashes(columns, float_features_count, cat_features_count, cat_features_buf); if (!api->CalcModelPredictionWithHashedCatFeatures( handle->get(), column_size, - float_features_buf, float_features_count_current, - cat_features_buf, cat_features_count_current, + float_features_buf, float_features_count, + cat_features_buf, cat_features_count, result_buf, column_size)) { throw Exception(error_msg + api->GetErrorString(), ErrorCodes::CANNOT_APPLY_CATBOOST_MODEL); @@ -453,7 +453,7 @@ CatBoostModel::CatBoostModel(std::string name_, std::string model_path_, std::st { try { - init(); + init(lib_path); } catch (...) { @@ -463,7 +463,7 @@ CatBoostModel::CatBoostModel(std::string name_, std::string model_path_, std::st creation_time = std::chrono::system_clock::now(); } -void CatBoostModel::init() +void CatBoostModel::init(const std::string & lib_path) { api_provider = getCatBoostWrapperHolder(lib_path); api = &api_provider->getAPI(); diff --git a/dbms/src/Dictionaries/CatBoostModel.h b/dbms/src/Dictionaries/CatBoostModel.h index b6a937fe048..735a9f4a7d3 100644 --- a/dbms/src/Dictionaries/CatBoostModel.h +++ b/dbms/src/Dictionaries/CatBoostModel.h @@ -80,7 +80,7 @@ private: std::chrono::time_point creation_time; std::exception_ptr creation_exception; - void init(); + void init(const std::string & lib_path); }; } diff --git a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp index 22cf1ae33c2..16e05333cf0 100644 --- a/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp +++ b/dbms/src/Dictionaries/ComplexKeyCacheDictionary_setAttributeValue.cpp @@ -28,12 +28,12 @@ void ComplexKeyCacheDictionary::setAttributeValue(Attribute & attribute, const s if (string_ref.data && string_ref.data != null_value_ref.data()) string_arena->free(const_cast(string_ref.data), string_ref.size); - const auto str_size = string.size(); - if (str_size != 0) + const auto size = string.size(); + if (size != 0) { - auto string_ptr = string_arena->alloc(str_size + 1); - std::copy(string.data(), string.data() + str_size + 1, string_ptr); - string_ref = StringRef{string_ptr, str_size}; + auto string_ptr = string_arena->alloc(size + 1); + std::copy(string.data(), string.data() + size + 1, string_ptr); + string_ref = StringRef{string_ptr, size}; } else string_ref = {}; diff --git a/dbms/src/Dictionaries/DictionarySourceFactory.cpp b/dbms/src/Dictionaries/DictionarySourceFactory.cpp index d6a97f8ecac..d4788d0c25e 100644 --- a/dbms/src/Dictionaries/DictionarySourceFactory.cpp +++ b/dbms/src/Dictionaries/DictionarySourceFactory.cpp @@ -20,7 +20,10 @@ #include #endif #if USE_POCO_SQLODBC || USE_POCO_DATAODBC +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" #include +#pragma GCC diagnostic pop #include #endif #if USE_MYSQL diff --git a/dbms/src/Dictionaries/DictionaryStructure.cpp b/dbms/src/Dictionaries/DictionaryStructure.cpp index 1f0d1542265..bb0d37c347a 100644 --- a/dbms/src/Dictionaries/DictionaryStructure.cpp +++ b/dbms/src/Dictionaries/DictionaryStructure.cpp @@ -240,22 +240,22 @@ std::vector DictionaryStructure::getAttributes( const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix, const bool hierarchy_allowed, const bool allow_null_values) { - Poco::Util::AbstractConfiguration::Keys config_elems; - config.keys(config_prefix, config_elems); + Poco::Util::AbstractConfiguration::Keys keys; + config.keys(config_prefix, keys); auto has_hierarchy = false; - std::vector res_attributes; + std::vector attributes; const FormatSettings format_settings; - for (const auto & config_elem : config_elems) + for (const auto & key : keys) { - if (!startsWith(config_elem.data(), "attribute")) + if (!startsWith(key.data(), "attribute")) continue; - const auto prefix = config_prefix + '.' + config_elem + '.'; + const auto prefix = config_prefix + '.' + key + '.'; Poco::Util::AbstractConfiguration::Keys attribute_keys; - config.keys(config_prefix + '.' + config_elem, attribute_keys); + config.keys(config_prefix + '.' + key, attribute_keys); checkAttributeKeys(attribute_keys); @@ -300,12 +300,12 @@ std::vector DictionaryStructure::getAttributes( has_hierarchy = has_hierarchy || hierarchical; - res_attributes.emplace_back(DictionaryAttribute{ + attributes.emplace_back(DictionaryAttribute{ name, underlying_type, type, expression, null_value, hierarchical, injective, is_object_id }); } - return res_attributes; + return attributes; } } diff --git a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp index 89ae0d0357c..dbbda76f1d9 100644 --- a/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp +++ b/dbms/src/Dictionaries/MongoDBBlockInputStream.cpp @@ -5,10 +5,13 @@ #include #include -#include -#include -#include -#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" + #include + #include + #include + #include +#pragma GCC diagnostic pop #include #include diff --git a/dbms/src/Dictionaries/MongoDBDictionarySource.cpp b/dbms/src/Dictionaries/MongoDBDictionarySource.cpp index 2fe8c91ac6b..90bf324da11 100644 --- a/dbms/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/dbms/src/Dictionaries/MongoDBDictionarySource.cpp @@ -2,11 +2,14 @@ #if USE_POCO_MONGODB #include -#include -#include -#include -#include -#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" + #include + #include + #include + #include + #include +#pragma GCC diagnostic pop #include diff --git a/dbms/src/Dictionaries/MySQLDictionarySource.cpp b/dbms/src/Dictionaries/MySQLDictionarySource.cpp index 63639ceb64c..605daaa3fc1 100644 --- a/dbms/src/Dictionaries/MySQLDictionarySource.cpp +++ b/dbms/src/Dictionaries/MySQLDictionarySource.cpp @@ -167,10 +167,10 @@ std::string MySQLDictionarySource::quoteForLike(const std::string s) LocalDateTime MySQLDictionarySource::getLastModification() const { - LocalDateTime modification_time{std::time(nullptr)}; + LocalDateTime update_time{std::time(nullptr)}; if (dont_check_update_time) - return modification_time; + return update_time; try { @@ -190,8 +190,8 @@ LocalDateTime MySQLDictionarySource::getLastModification() const if (!update_time_value.isNull()) { - modification_time = update_time_value.getDateTime(); - LOG_TRACE(log, "Got modification time: " << modification_time); + update_time = update_time_value.getDateTime(); + LOG_TRACE(log, "Got update time: " << update_time); } /// fetch remaining rows to avoid "commands out of sync" error @@ -211,15 +211,15 @@ LocalDateTime MySQLDictionarySource::getLastModification() const } /// we suppose failure to get modification time is not an error, therefore return current time - return modification_time; + return update_time; } std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request) const { - Block invalidate_sample_block; + Block sample_block; ColumnPtr column(ColumnString::create()); - invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - MySQLBlockInputStream block_input_stream(pool.Get(), request, invalidate_sample_block, 1); + sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); + MySQLBlockInputStream block_input_stream(pool.Get(), request, sample_block, 1); return readInvalidateQuery(block_input_stream); } diff --git a/dbms/src/Dictionaries/ODBCBlockInputStream.h b/dbms/src/Dictionaries/ODBCBlockInputStream.h index 8634b59e00c..efea8d6bf70 100644 --- a/dbms/src/Dictionaries/ODBCBlockInputStream.h +++ b/dbms/src/Dictionaries/ODBCBlockInputStream.h @@ -4,9 +4,12 @@ #include #include -#include -#include -#include +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" + #include + #include + #include +#pragma GCC diagnostic pop #include diff --git a/dbms/src/Dictionaries/ODBCDictionarySource.cpp b/dbms/src/Dictionaries/ODBCDictionarySource.cpp index 4813c1a80ba..0d5176c2bb0 100644 --- a/dbms/src/Dictionaries/ODBCDictionarySource.cpp +++ b/dbms/src/Dictionaries/ODBCDictionarySource.cpp @@ -1,58 +1,19 @@ -#include #include #include #include #include -#include #include #include -#include +#include +#include #include #include #include -#include -#include namespace DB { -namespace -{ - class ODBCBridgeBlockInputStream : public IProfilingBlockInputStream - { - public: - ODBCBridgeBlockInputStream(const Poco::URI & uri, - std::function callback, - const Block & sample_block, - const Context & context, - size_t max_block_size, - const ConnectionTimeouts & timeouts) - { - read_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, callback, timeouts); - reader = FormatFactory::instance().getInput(ODBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, context, max_block_size); - } - - Block getHeader() const override - { - return reader->getHeader(); - } - - String getName() const override - { - return "ODBCBridgeBlockInputStream"; - } - - private: - Block readImpl() override - { - return reader->read(); - } - - std::unique_ptr read_buf; - BlockInputStreamPtr reader; - }; -} static const size_t max_block_size = 8192; @@ -70,16 +31,20 @@ ODBCDictionarySource::ODBCDictionarySource(const DictionaryStructure & dict_stru sample_block{sample_block}, query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::None}, /// NOTE Better to obtain quoting style via ODBC interface. load_all_query{query_builder.composeLoadAllQuery()}, - invalidate_query{config.getString(config_prefix + ".invalidate_query", "")}, - odbc_bridge_helper{context.getConfigRef(), context.getSettingsRef().http_receive_timeout.value, config.getString(config_prefix + ".connection_string")}, - timeouts{ConnectionTimeouts::getHTTPTimeouts(context.getSettingsRef())}, - global_context(context) + invalidate_query{config.getString(config_prefix + ".invalidate_query", "")} { - bridge_url = odbc_bridge_helper.getMainURI(); + std::size_t field_size = context.getSettingsRef().odbc_max_field_size; - auto url_params = odbc_bridge_helper.getURLParams(sample_block.getNamesAndTypesList().toString(), max_block_size); - for (const auto & [name, value] : url_params) - bridge_url.addQueryParameter(name, value); + pool = createAndCheckResizePocoSessionPool([&] + { + auto session = std::make_shared( + config.getString(config_prefix + ".connector", "ODBC"), + config.getString(config_prefix + ".connection_string")); + + /// Default POCO value is 1024. Set property manually to make possible reading of longer strings. + session->setProperty("maxFieldSize", Poco::Any(field_size)); + return session; + }); } /// copy-constructor is provided in order to support cloneability @@ -92,14 +57,11 @@ ODBCDictionarySource::ODBCDictionarySource(const ODBCDictionarySource & other) where{other.where}, update_field{other.update_field}, sample_block{other.sample_block}, + pool{other.pool}, query_builder{dict_struct, db, table, where, IdentifierQuotingStyle::None}, load_all_query{other.load_all_query}, - invalidate_query{other.invalidate_query}, - invalidate_query_response{other.invalidate_query_response}, - odbc_bridge_helper{other.odbc_bridge_helper}, - global_context{other.global_context} + invalidate_query{other.invalidate_query}, invalidate_query_response{other.invalidate_query_response} { - } std::string ODBCDictionarySource::getUpdateFieldAndDate() @@ -123,7 +85,7 @@ std::string ODBCDictionarySource::getUpdateFieldAndDate() BlockInputStreamPtr ODBCDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); - return loadBase(load_all_query); + return std::make_shared(pool->get(), load_all_query, sample_block, max_block_size); } BlockInputStreamPtr ODBCDictionarySource::loadUpdatedAll() @@ -131,20 +93,20 @@ BlockInputStreamPtr ODBCDictionarySource::loadUpdatedAll() std::string load_query_update = getUpdateFieldAndDate(); LOG_TRACE(log, load_query_update); - return loadBase(load_query_update); + return std::make_shared(pool->get(), load_query_update, sample_block, max_block_size); } BlockInputStreamPtr ODBCDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); - return loadBase(query); + return std::make_shared(pool->get(), query, sample_block, max_block_size); } BlockInputStreamPtr ODBCDictionarySource::loadKeys( const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); - return loadBase(query); + return std::make_shared(pool->get(), query, sample_block, max_block_size); } bool ODBCDictionarySource::supportsSelectiveLoad() const @@ -182,31 +144,11 @@ bool ODBCDictionarySource::isModified() const std::string ODBCDictionarySource::doInvalidateQuery(const std::string & request) const { - Block invalidate_sample_block; + Block sample_block; ColumnPtr column(ColumnString::create()); - invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - odbc_bridge_helper.startODBCBridgeSync(); - - ODBCBridgeBlockInputStream stream( - bridge_url, - [request](std::ostream & os) { os << "query=" << request; }, - invalidate_sample_block, - global_context, - max_block_size, - timeouts); - - return readInvalidateQuery(stream); -} - -BlockInputStreamPtr ODBCDictionarySource::loadBase(const std::string & query) const -{ - odbc_bridge_helper.startODBCBridgeSync(); - return std::make_shared(bridge_url, - [query](std::ostream & os) { os << "query=" << query; }, - sample_block, - global_context, - max_block_size, - timeouts); + sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); + ODBCBlockInputStream block_input_stream(pool->get(), request, sample_block, 1); + return readInvalidateQuery(block_input_stream); } } diff --git a/dbms/src/Dictionaries/ODBCDictionarySource.h b/dbms/src/Dictionaries/ODBCDictionarySource.h index 7d7a0ca51e0..0f10dbd94ff 100644 --- a/dbms/src/Dictionaries/ODBCDictionarySource.h +++ b/dbms/src/Dictionaries/ODBCDictionarySource.h @@ -1,17 +1,13 @@ #pragma once +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wunused-parameter" #include -#include - -#include - +#pragma GCC diagnostic pop #include #include #include -#include - - namespace Poco { namespace Util @@ -63,8 +59,6 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; - BlockInputStreamPtr loadBase(const std::string & query) const; - Poco::Logger * log; std::chrono::time_point update_time; @@ -74,16 +68,11 @@ private: const std::string where; const std::string update_field; Block sample_block; + std::shared_ptr pool = nullptr; ExternalQueryBuilder query_builder; const std::string load_all_query; std::string invalidate_query; mutable std::string invalidate_query_response; - - ODBCBridgeHelper odbc_bridge_helper; - Poco::URI bridge_url; - ConnectionTimeouts timeouts; - const Context & global_context; - }; diff --git a/dbms/src/Dictionaries/tests/CMakeLists.txt b/dbms/src/Dictionaries/tests/CMakeLists.txt deleted file mode 100644 index e69de29bb2d..00000000000 diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 068454c8681..b6c6b131c77 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -60,7 +60,6 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & FormatSettings format_settings; format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers; format_settings.json.quote_denormals = settings.output_format_json_quote_denormals; - format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes; format_settings.csv.delimiter = settings.format_csv_delimiter; format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes; format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes; @@ -120,7 +119,6 @@ void registerOutputFormatJSON(FormatFactory & factory); void registerOutputFormatJSONCompact(FormatFactory & factory); void registerOutputFormatXML(FormatFactory & factory); void registerOutputFormatODBCDriver(FormatFactory & factory); -void registerOutputFormatODBCDriver2(FormatFactory & factory); void registerOutputFormatNull(FormatFactory & factory); /// Input only formats. @@ -154,7 +152,6 @@ FormatFactory::FormatFactory() registerOutputFormatJSONCompact(*this); registerOutputFormatXML(*this); registerOutputFormatODBCDriver(*this); - registerOutputFormatODBCDriver2(*this); registerOutputFormatNull(*this); } diff --git a/dbms/src/Formats/FormatSettings.h b/dbms/src/Formats/FormatSettings.h index d6779116807..ad951bd798f 100644 --- a/dbms/src/Formats/FormatSettings.h +++ b/dbms/src/Formats/FormatSettings.h @@ -17,7 +17,6 @@ struct FormatSettings { bool quote_64bit_integers = true; bool quote_denormals = true; - bool escape_forward_slashes = true; }; JSON json; diff --git a/dbms/src/Formats/JSONEachRowRowOutputStream.cpp b/dbms/src/Formats/JSONEachRowRowOutputStream.cpp index 40508b36856..fec80cd1421 100644 --- a/dbms/src/Formats/JSONEachRowRowOutputStream.cpp +++ b/dbms/src/Formats/JSONEachRowRowOutputStream.cpp @@ -18,7 +18,7 @@ JSONEachRowRowOutputStream::JSONEachRowRowOutputStream(WriteBuffer & ostr_, cons for (size_t i = 0; i < columns; ++i) { WriteBufferFromString out(fields[i]); - writeJSONString(sample.getByPosition(i).name, out, settings); + writeJSONString(sample.getByPosition(i).name, out); } } diff --git a/dbms/src/Formats/JSONRowOutputStream.cpp b/dbms/src/Formats/JSONRowOutputStream.cpp index 89f57669a3e..a549ff4395f 100644 --- a/dbms/src/Formats/JSONRowOutputStream.cpp +++ b/dbms/src/Formats/JSONRowOutputStream.cpp @@ -21,7 +21,7 @@ JSONRowOutputStream::JSONRowOutputStream(WriteBuffer & ostr_, const Block & samp need_validate_utf8 = true; WriteBufferFromOwnString out; - writeJSONString(fields[i].name, out, settings); + writeJSONString(fields[i].name, out); fields[i].name = out.str(); } @@ -50,7 +50,7 @@ void JSONRowOutputStream::writePrefix() writeString(fields[i].name, *ostr); writeCString(",\n", *ostr); writeCString("\t\t\t\"type\": ", *ostr); - writeJSONString(fields[i].type->getName(), *ostr, settings); + writeJSONString(fields[i].type->getName(), *ostr); writeChar('\n', *ostr); writeCString("\t\t}", *ostr); @@ -149,7 +149,7 @@ void JSONRowOutputStream::writeTotals() writeCString(",\n", *ostr); writeCString("\t\t", *ostr); - writeJSONString(column.name, *ostr, settings); + writeJSONString(column.name, *ostr); writeCString(": ", *ostr); column.type->serializeTextJSON(*column.column.get(), 0, *ostr, settings); } @@ -176,7 +176,7 @@ static void writeExtremesElement(const char * title, const Block & extremes, siz writeCString(",\n", ostr); writeCString("\t\t\t", ostr); - writeJSONString(column.name, ostr, settings); + writeJSONString(column.name, ostr); writeCString(": ", ostr); column.type->serializeTextJSON(*column.column.get(), row_num, ostr, settings); } diff --git a/dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp b/dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp deleted file mode 100644 index 3d02bd92e07..00000000000 --- a/dbms/src/Formats/ODBCDriver2BlockOutputStream.cpp +++ /dev/null @@ -1,96 +0,0 @@ -#include -#include -#include -#include -#include - - -#include - - -namespace DB -{ -ODBCDriver2BlockOutputStream::ODBCDriver2BlockOutputStream( - WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings) - : out(out_), header(header_), format_settings(format_settings) -{ -} - -void ODBCDriver2BlockOutputStream::flush() -{ - out.next(); -} - -void writeODBCString(WriteBuffer & out, const std::string & str) -{ - writeIntBinary(Int32(str.size()), out); - out.write(str.data(), str.size()); -} - -void ODBCDriver2BlockOutputStream::write(const Block & block) -{ - const size_t rows = block.rows(); - const size_t columns = block.columns(); - String text_value; - - for (size_t i = 0; i < rows; ++i) - { - for (size_t j = 0; j < columns; ++j) - { - text_value.resize(0); - const ColumnWithTypeAndName & col = block.getByPosition(j); - - if (col.column->isNullAt(i)) - { - writeIntBinary(Int32(-1), out); - } - else - { - { - WriteBufferFromString text_out(text_value); - col.type->serializeText(*col.column, i, text_out, format_settings); - } - writeODBCString(out, text_value); - } - } - } -} - -void ODBCDriver2BlockOutputStream::writePrefix() -{ - const size_t columns = header.columns(); - - /// Number of header rows. - writeIntBinary(Int32(2), out); - - /// Names of columns. - /// Number of columns + 1 for first name column. - writeIntBinary(Int32(columns + 1), out); - writeODBCString(out, "name"); - for (size_t i = 0; i < columns; ++i) - { - const ColumnWithTypeAndName & col = header.getByPosition(i); - writeODBCString(out, col.name); - } - - /// Types of columns. - writeIntBinary(Int32(columns + 1), out); - writeODBCString(out, "type"); - for (size_t i = 0; i < columns; ++i) - { - const ColumnWithTypeAndName & col = header.getByPosition(i); - writeODBCString(out, col.type->getName()); - } -} - - -void registerOutputFormatODBCDriver2(FormatFactory & factory) -{ - factory.registerOutputFormat( - "ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const Context &, const FormatSettings & format_settings) - { - return std::make_shared(buf, sample, format_settings); - }); -} - -} diff --git a/dbms/src/Formats/ODBCDriver2BlockOutputStream.h b/dbms/src/Formats/ODBCDriver2BlockOutputStream.h deleted file mode 100644 index 7e295b52d60..00000000000 --- a/dbms/src/Formats/ODBCDriver2BlockOutputStream.h +++ /dev/null @@ -1,44 +0,0 @@ -#pragma once - -#include -#include -#include -#include - - -namespace DB -{ -class WriteBuffer; - - -/** A data format designed to simplify the implementation of the ODBC driver. - * ODBC driver is designed to be build for different platforms without dependencies from the main code, - * so the format is made that way so that it can be as easy as possible to parse it. - * A header is displayed with the required information. - * The data is then output in the order of the rows. Each value is displayed as follows: length in Int32 format (-1 for NULL), then data in text form. - */ -class ODBCDriver2BlockOutputStream : public IBlockOutputStream -{ -public: - ODBCDriver2BlockOutputStream(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings); - - Block getHeader() const override - { - return header; - } - void write(const Block & block) override; - void writePrefix() override; - - void flush() override; - std::string getContentType() const override - { - return "application/octet-stream"; - } - -private: - WriteBuffer & out; - const Block header; - const FormatSettings format_settings; -}; - -} diff --git a/dbms/src/Functions/CMakeLists.txt b/dbms/src/Functions/CMakeLists.txt index ef285659be2..81b6c46eb79 100644 --- a/dbms/src/Functions/CMakeLists.txt +++ b/dbms/src/Functions/CMakeLists.txt @@ -41,7 +41,7 @@ generate_function_register(Array FunctionArrayEnumerate FunctionArrayEnumerateUniq FunctionArrayUniq - FunctionArrayDistinct + FunctionArrayDistinct FunctionEmptyArrayUInt8 FunctionEmptyArrayUInt16 FunctionEmptyArrayUInt32 @@ -91,7 +91,7 @@ list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h Func add_library(clickhouse_functions ${clickhouse_functions_sources}) -target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE libconsistent-hashing ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} murmurhash) +target_link_libraries(clickhouse_functions PUBLIC dbms PRIVATE libconsistent-hashing ${FARMHASH_LIBRARIES} ${METROHASH_LIBRARIES} ${MURMURHASH2_LIBRARIES}) target_include_directories (clickhouse_functions SYSTEM BEFORE PUBLIC ${DIVIDE_INCLUDE_DIR}) diff --git a/dbms/src/Functions/FunctionHelpers.h b/dbms/src/Functions/FunctionHelpers.h index 6b1d89560df..aaaca0bf26a 100644 --- a/dbms/src/Functions/FunctionHelpers.h +++ b/dbms/src/Functions/FunctionHelpers.h @@ -7,101 +7,6 @@ #include #include -namespace common -{ - template - inline bool addOverflow(T x, T y, T & res) - { - return __builtin_add_overflow(x, y, &res); - } - - template <> - inline bool addOverflow(Int32 x, Int32 y, Int32 & res) - { - return __builtin_sadd_overflow(x, y, &res); - } - - template <> - inline bool addOverflow(long x, long y, long & res) - { - return __builtin_saddl_overflow(x, y, &res); - } - - template <> - inline bool addOverflow(long long x, long long y, long long & res) - { - return __builtin_saddll_overflow(x, y, &res); - } - - template <> - inline bool addOverflow(__int128 x, __int128 y, __int128 & res) - { - res = x + y; - return (res - y) != x; - } - - template - inline bool subOverflow(T x, T y, T & res) - { - return __builtin_sub_overflow(x, y, &res); - } - - template <> - inline bool subOverflow(Int32 x, Int32 y, Int32 & res) - { - return __builtin_ssub_overflow(x, y, &res); - } - - template <> - inline bool subOverflow(long x, long y, long & res) - { - return __builtin_ssubl_overflow(x, y, &res); - } - - template <> - inline bool subOverflow(long long x, long long y, long long & res) - { - return __builtin_ssubll_overflow(x, y, &res); - } - - template <> - inline bool subOverflow(__int128 x, __int128 y, __int128 & res) - { - res = x - y; - return (res + y) != x; - } - - template - inline bool mulOverflow(T x, T y, T & res) - { - return __builtin_mul_overflow(x, y, &res); - } - - template <> - inline bool mulOverflow(Int32 x, Int32 y, Int32 & res) - { - return __builtin_smul_overflow(x, y, &res); - } - - template <> - inline bool mulOverflow(long x, long y, long & res) - { - return __builtin_smull_overflow(x, y, &res); - } - - template <> - inline bool mulOverflow(long long x, long long y, long long & res) - { - return __builtin_smulll_overflow(x, y, &res); - } - - template <> - inline bool mulOverflow(__int128 x, __int128 y, __int128 & res) - { - res = x * y; - return (res / y) != x; - } -} namespace DB { @@ -189,60 +94,5 @@ Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & ar /// Similar function as above. Additionally transform the result type if needed. Block createBlockWithNestedColumns(const Block & block, const ColumnNumbers & args, size_t result); -template -bool callByTypeAndNumber(UInt8 number, F && f) -{ - switch (number) - { - case TypeId::value: f(T(), UInt8()); break; - case TypeId::value: f(T(), UInt16()); break; - case TypeId::value: f(T(), UInt32()); break; - case TypeId::value: f(T(), UInt64()); break; - //case TypeId::value: f(T(), UInt128()); break; - - case TypeId::value: f(T(), Int8()); break; - case TypeId::value: f(T(), Int16()); break; - case TypeId::value: f(T(), Int32()); break; - case TypeId::value: f(T(), Int64()); break; - case TypeId::value: f(T(), Int128()); break; - - case TypeId::value: f(T(), Decimal32()); break; - case TypeId::value: f(T(), Decimal64()); break; - case TypeId::value: f(T(), Decimal128()); break; - default: - return false; - } - - return true; -} - -/// Unroll template using TypeNumber -template -inline bool callByNumbers(UInt8 type_num1, UInt8 type_num2, F && f) -{ - switch (type_num1) - { - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - //case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - case TypeId::value: return callByTypeAndNumber(type_num2, std::forward(f)); - - default: - break; - }; - - return false; -} } diff --git a/dbms/src/Functions/FunctionsArithmetic.h b/dbms/src/Functions/FunctionsArithmetic.h index 8e5743665ac..adbfdba175f 100644 --- a/dbms/src/Functions/FunctionsArithmetic.h +++ b/dbms/src/Functions/FunctionsArithmetic.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -38,7 +37,6 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int LOGICAL_ERROR; extern const int TOO_LESS_ARGUMENTS_FOR_FUNCTION; - extern const int DECIMAL_OVERFLOW; } @@ -90,10 +88,8 @@ template struct UnaryOperationImpl { using ResultType = typename Op::ResultType; - using ArrayA = typename ColumnVector::Container; - using ArrayC = typename ColumnVector::Container; - static void NO_INLINE vector(const ArrayA & a, ArrayC & c) + static void NO_INLINE vector(const PaddedPODArray & a, PaddedPODArray & c) { size_t size = a.size(); for (size_t i = 0; i < size; ++i) @@ -111,7 +107,6 @@ template struct PlusImpl { using ResultType = typename NumberTraits::ResultOfAdditionMultiplication::Type; - static const constexpr bool allow_decimal = true; template static inline Result apply(A a, B b) @@ -120,12 +115,6 @@ struct PlusImpl return static_cast(a) + b; } - template - static inline bool apply(A a, B b, Result & c) - { - return common::addOverflow(static_cast(a), b, c); - } - #if USE_EMBEDDED_COMPILER static constexpr bool compilable = true; @@ -141,7 +130,6 @@ template struct MultiplyImpl { using ResultType = typename NumberTraits::ResultOfAdditionMultiplication::Type; - static const constexpr bool allow_decimal = true; template static inline Result apply(A a, B b) @@ -149,12 +137,6 @@ struct MultiplyImpl return static_cast(a) * b; } - template - static inline bool apply(A a, B b, Result & c) - { - return common::mulOverflow(static_cast(a), b, c); - } - #if USE_EMBEDDED_COMPILER static constexpr bool compilable = true; @@ -169,7 +151,6 @@ template struct MinusImpl { using ResultType = typename NumberTraits::ResultOfSubtraction::Type; - static const constexpr bool allow_decimal = true; template static inline Result apply(A a, B b) @@ -177,12 +158,6 @@ struct MinusImpl return static_cast(a) - b; } - template - static inline bool apply(A a, B b, Result & c) - { - return common::subOverflow(static_cast(a), b, c); - } - #if USE_EMBEDDED_COMPILER static constexpr bool compilable = true; @@ -197,7 +172,6 @@ template struct DivideFloatingImpl { using ResultType = typename NumberTraits::ResultOfFloatingPointDivision::Type; - static const constexpr bool allow_decimal = true; template static inline Result apply(A a, B b) @@ -577,7 +551,7 @@ using GreatestImpl = std::conditional_t struct NegateImpl { - using ResultType = std::conditional_t, A, typename NumberTraits::ResultOfNegate::Type>; + using ResultType = typename NumberTraits::ResultOfNegate::Type; static inline ResultType apply(A a) { @@ -619,13 +593,11 @@ struct BitNotImpl template struct AbsImpl { - using ResultType = std::conditional_t, A, typename NumberTraits::ResultOfAbs::Type>; + using ResultType = typename NumberTraits::ResultOfAbs::Type; static inline ResultType apply(A a) { - if constexpr (IsDecimalNumber) - return a < 0 ? A(-a) : a; - else if constexpr (std::is_integral_v && std::is_signed_v) + if constexpr (std::is_integral_v && std::is_signed_v) return a < 0 ? static_cast(~a) + 1 : a; else if constexpr (std::is_integral_v && std::is_unsigned_v) return static_cast(a); @@ -715,199 +687,6 @@ struct IntExp10Impl #endif }; - -template struct NativeType { using Type = T; }; -template <> struct NativeType { using Type = Int32; }; -template <> struct NativeType { using Type = Int64; }; -template <> struct NativeType { using Type = Int128; }; - -/// Binary operations for Decimals need scale args -/// +|- scale one of args (which scale factor is not 1). ScaleR = oneof(Scale1, Scale2); -/// * no agrs scale. ScaleR = Scale1 + Scale2; -/// / first arg scale. ScaleR = Scale1 (scale_a = DecimalType::getScale()). -template typename Operation, typename ResultType_> -struct DecimalBinaryOperation -{ - using ResultType = ResultType_; - using NativeResultType = typename NativeType::Type; - using Op = Operation; - using ArrayA = typename ColumnVector::Container; - using ArrayB = typename ColumnVector::Container; - using ArrayC = typename ColumnVector::Container; - - static constexpr bool is_plus_minus = std::is_same_v, PlusImpl> || - std::is_same_v, MinusImpl>; - static constexpr bool is_multiply = std::is_same_v, MultiplyImpl>; - static constexpr bool is_division = std::is_same_v, DivideFloatingImpl>; - static constexpr bool is_compare = std::is_same_v, LeastBaseImpl> || - std::is_same_v, GreatestBaseImpl>; - static constexpr bool is_plus_minus_compare = is_plus_minus || is_compare; - static constexpr bool can_overflow = is_plus_minus || is_multiply; - - static void NO_INLINE vector_vector(const ArrayA & a, const ArrayB & b, ArrayC & c, - ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { - size_t size = a.size(); - if constexpr (is_plus_minus_compare) - { - if (scale_a != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b[i], scale_a); - return; - } - else if (scale_b != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b[i], scale_b); - return; - } - } - else if constexpr (is_division && IsDecimalNumber) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaledDiv(a[i], b[i], scale_a); - return; - } - - /// default: use it if no return before - for (size_t i = 0; i < size; ++i) - c[i] = apply(a[i], b[i]); - } - - static void NO_INLINE vector_constant(const ArrayA & a, B b, ArrayC & c, - ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { - size_t size = a.size(); - if constexpr (is_plus_minus_compare) - { - if (scale_a != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b, scale_a); - return; - } - else if (scale_b != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a[i], b, scale_b); - return; - } - } - else if constexpr (is_division && IsDecimalNumber) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaledDiv(a[i], b, scale_a); - return; - } - - /// default: use it if no return before - for (size_t i = 0; i < size; ++i) - c[i] = apply(a[i], b); - } - - static void NO_INLINE constant_vector(A a, const ArrayB & b, ArrayC & c, - ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { - size_t size = b.size(); - if constexpr (is_plus_minus_compare) - { - if (scale_a != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a, b[i], scale_a); - return; - } - else if (scale_b != 1) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaled(a, b[i], scale_b); - return; - } - } - else if constexpr (is_division && IsDecimalNumber) - { - for (size_t i = 0; i < size; ++i) - c[i] = applyScaledDiv(a, b[i], scale_a); - return; - } - - /// default: use it if no return before - for (size_t i = 0; i < size; ++i) - c[i] = apply(a, b[i]); - } - - static ResultType constant_constant(A a, B b, ResultType scale_a [[maybe_unused]], ResultType scale_b [[maybe_unused]]) - { - if constexpr (is_plus_minus_compare) - { - if (scale_a != 1) - return applyScaled(a, b, scale_a); - else if (scale_b != 1) - return applyScaled(a, b, scale_b); - } - else if constexpr (is_division && IsDecimalNumber) - return applyScaledDiv(a, b, scale_a); - return apply(a, b); - } - -private: - /// there's implicit type convertion here - static NativeResultType apply(NativeResultType a, NativeResultType b) - { - if constexpr (can_overflow) - { - NativeResultType res; - if (Op::template apply(a, b, res)) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - return res; - } - else - return Op::template apply(a, b); - } - - template - static NativeResultType applyScaled(NativeResultType a, NativeResultType b, NativeResultType scale) - { - if constexpr (is_plus_minus_compare) - { - NativeResultType res; - - bool overflow = false; - if constexpr (scale_left) - overflow |= common::mulOverflow(a, scale, a); - else - overflow |= common::mulOverflow(b, scale, b); - - if constexpr (can_overflow) - overflow |= Op::template apply(a, b, res); - else - res = Op::template apply(a, b); - - if (overflow) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - - return res; - } - } - - static NativeResultType applyScaledDiv(NativeResultType a, NativeResultType b, NativeResultType scale) - { - if constexpr (is_division) - { - bool overflow = false; - if constexpr (!IsDecimalNumber) - overflow |= common::mulOverflow(scale, scale, scale); - overflow |= common::mulOverflow(a, scale, a); - if (overflow) - throw Exception("Decimal math overflow", ErrorCodes::DECIMAL_OVERFLOW); - - return Op::template apply(a, b); - } - } -}; - - /// Used to indicate undefined operation struct InvalidType; @@ -930,16 +709,6 @@ template constexpr bool IsDateOrDateTime = false; template <> constexpr bool IsDateOrDateTime = true; template <> constexpr bool IsDateOrDateTime = true; -template constexpr bool IsDecimal = false; -template <> constexpr bool IsDecimal> = true; -template <> constexpr bool IsDecimal> = true; -template <> constexpr bool IsDecimal> = true; - -template constexpr bool UseLeftDecimal = false; -template <> constexpr bool UseLeftDecimal, DataTypeDecimal> = true; -template <> constexpr bool UseLeftDecimal, DataTypeDecimal> = true; -template <> constexpr bool UseLeftDecimal, DataTypeDecimal> = true; - template using DataTypeFromFieldType = std::conditional_t, InvalidType, DataTypeNumber>; template