From 19a54a59bbef8be40ca28227e78ed61a6f6d0af1 Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 16 Oct 2019 18:28:10 +0300 Subject: [PATCH 01/77] Created: function hex for Decimal32, Decimal64, Decimal128 --- dbms/src/Functions/FunctionsCoding.h | 51 ++++++++++++++++++- .../0_stateless/01013_hex_decimal.reference | 11 ++++ .../queries/0_stateless/01013_hex_decimal.sql | 8 +++ 3 files changed, 68 insertions(+), 2 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01013_hex_decimal.reference create mode 100644 dbms/tests/queries/0_stateless/01013_hex_decimal.sql diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 1ab00d725f6..4742501fdef 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -19,6 +19,7 @@ #include #include #include +#include #include #include @@ -949,7 +950,8 @@ public: if (!which.isStringOrFixedString() && !which.isDateOrDateTime() && !which.isUInt() && - !which.isFloat()) + !which.isFloat() && + !which.isDecimal()) throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); @@ -1061,6 +1063,48 @@ public: } } + template + bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) + { + const ColumnDecimal * col_dec = checkAndGetColumn>(col); + + static constexpr size_t DECIMAL_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. + + if (col_dec) + { + auto col_str = ColumnString::create(); + ColumnString::Chars & out_vec = col_str->getChars(); + ColumnString::Offsets & out_offsets = col_str->getOffsets(); + + const typename ColumnDecimal::Container & in_vec = col_dec->getData(); + + size_t size = in_vec.size(); + out_offsets.resize(size); + out_vec.resize(size * DECIMAL_HEX_LENGTH); + + size_t pos = 0; + char * out = reinterpret_cast(&out_vec[0]); + for (size_t i = 0; i < size; ++i) + { + const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); + executeOneString(in_pos, in_pos + sizeof(T), out); + + pos += DECIMAL_HEX_LENGTH; + out_offsets[i] = pos; + } + + col_res = std::move(col_str); + return true; + } + else + { + return false; + } + } + + + + void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out) { while (pos < end) @@ -1177,7 +1221,10 @@ public: tryExecuteString(column, res_column) || tryExecuteFixedString(column, res_column) || tryExecuteFloat(column, res_column) || - tryExecuteFloat(column, res_column)) + tryExecuteFloat(column, res_column) || + tryExecuteDecimal(column, res_column) || + tryExecuteDecimal(column, res_column) || + tryExecuteDecimal(column, res_column)) return; throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName() diff --git a/dbms/tests/queries/0_stateless/01013_hex_decimal.reference b/dbms/tests/queries/0_stateless/01013_hex_decimal.reference new file mode 100644 index 00000000000..8d2082511f0 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_hex_decimal.reference @@ -0,0 +1,11 @@ +64000000 +64000000 +42020000 +B61BFEFFFFFFFFFF +EF260000000000000000000000000000 +400D0300 +28110300 +403A340100000000 +E0C0350100000000 +00B08EF01B0000000000000000000000 +007A292C1C0000000000000000000000 diff --git a/dbms/tests/queries/0_stateless/01013_hex_decimal.sql b/dbms/tests/queries/0_stateless/01013_hex_decimal.sql new file mode 100644 index 00000000000..97bb57fc212 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01013_hex_decimal.sql @@ -0,0 +1,8 @@ +SELECT hex(toDecimal32(1.0, 2)); +SELECT hex(toDecimal32(1., 2)); +SELECT hex(toDecimal32(0.000578, 6)); +SELECT hex(toDecimal64(-123.978, 3)); +SELECT hex(toDecimal128(99.67, 2)); +SELECT hex(toDecimal32(number, 3)) FROM numbers(200, 2); +SELECT hex(toDecimal64(number, 5)) FROM numbers(202, 2); +SELECT hex(toDecimal128(number, 9)) FROM numbers(120, 2); From 3f1651fa294fb13365ad78f4566de762284d503b Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 29 Oct 2019 11:03:47 +0800 Subject: [PATCH 02/77] improve range --- dbms/src/Functions/array/range.cpp | 126 +++++++++++------- .../functions/array_functions.md | 5 +- 2 files changed, 81 insertions(+), 50 deletions(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 9a6f502beb8..913c2a4ad42 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -1,11 +1,14 @@ #include #include #include +#include #include #include #include #include +#include + namespace DB { @@ -15,6 +18,7 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int ILLEGAL_COLUMN; extern const int ILLEGAL_TYPE_OF_ARGUMENT; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -27,79 +31,105 @@ public: private: String getName() const override { return name; } - size_t getNumberOfArguments() const override { return 1; } + size_t getNumberOfArguments() const override { return 0; } + bool isVariadic() const override { return true; } bool useDefaultImplementationForConstants() const override { return true; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - const DataTypePtr & arg = arguments.front(); + if (arguments.size() > 2 || arguments.empty()) + { + throw Exception{"Function " + getName() + " needs [1-2] argument; passed " + + std::to_string(arguments.size()) + ".", + ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; + } - if (!isUnsignedInteger(arg)) - throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; - - return std::make_shared(arg); + for (const auto & arg : arguments) + { + if (!isUnsignedInteger(arg)) + throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; + } + return std::make_shared(arguments.back()); } - template - bool executeInternal(Block & block, const IColumn * arg, const size_t result) + template + bool executeInternal(Block & block, const IColumn * start_col, const IColumn * end_col, const size_t result) { static constexpr size_t max_elements = 100'000'000; + + auto start_column = checkAndGetColumn>(start_col); + auto end_column = checkAndGetColumn>(end_col); - if (const auto in = checkAndGetColumn>(arg)) + if (!start_column || !end_column) { - const auto & in_data = in->getData(); - const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), size_t{}, - [this] (const size_t lhs, const size_t rhs) - { - const auto sum = lhs + rhs; - if (sum < lhs) - throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", - ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + LOG_TRACE(&Logger::get("range function"), "some column is null-----"); + return false; + } - return sum; - }); + const auto & in_start_data = start_column->getData(); + const auto & in_end_data = end_column->getData(); + size_t total_values = 0; + + for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) + { + total_values += in_start_data[row_idx] >= in_end_data[row_idx] ? 0 : (in_end_data[row_idx] -in_start_data[row_idx]); if (total_values > max_elements) throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), ErrorCodes::ARGUMENT_OUT_OF_BOUND}; - - auto data_col = ColumnVector::create(total_values); - auto offsets_col = ColumnArray::ColumnOffsets::create(in->size()); - - auto & out_data = data_col->getData(); - auto & out_offsets = offsets_col->getData(); - - IColumn::Offset offset{}; - for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx) - { - for (size_t elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx) - out_data[offset + elem_idx] = elem_idx; - - offset += in_data[row_idx]; - out_offsets[row_idx] = offset; - } - - block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); - return true; } - else - return false; + + auto data_col = ColumnVector::create(total_values); + auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size()); + + auto & out_data = data_col->getData(); + auto & out_offsets = offsets_col->getData(); + + for (const auto & data : in_start_data) + { + LOG_TRACE(&Logger::get("range function"), "Test--" << UInt32(data) << std::endl); + } + + IColumn::Offset offset{}; + for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) + { + LOG_TRACE(&Logger::get("range function"), "DEBUG--" << UInt64(start_col->getUInt(row_idx)) << "---" << UInt64(in_end_data[row_idx]) << "--" << UInt64(offset)); + for (size_t st = in_start_data[row_idx], ed = in_end_data[row_idx]; st < ed; ++st) + out_data[offset++] = st; + + out_offsets[row_idx] = offset; + } + + block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); + return true; + } + + template + bool executeStartInternal(Block & block, const IColumn * start_col, const IColumn * end_col, const size_t result) + { + return executeInternal(block, start_col, end_col, result) + || executeInternal(block, start_col, end_col, result) + || executeInternal(block, start_col, end_col, result) + || executeInternal(block, start_col, end_col, result); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override - { - const auto col = block.getByPosition(arguments[0]).column.get(); + { + const auto end_col = block.getByPosition(arguments[arguments.size() == 1 ? 0 : 1]).column.get(); + const auto start_col = arguments.size() == 1? ColumnVector::create(end_col->size(), 0).get() + : block.getByPosition(arguments[0]).column.get(); - if (!executeInternal(block, col, result) && - !executeInternal(block, col, result) && - !executeInternal(block, col, result) && - !executeInternal(block, col, result)) + if (!executeStartInternal(block, start_col, end_col, result) && + !executeStartInternal(block, start_col, end_col, result) && + !executeStartInternal(block, start_col, end_col, result) && + !executeStartInternal(block, start_col, end_col, result)) { - throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + throw Exception{"Illegal column " + start_col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } + }; diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index a43f975254f..ed33ae5e512 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -34,9 +34,10 @@ Accepts zero arguments and returns an empty array of the appropriate type. Accepts an empty array and returns a one-element array that is equal to the default value. -## range(N) +## range([start,] end) -Returns an array of numbers from 0 to N-1. +Returns an array of numbers from start to end-1. +If the argument `start` is not specified, defaults to 0. Just in case, an exception is thrown if arrays with a total length of more than 100,000,000 elements are created in a data block. ## array(x1, ...), operator \[x1, ...\] From a653f50432255fc1d05c0440d55d3b4239e391d5 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 29 Oct 2019 13:01:45 +0800 Subject: [PATCH 03/77] remove logs --- dbms/src/Functions/array/range.cpp | 40 +++++++++++++++++------------- 1 file changed, 23 insertions(+), 17 deletions(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 913c2a4ad42..3106ddd3888 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -7,8 +7,6 @@ #include #include -#include - namespace DB { @@ -63,7 +61,6 @@ private: if (!start_column || !end_column) { - LOG_TRACE(&Logger::get("range function"), "some column is null-----"); return false; } @@ -87,15 +84,9 @@ private: auto & out_data = data_col->getData(); auto & out_offsets = offsets_col->getData(); - for (const auto & data : in_start_data) - { - LOG_TRACE(&Logger::get("range function"), "Test--" << UInt32(data) << std::endl); - } - IColumn::Offset offset{}; for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) { - LOG_TRACE(&Logger::get("range function"), "DEBUG--" << UInt64(start_col->getUInt(row_idx)) << "---" << UInt64(in_end_data[row_idx]) << "--" << UInt64(offset)); for (size_t st = in_start_data[row_idx], ed = in_end_data[row_idx]; st < ed; ++st) out_data[offset++] = st; @@ -117,16 +108,31 @@ private: void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override { - const auto end_col = block.getByPosition(arguments[arguments.size() == 1 ? 0 : 1]).column.get(); - const auto start_col = arguments.size() == 1? ColumnVector::create(end_col->size(), 0).get() - : block.getByPosition(arguments[0]).column.get(); + Columns columns_holder(2); + ColumnRawPtrs columns(2); + size_t idx = 0; + size_t rows = block.getByPosition(arguments[0]).column.get()->size(); - if (!executeStartInternal(block, start_col, end_col, result) && - !executeStartInternal(block, start_col, end_col, result) && - !executeStartInternal(block, start_col, end_col, result) && - !executeStartInternal(block, start_col, end_col, result)) + if (arguments.size() == 1) { - throw Exception{"Illegal column " + start_col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + columns_holder[idx] = std::move(DataTypeUInt8().createColumnConst(rows, 0)->convertToFullColumnIfConst()); + columns[idx] = columns_holder[idx].get(); + idx ++; + } + + for (size_t i = 0; i < arguments.size(); ++i) + { + columns_holder[idx] = std::move(block.getByPosition(arguments[i]).column->convertToFullColumnIfConst()); + columns[idx] = columns_holder[idx].get(); + idx ++; + } + + if (!executeStartInternal(block, columns[0], columns[1], result) && + !executeStartInternal(block, columns[0], columns[1], result) && + !executeStartInternal(block, columns[0], columns[1], result) && + !executeStartInternal(block, columns[0], columns[1], result)) + { + throw Exception{"Illegal column " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } From 66ffec32f7c2a98acfe1d07328f1707f027022b8 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 29 Oct 2019 14:33:03 +0800 Subject: [PATCH 04/77] improve range to support start,step args --- dbms/src/Functions/array/range.cpp | 85 ++++++++++++------- .../functions/array_functions.md | 9 +- 2 files changed, 61 insertions(+), 33 deletions(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 3106ddd3888..07991d6e999 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -35,9 +35,9 @@ private: DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override { - if (arguments.size() > 2 || arguments.empty()) + if (arguments.size() > 3 || arguments.empty()) { - throw Exception{"Function " + getName() + " needs [1-2] argument; passed " + throw Exception{"Function " + getName() + " needs [1-3] argument; passed " + std::to_string(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; } @@ -48,37 +48,47 @@ private: throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - return std::make_shared(arguments.back()); + return std::make_shared(arguments.size() == 3 ? arguments[1] : arguments.back()); } - template - bool executeInternal(Block & block, const IColumn * start_col, const IColumn * end_col, const size_t result) + template + bool executeStartEndStep(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result) { static constexpr size_t max_elements = 100'000'000; - - auto start_column = checkAndGetColumn>(start_col); - auto end_column = checkAndGetColumn>(end_col); - if (!start_column || !end_column) + auto start_column = checkAndGetColumn>(start_col); + auto end_column = checkAndGetColumn>(end_col); + auto step_column = checkAndGetColumn>(step_col); + + if (!start_column || !end_column || !step_column) { return false; } - const auto & in_start_data = start_column->getData(); - const auto & in_end_data = end_column->getData(); + const auto & start_data = start_column->getData(); + const auto & end_start = end_column->getData(); + const auto & step_data = step_column->getData(); size_t total_values = 0; + size_t pre_values = 0; for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) { - total_values += in_start_data[row_idx] >= in_end_data[row_idx] ? 0 : (in_end_data[row_idx] -in_start_data[row_idx]); + pre_values += start_data[row_idx] >= end_start[row_idx] ? 0 + : (end_start[row_idx] -start_data[row_idx] - 1) / (step_data[row_idx]) + 1; + + if (pre_values < total_values) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + total_values = pre_values; if (total_values > max_elements) throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), ErrorCodes::ARGUMENT_OUT_OF_BOUND}; } - auto data_col = ColumnVector::create(total_values); + auto data_col = ColumnVector::create(total_values); auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size()); auto & out_data = data_col->getData(); @@ -87,7 +97,7 @@ private: IColumn::Offset offset{}; for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) { - for (size_t st = in_start_data[row_idx], ed = in_end_data[row_idx]; st < ed; ++st) + for (size_t st = start_data[row_idx], ed = end_start[row_idx]; st < ed; st += step_data[row_idx]) out_data[offset++] = st; out_offsets[row_idx] = offset; @@ -96,23 +106,33 @@ private: block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); return true; } - - template - bool executeStartInternal(Block & block, const IColumn * start_col, const IColumn * end_col, const size_t result) + + template + bool executeStartEnd(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result) { - return executeInternal(block, start_col, end_col, result) - || executeInternal(block, start_col, end_col, result) - || executeInternal(block, start_col, end_col, result) - || executeInternal(block, start_col, end_col, result); + return executeStartEndStep(block, start_col, end_col, step_col,result) + || executeStartEndStep(block, start_col, end_col, step_col,result) + || executeStartEndStep(block, start_col, end_col, step_col,result) + || executeStartEndStep(block, start_col, end_col, step_col,result); + } + + template + bool executeStart(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result) + { + return executeStartEnd(block, start_col, end_col, step_col, result) + || executeStartEnd(block, start_col, end_col, step_col, result) + || executeStartEnd(block, start_col, end_col, step_col, result) + || executeStartEnd(block, start_col, end_col, step_col, result); } void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override - { - Columns columns_holder(2); - ColumnRawPtrs columns(2); + { + Columns columns_holder(3); + ColumnRawPtrs columns(3); size_t idx = 0; size_t rows = block.getByPosition(arguments[0]).column.get()->size(); + // for start column, default to 0 if (arguments.size() == 1) { columns_holder[idx] = std::move(DataTypeUInt8().createColumnConst(rows, 0)->convertToFullColumnIfConst()); @@ -127,12 +147,19 @@ private: idx ++; } - if (!executeStartInternal(block, columns[0], columns[1], result) && - !executeStartInternal(block, columns[0], columns[1], result) && - !executeStartInternal(block, columns[0], columns[1], result) && - !executeStartInternal(block, columns[0], columns[1], result)) + // for step column, defaults to 1 + if (arguments.size() <= 2) { - throw Exception{"Illegal column " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + columns_holder[idx] = std::move(DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst()); + columns[idx] = columns_holder[idx].get(); + } + + if (!executeStart(block, columns[0], columns[1], columns[2], result) && + !executeStart(block, columns[0], columns[1], columns[2], result) && + !executeStart(block, columns[0], columns[1], columns[2], result) && + !executeStart(block, columns[0], columns[1], columns[2], result)) + { + throw Exception{"Illegal columns " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } } diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index ed33ae5e512..89d6ca208df 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -34,10 +34,11 @@ Accepts zero arguments and returns an empty array of the appropriate type. Accepts an empty array and returns a one-element array that is equal to the default value. -## range([start,] end) - -Returns an array of numbers from start to end-1. -If the argument `start` is not specified, defaults to 0. +## range(end) or range(start, end [, step]) +Returns an array of numbers from start to end-1 by step. +If the argument `start` is not specified, defaults to 0. +If the argument `step` is not specified, default to 1. +It behaviors almost like pythonic `range`. But the difference is that all the arguments type must be `UInt` numbers. Just in case, an exception is thrown if arrays with a total length of more than 100,000,000 elements are created in a data block. ## array(x1, ...), operator \[x1, ...\] From 00389a67a0073f7b83dd46463b7c445a7fcdc5bd Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 29 Oct 2019 14:55:59 +0800 Subject: [PATCH 05/77] add more tests --- .../queries/0_stateless/00343_array_element_generic.reference | 3 +++ dbms/tests/queries/0_stateless/00343_array_element_generic.sql | 3 +++ 2 files changed, 6 insertions(+) diff --git a/dbms/tests/queries/0_stateless/00343_array_element_generic.reference b/dbms/tests/queries/0_stateless/00343_array_element_generic.reference index a1273ce2962..788a7c4b9fd 100644 --- a/dbms/tests/queries/0_stateless/00343_array_element_generic.reference +++ b/dbms/tests/queries/0_stateless/00343_array_element_generic.reference @@ -1,3 +1,6 @@ +1 +1 +1 0 0 0 diff --git a/dbms/tests/queries/0_stateless/00343_array_element_generic.sql b/dbms/tests/queries/0_stateless/00343_array_element_generic.sql index 9aa2e6db7be..33a5f4dc1f2 100644 --- a/dbms/tests/queries/0_stateless/00343_array_element_generic.sql +++ b/dbms/tests/queries/0_stateless/00343_array_element_generic.sql @@ -1,3 +1,6 @@ +SELECT range(100) == range(0, 100) and range(0, 100) == range(0, 100, 1); +SELECT distinct length(range(number, number + 100, 99)) == 2 FROM numbers(1000); +SELECT distinct length(range(number, number + 100, 100)) == 1 FROM numbers(1000); SELECT range(0)[-1]; SELECT range(0)[1]; SELECT range(number)[2] FROM system.numbers LIMIT 10; From 718fb99939a7385776003534693384ad89b3c57e Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 29 Oct 2019 19:20:35 +0800 Subject: [PATCH 06/77] fix move --- dbms/src/Functions/array/range.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 07991d6e999..2587f44d9aa 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -135,14 +135,14 @@ private: // for start column, default to 0 if (arguments.size() == 1) { - columns_holder[idx] = std::move(DataTypeUInt8().createColumnConst(rows, 0)->convertToFullColumnIfConst()); + columns_holder[idx] = DataTypeUInt8().createColumnConst(rows, 0)->convertToFullColumnIfConst(); columns[idx] = columns_holder[idx].get(); idx ++; } for (size_t i = 0; i < arguments.size(); ++i) { - columns_holder[idx] = std::move(block.getByPosition(arguments[i]).column->convertToFullColumnIfConst()); + columns_holder[idx] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst(); columns[idx] = columns_holder[idx].get(); idx ++; } @@ -150,7 +150,7 @@ private: // for step column, defaults to 1 if (arguments.size() <= 2) { - columns_holder[idx] = std::move(DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst()); + columns_holder[idx] = DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst(); columns[idx] = columns_holder[idx].get(); } From be9da37720bf5ad2ad6e6dda68728f3156c2b361 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 30 Oct 2019 09:36:50 +0800 Subject: [PATCH 07/77] fix zero divide --- dbms/src/Functions/array/range.cpp | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 2587f44d9aa..1465751331d 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -74,6 +74,10 @@ private: for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) { + if (step_data[row_idx] == 0) + throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + pre_values += start_data[row_idx] >= end_start[row_idx] ? 0 : (end_start[row_idx] -start_data[row_idx] - 1) / (step_data[row_idx]) + 1; From 72927480122376ae05a1c600e4b7a1c981e2f2a4 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 30 Oct 2019 10:06:30 +0800 Subject: [PATCH 08/77] fix docs --- dbms/src/Functions/array/range.cpp | 2 +- docs/en/query_language/functions/array_functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 1465751331d..be28bc6194a 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -74,7 +74,7 @@ private: for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) { - if (step_data[row_idx] == 0) + if (start_data[row_idx] < end_start[row_idx] && step_data[row_idx] == 0) throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero", ErrorCodes::ARGUMENT_OUT_OF_BOUND}; diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 89d6ca208df..f33fcc2e585 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -37,7 +37,7 @@ Accepts an empty array and returns a one-element array that is equal to the defa ## range(end) or range(start, end [, step]) Returns an array of numbers from start to end-1 by step. If the argument `start` is not specified, defaults to 0. -If the argument `step` is not specified, default to 1. +If the argument `step` is not specified, defaults to 1. It behaviors almost like pythonic `range`. But the difference is that all the arguments type must be `UInt` numbers. Just in case, an exception is thrown if arrays with a total length of more than 100,000,000 elements are created in a data block. From 864bfb20ad49492e770b4b058d8d8a2f39395047 Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 30 Oct 2019 18:34:09 +0300 Subject: [PATCH 09/77] Docs added --- docs/en/query_language/functions/encoding_functions.md | 2 +- docs/ru/query_language/functions/encoding_functions.md | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/query_language/functions/encoding_functions.md b/docs/en/query_language/functions/encoding_functions.md index 74ef53f82f7..7bc4760d7d2 100644 --- a/docs/en/query_language/functions/encoding_functions.md +++ b/docs/en/query_language/functions/encoding_functions.md @@ -2,7 +2,7 @@ ## hex -Accepts arguments of types: `String`, `unsigned integer`, `Date`, or `DateTime`. Returns a string containing the argument's hexadecimal representation. Uses uppercase letters `A-F`. Does not use `0x` prefixes or `h` suffixes. For strings, all bytes are simply encoded as two hexadecimal numbers. Numbers are converted to big endian ("human readable") format. For numbers, older zeros are trimmed, but only by entire bytes. For example, `hex (1) = '01'`. `Date` is encoded as the number of days since the beginning of the Unix epoch. `DateTime` is encoded as the number of seconds since the beginning of the Unix epoch. +Accepts arguments of types: `String`, `unsigned integer`, `float`, `decimal`, `Date`, or `DateTime`. Returns a string containing the argument's hexadecimal representation. Uses uppercase letters `A-F`. Does not use `0x` prefixes or `h` suffixes. For strings, all bytes are simply encoded as two hexadecimal numbers. Numbers are converted to big endian ("human readable") format. For numbers, older zeros are trimmed, but only by entire bytes. For example, `hex (1) = '01'`. `Date` is encoded as the number of days since the beginning of the Unix epoch. `DateTime` is encoded as the number of seconds since the beginning of the Unix epoch. `float` and `decimal` is encoded as their hexadecimal representation in memory. ## unhex(str) diff --git a/docs/ru/query_language/functions/encoding_functions.md b/docs/ru/query_language/functions/encoding_functions.md index c3825cd22af..6d9704d597a 100644 --- a/docs/ru/query_language/functions/encoding_functions.md +++ b/docs/ru/query_language/functions/encoding_functions.md @@ -2,7 +2,7 @@ ## hex -Принимает аргументы типов: `String`, `unsigned integer`, `Date`, or `DateTime`. Возвращает строку, содержащую шестнадцатеричное представление аргумента. Используются заглавные буквы `A-F`. Не используются префиксы `0x` и суффиксы `h`. Для строк просто все байты кодируются в виде двух шестнадцатеричных цифр. Числа выводятся в big endian ("человеческом") формате. Для чисел вырезаются старшие нули, но только по целым байтам. Например, `hex(1) = '01'`. `Date` кодируется как число дней с начала unix-эпохи. `DateTime` кодируются как число секунд с начала unix-эпохи. +Принимает аргументы типов: `String`, `unsigned integer`, `float`, `decimal`, `Date`, or `DateTime`. Возвращает строку, содержащую шестнадцатеричное представление аргумента. Используются заглавные буквы `A-F`. Не используются префиксы `0x` и суффиксы `h`. Для строк просто все байты кодируются в виде двух шестнадцатеричных цифр. Числа выводятся в big endian ("человеческом") формате. Для чисел вырезаются старшие нули, но только по целым байтам. Например, `hex(1) = '01'`. `Date` кодируется как число дней с начала unix-эпохи. `DateTime` кодируются как число секунд с начала unix-эпохи. `float` и `decimal` кодируются как их шетснадцатеричное представление в памяти. ## unhex(str) Принимает строку, содержащую произвольное количество шестнадцатеричных цифр, и возвращает строку, содержащую соответствующие байты. Поддерживаются как строчные, так и заглавные буквы A-F. Число шестнадцатеричных цифр не обязано быть чётным. Если оно нечётное - последняя цифра интерпретируется как младшая половинка байта 00-0F. Если строка-аргумент содержит что-либо кроме шестнадцатеричных цифр, то будет возвращён какой-либо implementation-defined результат (не кидается исключение). From 1ec7cb3546b502effcc2504d70490b5cb7f75b56 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 30 Oct 2019 23:39:29 +0300 Subject: [PATCH 10/77] Update range.cpp --- dbms/src/Functions/array/range.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index be28bc6194a..7e8005db82a 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -37,7 +37,7 @@ private: { if (arguments.size() > 3 || arguments.empty()) { - throw Exception{"Function " + getName() + " needs [1-3] argument; passed " + throw Exception{"Function " + getName() + " needs 1..3 arguments; passed " + std::to_string(arguments.size()) + ".", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH}; } From d2a785f759a548e821c677358309362c36d20eab Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Thu, 31 Oct 2019 12:59:50 +0800 Subject: [PATCH 11/77] fix style --- dbms/src/Functions/array/range.cpp | 45 +++++++++---------- .../functions/array_functions.md | 2 +- 2 files changed, 23 insertions(+), 24 deletions(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 7e8005db82a..85965c73b3c 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -52,7 +52,7 @@ private: } template - bool executeStartEndStep(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result) + bool executeStartEndStep(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) { static constexpr size_t max_elements = 100'000'000; @@ -72,7 +72,7 @@ private: size_t total_values = 0; size_t pre_values = 0; - for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { if (start_data[row_idx] < end_start[row_idx] && step_data[row_idx] == 0) throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero", @@ -99,7 +99,7 @@ private: auto & out_offsets = offsets_col->getData(); IColumn::Offset offset{}; - for (size_t row_idx = 0, rows = end_column->size(); row_idx < rows; ++row_idx) + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) { for (size_t st = start_data[row_idx], ed = end_start[row_idx]; st < ed; st += step_data[row_idx]) out_data[offset++] = st; @@ -112,56 +112,55 @@ private: } template - bool executeStartEnd(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result) + bool executeStartEnd(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) { - return executeStartEndStep(block, start_col, end_col, step_col,result) - || executeStartEndStep(block, start_col, end_col, step_col,result) - || executeStartEndStep(block, start_col, end_col, step_col,result) - || executeStartEndStep(block, start_col, end_col, step_col,result); + return executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result) + || executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result) + || executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result) + || executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result); } template - bool executeStart(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t result) + bool executeStart(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) { - return executeStartEnd(block, start_col, end_col, step_col, result) - || executeStartEnd(block, start_col, end_col, step_col, result) - || executeStartEnd(block, start_col, end_col, step_col, result) - || executeStartEnd(block, start_col, end_col, step_col, result); + return executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result) + || executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result) + || executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result) + || executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result); } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t) override + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { Columns columns_holder(3); ColumnRawPtrs columns(3); size_t idx = 0; - size_t rows = block.getByPosition(arguments[0]).column.get()->size(); // for start column, default to 0 if (arguments.size() == 1) { - columns_holder[idx] = DataTypeUInt8().createColumnConst(rows, 0)->convertToFullColumnIfConst(); + columns_holder[idx] = DataTypeUInt8().createColumnConst(input_rows_count, 0)->convertToFullColumnIfConst(); columns[idx] = columns_holder[idx].get(); - idx ++; + ++idx; } for (size_t i = 0; i < arguments.size(); ++i) { columns_holder[idx] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst(); columns[idx] = columns_holder[idx].get(); - idx ++; + ++idx; } // for step column, defaults to 1 if (arguments.size() <= 2) { - columns_holder[idx] = DataTypeUInt8().createColumnConst(rows, 1)->convertToFullColumnIfConst(); + columns_holder[idx] = DataTypeUInt8().createColumnConst(input_rows_count, 1)->convertToFullColumnIfConst(); columns[idx] = columns_holder[idx].get(); } - if (!executeStart(block, columns[0], columns[1], columns[2], result) && - !executeStart(block, columns[0], columns[1], columns[2], result) && - !executeStart(block, columns[0], columns[1], columns[2], result) && - !executeStart(block, columns[0], columns[1], columns[2], result)) + if (!executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result) && + !executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result) && + !executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result) && + !executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result)) { throw Exception{"Illegal columns " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index f33fcc2e585..7081cf2ecba 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -34,7 +34,7 @@ Accepts zero arguments and returns an empty array of the appropriate type. Accepts an empty array and returns a one-element array that is equal to the default value. -## range(end) or range(start, end [, step]) +## range(end), range(start, end [, step]) Returns an array of numbers from start to end-1 by step. If the argument `start` is not specified, defaults to 0. If the argument `step` is not specified, defaults to 1. From ddcbc2f9d3d6abb74ab1c9536df1b509e53a8937 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Nov 2019 12:45:33 +0300 Subject: [PATCH 12/77] Enable processors by default. --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index fb3021275be..6f106044d10 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -360,7 +360,7 @@ struct Settings : public SettingsCollection M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \ \ - M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \ + M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ From 1f7a798c975a595dc440e4f5cc5e5390bc43462c Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Nov 2019 13:58:29 +0300 Subject: [PATCH 13/77] Add flag to set that set is created. --- .../DataStreams/CreatingSetsBlockInputStream.cpp | 3 +++ dbms/src/Interpreters/ExpressionActions.cpp | 6 ++++-- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 2 ++ dbms/src/Interpreters/Set.cpp | 1 + dbms/src/Interpreters/Set.h | 7 +++++++ .../Transforms/CreatingSetsTransform.cpp | 3 +++ .../src/Processors/Transforms/FilterTransform.cpp | 15 ++------------- dbms/src/Processors/Transforms/FilterTransform.h | 2 -- dbms/src/Storages/StorageSet.cpp | 5 +++++ dbms/src/Storages/StorageSet.h | 3 +++ 10 files changed, 30 insertions(+), 17 deletions(-) diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 14912b4ebc3..495770f318e 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -146,6 +146,9 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) } } + if (subquery.set) + subquery.set->finishInsert(); + if (table_out) table_out->writeSuffix(); diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index e1e3fa3cedc..c975b8461fa 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -1188,8 +1188,9 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con /// Check has column in (empty set). String set_to_check; - for (auto & action : actions) + for (auto it = actions.rbegin(); it != actions.rend(); ++it) { + auto & action = *it; if (action.type == action.APPLY_FUNCTION && action.function_base) { auto name = action.function_base->getName(); @@ -1198,6 +1199,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con && action.argument_names.size() > 1) { set_to_check = action.argument_names[1]; + break; } } } @@ -1210,7 +1212,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con { if (auto * column_set = typeid_cast(action.added_column.get())) { - if (column_set->getData()->getTotalRowCount() == 0) + if (column_set->getData()->isCreated() && column_set->getData()->empty()) return true; } } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index f694f74989a..fc5d697d005 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -249,6 +249,8 @@ void SelectQueryExpressionAnalyzer::tryMakeSetForIndexFromSubquery(const ASTPtr if (!set->insertFromBlock(block)) return; } + + set->finishInsert(); res.in->readSuffix(); prepared_sets[set_key] = std::move(set); diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index fbed7957f2f..dc22b497dca 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -291,6 +291,7 @@ void Set::createFromAST(const DataTypes & types, ASTPtr node, const Context & co Block block = header.cloneWithColumns(std::move(columns)); insertFromBlock(block); + finishInsert(); } diff --git a/dbms/src/Interpreters/Set.h b/dbms/src/Interpreters/Set.h index 987252e37ba..9d837857ef3 100644 --- a/dbms/src/Interpreters/Set.h +++ b/dbms/src/Interpreters/Set.h @@ -56,6 +56,10 @@ public: /// Returns false, if some limit was exceeded and no need to insert more data. bool insertFromBlock(const Block & block); + /// Call after all blocks were inserted. To get the information that set is already created. + void finishInsert() { is_created = true; } + + bool isCreated() const { return is_created; } /** For columns of 'block', check belonging of corresponding rows to the set. * Return UInt8 column with the result. @@ -107,6 +111,9 @@ private: /// Do we need to additionally store all elements of the set in explicit form for subsequent use for index. bool fill_set_elements; + /// Check if set contains all the data. + bool is_created = false; + /// If in the left part columns contains the same types as the elements of the set. void executeOrdinary( const ColumnRawPtrs & key_columns, diff --git a/dbms/src/Processors/Transforms/CreatingSetsTransform.cpp b/dbms/src/Processors/Transforms/CreatingSetsTransform.cpp index c69f5f42e1c..f21d68afea2 100644 --- a/dbms/src/Processors/Transforms/CreatingSetsTransform.cpp +++ b/dbms/src/Processors/Transforms/CreatingSetsTransform.cpp @@ -141,6 +141,9 @@ void CreatingSetsTransform::work() auto finishCurrentSubquery = [&]() { + if (subquery.set) + subquery.set->finishInsert(); + if (table_out) table_out->writeSuffix(); diff --git a/dbms/src/Processors/Transforms/FilterTransform.cpp b/dbms/src/Processors/Transforms/FilterTransform.cpp index e4f82025fb1..202a5c1160f 100644 --- a/dbms/src/Processors/Transforms/FilterTransform.cpp +++ b/dbms/src/Processors/Transforms/FilterTransform.cpp @@ -64,7 +64,8 @@ FilterTransform::FilterTransform( IProcessor::Status FilterTransform::prepare() { - if (constant_filter_description.always_false) + if (constant_filter_description.always_false + || expression->checkColumnIsAlwaysFalse(filter_column_name)) { input.close(); output.finish(); @@ -83,18 +84,6 @@ void FilterTransform::removeFilterIfNeed(Chunk & chunk) void FilterTransform::transform(Chunk & chunk) { - if (!initialized) - { - initialized = true; - /// Cannot check this in prepare. Because in prepare columns for set may be not created yet. - if (expression->checkColumnIsAlwaysFalse(filter_column_name)) - { - stopReading(); - chunk = Chunk(getOutputPort().getHeader().getColumns(), 0); - return; - } - } - size_t num_rows_before_filtration = chunk.getNumRows(); auto columns = chunk.detachColumns(); diff --git a/dbms/src/Processors/Transforms/FilterTransform.h b/dbms/src/Processors/Transforms/FilterTransform.h index c595d72a70b..127eb5a8039 100644 --- a/dbms/src/Processors/Transforms/FilterTransform.h +++ b/dbms/src/Processors/Transforms/FilterTransform.h @@ -36,8 +36,6 @@ private: /// Header after expression, but before removing filter column. Block transformed_header; - bool initialized = false; - void removeFilterIfNeed(Chunk & chunk); }; diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index c76857bf610..f179565e61c 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -70,6 +70,7 @@ void SetOrJoinBlockOutputStream::write(const Block & block) void SetOrJoinBlockOutputStream::writeSuffix() { + table.finishInsert(); backup_stream.flush(); compressed_backup_buf.next(); backup_buf.next(); @@ -123,6 +124,7 @@ StorageSet::StorageSet( void StorageSet::insertBlock(const Block & block) { set->insertFromBlock(block); } +void StorageSet::finishInsert() { set->finishInsert(); } size_t StorageSet::getSize() const { return set->getTotalRowCount(); } @@ -180,8 +182,11 @@ void StorageSetOrJoinBase::restoreFromFile(const String & file_path) NativeBlockInputStream backup_stream(compressed_backup_buf, 0); backup_stream.readPrefix(); + while (Block block = backup_stream.read()) insertBlock(block); + + finishInsert(); backup_stream.readSuffix(); /// TODO Add speed, compressed bytes, data volume in memory, compression ratio ... Generalize all statistics logging in project. diff --git a/dbms/src/Storages/StorageSet.h b/dbms/src/Storages/StorageSet.h index 954e58d3929..e1ca91ad22c 100644 --- a/dbms/src/Storages/StorageSet.h +++ b/dbms/src/Storages/StorageSet.h @@ -50,6 +50,8 @@ private: /// Insert the block into the state. virtual void insertBlock(const Block & block) = 0; + /// Call after all blocks were inserted. + virtual void finishInsert() = 0; virtual size_t getSize() const = 0; }; @@ -75,6 +77,7 @@ private: SetPtr set; void insertBlock(const Block & block) override; + void finishInsert() override; size_t getSize() const override; protected: From 388bbb950b83742f317d1580313f5a0f68b9edc7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Nov 2019 14:03:35 +0300 Subject: [PATCH 14/77] Add flag to set that set is created. --- dbms/src/Storages/StorageJoin.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Storages/StorageJoin.h b/dbms/src/Storages/StorageJoin.h index d770078ac39..cfafd118768 100644 --- a/dbms/src/Storages/StorageJoin.h +++ b/dbms/src/Storages/StorageJoin.h @@ -55,6 +55,7 @@ private: HashJoinPtr join; void insertBlock(const Block & block) override; + void finishInsert() override {} size_t getSize() const override; protected: From 575eabd4ff71c1ac032de7dff1078cdecc9cc4d1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 1 Nov 2019 16:56:33 +0300 Subject: [PATCH 15/77] Add flag to set that set is created. --- 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 c975b8461fa..cbfedcd5cff 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -1212,7 +1212,7 @@ bool ExpressionActions::checkColumnIsAlwaysFalse(const String & column_name) con { if (auto * column_set = typeid_cast(action.added_column.get())) { - if (column_set->getData()->isCreated() && column_set->getData()->empty()) + if (column_set->getData()->isCreated() && column_set->getData()->getTotalRowCount() == 0) return true; } } From 71ec49cc2fdeba910c17e868dc3147afb739f326 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Tue, 5 Nov 2019 15:07:25 +0800 Subject: [PATCH 16/77] add range performance tests --- dbms/src/Functions/array/range.cpp | 73 ++++++++++++++++++++++++------ dbms/tests/performance/range.xml | 17 +++++++ 2 files changed, 77 insertions(+), 13 deletions(-) create mode 100644 dbms/tests/performance/range.xml diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index 85965c73b3c..faafbd637b2 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -24,6 +24,7 @@ class FunctionRange : public IFunction { public: static constexpr auto name = "range"; + static constexpr size_t max_elements = 100'000'000; static FunctionPtr create(const Context &) { return std::make_shared(); } private: @@ -51,11 +52,54 @@ private: return std::make_shared(arguments.size() == 3 ? arguments[1] : arguments.back()); } + template + bool executeInternal(Block & block, const IColumn * arg, const size_t result) + { + if (const auto in = checkAndGetColumn>(arg)) + { + const auto & in_data = in->getData(); + const auto total_values = std::accumulate(std::begin(in_data), std::end(in_data), size_t{}, + [this] (const size_t lhs, const size_t rhs) + { + const auto sum = lhs + rhs; + if (sum < lhs) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + return sum; + }); + + if (total_values > max_elements) + throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + auto data_col = ColumnVector::create(total_values); + auto offsets_col = ColumnArray::ColumnOffsets::create(in->size()); + + auto & out_data = data_col->getData(); + auto & out_offsets = offsets_col->getData(); + + IColumn::Offset offset{}; + for (size_t row_idx = 0, rows = in->size(); row_idx < rows; ++row_idx) + { + for (size_t elem_idx = 0, elems = in_data[row_idx]; elem_idx < elems; ++elem_idx) + out_data[offset + elem_idx] = elem_idx; + + offset += in_data[row_idx]; + out_offsets[row_idx] = offset; + } + + block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); + return true; + } + else + return false; + } + template bool executeStartEndStep(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) { - static constexpr size_t max_elements = 100'000'000; - auto start_column = checkAndGetColumn>(start_col); auto end_column = checkAndGetColumn>(end_col); auto step_column = checkAndGetColumn>(step_col); @@ -133,28 +177,31 @@ private: { Columns columns_holder(3); ColumnRawPtrs columns(3); - size_t idx = 0; - // for start column, default to 0 if (arguments.size() == 1) { - columns_holder[idx] = DataTypeUInt8().createColumnConst(input_rows_count, 0)->convertToFullColumnIfConst(); - columns[idx] = columns_holder[idx].get(); - ++idx; + const auto col = block.getByPosition(arguments[0]).column.get(); + if (!executeInternal(block, col, result) && + !executeInternal(block, col, result) && + !executeInternal(block, col, result) && + !executeInternal(block, col, result)) + { + throw Exception{"Illegal column " + col->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; + } + return; } for (size_t i = 0; i < arguments.size(); ++i) { - columns_holder[idx] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst(); - columns[idx] = columns_holder[idx].get(); - ++idx; + columns_holder[i] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst(); + columns[i] = columns_holder[i].get(); } // for step column, defaults to 1 - if (arguments.size() <= 2) + if (arguments.size() == 2) { - columns_holder[idx] = DataTypeUInt8().createColumnConst(input_rows_count, 1)->convertToFullColumnIfConst(); - columns[idx] = columns_holder[idx].get(); + columns_holder[2] = DataTypeUInt8().createColumnConst(input_rows_count, 1)->convertToFullColumnIfConst(); + columns[2] = columns_holder[2].get(); } if (!executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result) && diff --git a/dbms/tests/performance/range.xml b/dbms/tests/performance/range.xml new file mode 100644 index 00000000000..b075bad5e43 --- /dev/null +++ b/dbms/tests/performance/range.xml @@ -0,0 +1,17 @@ + + once + + + + 5000 + 10000 + + + + + + + + SELECT count() FROM (SELECT range(number % 100) FROM system.numbers limit 10000000) + SELECT count() FROM (SELECT range(0, number % 100, 1) FROM system.numbers limit 10000000) + From abccddb6b9584b5a8565db4f823485d38cc417f1 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Nov 2019 12:26:15 +0300 Subject: [PATCH 17/77] Check number of rows in OneBlockInputStream. --- dbms/src/DataStreams/OneBlockInputStream.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/OneBlockInputStream.h b/dbms/src/DataStreams/OneBlockInputStream.h index 168053b4fb3..06f945b5803 100644 --- a/dbms/src/DataStreams/OneBlockInputStream.h +++ b/dbms/src/DataStreams/OneBlockInputStream.h @@ -12,7 +12,7 @@ namespace DB class OneBlockInputStream : public IBlockInputStream { public: - explicit OneBlockInputStream(const Block & block_) : block(block_) {} + explicit OneBlockInputStream(Block block_) : block(std::move(block_)) { block.checkNumberOfRows(); } String getName() const override { return "One"; } From a0e9f9fdcf9aa4aea42f32ed1f402330023c7c34 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Nov 2019 15:50:47 +0300 Subject: [PATCH 18/77] Skip empty totals in SourceFromInputStream. --- dbms/src/Processors/Sources/SourceFromInputStream.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/dbms/src/Processors/Sources/SourceFromInputStream.cpp b/dbms/src/Processors/Sources/SourceFromInputStream.cpp index 691a9785942..08eb0254097 100644 --- a/dbms/src/Processors/Sources/SourceFromInputStream.cpp +++ b/dbms/src/Processors/Sources/SourceFromInputStream.cpp @@ -115,8 +115,11 @@ Chunk SourceFromInputStream::generate() if (auto totals_block = stream->getTotals()) { - totals.setColumns(totals_block.getColumns(), 1); - has_totals = true; + if (totals_block.rows() == 1) /// Sometimes we can get empty totals. Skip it. + { + totals.setColumns(totals_block.getColumns(), 1); + has_totals = true; + } } is_stream_finished = true; From 055f21c9b746d404137ef9d30563c9316ad89fcb Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Tue, 5 Nov 2019 20:33:03 +0300 Subject: [PATCH 19/77] Less wrappers for procesors pipeline. --- .../Interpreters/InterpreterSelectQuery.cpp | 82 ++++++++++++++----- .../InterpreterSelectWithUnionQuery.cpp | 3 +- dbms/src/Processors/Pipe.cpp | 35 +++++++- dbms/src/Processors/Pipe.h | 15 ++++ dbms/src/Processors/QueryPipeline.cpp | 32 ++++---- dbms/src/Processors/QueryPipeline.h | 10 ++- dbms/src/Storages/IStorage.h | 2 + dbms/src/Storages/StorageMergeTree.h | 2 + .../src/Storages/StorageReplicatedMergeTree.h | 2 + 9 files changed, 143 insertions(+), 40 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 61fa0d05072..832e313f94b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -92,6 +92,7 @@ #include #include #include +#include namespace DB @@ -955,7 +956,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (options.only_analyze) { if constexpr (pipeline_with_processors) - pipeline.init({std::make_shared(source_header)}); + pipeline.init(Pipe(std::make_shared(source_header))); else pipeline.streams.emplace_back(std::make_shared(source_header)); @@ -999,7 +1000,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS if (prepared_input) { if constexpr (pipeline_with_processors) - pipeline.init({std::make_shared(prepared_input)}); + pipeline.init(Pipe(std::make_shared(prepared_input))); else pipeline.streams.push_back(prepared_input); } @@ -1322,7 +1323,7 @@ void InterpreterSelectQuery::executeFetchColumns( auto istream = std::make_shared(block_with_count); if constexpr (pipeline_with_processors) - pipeline.init({std::make_shared(istream)}); + pipeline.init(Pipe(std::make_shared(istream))); else pipeline.streams.emplace_back(istream); from_stage = QueryProcessingStage::WithMergeableState; @@ -1587,9 +1588,19 @@ void InterpreterSelectQuery::executeFetchColumns( query_info.prewhere_info = prewhere_info; query_info.sorting_info = sorting_info; - auto streams = storage->read(required_columns, query_info, context, processing_stage, max_block_size, max_streams); + BlockInputStreams streams; + Pipes pipes; - if (streams.empty()) + /// Will work with pipes directly if storage support processors. + /// Code is temporarily copy-pasted while moving to new pipeline. + bool use_pipes = pipeline_with_processors && storage->supportProcessorsPipeline(); + + if (use_pipes) + pipes = storage->readWithProcessors(required_columns, query_info, context, processing_stage, max_block_size, max_streams); + else + streams = storage->read(required_columns, query_info, context, processing_stage, max_block_size, max_streams); + + if (streams.empty() && !use_pipes) { streams = {std::make_shared(storage->getSampleBlockForColumns(required_columns))}; @@ -1612,9 +1623,34 @@ void InterpreterSelectQuery::executeFetchColumns( } } + /// Copy-paste from prev if. + if (pipes.empty() && use_pipes) + { + Pipe pipe(std::make_shared(storage->getSampleBlockForColumns(required_columns))); + + if (query_info.prewhere_info) + { + pipe.addSimpleTransform(std::make_shared( + pipe.getHeader(), + prewhere_info->prewhere_actions, + prewhere_info->prewhere_column_name, + prewhere_info->remove_prewhere_column)); + + if (query_info.prewhere_info->remove_columns_actions) + pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); + } + } + for (auto & stream : streams) stream->addTableLock(table_lock); + if constexpr (pipeline_with_processors) + { + /// Table lock is stored inside pipeline here. + if (use_pipes) + pipeline.addTableLock(table_lock); + } + /// Set the limits and quota for reading data, the speed and time of the query. { IBlockInputStream::LocalLimits limits; @@ -1649,11 +1685,21 @@ void InterpreterSelectQuery::executeFetchColumns( if (options.to_stage == QueryProcessingStage::Complete) stream->setQuota(quota); } + + /// Copy-paste + for (auto & pipe : pipes) + { + if (!options.ignore_limits) + pipe.setLimits(limits); + + if (options.to_stage == QueryProcessingStage::Complete) + pipe.setQuota(quota); + } } if constexpr (pipeline_with_processors) { - if (streams.size() == 1) + if (streams.size() == 1 || pipes.size() == 1) pipeline.setMaxThreads(streams.size()); /// Unify streams. They must have same headers. @@ -1665,9 +1711,8 @@ void InterpreterSelectQuery::executeFetchColumns( if (first_header.columns() > 1 && first_header.has("_dummy")) first_header.erase("_dummy"); - for (size_t i = 0; i < streams.size(); ++i) + for (auto & stream : streams) { - auto & stream = streams[i]; auto header = stream->getHeader(); auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; if (!blocksHaveEqualStructure(first_header, header)) @@ -1675,12 +1720,6 @@ void InterpreterSelectQuery::executeFetchColumns( } } - Processors sources; - sources.reserve(streams.size()); - - /// Pin sources for merge tree tables. - bool pin_sources = dynamic_cast(storage.get()) != nullptr; - for (auto & stream : streams) { bool force_add_agg_info = processing_stage == QueryProcessingStage::WithMergeableState; @@ -1689,13 +1728,18 @@ void InterpreterSelectQuery::executeFetchColumns( if (processing_stage == QueryProcessingStage::Complete) source->addTotalsPort(); - if (pin_sources) - source->setStream(sources.size()); - - sources.emplace_back(std::move(source)); + pipes.emplace_back(std::move(source)); } - pipeline.init(std::move(sources)); + /// Pin sources for merge tree tables. + bool pin_sources = dynamic_cast(storage.get()) != nullptr; + if (pin_sources) + { + for (size_t i = 0; i < pipes.size(); ++i) + pipes[i].pinSources(i); + } + + pipeline.init(std::move(pipes)); } else pipeline.streams = std::move(streams); diff --git a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp index 0899fed9872..e9671eb7358 100644 --- a/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectWithUnionQuery.cpp @@ -14,6 +14,7 @@ #include #include +#include namespace DB @@ -236,7 +237,7 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors() } if (!has_main_pipeline) - main_pipeline.init({ std::make_shared(getSampleBlock()) }); + main_pipeline.init(Pipe(std::make_shared(getSampleBlock()))); if (!pipelines.empty()) { diff --git a/dbms/src/Processors/Pipe.cpp b/dbms/src/Processors/Pipe.cpp index 4511b468061..45bd1c96a7a 100644 --- a/dbms/src/Processors/Pipe.cpp +++ b/dbms/src/Processors/Pipe.cpp @@ -44,8 +44,8 @@ static void checkSource(const IProcessor & source) throw Exception("Source for pipe should have single output, but it doesn't have any", ErrorCodes::LOGICAL_ERROR); - if (source.getOutputs().size() != 1) - throw Exception("Source for pipe should have single output, but " + source.getName() + " has " + + if (source.getOutputs().size() > 2) + throw Exception("Source for pipe should have single or two outputs, but " + source.getName() + " has " + toString(source.getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } @@ -54,6 +54,10 @@ Pipe::Pipe(ProcessorPtr source) { checkSource(*source); output_port = &source->getOutputs().front(); + + if (source->getOutputs().size() > 1) + totals = &source->getOutputs().back(); + processors.emplace_back(std::move(source)); } @@ -84,4 +88,31 @@ void Pipe::addSimpleTransform(ProcessorPtr transform) processors.emplace_back(std::move(transform)); } +void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) +{ + for (auto & processor : processors) + { + if (auto * source_with_progress = typeid_cast(processor.get())) + source_with_progress->setLimits(limits); + } +} + +void Pipe::setQuota(QuotaForIntervals & quota) +{ + for (auto & processor : processors) + { + if (auto * source_with_progress = typeid_cast(processor.get())) + source_with_progress->setQuota(quota); + } +} + +void Pipe::pinSources(size_t executor_number) +{ + for (auto & processor : processors) + { + if (auto * source = typeid_cast(processor.get())) + source->setStream(executor_number); + } +} + } diff --git a/dbms/src/Processors/Pipe.h b/dbms/src/Processors/Pipe.h index 872f04c339b..72cb90c4b9e 100644 --- a/dbms/src/Processors/Pipe.h +++ b/dbms/src/Processors/Pipe.h @@ -1,4 +1,6 @@ +#pragma once #include +#include namespace DB { @@ -6,6 +8,8 @@ namespace DB class Pipe; using Pipes = std::vector; +class QuotaForIntervals; + /// Pipe is a set of processors which represents the part of pipeline with single output. /// All processors in pipe are connected. All ports are connected except the output one. class Pipe @@ -33,9 +37,20 @@ public: Processors detachProcessors() && { return std::move(processors); } + /// Specify quotas and limits for every ISourceWithProgress. + void setLimits(const SourceWithProgress::LocalLimits & limits); + void setQuota(QuotaForIntervals & quota); + + /// Set information about preferred executor number for sources. + void pinSources(size_t executor_number); + + void setTotalsPort(OutputPort * totals_) { totals = totals_; } + OutputPort * getTotalsPort() const { return totals; } + private: Processors processors; OutputPort * output_port = nullptr; + OutputPort * totals = nullptr; }; } diff --git a/dbms/src/Processors/QueryPipeline.cpp b/dbms/src/Processors/QueryPipeline.cpp index 445e12a2a2d..55829338b07 100644 --- a/dbms/src/Processors/QueryPipeline.cpp +++ b/dbms/src/Processors/QueryPipeline.cpp @@ -14,7 +14,6 @@ #include #include #include - #include #include #include @@ -48,36 +47,41 @@ void QueryPipeline::checkSource(const ProcessorPtr & source, bool can_have_total toString(source->getOutputs().size()) + " outputs.", ErrorCodes::LOGICAL_ERROR); } -void QueryPipeline::init(Processors sources) +void QueryPipeline::init(Pipe pipe) +{ + Pipes pipes; + pipes.emplace_back(std::move(pipe)); + init(std::move(pipes)); +} + +void QueryPipeline::init(Pipes pipes) { if (initialized()) throw Exception("Pipeline has already been initialized.", ErrorCodes::LOGICAL_ERROR); - if (sources.empty()) - throw Exception("Can't initialize pipeline with empty source list.", ErrorCodes::LOGICAL_ERROR); + if (pipes.empty()) + throw Exception("Can't initialize pipeline with empty pipes list.", ErrorCodes::LOGICAL_ERROR); std::vector totals; - for (auto & source : sources) + for (auto & pipe : pipes) { - checkSource(source, true); - - auto & header = source->getOutputs().front().getHeader(); + auto & header = pipe.getHeader(); if (current_header) assertBlocksHaveEqualStructure(current_header, header, "QueryPipeline"); else current_header = header; - if (source->getOutputs().size() > 1) + if (auto * totals_port = pipe.getTotalsPort()) { - assertBlocksHaveEqualStructure(current_header, source->getOutputs().back().getHeader(), "QueryPipeline"); - totals.emplace_back(&source->getOutputs().back()); + assertBlocksHaveEqualStructure(current_header, totals_port->getHeader(), "QueryPipeline"); + totals.emplace_back(totals_port); } - /// source->setStream(streams.size()); - streams.emplace_back(&source->getOutputs().front()); - processors.emplace_back(std::move(source)); + streams.emplace_back(&pipe.getPort()); + auto cur_processors = std::move(pipe).detachProcessors(); + processors.insert(processors.end(), cur_processors.begin(), cur_processors.end()); } if (!totals.empty()) diff --git a/dbms/src/Processors/QueryPipeline.h b/dbms/src/Processors/QueryPipeline.h index 9c65ec69bbe..dbeb5166ea2 100644 --- a/dbms/src/Processors/QueryPipeline.h +++ b/dbms/src/Processors/QueryPipeline.h @@ -1,6 +1,7 @@ #pragma once #include #include +#include #include #include @@ -11,7 +12,7 @@ namespace DB class TableStructureReadLock; using TableStructureReadLockPtr = std::shared_ptr; -using TableStructureReadLocks = std::vector; +using TableStructureReadLocks = std::vector; class Context; @@ -22,8 +23,9 @@ class QueryPipeline public: QueryPipeline() = default; - /// Each source must have single output port and no inputs. All outputs must have same header. - void init(Processors sources); + /// All pipes must have same header. + void init(Pipes pipes); + void init(Pipe pipe); /// Simple init for single pipe bool initialized() { return !processors.empty(); } enum class StreamType @@ -72,7 +74,7 @@ public: const Block & getHeader() const { return current_header; } - void addTableLock(const TableStructureReadLockPtr & lock) { table_locks.push_back(lock); } + void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } /// For compatibility with IBlockInputStream. void setProgressCallback(const ProgressCallback & callback); diff --git a/dbms/src/Storages/IStorage.h b/dbms/src/Storages/IStorage.h index 6ae8870b377..684e20efcd5 100644 --- a/dbms/src/Storages/IStorage.h +++ b/dbms/src/Storages/IStorage.h @@ -268,6 +268,8 @@ public: throw Exception("Method read is not supported by storage " + getName(), ErrorCodes::NOT_IMPLEMENTED); } + virtual bool supportProcessorsPipeline() const { return false; } + /** Writes the data to a table. * Receives a description of the query, which can contain information about the data write method. * Returns an object by which you can write data sequentially. diff --git a/dbms/src/Storages/StorageMergeTree.h b/dbms/src/Storages/StorageMergeTree.h index 8b426bc1f6f..236f1224468 100644 --- a/dbms/src/Storages/StorageMergeTree.h +++ b/dbms/src/Storages/StorageMergeTree.h @@ -45,6 +45,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool supportProcessorsPipeline() const override { return true; } + std::optional totalRows() const override; BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.h b/dbms/src/Storages/StorageReplicatedMergeTree.h index d6e25ceb3d0..3727d5de0d8 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.h +++ b/dbms/src/Storages/StorageReplicatedMergeTree.h @@ -97,6 +97,8 @@ public: size_t max_block_size, unsigned num_streams) override; + bool supportProcessorsPipeline() const override { return true; } + std::optional totalRows() const override; BlockOutputStreamPtr write(const ASTPtr & query, const Context & context) override; From 0226ba406c0e04b137aee53297dbd2e8ce0da1ca Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 6 Nov 2019 10:42:10 +0300 Subject: [PATCH 20/77] Functions tryExecuteFloat and tryExecuteDecimal are combined --- dbms/src/Functions/FunctionsCoding.h | 93 ++++++++++------------------ 1 file changed, 34 insertions(+), 59 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 7464ad16736..2061b64e2c1 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1025,86 +1025,61 @@ public: } template - bool tryExecuteFloat(const IColumn * col, ColumnPtr & col_res) + void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, size_t HEX_LENGTH) { - const ColumnVector * col_vec = checkAndGetColumn>(col); + auto col_str = ColumnString::create(); - static constexpr size_t FLOAT_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. + ColumnString::Chars & out_vec = col_str->getChars(); + ColumnString::Offsets & out_offsets = col_str->getOffsets(); - if (col_vec) + size_t size = in_vec.size(); + out_offsets.resize(size); + out_vec.resize(size * HEX_LENGTH); + + size_t pos = 0; + char * out = reinterpret_cast(&out_vec[0]); + for (size_t i = 0; i < size; ++i) { - auto col_str = ColumnString::create(); - ColumnString::Chars & out_vec = col_str->getChars(); - ColumnString::Offsets & out_offsets = col_str->getOffsets(); + const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); + executeOneString(in_pos, in_pos + sizeof(T), out); - const typename ColumnVector::Container & in_vec = col_vec->getData(); - - size_t size = in_vec.size(); - out_offsets.resize(size); - out_vec.resize(size * FLOAT_HEX_LENGTH); - - size_t pos = 0; - char * out = reinterpret_cast(&out_vec[0]); - for (size_t i = 0; i < size; ++i) - { - const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); - executeOneString(in_pos, in_pos + sizeof(T), out); - - pos += FLOAT_HEX_LENGTH; - out_offsets[i] = pos; - } - - col_res = std::move(col_str); - return true; - } - else - { - return false; + pos += HEX_LENGTH; + out_offsets[i] = pos; } + + col_res = std::move(col_str); } template - bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) + bool tryExecuteFloat(const IColumn * col, ColumnPtr & col_res) { + const ColumnVector * col_vec = checkAndGetColumn>(col); + static constexpr size_t FLOAT_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. + if (col_vec) + { + const typename ColumnVector::Container & in_vec = col_vec->getData(); + executeFloatAndDecimal::Container>(in_vec, col_res, FLOAT_HEX_LENGTH); + return true; + } + return false; + } + + template + bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) { const ColumnDecimal * col_dec = checkAndGetColumn>(col); - static constexpr size_t DECIMAL_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. + static constexpr size_t DECIMAL_HEX_LENGTH = sizeof(T) * 2 + 1; if (col_dec) { - auto col_str = ColumnString::create(); - ColumnString::Chars & out_vec = col_str->getChars(); - ColumnString::Offsets & out_offsets = col_str->getOffsets(); - const typename ColumnDecimal::Container & in_vec = col_dec->getData(); - - size_t size = in_vec.size(); - out_offsets.resize(size); - out_vec.resize(size * DECIMAL_HEX_LENGTH); - - size_t pos = 0; - char * out = reinterpret_cast(&out_vec[0]); - for (size_t i = 0; i < size; ++i) - { - const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); - executeOneString(in_pos, in_pos + sizeof(T), out); - - pos += DECIMAL_HEX_LENGTH; - out_offsets[i] = pos; - } - - col_res = std::move(col_str); + executeFloatAndDecimal::Container>(in_vec, col_res, DECIMAL_HEX_LENGTH); return true; } - else - { - return false; - } + return false; } - - void executeOneString(const UInt8 * pos, const UInt8 * end, char *& out) { while (pos < end) From 0168620fad976666222d39c05b54905f056406d0 Mon Sep 17 00:00:00 2001 From: Mikhail Korotov <55493615+millb@users.noreply.github.com> Date: Wed, 6 Nov 2019 11:17:31 +0300 Subject: [PATCH 21/77] Fixed style commit --- dbms/src/Functions/FunctionsCoding.h | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 2061b64e2c1..6b86e760b62 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1065,7 +1065,8 @@ public: } template - bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) { + bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) + { const ColumnDecimal * col_dec = checkAndGetColumn>(col); static constexpr size_t DECIMAL_HEX_LENGTH = sizeof(T) * 2 + 1; From 695f3c58cc7ba3b7fe3458d9dfb8d80defbfadbd Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 6 Nov 2019 11:29:18 +0300 Subject: [PATCH 22/77] Working Commmit --- dbms/src/Functions/FunctionsCoding.h | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 2061b64e2c1..86a08fbadf3 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1027,6 +1027,8 @@ public: template void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, size_t HEX_LENGTH) { + std::cerr << "!!! "<< HEX_LENGTH << " !!!" << std::endl; + auto col_str = ColumnString::create(); ColumnString::Chars & out_vec = col_str->getChars(); @@ -1057,26 +1059,34 @@ public: static constexpr size_t FLOAT_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. if (col_vec) { + std::cerr << "ENTER in tryExecuteFloat " << sizeof(T) << std::endl; const typename ColumnVector::Container & in_vec = col_vec->getData(); executeFloatAndDecimal::Container>(in_vec, col_res, FLOAT_HEX_LENGTH); return true; } - return false; + else + { + return false; + } } template bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) { const ColumnDecimal * col_dec = checkAndGetColumn>(col); - static constexpr size_t DECIMAL_HEX_LENGTH = sizeof(T) * 2 + 1; + static constexpr size_t DECIMAL_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. if (col_dec) { + std::cerr << "ENTER in tryExecuteDecimal " << sizeof(T) << std::endl; const typename ColumnDecimal::Container & in_vec = col_dec->getData(); executeFloatAndDecimal::Container>(in_vec, col_res, DECIMAL_HEX_LENGTH); return true; } - return false; + else + { + return false; + } } From 043ca0d4700d691368f2d2bfcc4bba14f2414d33 Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 6 Nov 2019 11:42:34 +0300 Subject: [PATCH 23/77] Working commit --- dbms/src/Functions/FunctionsCoding.h | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index b44ebc9aab5..933f12efa28 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1056,12 +1056,11 @@ public: bool tryExecuteFloat(const IColumn * col, ColumnPtr & col_res) { const ColumnVector * col_vec = checkAndGetColumn>(col); - static constexpr size_t FLOAT_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. if (col_vec) { std::cerr << "ENTER in tryExecuteFloat " << sizeof(T) << std::endl; const typename ColumnVector::Container & in_vec = col_vec->getData(); - executeFloatAndDecimal::Container>(in_vec, col_res, FLOAT_HEX_LENGTH); + executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T) * 2 + 1); return true; } else @@ -1075,13 +1074,11 @@ public: { const ColumnDecimal * col_dec = checkAndGetColumn>(col); - static constexpr size_t DECIMAL_HEX_LENGTH = sizeof(T) * 2 + 1; /// Including trailing zero byte. - if (col_dec) { std::cerr << "ENTER in tryExecuteDecimal " << sizeof(T) << std::endl; const typename ColumnDecimal::Container & in_vec = col_dec->getData(); - executeFloatAndDecimal::Container>(in_vec, col_res, DECIMAL_HEX_LENGTH); + executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T) * 2 + 1); return true; } else From 556f6fbdc54c488f1cf8722a5df33930d39d2b28 Mon Sep 17 00:00:00 2001 From: Mikhail Korotov <55493615+millb@users.noreply.github.com> Date: Wed, 6 Nov 2019 11:47:25 +0300 Subject: [PATCH 24/77] Working commit --- dbms/src/Functions/FunctionsCoding.h | 5 ----- 1 file changed, 5 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 933f12efa28..139d86b07b3 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1027,8 +1027,6 @@ public: template void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, size_t HEX_LENGTH) { - std::cerr << "!!! "<< HEX_LENGTH << " !!!" << std::endl; - auto col_str = ColumnString::create(); ColumnString::Chars & out_vec = col_str->getChars(); @@ -1058,7 +1056,6 @@ public: const ColumnVector * col_vec = checkAndGetColumn>(col); if (col_vec) { - std::cerr << "ENTER in tryExecuteFloat " << sizeof(T) << std::endl; const typename ColumnVector::Container & in_vec = col_vec->getData(); executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T) * 2 + 1); return true; @@ -1072,8 +1069,6 @@ public: template bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) { - const ColumnDecimal * col_dec = checkAndGetColumn>(col); - if (col_dec) { std::cerr << "ENTER in tryExecuteDecimal " << sizeof(T) << std::endl; From 281b0062bc943acb967e5a0a61ef3106aac9821d Mon Sep 17 00:00:00 2001 From: Mikhail Korotov <55493615+millb@users.noreply.github.com> Date: Wed, 6 Nov 2019 11:48:51 +0300 Subject: [PATCH 25/77] Working commit --- dbms/src/Functions/FunctionsCoding.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 139d86b07b3..ebb009fc3fe 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1071,7 +1071,6 @@ public: { if (col_dec) { - std::cerr << "ENTER in tryExecuteDecimal " << sizeof(T) << std::endl; const typename ColumnDecimal::Container & in_vec = col_dec->getData(); executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T) * 2 + 1); return true; From 86a9c2b98ef2a2ce0d36a7ba8fc8ccec057875b9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Nov 2019 12:45:27 +0300 Subject: [PATCH 26/77] Try to fix tests. --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 2 ++ 1 file changed, 2 insertions(+) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 832e313f94b..c13f6031e2a 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1639,6 +1639,8 @@ void InterpreterSelectQuery::executeFetchColumns( if (query_info.prewhere_info->remove_columns_actions) pipe.addSimpleTransform(std::make_shared(pipe.getHeader(), query_info.prewhere_info->remove_columns_actions)); } + + pipes.emplace_back(std::move(pipe)); } for (auto & stream : streams) From 76b75dc0e8ce94f7d5082272101da8aed4d6e830 Mon Sep 17 00:00:00 2001 From: sundy-li <543950155@qq.com> Date: Wed, 6 Nov 2019 18:19:08 +0800 Subject: [PATCH 27/77] args to supertype --- dbms/src/Functions/array/range.cpp | 263 +++++++++++++++--- .../functions/array_functions.md | 1 + 2 files changed, 229 insertions(+), 35 deletions(-) diff --git a/dbms/src/Functions/array/range.cpp b/dbms/src/Functions/array/range.cpp index faafbd637b2..799cb233f3c 100644 --- a/dbms/src/Functions/array/range.cpp +++ b/dbms/src/Functions/array/range.cpp @@ -3,8 +3,10 @@ #include #include #include +#include #include #include +#include #include @@ -25,9 +27,11 @@ class FunctionRange : public IFunction public: static constexpr auto name = "range"; static constexpr size_t max_elements = 100'000'000; - static FunctionPtr create(const Context &) { return std::make_shared(); } + static FunctionPtr create(const Context & context_) { return std::make_shared(context_); } + FunctionRange(const Context & context_) : context(context_) {} private: + const Context & context; String getName() const override { return name; } size_t getNumberOfArguments() const override { return 0; } @@ -49,7 +53,9 @@ private: throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } - return std::make_shared(arguments.size() == 3 ? arguments[1] : arguments.back()); + + DataTypePtr common_type = getLeastSupertype(arguments); + return std::make_shared(common_type); } template @@ -97,12 +103,175 @@ private: return false; } - template - bool executeStartEndStep(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) + template + bool executeConstStartStep(Block & block, const IColumn * end_arg, const T start, const T step, const size_t input_rows_count, const size_t result) { - auto start_column = checkAndGetColumn>(start_col); - auto end_column = checkAndGetColumn>(end_col); - auto step_column = checkAndGetColumn>(step_col); + auto end_column = checkAndGetColumn>(end_arg); + if (!end_column) + { + return false; + } + + const auto & end_data = end_column->getData(); + + size_t total_values = 0; + size_t pre_values = 0; + + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) + { + if (start < end_data[row_idx] && step == 0) + throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + pre_values += start >= end_data[row_idx] ? 0 + : (end_data[row_idx] - start - 1) / step + 1; + + if (pre_values < total_values) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + total_values = pre_values; + if (total_values > max_elements) + throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + } + + auto data_col = ColumnVector::create(total_values); + auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size()); + + auto & out_data = data_col->getData(); + auto & out_offsets = offsets_col->getData(); + + IColumn::Offset offset{}; + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) + { + for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step) + out_data[offset++] = st; + + out_offsets[row_idx] = offset; + } + + block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); + return true; + } + + template + bool executeConstStep(Block & block, const IColumn * start_arg, const IColumn * end_arg, const T step, const size_t input_rows_count, const size_t result) + { + auto start_column = checkAndGetColumn>(start_arg); + auto end_column = checkAndGetColumn>(end_arg); + if (!end_column || !start_column) + { + return false; + } + + const auto & start_data = start_column->getData(); + const auto & end_data = end_column->getData(); + + size_t total_values = 0; + size_t pre_values = 0; + + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) + { + if (start_data[row_idx] < end_data[row_idx] && step == 0) + throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + pre_values += start_data[row_idx] >= end_data[row_idx] ? 0 + : (end_data[row_idx] - start_data[row_idx] - 1) / step + 1; + + if (pre_values < total_values) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + total_values = pre_values; + if (total_values > max_elements) + throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + } + + auto data_col = ColumnVector::create(total_values); + auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size()); + + auto & out_data = data_col->getData(); + auto & out_offsets = offsets_col->getData(); + + IColumn::Offset offset{}; + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) + { + for (size_t st = start_data[row_idx], ed = end_data[row_idx]; st < ed; st += step) + out_data[offset++] = st; + + out_offsets[row_idx] = offset; + } + + block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); + return true; + } + + template + bool executeConstStart(Block & block, const IColumn * end_arg, const IColumn * step_arg, const T start, const size_t input_rows_count, const size_t result) + { + auto end_column = checkAndGetColumn>(end_arg); + auto step_column = checkAndGetColumn>(step_arg); + if (!end_column || !step_column) + { + return false; + } + + const auto & end_data = end_column->getData(); + const auto & step_data = step_column->getData(); + + size_t total_values = 0; + size_t pre_values = 0; + + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) + { + if (start < end_data[row_idx] && step_data[row_idx] == 0) + throw Exception{"A call to function " + getName() + " overflows, the 3rd argument step can't be zero", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + pre_values += start >= end_data[row_idx] ? 0 + : (end_data[row_idx] - start - 1) / step_data[row_idx] + 1; + + if (pre_values < total_values) + throw Exception{"A call to function " + getName() + " overflows, investigate the values of arguments you are passing", + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + + total_values = pre_values; + if (total_values > max_elements) + throw Exception{"A call to function " + getName() + " would produce " + std::to_string(total_values) + + " array elements, which is greater than the allowed maximum of " + std::to_string(max_elements), + ErrorCodes::ARGUMENT_OUT_OF_BOUND}; + } + + auto data_col = ColumnVector::create(total_values); + auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size()); + + auto & out_data = data_col->getData(); + auto & out_offsets = offsets_col->getData(); + + IColumn::Offset offset{}; + for (size_t row_idx = 0; row_idx < input_rows_count; ++row_idx) + { + for (size_t st = start, ed = end_data[row_idx]; st < ed; st += step_data[row_idx]) + out_data[offset++] = st; + + out_offsets[row_idx] = offset; + } + + block.getByPosition(result).column = ColumnArray::create(std::move(data_col), std::move(offsets_col)); + return true; + } + + template + bool executeGeneric(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) + { + auto start_column = checkAndGetColumn>(start_col); + auto end_column = checkAndGetColumn>(end_col); + auto step_column = checkAndGetColumn>(step_col); if (!start_column || !end_column || !step_column) { @@ -136,7 +305,7 @@ private: ErrorCodes::ARGUMENT_OUT_OF_BOUND}; } - auto data_col = ColumnVector::create(total_values); + auto data_col = ColumnVector::create(total_values); auto offsets_col = ColumnArray::ColumnOffsets::create(end_column->size()); auto & out_data = data_col->getData(); @@ -155,29 +324,8 @@ private: return true; } - template - bool executeStartEnd(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) - { - return executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result) - || executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result) - || executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result) - || executeStartEndStep(block, start_col, end_col, step_col, input_rows_count, result); - } - - template - bool executeStart(Block & block, const IColumn * start_col, const IColumn * end_col, const IColumn * step_col, const size_t input_rows_count, const size_t result) - { - return executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result) - || executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result) - || executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result) - || executeStartEnd(block, start_col, end_col, step_col, input_rows_count, result); - } - void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override { - Columns columns_holder(3); - ColumnRawPtrs columns(3); - if (arguments.size() == 1) { const auto col = block.getByPosition(arguments[0]).column.get(); @@ -191,23 +339,68 @@ private: return; } + Columns columns_holder(3); + ColumnRawPtrs columns(3); + + const auto return_type = checkAndGetDataType(block.getByPosition(result).type.get())->getNestedType(); + for (size_t i = 0; i < arguments.size(); ++i) { - columns_holder[i] = block.getByPosition(arguments[i]).column->convertToFullColumnIfConst(); + if (i == 1) + columns_holder[i] = castColumn(block.getByPosition(arguments[i]), return_type, context)->convertToFullColumnIfConst(); + else + columns_holder[i] = castColumn(block.getByPosition(arguments[i]), return_type, context); + columns[i] = columns_holder[i].get(); } // for step column, defaults to 1 if (arguments.size() == 2) { - columns_holder[2] = DataTypeUInt8().createColumnConst(input_rows_count, 1)->convertToFullColumnIfConst(); + columns_holder[2] = return_type->createColumnConst(input_rows_count, 1); columns[2] = columns_holder[2].get(); } - if (!executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result) && - !executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result) && - !executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result) && - !executeStart(block, columns[0], columns[1], columns[2], input_rows_count, result)) + bool is_start_const = isColumnConst(*columns[0]); + bool is_step_const = isColumnConst(*columns[2]); + bool ok; + if (is_start_const && is_step_const) + { + UInt64 start = assert_cast(*columns[0]).getUInt(0); + UInt64 step = assert_cast(*columns[2]).getUInt(0); + + ok = executeConstStartStep(block, columns[1], start, step, input_rows_count, result) || + executeConstStartStep(block, columns[1], start, step, input_rows_count, result) || + executeConstStartStep(block, columns[1], start, step, input_rows_count, result) || + executeConstStartStep(block, columns[1], start, step, input_rows_count, result); + } + else if (is_start_const && !is_step_const) + { + UInt64 start = assert_cast(*columns[0]).getUInt(0); + + ok = executeConstStart(block, columns[1], columns[2], start, input_rows_count, result) || + executeConstStart(block, columns[1], columns[2], start, input_rows_count, result) || + executeConstStart(block, columns[1], columns[2], start, input_rows_count, result) || + executeConstStart(block, columns[1], columns[2], start, input_rows_count, result); + } + else if (!is_start_const && is_step_const) + { + UInt64 step = assert_cast(*columns[2]).getUInt(0); + + ok = executeConstStep(block, columns[0], columns[1], step, input_rows_count, result) || + executeConstStep(block, columns[0], columns[1], step, input_rows_count, result) || + executeConstStep(block, columns[0], columns[1], step, input_rows_count, result) || + executeConstStep(block, columns[0], columns[1], step, input_rows_count, result); + } + else + { + ok = executeGeneric(block, columns[0], columns[1], columns[2], input_rows_count, result) || + executeGeneric(block, columns[0], columns[1], columns[2], input_rows_count, result) || + executeGeneric(block, columns[0], columns[1], columns[2], input_rows_count, result) || + executeGeneric(block, columns[0], columns[1], columns[2], input_rows_count, result); + } + + if (!ok) { throw Exception{"Illegal columns " + columns[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN}; } diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 7081cf2ecba..da96c518579 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -35,6 +35,7 @@ Accepts zero arguments and returns an empty array of the appropriate type. Accepts an empty array and returns a one-element array that is equal to the default value. ## range(end), range(start, end [, step]) + Returns an array of numbers from start to end-1 by step. If the argument `start` is not specified, defaults to 0. If the argument `step` is not specified, defaults to 1. From 68a82f78b4f7c3f069c5849e8309dc1149e094a6 Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Wed, 6 Nov 2019 19:46:37 +0900 Subject: [PATCH 28/77] Implemented javaHashUTF16LE() --- dbms/src/Functions/FunctionsHashing.cpp | 1 + dbms/src/Functions/FunctionsHashing.h | 38 +++++++++++++++++++ .../00800_function_java_hash.reference | 3 ++ .../0_stateless/00800_function_java_hash.sql | 3 ++ 4 files changed, 45 insertions(+) diff --git a/dbms/src/Functions/FunctionsHashing.cpp b/dbms/src/Functions/FunctionsHashing.cpp index 90b1057ce63..4a98a7993b4 100644 --- a/dbms/src/Functions/FunctionsHashing.cpp +++ b/dbms/src/Functions/FunctionsHashing.cpp @@ -23,6 +23,7 @@ void registerFunctionsHashing(FunctionFactory & factory) factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); + factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); factory.registerFunction(); diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 1b98191e224..2290e3b2050 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -353,6 +353,43 @@ struct JavaHashImpl static constexpr bool use_int_hash_for_pods = false; }; +struct JavaHashUTF16LEImpl +{ + static constexpr auto name = "javaHashUTF16LE"; + using ReturnType = Int32; + + static Int32 apply(const char * raw_data, const size_t raw_size) + { + char * data = const_cast(raw_data); + size_t size = raw_size; + + // Remove Byte-order-mark(0xFFFE) for UTF-16LE + if (size >= 2 && data[0] == -1 && data[1] == -2) + { + data += 2; + size -= 2; + } + + if (size % 2 != 0) + throw Exception("Arguments for javaHashUTF16LE must be in the form of UTF-16", ErrorCodes::LOGICAL_ERROR); + + UInt32 h = 0; + for (size_t i = 0; i < size; i += 2) + h = 31 * h + static_cast((data[i] & 0xFF) << HI_BYTE_SHIFT | (data[i+1] & 0xFF) << LO_BYTE_SHIFT); + + return static_cast(h); + } + + static Int32 combineHashes(Int32, Int32) + { + throw Exception("Java hash is not combineable for multiple arguments", ErrorCodes::NOT_IMPLEMENTED); + } + + static constexpr bool use_int_hash_for_pods = false; + static constexpr int HI_BYTE_SHIFT = 0; + static constexpr int LO_BYTE_SHIFT = 8; +}; + /// This is just JavaHash with zeroed out sign bit. /// This function is used in Hive for versions before 3.0, /// after 3.0, Hive uses murmur-hash3. @@ -1102,6 +1139,7 @@ using FunctionMurmurHash3_32 = FunctionAnyHash; using FunctionMurmurHash3_64 = FunctionAnyHash; using FunctionMurmurHash3_128 = FunctionStringHashFixedString; using FunctionJavaHash = FunctionAnyHash; +using FunctionJavaHashUTF16LE = FunctionAnyHash; using FunctionHiveHash = FunctionAnyHash; #if USE_XXHASH diff --git a/dbms/tests/queries/0_stateless/00800_function_java_hash.reference b/dbms/tests/queries/0_stateless/00800_function_java_hash.reference index 7f9c68298bd..6efefd41459 100644 --- a/dbms/tests/queries/0_stateless/00800_function_java_hash.reference +++ b/dbms/tests/queries/0_stateless/00800_function_java_hash.reference @@ -1,4 +1,7 @@ 96354 -676697544 +138768 +-2143570108 +2145564783 96354 1470786104 diff --git a/dbms/tests/queries/0_stateless/00800_function_java_hash.sql b/dbms/tests/queries/0_stateless/00800_function_java_hash.sql index c69cd412f57..2010b8d8311 100644 --- a/dbms/tests/queries/0_stateless/00800_function_java_hash.sql +++ b/dbms/tests/queries/0_stateless/00800_function_java_hash.sql @@ -1,4 +1,7 @@ select javaHash('abc'); select javaHash('874293087'); +select javaHashUTF16LE(convertCharset('a1가', 'utf-8', 'utf-16le')); +select javaHashUTF16LE(convertCharset('가나다라마바사아자차카타파하', 'utf-8', 'utf-16le')); +select javaHashUTF16LE(convertCharset('FJKLDSJFIOLD_389159837589429', 'utf-8', 'utf-16le')); select hiveHash('abc'); select hiveHash('874293087'); From 83c63171ade20919eff08054d3ece82b97135230 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Nov 2019 14:17:12 +0300 Subject: [PATCH 29/77] Added another mode for function "trap" #7601 --- dbms/src/Functions/trap.cpp | 16 +++++++++++++--- 1 file changed, 13 insertions(+), 3 deletions(-) diff --git a/dbms/src/Functions/trap.cpp b/dbms/src/Functions/trap.cpp index e05d5efa4f7..8beb901231e 100644 --- a/dbms/src/Functions/trap.cpp +++ b/dbms/src/Functions/trap.cpp @@ -1,4 +1,4 @@ -#if 0 +#if 1 #include #include @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -27,13 +28,18 @@ namespace ErrorCodes /// Various illegal actions to test diagnostic features of ClickHouse itself. Should not be enabled in production builds. class FunctionTrap : public IFunction { +private: + const Context & context; + public: static constexpr auto name = "trap"; - static FunctionPtr create(const Context &) + static FunctionPtr create(const Context & context) { - return std::make_shared(); + return std::make_shared(context); } + FunctionTrap(const Context & context_) : context(context_) {} + String getName() const override { return name; @@ -114,6 +120,10 @@ public: t1.join(); t2.join(); } + else if (mode == "access context") + { + (void)context.getCurrentQueryId(); + } else throw Exception("Unknown trap mode", ErrorCodes::BAD_ARGUMENTS); } From a120fba929ddf80922ca088d80dbd12e3aac205b Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Wed, 6 Nov 2019 14:20:31 +0300 Subject: [PATCH 30/77] Update FunctionsHashing.h --- dbms/src/Functions/FunctionsHashing.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 2290e3b2050..1882eb89d1c 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -375,7 +375,7 @@ struct JavaHashUTF16LEImpl UInt32 h = 0; for (size_t i = 0; i < size; i += 2) - h = 31 * h + static_cast((data[i] & 0xFF) << HI_BYTE_SHIFT | (data[i+1] & 0xFF) << LO_BYTE_SHIFT); + h = 31 * h + static_cast((data[i] & 0xFF) << HI_BYTE_SHIFT | (data[i + 1] & 0xFF) << LO_BYTE_SHIFT); return static_cast(h); } From a34023192d11325731c046cf851250a730a3246f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Nov 2019 14:36:33 +0300 Subject: [PATCH 31/77] Fix limits for pipe. --- dbms/src/Processors/Pipe.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Processors/Pipe.cpp b/dbms/src/Processors/Pipe.cpp index 45bd1c96a7a..b31cfd58848 100644 --- a/dbms/src/Processors/Pipe.cpp +++ b/dbms/src/Processors/Pipe.cpp @@ -92,7 +92,7 @@ void Pipe::setLimits(const ISourceWithProgress::LocalLimits & limits) { for (auto & processor : processors) { - if (auto * source_with_progress = typeid_cast(processor.get())) + if (auto * source_with_progress = dynamic_cast(processor.get())) source_with_progress->setLimits(limits); } } @@ -101,7 +101,7 @@ void Pipe::setQuota(QuotaForIntervals & quota) { for (auto & processor : processors) { - if (auto * source_with_progress = typeid_cast(processor.get())) + if (auto * source_with_progress = dynamic_cast(processor.get())) source_with_progress->setQuota(quota); } } @@ -110,7 +110,7 @@ void Pipe::pinSources(size_t executor_number) { for (auto & processor : processors) { - if (auto * source = typeid_cast(processor.get())) + if (auto * source = dynamic_cast(processor.get())) source->setStream(executor_number); } } From 4f6197c370f5316dbcade70534fcefc3d56f8183 Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 6 Nov 2019 14:48:17 +0300 Subject: [PATCH 32/77] Working commit --- dbms/src/Functions/FunctionsCoding.h | 1 + 1 file changed, 1 insertion(+) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index ebb009fc3fe..5371f724b74 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1069,6 +1069,7 @@ public: template bool tryExecuteDecimal(const IColumn * col, ColumnPtr & col_res) { + const ColumnDecimal * col_dec = checkAndGetColumn>(col); if (col_dec) { const typename ColumnDecimal::Container & in_vec = col_dec->getData(); From 81aeff2d2aacfcc81d4a1c4d13b596d8f6102c51 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Thu, 19 Sep 2019 00:17:00 +0300 Subject: [PATCH 33/77] Add _shard_num virtual column for the Distributed engine With JOIN from system.clusters one can figure out from which server data came. TODO: - optimization to avoid communicating with unrelated shards (for queries like "AND _shard_num = n") - fix aliases (see tests with serverError expected) v0: AddingConstColumnBlockInputStream v2: VirtualColumnUtils::rewriteEntityInAst v3: fix remote(Distributed) by appending _shard_num only if has been requested --- .../ClusterProxy/SelectStreamFactory.cpp | 32 +++++++-- .../ClusterProxy/SelectStreamFactory.h | 3 + dbms/src/Storages/StorageDistributed.cpp | 15 ++-- .../01018_Distributed__shard_num.reference | 36 ++++++++++ .../01018_Distributed__shard_num.sql | 72 +++++++++++++++++++ .../operations/table_engines/distributed.md | 11 +++ 6 files changed, 159 insertions(+), 10 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01018_Distributed__shard_num.reference create mode 100644 dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 3c141b56152..03acf6d9a7c 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -34,12 +35,14 @@ SelectStreamFactory::SelectStreamFactory( QueryProcessingStage::Enum processed_stage_, QualifiedTableName main_table_, const Scalars & scalars_, + bool has_virtual_shard_num_column_, const Tables & external_tables_) : header(header_), processed_stage{processed_stage_}, main_table(std::move(main_table_)), table_func_ptr{nullptr}, scalars{scalars_}, + has_virtual_shard_num_column(has_virtual_shard_num_column_), external_tables{external_tables_} { } @@ -49,11 +52,13 @@ SelectStreamFactory::SelectStreamFactory( QueryProcessingStage::Enum processed_stage_, ASTPtr table_func_ptr_, const Scalars & scalars_, + bool has_virtual_shard_num_column_, const Tables & external_tables_) : header(header_), processed_stage{processed_stage_}, table_func_ptr{table_func_ptr_}, scalars{scalars_}, + has_virtual_shard_num_column(has_virtual_shard_num_column_), external_tables{external_tables_} { } @@ -81,23 +86,38 @@ BlockInputStreamPtr createLocalStream(const ASTPtr & query_ast, const Context & return stream; } +static String formattedAST(const ASTPtr & ast) +{ + if (!ast) + return ""; + std::stringstream ss; + formatAST(*ast, ss, false, true); + return ss.str(); +} + } void SelectStreamFactory::createForShard( const Cluster::ShardInfo & shard_info, - const String & query, const ASTPtr & query_ast, + const String &, const ASTPtr & query_ast, const Context & context, const ThrottlerPtr & throttler, BlockInputStreams & res) { + auto modified_query_ast = query_ast->clone(); + if (has_virtual_shard_num_column) + VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, "_shard_num", shard_info.shard_num); + auto emplace_local_stream = [&]() { - res.emplace_back(createLocalStream(query_ast, context, processed_stage)); + res.emplace_back(createLocalStream(modified_query_ast, context, processed_stage)); }; + String modified_query = formattedAST(modified_query_ast); + auto emplace_remote_stream = [&]() { auto stream = std::make_shared( - shard_info.pool, query, header, context, nullptr, throttler, scalars, external_tables, processed_stage); + shard_info.pool, modified_query, header, context, nullptr, throttler, scalars, external_tables, processed_stage); stream->setPoolMode(PoolMode::GET_MANY); if (!table_func_ptr) stream->setMainTable(main_table); @@ -194,7 +214,7 @@ void SelectStreamFactory::createForShard( /// Do it lazily to avoid connecting in the main thread. auto lazily_create_stream = [ - pool = shard_info.pool, shard_num = shard_info.shard_num, query, header = header, query_ast, context, throttler, + pool = shard_info.pool, shard_num = shard_info.shard_num, modified_query, header = header, modified_query_ast, context, throttler, main_table = main_table, table_func_ptr = table_func_ptr, scalars = scalars, external_tables = external_tables, stage = processed_stage, local_delay]() -> BlockInputStreamPtr @@ -229,7 +249,7 @@ void SelectStreamFactory::createForShard( } if (try_results.empty() || local_delay < max_remote_delay) - return createLocalStream(query_ast, context, stage); + return createLocalStream(modified_query_ast, context, stage); else { std::vector connections; @@ -238,7 +258,7 @@ void SelectStreamFactory::createForShard( connections.emplace_back(std::move(try_result.entry)); return std::make_shared( - std::move(connections), query, header, context, nullptr, throttler, scalars, external_tables, stage); + std::move(connections), modified_query, header, context, nullptr, throttler, scalars, external_tables, stage); } }; diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h index c3a55f5348f..e2fbf422246 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -19,6 +19,7 @@ public: QueryProcessingStage::Enum processed_stage_, QualifiedTableName main_table_, const Scalars & scalars_, + bool has_virtual_shard_num_column_, const Tables & external_tables); /// TableFunction in a query. @@ -27,6 +28,7 @@ public: QueryProcessingStage::Enum processed_stage_, ASTPtr table_func_ptr_, const Scalars & scalars_, + bool has_virtual_shard_num_column_, const Tables & external_tables_); void createForShard( @@ -41,6 +43,7 @@ private: QualifiedTableName main_table; ASTPtr table_func_ptr; Scalars scalars; + bool has_virtual_shard_num_column = false; Tables external_tables; }; diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index dbcfcc57d75..152712a72f4 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -216,7 +216,10 @@ StorageDistributed::StorageDistributed( const ASTPtr & sharding_key_, const String & data_path_, bool attach_) - : table_name(table_name_), database_name(database_name_), + : IStorage(ColumnsDescription({ + {"_shard_num", std::make_shared()}, + }, true)), + table_name(table_name_), database_name(database_name_), remote_database(remote_database_), remote_table(remote_table_), global_context(context_), cluster_name(global_context.getMacros()->expand(cluster_name_)), has_sharding_key(sharding_key_), path(data_path_.empty() ? "" : (data_path_ + escapeForFileName(table_name) + '/')) @@ -305,7 +308,7 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(const Con } BlockInputStreams StorageDistributed::read( - const Names & /*column_names*/, + const Names & column_names, const SelectQueryInfo & query_info, const Context & context, QueryProcessingStage::Enum processed_stage, @@ -324,11 +327,15 @@ BlockInputStreams StorageDistributed::read( const Scalars & scalars = context.hasQueryContext() ? context.getQueryContext().getScalars() : Scalars{}; + bool has_virtual_shard_num_column = std::find(column_names.begin(), column_names.end(), "_shard_num") != column_names.end(); + if (has_virtual_shard_num_column && !isVirtualColumn("_shard_num")) + has_virtual_shard_num_column = false; + ClusterProxy::SelectStreamFactory select_stream_factory = remote_table_function_ptr ? ClusterProxy::SelectStreamFactory( - header, processed_stage, remote_table_function_ptr, scalars, context.getExternalTables()) + header, processed_stage, remote_table_function_ptr, scalars, has_virtual_shard_num_column, context.getExternalTables()) : ClusterProxy::SelectStreamFactory( - header, processed_stage, QualifiedTableName{remote_database, remote_table}, scalars, context.getExternalTables()); + header, processed_stage, QualifiedTableName{remote_database, remote_table}, scalars, has_virtual_shard_num_column, context.getExternalTables()); if (settings.optimize_skip_unused_shards) { diff --git a/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.reference b/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.reference new file mode 100644 index 00000000000..679383f969e --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.reference @@ -0,0 +1,36 @@ +remote(system.one) +0 +0 +0 +1 0 +1 0 +2 0 +1 0 +dist_1 +1 +1 10 +10 +1 +1 +1 10 +1 20 +10 +20 +dist_2 +1 +2 +1 100 +2 100 +100 +100 +remote(Distributed) +1 100 +1 100 +JOIN system.clusters +1 10 localhost ::1 9000 +1 20 localhost ::1 9000 +1 10 localhost ::1 9000 +1 20 localhost ::1 9000 +dist_3 +100 foo +foo 100 foo diff --git a/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql b/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql new file mode 100644 index 00000000000..3c4a0c409c4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql @@ -0,0 +1,72 @@ +-- make the order static +SET max_threads = 1; + +-- remote(system.one) +SELECT 'remote(system.one)'; +SELECT * FROM remote('127.0.0.1', system.one); +SELECT * FROM remote('127.0.0.{1,2}', system.one); +SELECT _shard_num, * FROM remote('127.0.0.1', system.one); +SELECT _shard_num, * FROM remote('127.0.0.{1,2}', system.one); +SELECT _shard_num, * FROM remote('127.0.0.{1,2}', system.one) WHERE _shard_num = 1; + +-- dist_1 using test_shard_localhost +SELECT 'dist_1'; +CREATE TABLE mem1 (key Int) Engine=Memory(); +CREATE TABLE dist_1 AS mem1 Engine=Distributed(test_shard_localhost, currentDatabase(), mem1); +SELECT _shard_num FROM dist_1; + +INSERT INTO mem1 VALUES (10); +SELECT _shard_num FROM dist_1; +SELECT _shard_num, key FROM dist_1; +SELECT key FROM dist_1; + +INSERT INTO dist_1 VALUES (20); +SELECT _shard_num FROM dist_1; +SELECT _shard_num, key FROM dist_1; +SELECT key FROM dist_1; + +-- dist_2 using test_cluster_two_shards_localhost +SELECT 'dist_2'; +CREATE TABLE mem2 (key Int) Engine=Memory(); +CREATE TABLE dist_2 AS mem2 Engine=Distributed(test_cluster_two_shards_localhost, currentDatabase(), mem2); +SELECT _shard_num FROM dist_2; + +INSERT INTO mem2 VALUES (100); +SELECT _shard_num FROM dist_2; +SELECT _shard_num, key FROM dist_2; +SELECT key FROM dist_2; + +-- multiple _shard_num +SELECT 'remote(Distributed)'; +SELECT _shard_num, key FROM remote('127.0.0.1', currentDatabase(), dist_2); + +-- JOIN system.clusters +SELECT 'JOIN system.clusters'; + +SELECT a._shard_num, a.key, b.host_name, b.host_address, b.port +FROM (SELECT *, _shard_num FROM dist_1) a +JOIN system.clusters b +ON a._shard_num = b.shard_num +WHERE b.cluster = 'test_cluster_two_shards_localhost'; +-- Requires toUInt32() otherwise Type mismatch of columns (53) +SELECT _shard_num, key, b.host_name, b.host_address, b.port +FROM dist_1 a +JOIN system.clusters b +ON toUInt32(_shard_num) = b.shard_num +WHERE b.cluster = 'test_cluster_two_shards_localhost'; + +-- rewrite does not work with aliases, hence Missing columns (47) +SELECT a._shard_num, key FROM dist_1 a; -- { serverError 47; } +-- the same with JOIN, just in case +SELECT a._shard_num, a.key, b.host_name, b.host_address, b.port +FROM dist_1 a +JOIN system.clusters b +ON a._shard_num = b.shard_num +WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- { serverError 47; } + +SELECT 'dist_3'; +CREATE TABLE mem3 (key Int, _shard_num String) Engine=Memory(); +CREATE TABLE dist_3 AS mem3 Engine=Distributed(test_shard_localhost, currentDatabase(), mem3); +INSERT INTO mem3 VALUES (100, 'foo'); +SELECT * FROM dist_3; +SELECT _shard_num, * FROM dist_3; diff --git a/docs/en/operations/table_engines/distributed.md b/docs/en/operations/table_engines/distributed.md index 67b6697d01f..38d085da568 100644 --- a/docs/en/operations/table_engines/distributed.md +++ b/docs/en/operations/table_engines/distributed.md @@ -121,5 +121,16 @@ If the server ceased to exist or had a rough restart (for example, after a devic When the max_parallel_replicas option is enabled, query processing is parallelized across all replicas within a single shard. For more information, see the section [max_parallel_replicas](../settings/settings.md#settings-max_parallel_replicas). +## Virtual Columns + +- `_shard_num` — Contains the `shard_num` (from `system.clusters`). Type: [UInt32](../../data_types/int_uint.md). + +!!! note "Note" + Since [`remote`](../../query_language/table_functions/remote.md)/`cluster` table functions internally create temporary instance of the same Distributed engine, `_shard_num` is available there too. + +**See Also** + +- [Virtual columns](index.md#table_engines-virtual_columns) + [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/distributed/) From ec8f16085928c0e3c2c742df1975ab08c9f56b69 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Nov 2019 15:26:51 +0300 Subject: [PATCH 34/77] Wailt for AsynchronousBlockInputStream inner thread before getting totals to avoid possible race. --- dbms/src/DataStreams/AsynchronousBlockInputStream.h | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/dbms/src/DataStreams/AsynchronousBlockInputStream.h b/dbms/src/DataStreams/AsynchronousBlockInputStream.h index 93c695f20c9..b05948faaf6 100644 --- a/dbms/src/DataStreams/AsynchronousBlockInputStream.h +++ b/dbms/src/DataStreams/AsynchronousBlockInputStream.h @@ -33,6 +33,12 @@ public: String getName() const override { return "Asynchronous"; } + void waitInnerThread() + { + if (started) + pool.wait(); + } + void readPrefix() override { /// Do not call `readPrefix` on the child, so that the corresponding actions are performed in a separate thread. From 3bde6636815ab113376f67cd940817ed3e1e0f61 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Nov 2019 15:34:51 +0300 Subject: [PATCH 35/77] Wailt for AsynchronousBlockInputStream inner thread before getting totals to avoid possible race. --- dbms/programs/server/TCPHandler.cpp | 3 +++ 1 file changed, 3 insertions(+) diff --git a/dbms/programs/server/TCPHandler.cpp b/dbms/programs/server/TCPHandler.cpp index cb1512a81e8..6d50a99cd93 100644 --- a/dbms/programs/server/TCPHandler.cpp +++ b/dbms/programs/server/TCPHandler.cpp @@ -523,6 +523,9 @@ void TCPHandler::processOrdinaryQuery() */ if (!block && !isQueryCancelled()) { + /// Wait till inner thread finish to avoid possible race with getTotals. + async_in.waitInnerThread(); + sendTotals(state.io.in->getTotals()); sendExtremes(state.io.in->getExtremes()); sendProfileInfo(state.io.in->getProfileInfo()); From 3f0476b52295b4660fd4e094a1375af5be5ca1f2 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 6 Nov 2019 15:36:56 +0300 Subject: [PATCH 36/77] Disabled "trap" function back --- dbms/src/Functions/trap.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/trap.cpp b/dbms/src/Functions/trap.cpp index 8beb901231e..14cd6075a88 100644 --- a/dbms/src/Functions/trap.cpp +++ b/dbms/src/Functions/trap.cpp @@ -1,4 +1,4 @@ -#if 1 +#if 0 #include #include From 7863b7f3c9c2809fc16c36898829acdea2ce8470 Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 6 Nov 2019 16:08:02 +0300 Subject: [PATCH 37/77] Working commit --- dbms/src/Functions/FunctionsCoding.h | 25 +++++++++++++++++++++++-- 1 file changed, 23 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 5371f724b74..7054bb1c49b 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1025,7 +1025,7 @@ public: } template - void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, size_t HEX_LENGTH) + void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, const size_t HEX_LENGTH) { auto col_str = ColumnString::create(); @@ -1057,7 +1057,28 @@ public: if (col_vec) { const typename ColumnVector::Container & in_vec = col_vec->getData(); - executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T) * 2 + 1); + const size_t HEX_LENGTH = sizeof(T) * 2 + 1; + auto col_str = ColumnString::create(); + + ColumnString::Chars & out_vec = col_str->getChars(); + ColumnString::Offsets & out_offsets = col_str->getOffsets(); + + size_t size = in_vec.size(); + out_offsets.resize(size); + out_vec.resize(size * HEX_LENGTH); + + size_t pos = 0; + char * out = reinterpret_cast(&out_vec[0]); + for (size_t i = 0; i < size; ++i) + { + const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); + executeOneString(in_pos, in_pos + sizeof(T), out); + + pos += HEX_LENGTH; + out_offsets[i] = pos; + } + + col_res = std::move(col_str); return true; } else From 300062e91aaa9a939f64a2085a28b2493855943e Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Wed, 6 Nov 2019 23:17:38 +0900 Subject: [PATCH 38/77] Fixed type conversions in javaHashUTF16LE --- dbms/src/Functions/FunctionsHashing.h | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 1882eb89d1c..38774ef96d4 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -364,7 +364,7 @@ struct JavaHashUTF16LEImpl size_t size = raw_size; // Remove Byte-order-mark(0xFFFE) for UTF-16LE - if (size >= 2 && data[0] == -1 && data[1] == -2) + if (size >= 2 && data[0] == '\xFF' && data[1] == '\xFE') { data += 2; size -= 2; @@ -373,11 +373,11 @@ struct JavaHashUTF16LEImpl if (size % 2 != 0) throw Exception("Arguments for javaHashUTF16LE must be in the form of UTF-16", ErrorCodes::LOGICAL_ERROR); - UInt32 h = 0; + Int32 h = 0; for (size_t i = 0; i < size; i += 2) - h = 31 * h + static_cast((data[i] & 0xFF) << HI_BYTE_SHIFT | (data[i + 1] & 0xFF) << LO_BYTE_SHIFT); + h = 31 * h + static_cast(static_cast(data[i]) | static_cast(data[i + 1]) << 8); - return static_cast(h); + return h; } static Int32 combineHashes(Int32, Int32) @@ -386,8 +386,6 @@ struct JavaHashUTF16LEImpl } static constexpr bool use_int_hash_for_pods = false; - static constexpr int HI_BYTE_SHIFT = 0; - static constexpr int LO_BYTE_SHIFT = 8; }; /// This is just JavaHash with zeroed out sign bit. From 7d46d359ea2f4ef2578c3a544e24b21e925f8b33 Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 6 Nov 2019 17:48:23 +0300 Subject: [PATCH 39/77] Working commit --- dbms/src/Functions/FunctionsCoding.h | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 7054bb1c49b..8c954beb51a 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1027,27 +1027,34 @@ public: template void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, const size_t HEX_LENGTH) { + std::cerr << "\n\n1!!!\n\n" << std::endl; auto col_str = ColumnString::create(); - + std::cerr << "\n\n2!!!\n\n" << std::endl; ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); - + std::cerr << "\n\n3!!!\n\n" << std::endl; size_t size = in_vec.size(); + std::cerr << "\n\n4!!!\n\n" << std::endl; out_offsets.resize(size); + std::cerr << "\n\n5!!!\n\n" << std::endl; out_vec.resize(size * HEX_LENGTH); - + std::cerr << "\n\n6!!!\n\n" << std::endl; size_t pos = 0; char * out = reinterpret_cast(&out_vec[0]); + std::cerr << "\n\n7!!!\n\n" << std::endl; for (size_t i = 0; i < size; ++i) { + std::cerr << "\n\n??" << i << "\n\n" << std::endl; const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); + std::cerr << "\n\n8!!!\n\n" << std::endl; executeOneString(in_pos, in_pos + sizeof(T), out); - + std::cerr << "\n\n9!!!\n\n" << std::endl; pos += HEX_LENGTH; out_offsets[i] = pos; + std::cerr << "\n\n10!!!\n\n" << std::endl; } - col_res = std::move(col_str); + std::cerr << "\n\n11!!!\n\n" << std::endl; } template @@ -1093,7 +1100,9 @@ public: const ColumnDecimal * col_dec = checkAndGetColumn>(col); if (col_dec) { + std::cerr << "1" << std::endl; const typename ColumnDecimal::Container & in_vec = col_dec->getData(); + std::cerr << "2" << std::endl; executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T) * 2 + 1); return true; } From 16ea323fffc77ae23552ac0ddac667420639eda3 Mon Sep 17 00:00:00 2001 From: "philip.han" Date: Thu, 7 Nov 2019 00:00:59 +0900 Subject: [PATCH 40/77] Removed undefined behavior of signed integer overflow. --- dbms/src/Functions/FunctionsHashing.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Functions/FunctionsHashing.h b/dbms/src/Functions/FunctionsHashing.h index 38774ef96d4..a0d6807b3e5 100644 --- a/dbms/src/Functions/FunctionsHashing.h +++ b/dbms/src/Functions/FunctionsHashing.h @@ -373,11 +373,11 @@ struct JavaHashUTF16LEImpl if (size % 2 != 0) throw Exception("Arguments for javaHashUTF16LE must be in the form of UTF-16", ErrorCodes::LOGICAL_ERROR); - Int32 h = 0; + UInt32 h = 0; for (size_t i = 0; i < size; i += 2) h = 31 * h + static_cast(static_cast(data[i]) | static_cast(data[i + 1]) << 8); - return h; + return static_cast(h); } static Int32 combineHashes(Int32, Int32) From 254c94ebfa7530a5b4419d5fbd257f38a4ebe7de Mon Sep 17 00:00:00 2001 From: millb Date: Wed, 6 Nov 2019 18:06:15 +0300 Subject: [PATCH 41/77] Fixed bugs --- dbms/src/Functions/FunctionsCoding.h | 54 +++++++--------------------- 1 file changed, 12 insertions(+), 42 deletions(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 8c954beb51a..03becb1c6e3 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1025,36 +1025,29 @@ public: } template - void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, const size_t HEX_LENGTH) + void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, const size_t type_size_in_bytes) { - std::cerr << "\n\n1!!!\n\n" << std::endl; - auto col_str = ColumnString::create(); - std::cerr << "\n\n2!!!\n\n" << std::endl; + const size_t hex_length = type_size_in_bytes * 2 + 1; /// Including trailing zero byte. + auto col_str = ColumnString::create();; + ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); - std::cerr << "\n\n3!!!\n\n" << std::endl; + size_t size = in_vec.size(); - std::cerr << "\n\n4!!!\n\n" << std::endl; out_offsets.resize(size); - std::cerr << "\n\n5!!!\n\n" << std::endl; - out_vec.resize(size * HEX_LENGTH); - std::cerr << "\n\n6!!!\n\n" << std::endl; + out_vec.resize(size * hex_length); + size_t pos = 0; char * out = reinterpret_cast(&out_vec[0]); - std::cerr << "\n\n7!!!\n\n" << std::endl; for (size_t i = 0; i < size; ++i) { - std::cerr << "\n\n??" << i << "\n\n" << std::endl; const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); - std::cerr << "\n\n8!!!\n\n" << std::endl; - executeOneString(in_pos, in_pos + sizeof(T), out); - std::cerr << "\n\n9!!!\n\n" << std::endl; - pos += HEX_LENGTH; + executeOneString(in_pos, in_pos + type_size_in_bytes, out); + + pos += hex_length; out_offsets[i] = pos; - std::cerr << "\n\n10!!!\n\n" << std::endl; } col_res = std::move(col_str); - std::cerr << "\n\n11!!!\n\n" << std::endl; } template @@ -1064,28 +1057,7 @@ public: if (col_vec) { const typename ColumnVector::Container & in_vec = col_vec->getData(); - const size_t HEX_LENGTH = sizeof(T) * 2 + 1; - auto col_str = ColumnString::create(); - - ColumnString::Chars & out_vec = col_str->getChars(); - ColumnString::Offsets & out_offsets = col_str->getOffsets(); - - size_t size = in_vec.size(); - out_offsets.resize(size); - out_vec.resize(size * HEX_LENGTH); - - size_t pos = 0; - char * out = reinterpret_cast(&out_vec[0]); - for (size_t i = 0; i < size; ++i) - { - const UInt8 * in_pos = reinterpret_cast(&in_vec[i]); - executeOneString(in_pos, in_pos + sizeof(T), out); - - pos += HEX_LENGTH; - out_offsets[i] = pos; - } - - col_res = std::move(col_str); + executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T)); return true; } else @@ -1100,10 +1072,8 @@ public: const ColumnDecimal * col_dec = checkAndGetColumn>(col); if (col_dec) { - std::cerr << "1" << std::endl; const typename ColumnDecimal::Container & in_vec = col_dec->getData(); - std::cerr << "2" << std::endl; - executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T) * 2 + 1); + executeFloatAndDecimal::Container>(in_vec, col_res, sizeof(T)); return true; } else From ed7971ea254d5da1a8a349fd1721b85e02f65db6 Mon Sep 17 00:00:00 2001 From: Mikhail Korotov Date: Wed, 6 Nov 2019 18:15:24 +0300 Subject: [PATCH 42/77] Fixed small bug --- dbms/src/Functions/FunctionsCoding.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index 03becb1c6e3..78c0dd84fa6 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1028,7 +1028,7 @@ public: void executeFloatAndDecimal(const T & in_vec, ColumnPtr & col_res, const size_t type_size_in_bytes) { const size_t hex_length = type_size_in_bytes * 2 + 1; /// Including trailing zero byte. - auto col_str = ColumnString::create();; + auto col_str = ColumnString::create(); ColumnString::Chars & out_vec = col_str->getChars(); ColumnString::Offsets & out_offsets = col_str->getOffsets(); From 59ec541c1572468a28fab4b806bf279e59ed75ae Mon Sep 17 00:00:00 2001 From: Mikhail Korotov <55493615+millb@users.noreply.github.com> Date: Wed, 6 Nov 2019 19:02:26 +0300 Subject: [PATCH 43/77] Update encoding_functions.md --- docs/ru/query_language/functions/encoding_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/functions/encoding_functions.md b/docs/ru/query_language/functions/encoding_functions.md index 6d9704d597a..9c7737e5d14 100644 --- a/docs/ru/query_language/functions/encoding_functions.md +++ b/docs/ru/query_language/functions/encoding_functions.md @@ -2,7 +2,7 @@ ## hex -Принимает аргументы типов: `String`, `unsigned integer`, `float`, `decimal`, `Date`, or `DateTime`. Возвращает строку, содержащую шестнадцатеричное представление аргумента. Используются заглавные буквы `A-F`. Не используются префиксы `0x` и суффиксы `h`. Для строк просто все байты кодируются в виде двух шестнадцатеричных цифр. Числа выводятся в big endian ("человеческом") формате. Для чисел вырезаются старшие нули, но только по целым байтам. Например, `hex(1) = '01'`. `Date` кодируется как число дней с начала unix-эпохи. `DateTime` кодируются как число секунд с начала unix-эпохи. `float` и `decimal` кодируются как их шетснадцатеричное представление в памяти. +Принимает аргументы типов: `String`, `unsigned integer`, `float`, `decimal`, `Date`, or `DateTime`. Возвращает строку, содержащую шестнадцатеричное представление аргумента. Используются заглавные буквы `A-F`. Не используются префиксы `0x` и суффиксы `h`. Для строк просто все байты кодируются в виде двух шестнадцатеричных цифр. Числа выводятся в big endian ("человеческом") формате. Для чисел вырезаются старшие нули, но только по целым байтам. Например, `hex(1) = '01'`. `Date` кодируется как число дней с начала unix-эпохи. `DateTime` кодируются как число секунд с начала unix-эпохи. `float` и `decimal` кодируются как их шестнадцатеричное представление в памяти. ## unhex(str) Принимает строку, содержащую произвольное количество шестнадцатеричных цифр, и возвращает строку, содержащую соответствующие байты. Поддерживаются как строчные, так и заглавные буквы A-F. Число шестнадцатеричных цифр не обязано быть чётным. Если оно нечётное - последняя цифра интерпретируется как младшая половинка байта 00-0F. Если строка-аргумент содержит что-либо кроме шестнадцатеричных цифр, то будет возвращён какой-либо implementation-defined результат (не кидается исключение). From 3f8cc216a8fc962a4b2828bed51ba7f5d5bb0b42 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Nov 2019 19:50:19 +0300 Subject: [PATCH 44/77] Update memory limit for external aggregation test. --- dbms/tests/queries/1_stateful/00084_external_aggregation.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/1_stateful/00084_external_aggregation.sql b/dbms/tests/queries/1_stateful/00084_external_aggregation.sql index 1423f026bbb..c429661a953 100644 --- a/dbms/tests/queries/1_stateful/00084_external_aggregation.sql +++ b/dbms/tests/queries/1_stateful/00084_external_aggregation.sql @@ -1,6 +1,6 @@ SET max_bytes_before_external_group_by = 200000000; -SET max_memory_usage = 1000000000; +SET max_memory_usage = 1200000000; SET max_threads = 12; SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u DESC, URL LIMIT 10; From 8c7deaa70d1f863e0618a07eacd4741c4f33f8a1 Mon Sep 17 00:00:00 2001 From: alesapin Date: Wed, 6 Nov 2019 20:28:03 +0300 Subject: [PATCH 45/77] More safier, but still ugly mutations interpreter --- dbms/src/Functions/FunctionsJSON.h | 2 +- .../Interpreters/InterpreterAlterQuery.cpp | 2 +- .../src/Interpreters/MutationsInterpreter.cpp | 41 ++++++++++++++----- dbms/src/Interpreters/MutationsInterpreter.h | 16 +++++--- .../MergeTree/MergeTreeDataMergerMutator.cpp | 2 +- dbms/src/Storages/StorageMergeTree.cpp | 2 +- ...utations_interpreter_and_context.reference | 2 + ...01031_mutations_interpreter_and_context.sh | 23 +++++++++++ 8 files changed, 69 insertions(+), 21 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.reference create mode 100755 dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.sh diff --git a/dbms/src/Functions/FunctionsJSON.h b/dbms/src/Functions/FunctionsJSON.h index 08f42dd2b05..85088bed61c 100644 --- a/dbms/src/Functions/FunctionsJSON.h +++ b/dbms/src/Functions/FunctionsJSON.h @@ -65,7 +65,7 @@ public: { /// Choose JSONParser. #if USE_SIMDJSON - if (context.getSettings().allow_simdjson && Cpu::CpuFlagsCache::have_SSE42 && Cpu::CpuFlagsCache::have_PCLMUL) + if (context.getSettingsRef().allow_simdjson && Cpu::CpuFlagsCache::have_SSE42 && Cpu::CpuFlagsCache::have_PCLMUL) { Executor::run(block, arguments, result_pos, input_rows_count); return; diff --git a/dbms/src/Interpreters/InterpreterAlterQuery.cpp b/dbms/src/Interpreters/InterpreterAlterQuery.cpp index 5798e182b69..94d27a7157b 100644 --- a/dbms/src/Interpreters/InterpreterAlterQuery.cpp +++ b/dbms/src/Interpreters/InterpreterAlterQuery.cpp @@ -75,7 +75,7 @@ BlockIO InterpreterAlterQuery::execute() if (!mutation_commands.empty()) { auto table_lock_holder = table->lockStructureForShare(false /* because mutation is executed asyncronously */, context.getCurrentQueryId()); - MutationsInterpreter(table, mutation_commands, context).validate(table_lock_holder); + MutationsInterpreter(table, mutation_commands, context, false).validate(table_lock_holder); table->mutate(mutation_commands, context); } diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 2641ab2a5c4..9c6982d6f22 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -91,6 +91,21 @@ std::optional findFirstNonDeterministicFuncName(const MutationCommand & } }; +MutationsInterpreter::MutationsInterpreter( + StoragePtr storage_, + std::vector commands_, + const Context & context_, + bool can_execute_) + : storage(std::move(storage_)) + , commands(std::move(commands_)) + , context(context_) + , can_execute(can_execute_) +{ + mutation_ast = prepare(!can_execute); + auto limits = SelectQueryOptions().analyze(!can_execute).ignoreLimits(); + select_interpreter = std::make_unique(mutation_ast, context, storage, limits); +} + bool MutationsInterpreter::isStorageTouchedByMutations() const { @@ -103,12 +118,17 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const return true; } - auto context_copy = context; + Context context_copy = context; context_copy.getSettingsRef().merge_tree_uniform_read_distribution = 0; context_copy.getSettingsRef().max_threads = 1; - const ASTPtr & select_query = prepareQueryAffectedAST(); - BlockInputStreamPtr in = InterpreterSelectQuery(select_query, context_copy, storage, SelectQueryOptions().ignoreLimits()).execute().in; + ASTPtr select_query = prepareQueryAffectedAST(); + + /// Interpreter must be alive, when we use result of execute() method. + /// For some reason it may copy context and and give it into ExpressionBlockInputStream + /// after that we will use context from destroyed stack frame in our stream. + InterpreterSelectQuery interpreter(select_query, context_copy, storage, SelectQueryOptions().ignoreLimits()); + BlockInputStreamPtr in = interpreter.execute().in; Block block = in->read(); if (!block.rows()) @@ -520,19 +540,18 @@ void MutationsInterpreter::validate(TableStructureReadLockHolder &) } } - const auto & select_query = prepare(/* dry_run = */ true); - InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ true).ignoreLimits()}; /// Do not use getSampleBlock in order to check the whole pipeline. - Block first_stage_header = interpreter.execute().in->getHeader(); + Block first_stage_header = select_interpreter->execute().in->getHeader(); BlockInputStreamPtr in = std::make_shared(first_stage_header); addStreamsForLaterStages(stages, in)->getHeader(); } BlockInputStreamPtr MutationsInterpreter::execute(TableStructureReadLockHolder &) { - const auto & select_query = prepare(/* dry_run = */ false); - InterpreterSelectQuery interpreter{select_query, context, storage, SelectQueryOptions().analyze(/* dry_run = */ false).ignoreLimits()}; - BlockInputStreamPtr in = interpreter.execute().in; + if (!can_execute) + throw Exception("Cannot execute mutations interpreter because can_execute flag set to false", ErrorCodes::LOGICAL_ERROR); + + BlockInputStreamPtr in = select_interpreter->execute().in; auto result_stream = addStreamsForLaterStages(stages, in); if (!updated_header) updated_header = std::make_unique(result_stream->getHeader()); @@ -581,9 +600,9 @@ size_t MutationsInterpreter::evaluateCommandsSize() { for (const MutationCommand & command : commands) if (unlikely(!command.predicate)) /// The command touches all rows. - return prepare(/* dry_run = */ true)->size(); + return mutation_ast->size(); - return std::max(prepareQueryAffectedAST()->size(), prepare(/* dry_run = */ true)->size()); + return std::max(prepareQueryAffectedAST()->size(), mutation_ast->size()); } } diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index 5df5b1ca2f7..ff9a8ddccc3 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -18,12 +18,9 @@ class Context; class MutationsInterpreter { public: - MutationsInterpreter(StoragePtr storage_, std::vector commands_, const Context & context_) - : storage(std::move(storage_)) - , commands(std::move(commands_)) - , context(context_) - { - } + /// Storage to mutate, array of mutations commands and context. If you really want to execute mutation + /// use can_execute = true, in other cases (validation, amount of commands) it can be false + MutationsInterpreter(StoragePtr storage_, std::vector commands_, const Context & context_, bool can_execute_); void validate(TableStructureReadLockHolder & table_lock_holder); @@ -50,6 +47,13 @@ private: StoragePtr storage; std::vector commands; const Context & context; + bool can_execute; + + ASTPtr mutation_ast; + + /// We have to store interpreter because it use own copy of context + /// and some streams from execute method may use it. + std::unique_ptr select_interpreter; /// A sequence of mutation commands is executed as a sequence of stages. Each stage consists of several /// filters, followed by updating values of some columns. Commands can reuse expressions calculated by the diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index a4332f7df56..47a796fbb3c 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -943,7 +943,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor command.partition, context_for_reading); }); - MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading); + MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading, true); if (!mutations_interpreter.isStorageTouchedByMutations()) { diff --git a/dbms/src/Storages/StorageMergeTree.cpp b/dbms/src/Storages/StorageMergeTree.cpp index bb3f70afd79..b9f5f8dfeda 100644 --- a/dbms/src/Storages/StorageMergeTree.cpp +++ b/dbms/src/Storages/StorageMergeTree.cpp @@ -689,7 +689,7 @@ bool StorageMergeTree::tryMutatePart() size_t current_ast_elements = 0; for (auto it = mutations_begin_it; it != mutations_end_it; ++it) { - MutationsInterpreter interpreter(shared_from_this(), it->second.commands, global_context); + MutationsInterpreter interpreter(shared_from_this(), it->second.commands, global_context, false); size_t commands_size = interpreter.evaluateCommandsSize(); if (current_ast_elements + commands_size >= max_ast_elements) diff --git a/dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.reference b/dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.reference new file mode 100644 index 00000000000..5f1d0ecea5d --- /dev/null +++ b/dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.reference @@ -0,0 +1,2 @@ +2 +1 diff --git a/dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.sh b/dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.sh new file mode 100755 index 00000000000..8ac912bfa0c --- /dev/null +++ b/dbms/tests/queries/0_stateless/01031_mutations_interpreter_and_context.sh @@ -0,0 +1,23 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +. $CURDIR/mergetree_mutations.lib + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS json_test" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE json_test (id UInt32, metadata String) ENGINE = MergeTree() ORDER BY id" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO json_test VALUES (1, '{\"date\": \"2018-01-01\", \"task_id\": \"billing_history__billing_history.load_history_payments_data__20180101\"}'), (2, '{\"date\": \"2018-01-02\", \"task_id\": \"billing_history__billing_history.load_history_payments_data__20180101\"}')" + +${CLICKHOUSE_CLIENT} --query="SELECT COUNT() FROM json_test" + +${CLICKHOUSE_CLIENT} --query="ALTER TABLE json_test DELETE WHERE JSONExtractString(metadata, 'date') = '2018-01-01'" + +wait_for_mutation "json_test" "mutation_2.txt" + +${CLICKHOUSE_CLIENT} --query="SELECT COUNT() FROM json_test" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS json_test" From 92bd96fc0dd49b0dc43f639581089026d38c4b27 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 6 Nov 2019 20:37:30 +0300 Subject: [PATCH 46/77] Simplify buildWhereExpression() by using makeASTFunction() helper --- dbms/src/Storages/VirtualColumnUtils.cpp | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index cbb1feef3af..8cba0fa42e0 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -61,13 +61,7 @@ ASTPtr buildWhereExpression(const ASTs & functions) return nullptr; if (functions.size() == 1) return functions[0]; - ASTPtr new_query = std::make_shared(); - auto & new_function = new_query->as(); - new_function.name = "and"; - new_function.arguments = std::make_shared(); - new_function.arguments->children = functions; - new_function.children.push_back(new_function.arguments); - return new_query; + return makeASTFunction("and", functions); } } From 531fb7f8671486cbe4955080012182fd814139fa Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 3 Nov 2019 22:31:20 +0300 Subject: [PATCH 47/77] Fix scope of the InterpreterSelectQuery for views with query ASAN report (s/#//): ================================================================= ==1==ERROR: AddressSanitizer: stack-use-after-scope on address 0x7fff5d89a080 at pc 0x0000123398bb bp 0x7fff5d8941b0 sp 0x7fff5d8941a8 READ of size 8 at 0x7fff5d89a080 thread T55 (AsyncBlockInput) 0 0x123398ba in std::__1::shared_ptr::operator->() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3933:56 1 0x123398ba in DB::Context::getLock() const /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/Context.cpp:340 2 0x123398ba in DB::Context::hasDictionaryAccessRights(std::__1::basic_string, std::__1::allocator > const&) const /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/Context.cpp:711 3 0x88faf6b in DB::FunctionDictGet, DB::NameDictGetUInt16>::executeImpl(DB::Block&, std::__1::vector > const&, unsigned long, unsigned long) (/usr/bin/clickhouse+0x88faf6b) 4 0x11da8091 in DB::PreparedFunctionImpl::executeWithoutLowCardinalityColumns(DB::Block&, std::__1::vector > const&, unsigned long, unsigned long, bool) /build/obj-x86_64-linux-gnu/../dbms/src/Functions/IFunction.cpp:312:9 5 0x11da8091 in DB::PreparedFunctionImpl::execute(DB::Block&, std::__1::vector > const&, unsigned long, unsigned long, bool) /build/obj-x86_64-linux-gnu/../dbms/src/Functions/IFunction.cpp:473 6 0x125278c5 in DB::ExpressionAction::execute(DB::Block&, bool) const /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/ExpressionActions.cpp:381:23 7 0x1254250d in DB::ExpressionActions::execute(DB::Block&, bool) const /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/ExpressionActions.cpp:762:16 8 0x129e9811 in DB::ExpressionBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/ExpressionBlockInputStream.cpp:43:21 9 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 10 0x12a46ef2 in DB::PartialSortingBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/PartialSortingBlockInputStream.cpp:12:34 11 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 12 0x129f3968 in DB::MergeSortingBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/MergeSortingBlockInputStream.cpp:51:47 13 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 14 0x129e97a3 in DB::ExpressionBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/ExpressionBlockInputStream.cpp:41:34 15 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 16 0x125d5a29 in DB::MaterializingBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/MaterializingBlockInputStream.cpp:25:46 17 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 18 0x12098079 in DB::SquashingBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/SquashingBlockInputStream.cpp:22:36 19 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 20 0x125c677b in DB::ConvertingBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/ConvertingBlockInputStream.cpp:95:34 21 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 22 0x125d043e in DB::PushingToViewsBlockOutputStream::process(DB::Block const&, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp:230:41 23 0x125cedde in DB::PushingToViewsBlockOutputStream::write(DB::Block const&) /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp:147:13 24 0x125c8600 in DB::SquashingBlockOutputStream::finalize() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/SquashingBlockOutputStream.cpp:30:17 25 0x125c89ed in DB::SquashingBlockOutputStream::writeSuffix() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/SquashingBlockOutputStream.cpp:50:5 26 0x13402808 in DB::StorageBuffer::writeBlockToDestination(DB::Block const&, std::__1::shared_ptr) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/StorageBuffer.cpp:678:19 27 0x1340c4b2 in DB::BufferBlockOutputStream::write(DB::Block const&) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/StorageBuffer.cpp:340:25 28 0x125ce657 in DB::PushingToViewsBlockOutputStream::write(DB::Block const&) /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp:116:21 29 0x125c8600 in DB::SquashingBlockOutputStream::finalize() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/SquashingBlockOutputStream.cpp:30:17 30 0x125c89ed in DB::SquashingBlockOutputStream::writeSuffix() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/SquashingBlockOutputStream.cpp:50:5 31 0x134652a0 in DB::DistributedBlockOutputStream::writeToLocal(DB::Block const&, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp:550:19 32 0x1345d13f in DB::DistributedBlockOutputStream::writeAsyncImpl(DB::Block const&, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp:513:13 33 0x1345abdd in DB::DistributedBlockOutputStream::writeAsync(DB::Block const&) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp:111:5 34 0x1345abdd in DB::DistributedBlockOutputStream::write(DB::Block const&) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp:103 35 0x125ce657 in DB::PushingToViewsBlockOutputStream::write(DB::Block const&) /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp:116:21 36 0x125c8600 in DB::SquashingBlockOutputStream::finalize() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/SquashingBlockOutputStream.cpp:30:17 37 0x125c89ed in DB::SquashingBlockOutputStream::writeSuffix() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/SquashingBlockOutputStream.cpp:50:5 38 0x1209ae2d in void DB::copyDataImpl*)::$_0&, void (&)(DB::Block const&)>(DB::IBlockInputStream&, DB::IBlockOutputStream&, DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::atomic*)::$_0&, void (&)(DB::Block const&)) /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/copyData.cpp:48:8 39 0x1209ae2d in DB::copyData(DB::IBlockInputStream&, DB::IBlockOutputStream&, std::__1::atomic*) /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/copyData.cpp:61 40 0x125c1d43 in DB::NullAndDoCopyBlockInputStream::readImpl() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/NullAndDoCopyBlockInputStream.h:42:9 41 0x1205391a in DB::IBlockInputStream::read() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/IBlockInputStream.cpp:61:15 42 0x1204e769 in DB::AsynchronousBlockInputStream::calculate() /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/AsynchronousBlockInputStream.cpp:74:34 43 0x1204f7d6 in DB::AsynchronousBlockInputStream::next()::$_0::operator()() const /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/AsynchronousBlockInputStream.cpp:59:9 44 0x1204f7d6 in decltype(std::__1::forward(fp)()) std::__1::__invoke(DB::AsynchronousBlockInputStream::next()::$_0&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4410 45 0x1204f7d6 in void std::__1::__invoke_void_return_wrapper::__call(DB::AsynchronousBlockInputStream::next()::$_0&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348 46 0x1204f7d6 in std::__1::__function::__alloc_func, void ()>::operator()() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1527 47 0x1204f7d6 in std::__1::__function::__func, void ()>::operator()() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1651 48 0x77324d6 in std::__1::__function::__value_func::operator()() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1799:16 49 0x77324d6 in std::__1::function::operator()() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2347 50 0x77324d6 in ThreadPoolImpl::worker(std::__1::__list_iterator) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:221 51 0x773ad16 in void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()::operator()() const /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:102:73 52 0x773ad16 in decltype(std::__1::forward(fp)()) std::__1::__invoke_constexpr::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'() const&>(void&&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4416 53 0x773ad16 in decltype(auto) std::__1::__apply_tuple_impl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'() const&, std::__1::tuple<> const&>(void&&, std::__1::tuple<> const&, std::__1::__tuple_indices<>) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1358 54 0x773ad16 in decltype(auto) std::__1::apply::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'() const&, std::__1::tuple<> const&>(void&&, std::__1::tuple<> const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1367 55 0x773ad16 in ThreadFromGlobalPool::ThreadFromGlobalPool::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'()::operator()() const /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.h:156 56 0x773ad16 in decltype(std::__1::forward(fp)()) std::__1::__invoke::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'()&>(void&&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4410 57 0x773ad16 in void std::__1::__invoke_void_return_wrapper::__call::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'()&>(ThreadFromGlobalPool::ThreadFromGlobalPool::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'()&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348 58 0x773ad16 in std::__1::__function::__alloc_func::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'(), std::__1::allocator::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'()>, void ()>::operator()() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1527 59 0x773ad16 in std::__1::__function::__func::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'(), std::__1::allocator::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&)::'lambda'()>, void ()>::operator()() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1651 60 0x772d9c3 in std::__1::__function::__value_func::operator()() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1799:16 61 0x772d9c3 in std::__1::function::operator()() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2347 62 0x772d9c3 in ThreadPoolImpl::worker(std::__1::__list_iterator) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:221 63 0x7735e87 in void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()::operator()() const /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:102:73 64 0x7735e87 in decltype(std::__1::forward(fp)()) std::__1::__invoke::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4410 65 0x7735e87 in void std::__1::__thread_execute >, void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(std::__1::tuple::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>&, std::__1::__tuple_indices<>) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:341 66 0x7735e87 in void* std::__1::__thread_proxy >, void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()> >(void*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:351 67 0x7ffff7fb3fb6 in start_thread (/lib/x86_64-linux-gnu/libpthread.so.0+0x8fb6) 68 0x7ffff7ed52ee in clone (/lib/x86_64-linux-gnu/libc.so.6+0xfa2ee) Address 0x7fff5d89a080 is located in stack of thread T55 (AsyncBlockInput) at offset 4448 in frame 0 0x125cf5df in DB::PushingToViewsBlockOutputStream::process(DB::Block const&, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp:199 This frame has 18 object(s): [32, 48) 'in' (line 204) [64, 4112) 'local_context' (line 211) [4240, 4256) 'ref.tmp' (line 212) [4272, 4296) 'ref.tmp6' (line 212) [4336, 4360) 'ref.tmp10' (line 212) [4400, 9136) 'select' (line 215) <== Memory access at offset 4448 is inside this variable [9392, 9416) 'ref.tmp33' (line 215) [9456, 9480) 'ref.tmp36' (line 215) [9520, 9840) 'ref.tmp43' (line 216) [9904, 9960) 'ref.tmp71' (line 223) [10000, 10056) 'result_block' (line 230) [10096, 10120) 'ref.tmp126' (line 240) [10160, 10184) 'ref.tmp127' (line 240) [10224, 10248) 'ref.tmp128' (line 240) [10288, 10312) 'ref.tmp129' (line 240) [10352, 10368) 'ref.tmp130' (line 240) [10384, 10408) 'ref.tmp138' (line 240) [10448, 10464) 'ref.tmp139' (line 240) HINT: this may be a false positive if your program uses some custom stack unwind mechanism, swapcontext or vfork (longjmp and C++ exceptions *are* supported) Thread T55 (AsyncBlockInput) created by T4 (SystemLogFlush) here: 0 0x759ffad in pthread_create (/usr/bin/clickhouse+0x759ffad) 1 0x7733f3e in std::__1::__libcpp_thread_create(unsigned long*, void* (*)(void*), void*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:327:10 2 0x7733f3e in std::__1::thread::thread::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'(), void>(void&&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:367 3 0x772b172 in void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:102:35 4 0x772c613 in ThreadPoolImpl::scheduleOrThrow(std::__1::function, int, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:138:5 5 0x77383c2 in ThreadFromGlobalPool::ThreadFromGlobalPool::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.h:149:38 6 0x772f1de in void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:102:35 7 0x772e8db in ThreadPoolImpl::scheduleOrThrowOnError(std::__1::function, int) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:126:5 8 0x134e4d0e in DB::MergeTreeData::loadDataParts(bool) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/MergeTree/MergeTreeData.cpp:805:14 9 0x1318d956 in DB::StorageMergeTree::StorageMergeTree(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::ColumnsDescription const&, DB::IndicesDescription const&, DB::ConstraintsDescription const&, bool, DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::shared_ptr const&, std::__1::shared_ptr const&, std::__1::shared_ptr const&, std::__1::shared_ptr const&, std::__1::shared_ptr const&, DB::MergeTreeData::MergingParams const&, std::__1::unique_ptr >, bool) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/StorageMergeTree.cpp:80:5 10 0x1393e5c8 in std::__1::shared_ptr ext::shared_ptr_helper::create, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::ColumnsDescription const&, DB::IndicesDescription&, DB::ConstraintsDescription const&, bool const&, DB::Context&, std::__1::basic_string, std::__1::allocator >&, std::__1::shared_ptr&, std::__1::shared_ptr&, std::__1::shared_ptr&, std::__1::shared_ptr&, std::__1::shared_ptr&, DB::MergeTreeData::MergingParams&, std::__1::unique_ptr >, bool const&>(std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::ColumnsDescription const&, DB::IndicesDescription&, DB::ConstraintsDescription const&, bool const&, DB::Context&, std::__1::basic_string, std::__1::allocator >&, std::__1::shared_ptr&, std::__1::shared_ptr&, std::__1::shared_ptr&, std::__1::shared_ptr&, std::__1::shared_ptr&, DB::MergeTreeData::MergingParams&, std::__1::unique_ptr >&&, bool const&) /build/obj-x86_64-linux-gnu/../libs/libcommon/include/ext/shared_ptr_helper.h:19:39 11 0x1392c4ff in DB::create(DB::StorageFactory::Arguments const&) /build/obj-x86_64-linux-gnu/../dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp:648:16 12 0x13940444 in decltype(std::__1::forward (*&)(DB::StorageFactory::Arguments const&)>(fp)(std::__1::forward(fp0))) std::__1::__invoke (*&)(DB::StorageFactory::Arguments const&), DB::StorageFactory::Arguments const&>(std::__1::shared_ptr (*&)(DB::StorageFactory::Arguments const&), DB::StorageFactory::Arguments const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4410:1 13 0x13940444 in std::__1::shared_ptr std::__1::__invoke_void_return_wrapper >::__call (*&)(DB::StorageFactory::Arguments const&), DB::StorageFactory::Arguments const&>(std::__1::shared_ptr (*&)(DB::StorageFactory::Arguments const&), DB::StorageFactory::Arguments const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:317 14 0x13940444 in std::__1::__function::__alloc_func (*)(DB::StorageFactory::Arguments const&), std::__1::allocator (*)(DB::StorageFactory::Arguments const&)>, std::__1::shared_ptr (DB::StorageFactory::Arguments const&)>::operator()(DB::StorageFactory::Arguments const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1527 15 0x13940444 in std::__1::__function::__func (*)(DB::StorageFactory::Arguments const&), std::__1::allocator (*)(DB::StorageFactory::Arguments const&)>, std::__1::shared_ptr (DB::StorageFactory::Arguments const&)>::operator()(DB::StorageFactory::Arguments const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1651 16 0x131075c5 in std::__1::__function::__value_func (DB::StorageFactory::Arguments const&)>::operator()(DB::StorageFactory::Arguments const&) const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1799:16 17 0x131075c5 in std::__1::function (DB::StorageFactory::Arguments const&)>::operator()(DB::StorageFactory::Arguments const&) const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2347 18 0x131075c5 in DB::StorageFactory::get(DB::ASTCreateQuery&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, DB::Context&, DB::Context&, DB::ColumnsDescription const&, DB::ConstraintsDescription const&, bool, bool) const /build/obj-x86_64-linux-gnu/../dbms/src/Storages/StorageFactory.cpp:164 19 0x124414ad in DB::InterpreterCreateQuery::createTable(DB::ASTCreateQuery&) /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/InterpreterCreateQuery.cpp:657:46 20 0x12446e45 in DB::InterpreterCreateQuery::execute() /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/InterpreterCreateQuery.cpp:753:16 21 0x76760da in DB::SystemLog::prepareTable() /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/SystemLog.h:438:21 22 0x767006d in DB::SystemLog::flushImpl(DB::SystemLog::EntryType) /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/SystemLog.h:327:9 23 0x766eb18 in DB::SystemLog::threadFunction() /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/SystemLog.h:300:17 24 0x766e445 in DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()::operator()() const /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/SystemLog.h:187:51 25 0x766e445 in decltype(std::__1::forward::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'() const&>(fp)()) std::__1::__invoke_constexpr::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'() const&>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'() const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4416 26 0x766e445 in decltype(auto) std::__1::__apply_tuple_impl::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'() const&, std::__1::tuple<> const&>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'() const&, std::__1::tuple<> const&, std::__1::__tuple_indices<>) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1358 27 0x766e445 in decltype(auto) std::__1::apply::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'() const&, std::__1::tuple<> const&>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'() const&, std::__1::tuple<> const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/tuple:1367 28 0x766e445 in ThreadFromGlobalPool::ThreadFromGlobalPool::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'()::operator()() const /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.h:156 29 0x766e445 in decltype(std::__1::forward::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(fp)()) std::__1::__invoke::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'()&>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4410 30 0x766e445 in void std::__1::__invoke_void_return_wrapper::__call::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'()&>(ThreadFromGlobalPool::ThreadFromGlobalPool::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'()&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/__functional_base:348 31 0x766e445 in std::__1::__function::__alloc_func::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'(), std::__1::allocator::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'()>, void ()>::operator()() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1527 32 0x766e445 in std::__1::__function::__func::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'(), std::__1::allocator::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&)::'lambda'()>, void ()>::operator()() /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1651 33 0x772d9c3 in std::__1::__function::__value_func::operator()() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:1799:16 34 0x772d9c3 in std::__1::function::operator()() const /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/functional:2347 35 0x772d9c3 in ThreadPoolImpl::worker(std::__1::__list_iterator) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:221 36 0x7735e87 in void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()::operator()() const /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:102:73 37 0x7735e87 in decltype(std::__1::forward(fp)()) std::__1::__invoke::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(void&&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/type_traits:4410 38 0x7735e87 in void std::__1::__thread_execute >, void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>(std::__1::tuple::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()>&, std::__1::__tuple_indices<>) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:341 39 0x7735e87 in void* std::__1::__thread_proxy >, void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'()> >(void*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:351 40 0x7ffff7fb3fb6 in start_thread (/lib/x86_64-linux-gnu/libpthread.so.0+0x8fb6) Thread T4 (SystemLogFlush) created by T0 here: 0 0x759ffad in pthread_create (/usr/bin/clickhouse+0x759ffad) 1 0x7733f3e in std::__1::__libcpp_thread_create(unsigned long*, void* (*)(void*), void*) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/__threading_support:327:10 2 0x7733f3e in std::__1::thread::thread::scheduleImpl(std::__1::function, int, std::__1::optional)::'lambda1'(), void>(void&&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/thread:367 3 0x772b172 in void ThreadPoolImpl::scheduleImpl(std::__1::function, int, std::__1::optional) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:102:35 4 0x772c613 in ThreadPoolImpl::scheduleOrThrow(std::__1::function, int, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.cpp:138:5 5 0x766b5f6 in ThreadFromGlobalPool::ThreadFromGlobalPool::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()>(DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long)::'lambda'()&&) /build/obj-x86_64-linux-gnu/../dbms/src/Common/ThreadPool.h:149:38 6 0x766a214 in DB::SystemLog::SystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/SystemLog.h:187:21 7 0x75ecf52 in DB::QueryLog::QueryLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, unsigned long) /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/QueryLog.h:79:39 8 0x75ecf52 in std::__1::__compressed_pair_elem::__compressed_pair_elem, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&, 0ul, 1ul, 2ul, 3ul, 4ul>(std::__1::piecewise_construct_t, std::__1::tuple, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&>, std::__1::__tuple_indices<0ul, 1ul, 2ul, 3ul, 4ul>) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2155 9 0x75ecf52 in std::__1::__compressed_pair, DB::QueryLog>::__compressed_pair&, DB::Context&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&>(std::__1::piecewise_construct_t, std::__1::tuple&>, std::__1::tuple, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&>) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:2258 10 0x75ecf52 in std::__1::__shared_ptr_emplace >::__shared_ptr_emplace, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&>(std::__1::allocator, DB::Context&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3671 11 0x75ecf52 in std::__1::shared_ptr std::__1::shared_ptr::make_shared, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&>(DB::Context&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:4330 12 0x75ecf52 in std::__1::enable_if::value), std::__1::shared_ptr >::type std::__1::make_shared, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&>(DB::Context&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, std::__1::basic_string, std::__1::allocator >&, unsigned long&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:4709 13 0x75ecf52 in std::__1::shared_ptr DB::(anonymous namespace)::createSystemLog(DB::Context&, std::__1::basic_string, std::__1::allocator > const&, std::__1::basic_string, std::__1::allocator > const&, Poco::Util::AbstractConfiguration const&, std::__1::basic_string, std::__1::allocator > const&) /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/SystemLog.cpp:39 14 0x75ecf52 in DB::SystemLogs::SystemLogs(DB::Context&, Poco::Util::AbstractConfiguration const&) /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/SystemLog.cpp:47 15 0x12350225 in void std::__1::__optional_storage_base::__construct(DB::Context&, Poco::Util::AbstractConfiguration const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:323:54 16 0x12350225 in DB::SystemLogs& std::__1::optional::emplace(DB::Context&, Poco::Util::AbstractConfiguration const&) /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/optional:826 17 0x12350225 in DB::Context::initializeSystemLogs() /build/obj-x86_64-linux-gnu/../dbms/src/Interpreters/Context.cpp:1679 18 0x760b2c6 in DB::Server::main(std::__1::vector, std::__1::allocator >, std::__1::allocator, std::__1::allocator > > > const&) /build/obj-x86_64-linux-gnu/../dbms/programs/server/Server.cpp:528:25 19 0x147fbce0 in Poco::Util::Application::run() /build/obj-x86_64-linux-gnu/../contrib/poco/Util/src/Application.cpp:335:8 20 0x75f5df5 in DB::Server::run() /build/obj-x86_64-linux-gnu/../dbms/programs/server/Server.cpp:149:25 21 0x76463e3 in mainEntryClickHouseServer(int, char**) /build/obj-x86_64-linux-gnu/../dbms/programs/server/Server.cpp:962:20 22 0x75eaa8b in main /build/obj-x86_64-linux-gnu/../dbms/programs/main.cpp:178:12 23 0x7ffff7e01bba in __libc_start_main (/lib/x86_64-linux-gnu/libc.so.6+0x26bba) SUMMARY: AddressSanitizer: stack-use-after-scope /build/obj-x86_64-linux-gnu/../contrib/libcxx/include/memory:3933:56 in std::__1::shared_ptr::operator->() const Shadow bytes around the buggy address: 0x10006bb0b3c0: f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 0x10006bb0b3d0: f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 0x10006bb0b3e0: f8 f8 f8 f8 f8 f8 f2 f2 f2 f2 f2 f2 f2 f2 f2 f2 0x10006bb0b3f0: f2 f2 f2 f2 f2 f2 f8 f8 f2 f2 f8 f8 f8 f2 f2 f2 0x10006bb0b400: f2 f2 f8 f8 f8 f2 f2 f2 f2 f2 f8 f8 f8 f8 f8 f8 =>0x10006bb0b410:[f8]f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 0x10006bb0b420: f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 0x10006bb0b430: f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 0x10006bb0b440: f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 0x10006bb0b450: f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 0x10006bb0b460: f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 f8 Shadow byte legend (one shadow byte represents 8 application bytes): Addressable: 00 Partially addressable: 01 02 03 04 05 06 07 Heap left redzone: fa Freed heap region: fd Stack left redzone: f1 Stack mid redzone: f2 Stack right redzone: f3 Stack after return: f5 Stack use after scope: f8 Global redzone: f9 Global init order: f6 Poisoned by user: f7 Container overflow: fc Array cookie: ac Intra object redzone: bb ASan internal: fe Left alloca redzone: ca Right alloca redzone: cb Shadow gap: cc ==1==ABORTING --- .../PushingToViewsBlockOutputStream.cpp | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 84a5bd78293..d7fb7bad343 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -203,6 +203,19 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n { BlockInputStreamPtr in; + /// We need keep InterpreterSelectQuery, until the processing will be finished, since: + /// + /// - We copy Context inside InterpreterSelectQuery to support + /// modification of context (Settings) for subqueries + /// - InterpreterSelectQuery lives shorter than query pipeline. + /// It's used just to build the query pipeline and no longer needed + /// - ExpressionAnalyzer and then, Functions, that created in InterpreterSelectQuery, + /// **can** take a reference to Context from InterpreterSelectQuery + /// (the problem raises only when function uses context from the + /// execute*() method, like FunctionDictGet do) + /// - These objects live inside query pipeline (DataStreams) and the reference become dangling. + std::optional select; + if (view.query) { /// We create a table with the same name as original table and the same alias columns, @@ -212,8 +225,8 @@ void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_n local_context.addViewSource( StorageValues::create(storage->getDatabaseName(), storage->getTableName(), storage->getColumns(), block)); - InterpreterSelectQuery select(view.query, local_context, SelectQueryOptions()); - in = std::make_shared(select.execute().in); + select.emplace(view.query, local_context, SelectQueryOptions()); + in = std::make_shared(select->execute().in); /// Squashing is needed here because the materialized view query can generate a lot of blocks /// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY From f58f1517e0a248445eb2c779f371d466333b91a8 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 3 Nov 2019 22:50:34 +0300 Subject: [PATCH 48/77] Regression test for InterpreterSelectQuery scope in PushingToViewsBlockOutputStream It is too complex, so I'm not sure that it is worth it. --- ..._materialized_view_query_context.reference | 1 + .../01023_materialized_view_query_context.sql | 28 +++++++++++++++++++ 2 files changed, 29 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/01023_materialized_view_query_context.reference create mode 100644 dbms/tests/queries/0_stateless/01023_materialized_view_query_context.sql diff --git a/dbms/tests/queries/0_stateless/01023_materialized_view_query_context.reference b/dbms/tests/queries/0_stateless/01023_materialized_view_query_context.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01023_materialized_view_query_context.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/01023_materialized_view_query_context.sql b/dbms/tests/queries/0_stateless/01023_materialized_view_query_context.sql new file mode 100644 index 00000000000..1a7f8c15678 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01023_materialized_view_query_context.sql @@ -0,0 +1,28 @@ +-- Create dictionary, since dictGet*() uses DB::Context in executeImpl() +-- (To cover scope of the Context in DB::PushingToViewsBlockOutputStream::process) +DROP DATABASE IF EXISTS dict_in_01023; +CREATE DATABASE dict_in_01023; + +CREATE TABLE dict_in_01023.input (key UInt64, val UInt64) Engine=Memory(); + +CREATE DICTIONARY dict_in_01023.dict +( + key UInt64 DEFAULT 0, + val UInt64 DEFAULT 1 +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT 9000 USER 'default' TABLE 'input' PASSWORD '' DB 'dict_in_01023')) +LIFETIME(MIN 0 MAX 0) +LAYOUT(HASHED()); + +CREATE TABLE input (key UInt64) Engine=Distributed(test_shard_localhost, currentDatabase(), buffer_, key); +CREATE TABLE null_ (key UInt64) Engine=Null(); +CREATE TABLE buffer_ (key UInt64) Engine=Buffer(currentDatabase(), dist_out, 1, 0, 0, 0, 0, 0, 0); +CREATE TABLE dist_out (key UInt64) Engine=Distributed(test_shard_localhost, currentDatabase(), null_, key); + +CREATE TABLE output (key UInt64, val UInt64) Engine=Memory(); +CREATE MATERIALIZED VIEW mv TO output AS SELECT key, dictGetUInt64('dict_in_01023.dict', 'val', key) val FROM dist_out; + +INSERT INTO input VALUES (1); + +SELECT count() FROM output; From 0677ae048c0cc5f62c5de31b762c7dc6c2d92557 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Wed, 6 Nov 2019 20:37:30 +0300 Subject: [PATCH 49/77] Wrap _shard_num with toUInt32 for simpler JOIN with system.clusters.shard_num --- .../ClusterProxy/SelectStreamFactory.cpp | 2 +- dbms/src/Storages/VirtualColumnUtils.cpp | 26 +++++++++++++++++++ dbms/src/Storages/VirtualColumnUtils.h | 10 ++++--- .../01018_Distributed__shard_num.sql | 3 +-- 4 files changed, 35 insertions(+), 6 deletions(-) diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp index 03acf6d9a7c..a3474b466d5 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.cpp @@ -105,7 +105,7 @@ void SelectStreamFactory::createForShard( { auto modified_query_ast = query_ast->clone(); if (has_virtual_shard_num_column) - VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, "_shard_num", shard_info.shard_num); + VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, "_shard_num", shard_info.shard_num, "toUInt32"); auto emplace_local_stream = [&]() { diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index 8cba0fa42e0..77069d97c10 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -81,6 +81,32 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va select.with()->children.push_back(literal); } +void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value, const String & func) +{ + auto & select = ast->as(); + if (!select.with()) + select.setExpression(ASTSelectQuery::Expression::WITH, std::make_shared()); + + + if (func.empty()) + { + auto literal = std::make_shared(value); + literal->alias = column_name; + literal->prefer_alias_to_column_name = true; + select.with()->children.push_back(literal); + } + else + { + auto literal = std::make_shared(value); + literal->prefer_alias_to_column_name = true; + + auto function = makeASTFunction(func, literal); + function->alias = column_name; + function->prefer_alias_to_column_name = true; + select.with()->children.push_back(function); + } +} + void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context) { const auto & select = query->as(); diff --git a/dbms/src/Storages/VirtualColumnUtils.h b/dbms/src/Storages/VirtualColumnUtils.h index 4976deaa4c9..e1c7e400249 100644 --- a/dbms/src/Storages/VirtualColumnUtils.h +++ b/dbms/src/Storages/VirtualColumnUtils.h @@ -16,9 +16,13 @@ class NamesAndTypesList; namespace VirtualColumnUtils { -/// Adds to the select query section `select column_name as value` -/// For example select _port as 9000. -void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value); +/// Adds to the select query section `WITH value AS column_name`, and uses func +/// to wrap the value (if any) +/// +/// For example: +/// - `WITH 9000 as _port`. +/// - `WITH toUInt16(9000) as _port`. +void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & value, const String & func = ""); /// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query. /// Only elements of the outer conjunction are considered, depending only on the columns present in the block. diff --git a/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql b/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql index 3c4a0c409c4..cd338aa3985 100644 --- a/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql +++ b/dbms/tests/queries/0_stateless/01018_Distributed__shard_num.sql @@ -48,11 +48,10 @@ FROM (SELECT *, _shard_num FROM dist_1) a JOIN system.clusters b ON a._shard_num = b.shard_num WHERE b.cluster = 'test_cluster_two_shards_localhost'; --- Requires toUInt32() otherwise Type mismatch of columns (53) SELECT _shard_num, key, b.host_name, b.host_address, b.port FROM dist_1 a JOIN system.clusters b -ON toUInt32(_shard_num) = b.shard_num +ON _shard_num = b.shard_num WHERE b.cluster = 'test_cluster_two_shards_localhost'; -- rewrite does not work with aliases, hence Missing columns (47) From da07018a1a95107286d5c78fd7171fc2fb06e3b0 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 6 Nov 2019 21:11:06 +0300 Subject: [PATCH 50/77] Update memory limit for external aggregation test. --- dbms/tests/queries/1_stateful/00084_external_aggregation.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/queries/1_stateful/00084_external_aggregation.sql b/dbms/tests/queries/1_stateful/00084_external_aggregation.sql index c429661a953..b3922eae049 100644 --- a/dbms/tests/queries/1_stateful/00084_external_aggregation.sql +++ b/dbms/tests/queries/1_stateful/00084_external_aggregation.sql @@ -1,6 +1,6 @@ SET max_bytes_before_external_group_by = 200000000; -SET max_memory_usage = 1200000000; +SET max_memory_usage = 1500000000; SET max_threads = 12; SELECT URL, uniq(SearchPhrase) AS u FROM test.hits GROUP BY URL ORDER BY u DESC, URL LIMIT 10; From 544af39d23c7e556caaa153d85444bdef086e6e4 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Mon, 4 Nov 2019 13:29:54 +0800 Subject: [PATCH 51/77] Vectorize arrayReduce and -Array combinator Also devirtualize -State combinator. --- .../AggregateFunctionArray.h | 2 + .../AggregateFunctions/IAggregateFunction.h | 116 ++++++++++++++---- .../AggregatingSortedBlockInputStream.h | 2 +- .../SummingSortedBlockInputStream.h | 2 +- dbms/src/Functions/FunctionHelpers.cpp | 25 ++++ dbms/src/Functions/FunctionHelpers.h | 5 + dbms/src/Functions/array/arrayReduce.cpp | 47 ++++--- dbms/src/Interpreters/Aggregator.cpp | 45 ++++++- dbms/src/Interpreters/Aggregator.h | 6 +- .../vectorize_aggregation_combinators.xml | 35 ++++++ 10 files changed, 233 insertions(+), 52 deletions(-) create mode 100644 dbms/tests/performance/vectorize_aggregation_combinators.xml diff --git a/dbms/src/AggregateFunctions/AggregateFunctionArray.h b/dbms/src/AggregateFunctions/AggregateFunctionArray.h index 2a4780b6c9d..66dbcd865d5 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionArray.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionArray.h @@ -129,6 +129,8 @@ public: return nested_func->allocatesMemoryInArena(); } + AggregateFunctionPtr getNestedFunction() const { return nested_func; } + const char * getHeaderFilePath() const override { return __FILE__; } }; diff --git a/dbms/src/AggregateFunctions/IAggregateFunction.h b/dbms/src/AggregateFunctions/IAggregateFunction.h index e35041ab560..99a6da781d3 100644 --- a/dbms/src/AggregateFunctions/IAggregateFunction.h +++ b/dbms/src/AggregateFunctions/IAggregateFunction.h @@ -119,23 +119,56 @@ public: */ virtual bool isState() const { return false; } + using AddFunc = void (*)(const IAggregateFunction *, AggregateDataPtr, const IColumn **, size_t, Arena *); + + /** Contains a loop with calls to "add" function. You can collect arguments into array "places" + * and do a single call to "addBatch" for devirtualization and inlining. When offsets is not + * null, behave like AddBatchArrayFunc (it's used to work around unknown regressions). + */ + using AddBatchFunc = void (*)( + const IAggregateFunction *, + size_t batch_size, + AggregateDataPtr * places, + size_t place_offset, + const IColumn ** columns, + const UInt64 * offsets, + Arena * arena); + + /** The same for single place. + */ + using AddBatchSinglePlaceFunc + = void (*)(const IAggregateFunction *, size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena); + + /** In addition to the above method, this variant accepts an array of "offsets" which allows + * collecting multiple rows of arguments into array "places" as long as they are between + * offsets[i-1] and offsets[i]. It is used for arrayReduce and might be used generally to + * break data dependency when array "places" contains a large number of same values + * consecutively. + */ + using AddBatchArrayFunc = void (*)( + const IAggregateFunction *, + size_t batch_size, + AggregateDataPtr * places, + size_t place_offset, + const IColumn ** columns, + const UInt64 * offsets, + Arena * arena); + + struct AddFuncs + { + AddFunc add; + AddBatchFunc add_batch; + AddBatchSinglePlaceFunc add_batch_single_place; + AddBatchArrayFunc add_batch_array; + }; + /** The inner loop that uses the function pointer is better than using the virtual function. * The reason is that in the case of virtual functions GCC 5.1.2 generates code, * which, at each iteration of the loop, reloads the function address (the offset value in the virtual function table) from memory to the register. * This gives a performance drop on simple queries around 12%. * After the appearance of better compilers, the code can be removed. */ - using AddFunc = void (*)(const IAggregateFunction *, AggregateDataPtr, const IColumn **, size_t, Arena *); - virtual AddFunc getAddressOfAddFunction() const = 0; - - /** Contains a loop with calls to "add" function. You can collect arguments into array "places" - * and do a single call to "addBatch" for devirtualization and inlining. - */ - virtual void addBatch(size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, Arena * arena) const = 0; - - /** The same for single place. - */ - virtual void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const = 0; + virtual AddFuncs getAddressOfAddFunctions() const = 0; /** This is used for runtime code generation to determine, which header files to include in generated source. * Always implement it as @@ -162,23 +195,58 @@ private: static_cast(*that).add(place, columns, row_num, arena); } + static void addBatch( + const IAggregateFunction * that, + size_t batch_size, + AggregateDataPtr * places, + size_t place_offset, + const IColumn ** columns, + const UInt64 * offsets, + Arena * arena) + { + if (offsets) + { + size_t current_offset = 0; + for (size_t i = 0; i < batch_size; ++i) + { + size_t next_offset = offsets[i]; + for (size_t j = current_offset; j < next_offset; ++j) + static_cast(that)->add(places[i] + place_offset, columns, j, arena); + current_offset = next_offset; + } + } + else + for (size_t i = 0; i < batch_size; ++i) + static_cast(that)->add(places[i] + place_offset, columns, i, arena); + } + + static void + addBatchSinglePlaceFree(const IAggregateFunction * that, size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) + { + for (size_t i = 0; i < batch_size; ++i) + static_cast(that)->add(place, columns, i, arena); + } + + /// TODO: We cannot use this function directly as it slows down aggregate functions like uniqCombined due to unknown reasons. + static void addBatchArrayFree(const IAggregateFunction * that, + size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, const UInt64 * offsets, Arena * arena) + { + size_t current_offset = 0; + for (size_t i = 0; i < batch_size; ++i) + { + size_t next_offset = offsets[i]; + for (size_t j = current_offset; j < next_offset; ++j) + static_cast(that)->add(places[i] + place_offset, columns, j, arena); + current_offset = next_offset; + } + } + public: IAggregateFunctionHelper(const DataTypes & argument_types_, const Array & parameters_) : IAggregateFunction(argument_types_, parameters_) {} - AddFunc getAddressOfAddFunction() const override { return &addFree; } - - void addBatch(size_t batch_size, AggregateDataPtr * places, size_t place_offset, const IColumn ** columns, Arena * arena) const override - { - for (size_t i = 0; i < batch_size; ++i) - static_cast(this)->add(places[i] + place_offset, columns, i, arena); - } - - void addBatchSinglePlace(size_t batch_size, AggregateDataPtr place, const IColumn ** columns, Arena * arena) const override - { - for (size_t i = 0; i < batch_size; ++i) - static_cast(this)->add(place, columns, i, arena); - } + /// If we return addBatchArrayFree instead of nullptr, it leads to regression. + AddFuncs getAddressOfAddFunctions() const override { return {&addFree, &addBatch, &addBatchSinglePlaceFree, nullptr}; } }; diff --git a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h index 0cf4bd64d87..63e2e7cb37a 100644 --- a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h @@ -83,7 +83,7 @@ private: SimpleAggregateDescription(const AggregateFunctionPtr & function_, const size_t column_number_) : function(function_), column_number(column_number_) { - add_function = function->getAddressOfAddFunction(); + add_function = function->getAddressOfAddFunctions().add; state.reset(function->sizeOfData(), function->alignOfData()); } diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.h b/dbms/src/DataStreams/SummingSortedBlockInputStream.h index 4412e5529f8..e041a9b7300 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.h @@ -84,7 +84,7 @@ private: void init(const char * function_name, const DataTypes & argument_types) { function = AggregateFunctionFactory::instance().get(function_name, argument_types); - add_function = function->getAddressOfAddFunction(); + add_function = function->getAddressOfAddFunctions().add; state.reset(function->sizeOfData(), function->alignOfData()); } diff --git a/dbms/src/Functions/FunctionHelpers.cpp b/dbms/src/Functions/FunctionHelpers.cpp index 1edfbfebf78..5f625ec6d26 100644 --- a/dbms/src/Functions/FunctionHelpers.cpp +++ b/dbms/src/Functions/FunctionHelpers.cpp @@ -18,6 +18,7 @@ namespace ErrorCodes { extern const int ILLEGAL_COLUMN; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; + extern const int SIZES_OF_ARRAYS_DOESNT_MATCH; } const ColumnConst * checkAndGetColumnConstStringOrFixedString(const IColumn * column) @@ -118,4 +119,28 @@ void validateArgumentType(const IFunction & func, const DataTypes & arguments, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); } +std::pair, const ColumnArray::Offset *> +checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments) +{ + assert(num_arguments > 0); + std::vector nested_columns(num_arguments); + const ColumnArray::Offsets * offsets = nullptr; + for (size_t i = 0; i < num_arguments; ++i) + { + const ColumnArray::Offsets * offsets_i = nullptr; + if (const ColumnArray * arr = checkAndGetColumn(columns[i])) + { + nested_columns[i] = &arr->getData(); + offsets_i = &arr->getOffsets(); + } + else + throw Exception("Illegal column " + columns[i]->getName() + " as argument of function", ErrorCodes::ILLEGAL_COLUMN); + if (i == 0) + offsets = offsets_i; + else if (*offsets_i != *offsets) + throw Exception("Lengths of all arrays passed to aggregate function must be equal.", ErrorCodes::SIZES_OF_ARRAYS_DOESNT_MATCH); + } + return {nested_columns, offsets->data()}; +} + } diff --git a/dbms/src/Functions/FunctionHelpers.h b/dbms/src/Functions/FunctionHelpers.h index ac116510b7e..827ea53217a 100644 --- a/dbms/src/Functions/FunctionHelpers.h +++ b/dbms/src/Functions/FunctionHelpers.h @@ -4,6 +4,7 @@ #include #include #include +#include #include #include #include @@ -89,4 +90,8 @@ void validateArgumentType(const IFunction & func, const DataTypes & arguments, size_t argument_index, bool (* validator_func)(const IDataType &), const char * expected_type_description); +/// Checks if a list of array columns have equal offsets. Return a pair of nested columns and offsets if true, otherwise throw. +std::pair, const ColumnArray::Offset *> +checkAndGetNestedArrayOffset(const IColumn ** columns, size_t num_arguments); + } diff --git a/dbms/src/Functions/array/arrayReduce.cpp b/dbms/src/Functions/array/arrayReduce.cpp index 516449a4872..ef566345acc 100644 --- a/dbms/src/Functions/array/arrayReduce.cpp +++ b/dbms/src/Functions/array/arrayReduce.cpp @@ -7,11 +7,14 @@ #include #include #include +#include #include #include #include #include +#include + namespace DB { @@ -106,10 +109,7 @@ DataTypePtr FunctionArrayReduce::getReturnTypeImpl(const ColumnsWithTypeAndName void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) { IAggregateFunction & agg_func = *aggregate_function.get(); - AlignedBuffer place_holder(agg_func.sizeOfData(), agg_func.alignOfData()); - AggregateDataPtr place = place_holder.data(); - - std::unique_ptr arena = agg_func.allocatesMemoryInArena() ? std::make_unique() : nullptr; + std::unique_ptr arena = std::make_unique(); /// Aggregate functions do not support constant columns. Therefore, we materialize them. std::vector materialized_columns; @@ -157,32 +157,41 @@ void FunctionArrayReduce::executeImpl(Block & block, const ColumnNumbers & argum throw Exception("State function " + agg_func.getName() + " inserts results into non-state column " + block.getByPosition(result).type->getName(), ErrorCodes::ILLEGAL_COLUMN); - ColumnArray::Offset current_offset = 0; + PODArray places(input_rows_count); for (size_t i = 0; i < input_rows_count; ++i) { - agg_func.create(place); - ColumnArray::Offset next_offset = (*offsets)[i]; - + places[i] = arena->alignedAlloc(agg_func.sizeOfData(), agg_func.alignOfData()); try { - for (size_t j = current_offset; j < next_offset; ++j) - agg_func.add(place, aggregate_arguments, j, arena.get()); - - if (!res_col_aggregate_function) - agg_func.insertResultInto(place, res_col); - else - res_col_aggregate_function->insertFrom(place); + agg_func.create(places[i]); } catch (...) { - agg_func.destroy(place); + agg_func.destroy(places[i]); throw; } - - agg_func.destroy(place); - current_offset = next_offset; } + SCOPE_EXIT({ + for (size_t i = 0; i < input_rows_count; ++i) + agg_func.destroy(places[i]); + }); + + { + auto that = &agg_func; + /// Unnest consecutive trailing -State combinators + while (auto func = typeid_cast(that)) + that = func->getNestedFunction().get(); + + that->getAddressOfAddFunctions().add_batch( + that, input_rows_count, places.data(), 0, aggregate_arguments, offsets->data(), arena.get()); + } + + for (size_t i = 0; i < input_rows_count; ++i) + if (!res_col_aggregate_function) + agg_func.insertResultInto(places[i], res_col); + else + res_col_aggregate_function->insertFrom(places[i]); block.getByPosition(result).column = std::move(result_holder); } diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index fc358c22189..f0c0aec8ee6 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -26,6 +26,8 @@ #include #include #include +#include +#include namespace ProfileEvents @@ -450,7 +452,7 @@ void NO_INLINE Aggregator::executeImplCase( /// Add values to the aggregate functions. for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) - (*inst->func)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool); + (*inst->funcs.add)(inst->that, value + inst->state_offset, inst->arguments, i, aggregates_pool); } } @@ -492,7 +494,10 @@ void NO_INLINE Aggregator::executeImplBatch( /// Add values to the aggregate functions. for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) - inst->that->addBatch(rows, places.data(), inst->state_offset, inst->arguments, aggregates_pool); + { + (*inst->batch_funcs.add_batch)( + inst->batch_that, rows, places.data(), inst->state_offset, inst->batch_arguments, inst->offsets, aggregates_pool); + } } @@ -504,7 +509,13 @@ void NO_INLINE Aggregator::executeWithoutKeyImpl( { /// Adding values for (AggregateFunctionInstruction * inst = aggregate_instructions; inst->that; ++inst) - inst->that->addBatchSinglePlace(rows, res + inst->state_offset, inst->arguments, arena); + { + if (inst->offsets) + (*inst->batch_funcs.add_batch_single_place)( + inst->batch_that, inst->offsets[static_cast(rows - 1)], res + inst->state_offset, inst->batch_arguments, arena); + else + (*inst->batch_funcs.add_batch_single_place)(inst->batch_that, rows, res + inst->state_offset, inst->batch_arguments, arena); + } } @@ -564,6 +575,7 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData AggregateFunctionInstructions aggregate_functions_instructions(params.aggregates_size + 1); aggregate_functions_instructions[params.aggregates_size].that = nullptr; + std::vector> nested_columns_holder; for (size_t i = 0; i < params.aggregates_size; ++i) { for (size_t j = 0; j < aggregate_columns[i].size(); ++j) @@ -579,10 +591,31 @@ bool Aggregator::executeOnBlock(Columns columns, UInt64 num_rows, AggregatedData } } - aggregate_functions_instructions[i].that = aggregate_functions[i]; - aggregate_functions_instructions[i].func = aggregate_functions[i]->getAddressOfAddFunction(); - aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; aggregate_functions_instructions[i].arguments = aggregate_columns[i].data(); + aggregate_functions_instructions[i].state_offset = offsets_of_aggregate_states[i]; + auto that = aggregate_functions[i]; + /// Unnest consecutive trailing -State combinators + while (auto func = typeid_cast(that)) + that = func->getNestedFunction().get(); + aggregate_functions_instructions[i].that = that; + aggregate_functions_instructions[i].funcs = that->getAddressOfAddFunctions(); + + if (auto func = typeid_cast(that)) + { + /// Unnest consecutive -State combinators before -Array + that = func->getNestedFunction().get(); + while (auto nested_func = typeid_cast(that)) + that = nested_func->getNestedFunction().get(); + auto [nested_columns, offsets] = checkAndGetNestedArrayOffset(aggregate_columns[i].data(), that->getArgumentTypes().size()); + nested_columns_holder.push_back(std::move(nested_columns)); + aggregate_functions_instructions[i].batch_arguments = nested_columns_holder.back().data(); + aggregate_functions_instructions[i].offsets = offsets; + } + else + aggregate_functions_instructions[i].batch_arguments = aggregate_columns[i].data(); + + aggregate_functions_instructions[i].batch_that = that; + aggregate_functions_instructions[i].batch_funcs = that->getAddressOfAddFunctions(); } if (isCancelled()) diff --git a/dbms/src/Interpreters/Aggregator.h b/dbms/src/Interpreters/Aggregator.h index 0f0faaecb44..6b0fca08e5f 100644 --- a/dbms/src/Interpreters/Aggregator.h +++ b/dbms/src/Interpreters/Aggregator.h @@ -1009,9 +1009,13 @@ protected: struct AggregateFunctionInstruction { const IAggregateFunction * that; - IAggregateFunction::AddFunc func; + IAggregateFunction::AddFuncs funcs; size_t state_offset; const IColumn ** arguments; + const IAggregateFunction * batch_that; + IAggregateFunction::AddFuncs batch_funcs; + const IColumn ** batch_arguments; + const UInt64 * offsets = nullptr; }; using AggregateFunctionInstructions = std::vector; diff --git a/dbms/tests/performance/vectorize_aggregation_combinators.xml b/dbms/tests/performance/vectorize_aggregation_combinators.xml new file mode 100644 index 00000000000..a1afb2e6cc8 --- /dev/null +++ b/dbms/tests/performance/vectorize_aggregation_combinators.xml @@ -0,0 +1,35 @@ + + + loop + + + + 30000 + + + 6000 + 60000 + + + + + + + + + 1 + + + CREATE TABLE array_data(k UInt16, v Array(UInt64)) ENGINE Log + + INSERT INTO array_data SELECT number % 1024, arrayWithConstant(16, number) from numbers(10000000) + + SELECT countMerge(v) FROM (SELECT countState() v FROM numbers(1000000000)) FORMAT Null + SELECT countMerge(v) FROM (SELECT number % 1024 k, countState() v FROM numbers(1000000000) GROUP BY k) FORMAT Null + + SELECT sumArray(v) FROM array_data FORMAT Null + SELECT k, sumArray(v) FROM array_data GROUP BY k FORMAT Null + SELECT arrayReduce('avg', v) FROM array_data FORMAT Null + + DROP TABLE IF EXISTS array_data + From 18e59f2771f38ac1b3ab942ea4dcc368e20491ee Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 7 Nov 2019 11:40:36 +0300 Subject: [PATCH 52/77] Remove some functions from mutations interpreter --- .../src/Interpreters/MutationsInterpreter.cpp | 101 +++++++++--------- dbms/src/Interpreters/MutationsInterpreter.h | 6 +- .../MergeTree/MergeTreeDataMergerMutator.cpp | 4 +- 3 files changed, 57 insertions(+), 54 deletions(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.cpp b/dbms/src/Interpreters/MutationsInterpreter.cpp index 9c6982d6f22..ab26efabdbf 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.cpp +++ b/dbms/src/Interpreters/MutationsInterpreter.cpp @@ -89,25 +89,46 @@ std::optional findFirstNonDeterministicFuncName(const MutationCommand & return {}; } -}; -MutationsInterpreter::MutationsInterpreter( - StoragePtr storage_, - std::vector commands_, - const Context & context_, - bool can_execute_) - : storage(std::move(storage_)) - , commands(std::move(commands_)) - , context(context_) - , can_execute(can_execute_) +ASTPtr prepareQueryAffectedAST(const std::vector & commands) { - mutation_ast = prepare(!can_execute); - auto limits = SelectQueryOptions().analyze(!can_execute).ignoreLimits(); - select_interpreter = std::make_unique(mutation_ast, context, storage, limits); + /// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query. + /// The result can differ from tne number of affected rows (e.g. if there is an UPDATE command that + /// changes how many rows satisfy the predicates of the subsequent commands). + /// But we can be sure that if count = 0, then no rows will be touched. + + auto select = std::make_shared(); + + select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); + auto count_func = std::make_shared(); + count_func->name = "count"; + count_func->arguments = std::make_shared(); + select->select()->children.push_back(count_func); + + if (commands.size() == 1) + select->setExpression(ASTSelectQuery::Expression::WHERE, commands[0].predicate->clone()); + else + { + auto coalesced_predicates = std::make_shared(); + coalesced_predicates->name = "or"; + coalesced_predicates->arguments = std::make_shared(); + coalesced_predicates->children.push_back(coalesced_predicates->arguments); + + for (const MutationCommand & command : commands) + coalesced_predicates->arguments->children.push_back(command.predicate->clone()); + + select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(coalesced_predicates)); + } + + return select; } +}; -bool MutationsInterpreter::isStorageTouchedByMutations() const +bool isStorageTouchedByMutations( + StoragePtr storage, + const std::vector & commands, + Context context_copy) { if (commands.empty()) return false; @@ -118,11 +139,10 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const return true; } - Context context_copy = context; context_copy.getSettingsRef().merge_tree_uniform_read_distribution = 0; context_copy.getSettingsRef().max_threads = 1; - ASTPtr select_query = prepareQueryAffectedAST(); + ASTPtr select_query = prepareQueryAffectedAST(commands); /// Interpreter must be alive, when we use result of execute() method. /// For some reason it may copy context and and give it into ExpressionBlockInputStream @@ -139,8 +159,23 @@ bool MutationsInterpreter::isStorageTouchedByMutations() const auto count = (*block.getByName("count()").column)[0].get(); return count != 0; + } +MutationsInterpreter::MutationsInterpreter( + StoragePtr storage_, + std::vector commands_, + const Context & context_, + bool can_execute_) + : storage(std::move(storage_)) + , commands(std::move(commands_)) + , context(context_) + , can_execute(can_execute_) +{ + mutation_ast = prepare(!can_execute); + auto limits = SelectQueryOptions().analyze(!can_execute).ignoreLimits(); + select_interpreter = std::make_unique(mutation_ast, context, storage, limits); +} static NameSet getKeyColumns(const StoragePtr & storage) { @@ -563,38 +598,6 @@ const Block & MutationsInterpreter::getUpdatedHeader() const return *updated_header; } -ASTPtr MutationsInterpreter::prepareQueryAffectedAST() const -{ - /// Execute `SELECT count() FROM storage WHERE predicate1 OR predicate2 OR ...` query. - /// The result can differ from tne number of affected rows (e.g. if there is an UPDATE command that - /// changes how many rows satisfy the predicates of the subsequent commands). - /// But we can be sure that if count = 0, then no rows will be touched. - - auto select = std::make_shared(); - - select->setExpression(ASTSelectQuery::Expression::SELECT, std::make_shared()); - auto count_func = std::make_shared(); - count_func->name = "count"; - count_func->arguments = std::make_shared(); - select->select()->children.push_back(count_func); - - if (commands.size() == 1) - select->setExpression(ASTSelectQuery::Expression::WHERE, commands[0].predicate->clone()); - else - { - auto coalesced_predicates = std::make_shared(); - coalesced_predicates->name = "or"; - coalesced_predicates->arguments = std::make_shared(); - coalesced_predicates->children.push_back(coalesced_predicates->arguments); - - for (const MutationCommand & command : commands) - coalesced_predicates->arguments->children.push_back(command.predicate->clone()); - - select->setExpression(ASTSelectQuery::Expression::WHERE, std::move(coalesced_predicates)); - } - - return select; -} size_t MutationsInterpreter::evaluateCommandsSize() { @@ -602,7 +605,7 @@ size_t MutationsInterpreter::evaluateCommandsSize() if (unlikely(!command.predicate)) /// The command touches all rows. return mutation_ast->size(); - return std::max(prepareQueryAffectedAST()->size(), mutation_ast->size()); + return std::max(prepareQueryAffectedAST(commands)->size(), mutation_ast->size()); } } diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index ff9a8ddccc3..a8322e7c611 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -13,6 +13,9 @@ namespace DB class Context; +/// Return false if the data isn't going to be changed by mutations. +bool isStorageTouchedByMutations(StoragePtr storage, const std::vector & commands, Context context_copy); + /// Create an input stream that will read data from storage and apply mutation commands (UPDATEs, DELETEs, MATERIALIZEs) /// to this data. class MutationsInterpreter @@ -26,8 +29,6 @@ public: size_t evaluateCommandsSize(); - /// Return false if the data isn't going to be changed by mutations. - bool isStorageTouchedByMutations() const; /// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices. BlockInputStreamPtr execute(TableStructureReadLockHolder & table_lock_holder); @@ -40,7 +41,6 @@ private: struct Stage; - ASTPtr prepareQueryAffectedAST() const; ASTPtr prepareInterpreterSelectQuery(std::vector &prepared_stages, bool dry_run); BlockInputStreamPtr addStreamsForLaterStages(const std::vector & prepared_stages, BlockInputStreamPtr in) const; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp index 47a796fbb3c..e1b9ee656a3 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMergerMutator.cpp @@ -943,9 +943,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor command.partition, context_for_reading); }); - MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading, true); - if (!mutations_interpreter.isStorageTouchedByMutations()) + if (!isStorageTouchedByMutations(storage_from_source_part, commands_for_part, context_for_reading)) { LOG_TRACE(log, "Part " << source_part->name << " doesn't change up to mutation version " << future_part.part_info.mutation); return data.cloneAndLoadDataPartOnSameDisk(source_part, "tmp_clone_", future_part.part_info); @@ -973,6 +972,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMergerMutator::mutatePartToTempor Poco::File(new_part_tmp_path).createDirectories(); + MutationsInterpreter mutations_interpreter(storage_from_source_part, commands_for_part, context_for_reading, true); auto in = mutations_interpreter.execute(table_lock_holder); const auto & updated_header = mutations_interpreter.getUpdatedHeader(); From 982eae7b30866a99f2f02f2c215bc06bbc6c86d5 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 7 Nov 2019 11:49:09 +0300 Subject: [PATCH 53/77] Remove redundant change --- dbms/src/Interpreters/MutationsInterpreter.h | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/MutationsInterpreter.h b/dbms/src/Interpreters/MutationsInterpreter.h index a8322e7c611..a5c1df778cb 100644 --- a/dbms/src/Interpreters/MutationsInterpreter.h +++ b/dbms/src/Interpreters/MutationsInterpreter.h @@ -29,7 +29,6 @@ public: size_t evaluateCommandsSize(); - /// The resulting stream will return blocks containing only changed columns and columns, that we need to recalculate indices. BlockInputStreamPtr execute(TableStructureReadLockHolder & table_lock_holder); From f3c58cdf65c2185327996dc9b89b1364a39af913 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 7 Nov 2019 11:52:22 +0300 Subject: [PATCH 54/77] Fix idiotic bug in database object loading --- dbms/src/Databases/DatabaseOnDisk.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Databases/DatabaseOnDisk.cpp b/dbms/src/Databases/DatabaseOnDisk.cpp index cb5921d74b9..03119743a86 100644 --- a/dbms/src/Databases/DatabaseOnDisk.cpp +++ b/dbms/src/Databases/DatabaseOnDisk.cpp @@ -170,7 +170,7 @@ String getObjectDefinitionFromCreateQuery(const ASTPtr & query) if (!create) { std::ostringstream query_stream; - formatAST(*create, query_stream, true); + formatAST(*query, query_stream, true); throw Exception("Query '" + query_stream.str() + "' is not CREATE query", ErrorCodes::LOGICAL_ERROR); } From ed555c5e521a47893510145d4a71832d78367c17 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 7 Nov 2019 12:35:37 +0300 Subject: [PATCH 55/77] Disable processorsw by default. --- dbms/src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index 6f106044d10..fb3021275be 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -360,7 +360,7 @@ struct Settings : public SettingsCollection M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \ \ - M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \ + M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \ \ M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \ M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \ From 58cd0eccac3ffb6e0c714f4ba2fe9199e7f6e6e4 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 7 Nov 2019 15:24:42 +0300 Subject: [PATCH 56/77] DOCS-439: RU review. EN translation. Data storage policies. (#7597) * CLICKHOUSEDOCS-439: RU review. EN translation. * Update docs/en/operations/table_engines/mergetree.md Co-Authored-By: Ivan Blinkov * Update docs/en/operations/table_engines/mergetree.md Co-Authored-By: Ivan Blinkov * Update docs/en/operations/table_engines/mergetree.md Co-Authored-By: Ivan Blinkov * Update docs/en/query_language/alter.md Co-Authored-By: Ivan Blinkov * CLICKHOUSEDOCS-439: The RU version is syncronized with EN. --- docs/en/operations/system_tables.md | 36 +++++- docs/en/operations/table_engines/mergetree.md | 104 ++++++++++++------ docs/en/query_language/alter.md | 26 ++++- docs/ru/operations/system_tables.md | 75 +++++++------ docs/ru/operations/table_engines/mergetree.md | 104 ++++++++++++------ docs/ru/query_language/alter.md | 13 +-- 6 files changed, 241 insertions(+), 117 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index 18f046688f1..c234ae0b464 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -338,6 +338,7 @@ Columns: - `table` (`String`) – Name of the table. - `engine` (`String`) – Name of the table engine without parameters. - `path` (`String`) – Absolute path to the folder with data part files. +- `disk` (`String`) – Name of a disk that stores the data part. - `hash_of_all_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of compressed files. - `hash_of_uncompressed_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of uncompressed files (files with marks, index file etc.). - `uncompressed_hash_of_compressed_files` (`String`) – [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) of data in the compressed files as if they were uncompressed. @@ -354,11 +355,12 @@ This table contains information about events that occurred with [data parts](tab The `system.part_log` table contains the following columns: - `event_type` (Enum) — Type of the event that occurred with the data part. Can have one of the following values: - - `NEW_PART` — inserting - - `MERGE_PARTS` — merging - - `DOWNLOAD_PART` — downloading - - `REMOVE_PART` — removing or detaching using [DETACH PARTITION](../query_language/alter.md#alter_detach-partition) - - `MUTATE_PART` — updating. + - `NEW_PART` — Inserting of a new data part. + - `MERGE_PARTS` — Merging of data parts. + - `DOWNLOAD_PART` — Downloading a data part. + - `REMOVE_PART` — Removing or detaching a data part using [DETACH PARTITION](../query_language/alter.md#alter_detach-partition). + - `MUTATE_PART` — Mutating of a data part. + - `MOVE_PART` — Moving the data part from the one disk to another one. - `event_date` (Date) — Event date. - `event_time` (DateTime) — Event time. - `duration_ms` (UInt64) — Duration. @@ -761,6 +763,30 @@ If there were problems with mutating some parts, the following columns contain a ## system.disks {#system_tables-disks} +Contains information about disks defined in the [server configuration](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure). + +Columns: + +- `name` ([String](../data_types/string.md)) — Name of a disk in the server configuration. +- `path` ([String](../data_types/string.md)) — Path to the mount point in the file system. +- `free_space` ([UInt64](../data_types/int_uint.md)) — Free space on disk in bytes. +- `total_space` ([UInt64](../data_types/int_uint.md)) — Disk volume in bytes. +- `keep_free_space` ([UInt64](../data_types/int_uint.md)) — Amount of disk space that should stay free on disk in bytes. Defined in the `keep_free_space_bytes` parameter of disk configuration. + + ## system.storage_policies {#system_tables-storage_policies} +Contains information about storage policies and volumes defined in the [server configuration](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure). + +Columns: + +- `policy_name` ([String](../data_types/string.md)) — Name of the storage policy. +- `volume_name` ([String](../data_types/string.md)) — Volume name defined in the storage policy. +- `volume_priority` ([UInt64](../data_types/int_uint.md)) — Volume order number in the configuration. +- `disks` ([Array(String)](../data_types/array.md)) — Disk names, defined in the storage policy. +- `max_data_part_size` ([UInt64](../data_types/int_uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). +- `move_factor` ([Float64](..data_types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. + +If the storage policy contains more then one volume, then information for each volume is stored in the individual row of the table. + [Original article](https://clickhouse.yandex/docs/en/operations/system_tables/) diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index 465c883d684..014460981f9 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -85,7 +85,9 @@ For a description of parameters, see the [CREATE query description](../../query_ - `min_merge_bytes_to_use_direct_io` — The minimum data volume for merge operation that is required for using direct I/O access to the storage disk. When merging data parts, ClickHouse calculates the total storage volume of all the data to be merged. If the volume exceeds `min_merge_bytes_to_use_direct_io` bytes, ClickHouse reads and writes the data to the storage disk using the direct I/O interface (`O_DIRECT` option). If `min_merge_bytes_to_use_direct_io = 0`, then direct I/O is disabled. Default value: `10 * 1024 * 1024 * 1024` bytes. - `merge_with_ttl_timeout` — Minimum delay in seconds before repeating a merge with TTL. Default value: 86400 (1 day). - - `write_final_mark` — Enables or disables writing the final index mark at the end of the data part. Default value: 1. Don't turn it off. + - `write_final_mark` — Enables or disables writing the final index mark at the end of data part. Default value: 1. Don't turn it off. + - `storage_policy` — Storage policy. See [Using Multiple Block Devices for Data Storage](#table_engine-mergetree-multiple-volumes). + **Example of Sections Setting** @@ -462,53 +464,89 @@ If you perform the `SELECT` query between merges, you may get expired data. To a [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) -## Using multiple block devices for data storage {#table_engine-mergetree-multiple-volumes} +## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} -### General +### Introduction -Tables of the MergeTree family are able to store their data on multiple block devices, which may be useful when, for instance, the data of a certain table are implicitly split into "hot" and "cold". The most recent data is regularly requested but requires only a small amount of space. On the contrary, the fat-tailed historical data is requested rarely. If several disks are available, the "hot" data may be located on fast disks (NVMe SSDs or even in memory), while the "cold" data - on relatively slow ones (HDD). +`MergeTree` family table engines can store data on multiple block devices. For example, it can be useful when the data of a certain table are implicitly split into "hot" and "cold". The most recent data is regularly requested but requires only a small amount of space. On the contrary, the fat-tailed historical data is requested rarely. If several disks are available, the "hot" data may be located on fast disks (for example, NVMe SSDs or in memory), while the "cold" data - on relatively slow ones (for example, HDD). -Part is the minimum movable unit for MergeTree tables. The data belonging to one part are stored on one disk. Parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](../../query_language/alter.md#alter_move-partition) queries. +Data part is the minimum movable unit for `MergeTree`-engine tables. The data belonging to one part are stored on one disk. Data parts can be moved between disks in the background (according to user settings) as well as by means of the [ALTER](../../query_language/alter.md#alter_move-partition) queries. ### Terms -* Disk — a block device mounted to the filesystem. -* Default disk — a disk that contains the path specified in the `` tag in `config.xml`. -* Volume — an ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). -* Storage policy — a number of volumes together with the rules for moving data between them. -The names given to the described entities can be found in the system tables, [system.storage_policies](../system_tables.md#system_tables-storage_policies) and [system.disks](../system_tables.md#system_tables-disks). Storage policy name can be used as a parameter for tables of the MergeTree family. +- Disk — Block device mounted to the filesystem. +- Default disk — Disk that stores the path specified in the [path](../server_settings/settings.md#server_settings-path) server setting. +- Volume — Ordered set of equal disks (similar to [JBOD](https://en.wikipedia.org/wiki/Non-RAID_drive_architectures)). +- Storage policy — Set of volumes and the rules for moving data between them. + +The names given to the described entities can be found in the system tables, [system.storage_policies](../system_tables.md#system_tables-storage_policies) and [system.disks](../system_tables.md#system_tables-disks). To apply one of the configured storage policies for a table, use the `storage_policy` setting of `MergeTree`-engine family tables. ### Configuration {#table_engine-mergetree-multiple-volumes_configure} -Disks, volumes and storage policies should be declared inside the `` tag either in the main file `config.xml` or in a distinct file in the `config.d` directory. This section in a configuration file has the following structure: +Disks, volumes and storage policies should be declared inside the `` tag either in the main file `config.xml` or in a distinct file in the `config.d` directory. + +Configuration structure: ```xml - + /mnt/fast_ssd/clickhouse - - + + /mnt/hdd1/clickhouse 10485760_ - - + + /mnt/hdd2/clickhouse 10485760_ - + ... ``` -where +Tags: -* the disk name is given as a tag name. -* `path` — path under which a server will store data (`data` and `shadow` folders), should be terminated with '/'. -* `keep_free_space_bytes` — the amount of free disk space to be reserved. +- `` — Disk name. Names must be different for all disks. +- `path` — path under which a server will store data (`data` and `shadow` folders), should be terminated with '/'. +- `keep_free_space_bytes` — the amount of free disk space to be reserved. The order of the disk definition is not important. -Storage policies configuration: +Storage policies configuration markup: + +```xml + + + + + disk_name_from_disks_configuration + 1073741824 + + + + + + + 0.2 + + + + + + + +``` + +Tags: + +- `policy_name_N` — Policy name. Policy names must be unique. +- `volume_name_N` — Volume name. Volume names must be unique. +- `disk` — a disk within a volume. +- `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volume's disks. +- `move_factor` — when the amount of available space gets lower than this factor, data automatically start to move on the next volume if any (by default, 0.1). + +Cofiguration examples: ```xml @@ -536,16 +574,9 @@ Storage policies configuration: ``` -where +In given example, the `hdd_in_order` policy implements the [round-robin](https://en.wikipedia.org/wiki/Round-robin_scheduling) approach. Thus this policy defines only one volume (`single`), the data parts are stored on all its disks in circular order. Such policy can be quite useful if there are several similar disks are mounted to the system, but RAID is not configured. Keep in mind that each individual disk drive is not reliable and you might want to compensate it with replication factor of 3 or more. -* volume and storage policy names are given as tag names. -* `disk` — a disk within a volume. -* `max_data_part_size_bytes` — the maximum size of a part that can be stored on any of the volume's disks. -* `move_factor` — when the amount of available space gets lower than this factor, data automatically start to move on the next volume if any (by default, 0.1). - - -In the given example, the `hdd_in_order` policy implements the [round-robin](https://en.wikipedia.org/wiki/Round-robin_scheduling) approach. Since the policy defines only one volume (`single`), the data are stored on all its disks in circular order. Such a policy can be quite useful if there are several similar disks mounted to the system. If there are different disks, the policy `moving_from_ssd_to_hdd` can be used instead. -The volume `hot` consists of an SSD disk (`fast_ssd`), and the maximum size of a part that can be stored on this volume is 1GB. All the parts with the size larger than 1GB will be stored directly on the `cold` volume, which contains an HDD disk `disk1`. +If there are different kinds of disks available in the system, `moving_from_ssd_to_hdd` policy can be used instead. The volume `hot` consists of an SSD disk (`fast_ssd`), and the maximum size of a part that can be stored on this volume is 1GB. All the parts with the size larger than 1GB will be stored directly on the `cold` volume, which contains an HDD disk `disk1`. Also, once the disk `fast_ssd` gets filled by more than 80%, data will be transferred to the `disk1` by a background process. The order of volume enumeration within a storage policy is important. Once a volume is overfilled, data are moved to the next one. The order of disk enumeration is important as well because data are stored on them in turns. @@ -568,12 +599,12 @@ The `default` storage policy implies using only one volume, which consists of on ### Details -In the case of MergeTree tables, data is getting to disk in different ways: +In the case of `MergeTree` tables, data is getting to disk in different ways: -* as a result of an insert (`INSERT` query). -* during background merges and [mutations](../../query_language/alter.md#alter-mutations). -* when downloading from another replica. -* as a result of partition freezing [ALTER TABLE ... FREEZE PARTITION](../../query_language/alter.md#alter_freeze-partition). +- As a result of an insert (`INSERT` query). +- During background merges and [mutations](../../query_language/alter.md#alter-mutations). +- When downloading from another replica. +- As a result of partition freezing [ALTER TABLE ... FREEZE PARTITION](../../query_language/alter.md#alter_freeze-partition). In all these cases except for mutations and partition freezing, a part is stored on a volume and a disk according to the given storage policy: @@ -592,3 +623,4 @@ Moving data does not interfere with data replication. Therefore, different stora After the completion of background merges and mutations, old parts are removed only after a certain amount of time (`old_parts_lifetime`). During this time, they are not moved to other volumes or disks. Therefore, until the parts are finally removed, they are still taken into account for evaluation of the occupied disk space. +[Original article](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) diff --git a/docs/en/query_language/alter.md b/docs/en/query_language/alter.md index a8d4588d858..eee707fd176 100644 --- a/docs/en/query_language/alter.md +++ b/docs/en/query_language/alter.md @@ -194,7 +194,7 @@ The following operations with [partitions](../operations/table_engines/custom_pa - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the specified secondary index in a partition. - [FREEZE PARTITION](#alter_freeze-partition) – Creates a backup of a partition. - [FETCH PARTITION](#alter_fetch-partition) – Downloads a partition from another server. - +- [MOVE PARTITION|PART](#alter_move-partition) – Move partition/data part to another disk or volume. #### DETACH PARTITION {#alter_detach-partition} ```sql @@ -291,7 +291,7 @@ ALTER TABLE table_name FREEZE [PARTITION partition_expr] This query creates a local backup of a specified partition. If the `PARTITION` clause is omitted, the query creates the backup of all partitions at once. -!!! note +!!! note "Note" The entire backup process is performed without stopping the server. Note that for old-styled tables you can specify the prefix of the partition name (for example, '2019') - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). @@ -301,6 +301,9 @@ At the time of execution, for a data snapshot, the query creates hardlinks to a - `/var/lib/clickhouse/` is the working ClickHouse directory specified in the config. - `N` is the incremental number of the backup. +!!! note "Note" + If you use [a set of disks for data storage in a table](../operations/table_engines/mergetree.md#table_engine-mergetree-multiple-volumes), the `shadow/N` directory appears on every disk, storing data parts that matched by the `PARTITION` expression. + The same structure of directories is created inside the backup as inside `/var/lib/clickhouse/`. The query performs 'chmod' for all files, forbidding writing into them. After creating the backup, you can copy the data from `/var/lib/clickhouse/shadow/` to the remote server and then delete it from the local server. Note that the `ALTER t FREEZE PARTITION` query is not replicated. It creates a local backup only on the local server. @@ -357,6 +360,25 @@ Although the query is called `ALTER TABLE`, it does not change the table structu #### MOVE PARTITION|PART {#alter_move-partition} +Moves partitions or data parts to another volume or disk for `MergeTree`-engine tables. See [Using Multiple Block Devices for Data Storage](../operations/table_engines/mergetree.md#table_engine-mergetree-multiple-volumes). + +```sql +ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' +``` + +The `ALTER TABLE t MOVE` query: + +- Not replicated, because different replicas can have different storage policies. +- Returns an error if the specified disk or volume is not configured. Query also returns an error if conditions of data moving, that specified in the storage policy, can't be applied. +- Can return an error in the case, when data to be moved is already moved by a background process, concurrent `ALTER TABLE t MOVE` query or as a result of background data merging. A user shouldn't perform any additional actions in this case. + +Example: + +```sql +ALTER TABLE hits MOVE PART '20190301_14343_16206_438' TO VOLUME 'slow' +ALTER TABLE hits MOVE PARTITION '2019-09-01' TO DISK 'fast_ssd' +``` + #### How To Set Partition Expression {#alter-how-to-specify-part-expr} You can specify the partition expression in `ALTER ... PARTITION` queries in different ways: diff --git a/docs/ru/operations/system_tables.md b/docs/ru/operations/system_tables.md index 8e5531ecdf9..298a0b5ad90 100644 --- a/docs/ru/operations/system_tables.md +++ b/docs/ru/operations/system_tables.md @@ -329,7 +329,8 @@ SELECT * FROM system.metrics LIMIT 10 - `database` (`String`) – имя базы данных. - `table` (`String`) – имя таблицы. - `engine` (`String`) – имя движка таблицы, без параметров. -- `path` (`String`) – абсолютный путь к папке с файлами кусков данных.. +- `path` (`String`) – абсолютный путь к папке с файлами кусков данных. +- `disk` (`String`) – имя диска, на котором находится кусок данных. - `hash_of_all_files` (`String`) – значение [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) для сжатых файлов. - `hash_of_uncompressed_files` (`String`) – значение [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) несжатых файлов (файлы с засечками, первичным ключом и пр.) - `uncompressed_hash_of_compressed_files` (`String`) – значение [sipHash128](../query_language/functions/hash_functions.md#hash_functions-siphash128) данных в сжатых файлах как если бы они были разжатыми. @@ -345,21 +346,27 @@ SELECT * FROM system.metrics LIMIT 10 Столбцы: -- `event_type` (Enum) — тип события. Столбец может содержать одно из следующих значений: `NEW_PART` — вставка нового куска; `MERGE_PARTS` — слияние кусков; `DOWNLOAD_PART` — загрузка с реплики; `REMOVE_PART` — удаление или отсоединение из таблицы с помощью [DETACH PARTITION](../query_language/alter.md#alter_detach-partition); `MUTATE_PART` — изменение куска; `MOVE_PART` — перемещение куска между дисками. -- `event_date` (Date) — дата события; -- `event_time` (DateTime) — время события; -- `duration_ms` (UInt64) — длительность; -- `database` (String) — имя базы данных, в которой находится кусок; -- `table` (String) — имя таблицы, в которой находится кусок; -- `part_name` (String) — имя куска; -- `partition_id` (String) — идентификатор партиции, в которую был добавлен кусок. В столбце будет значение 'all', если таблица партициируется по выражению `tuple()`; -- `rows` (UInt64) — число строк в куске; -- `size_in_bytes` (UInt64) — размер куска данных в байтах; -- `merged_from` (Array(String)) — массив имён кусков, из которых образован текущий кусок в результате слияния (также столбец заполняется в случае скачивания уже смерженного куска); -- `bytes_uncompressed` (UInt64) — количество прочитанных разжатых байт; -- `read_rows` (UInt64) — сколько было прочитано строк при слиянии кусков; -- `read_bytes` (UInt64) — сколько было прочитано байт при слиянии кусков; -- `error` (UInt16) — код ошибки, возникшей при текущем событии; +- `event_type` (Enum) — тип события. Столбец может содержать одно из следующих значений: + - `NEW_PART` — вставка нового куска. + - `MERGE_PARTS` — слияние кусков. + - `DOWNLOAD_PART` — загрузка с реплики. + - `REMOVE_PART` — удаление или отсоединение из таблицы с помощью [DETACH PARTITION](../query_language/alter.md#alter_detach-partition). + - `MUTATE_PART` — изменение куска. + - `MOVE_PART` — перемещение куска между дисками. +- `event_date` (Date) — дата события. +- `event_time` (DateTime) — время события. +- `duration_ms` (UInt64) — длительность. +- `database` (String) — имя базы данных, в которой находится кусок. +- `table` (String) — имя таблицы, в которой находится кусок. +- `part_name` (String) — имя куска. +- `partition_id` (String) — идентификатор партиции, в которую был добавлен кусок. В столбце будет значение 'all', если таблица партициируется по выражению `tuple()`. +- `rows` (UInt64) — число строк в куске. +- `size_in_bytes` (UInt64) — размер куска данных в байтах. +- `merged_from` (Array(String)) — массив имён кусков, из которых образован текущий кусок в результате слияния (также столбец заполняется в случае скачивания уже смерженного куска). +- `bytes_uncompressed` (UInt64) — количество прочитанных разжатых байт. +- `read_rows` (UInt64) — сколько было прочитано строк при слиянии кусков. +- `read_bytes` (UInt64) — сколько было прочитано байт при слиянии кусков. +- `error` (UInt16) — код ошибки, возникшей при текущем событии. - `exception` (String) — текст ошибки. Системная таблица `system.part_log` будет создана после первой вставки данных в таблицу `MergeTree`. @@ -747,26 +754,32 @@ path: /clickhouse/tables/01-08/visits/replicas **latest_fail_reason** — причина последней ошибки мутации. -[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/system_tables/) - ## system.disks {#system_tables-disks} -Таблица содержит информацию о дисках, заданных в [конфигурации сервера](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure). Имеет следующие столбцы: +Cодержит информацию о дисках, заданных в [конфигурации сервера](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure). -- `name String` — имя диска в конфигурации сервера. -- `path String` — путь к точке монтирования на файловой системе. -- `free_space UInt64` — свободное место на диске в данный момент времени в байтах. -- `total_space UInt64` — общее количество места на диске в данный момент времени в байтах. -- `keep_free_space UInt64` — количество байт, которое должно оставаться свободным (задается в конфигурации). +Столбцы: + +- `name` ([String](../data_types/string.md)) — имя диска в конфигурации сервера. +- `path` ([String](../data_types/string.md)) — путь к точке монтирования в файловой системе. +- `free_space` ([UInt64](../data_types/int_uint.md)) — свободное место на диске в байтах. +- `total_space` ([UInt64](../data_types/int_uint.md)) — объём диска в байтах. +- `keep_free_space` ([UInt64](../data_types/int_uint.md)) — место, которое должно остаться свободным на диске в байтах. Задаётся значением параметра `keep_free_space_bytes` конфигурации дисков. ## system.storage_policies {#system_tables-storage_policies} -Таблица содержит информацию о политиках хранения и томах, заданных в [конфигурации сервера](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure). Данные в таблице денормализованны, имя одной политики хранения может содержаться несколько раз, по количеству томов в ней. Имеет следующие столбцы: +Содержит информацию о политиках хранения и томах, заданных в [конфигурации сервера](table_engines/mergetree.md#table_engine-mergetree-multiple-volumes_configure). -- `policy_name String` — имя политики хранения в конфигурации сервера. -- `volume_name String` — имя тома, который содержится в данной политике хранения. -- `volume_priority UInt64` — порядковый номер тома, согласно конфигурации. -- `disks Array(String)` — имена дисков, содержащихся в данной политике хранения. -- `max_data_part_size UInt64` — максимальный размер куска, который может храниться на дисках этого тома (0 — без ограничений). -- `move_factor Float64` — доля свободного места, при превышении которой данные начинают перемещаться на следующий том. +Столбцы: + +- `policy_name` ([String](../data_types/string.md)) — имя политики хранения. +- `volume_name` ([String](../data_types/string.md)) — имя тома, который содержится в политике хранения. +- `volume_priority` ([UInt64](../data_types/int_uint.md)) — порядковый номер тома согласно конфигурации. +- `disks` ([Array(String)](../data_types/array.md)) — имена дисков, содержащихся в политике хранения. +- `max_data_part_size` ([UInt64](../data_types/int_uint.md)) — максимальный размер куска данных, который может храниться на дисках тома (0 — без ограничений). +- `move_factor` ([Float64](..data_types/float.md))` — доля свободного места, при превышении которой данные начинают перемещаться на следующий том. + +Если политика хранения содержит несколько томов, то каждому тому соответствует отдельная запись в таблице. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/system_tables/) diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index 1de59ad0449..303cd641038 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -86,7 +86,8 @@ CREATE TABLE [IF NOT EXISTS] [db.]table_name [ON CLUSTER cluster] - `merge_with_ttl_timeout` — минимальное время в секундах перед повторным слиянием с TTL. По умолчанию — 86400 (1 день). - `write_final_mark` — включает или отключает запись последней засечки индекса в конце куска данных. По умолчанию — 1. Не отключайте её. - + - `storage_policy` — политика хранения данных. Смотрите [Хранение данных таблицы на нескольких блочных устройствах](#table_engine-mergetree-multiple-volumes). + **Пример задания секций** ```sql @@ -450,55 +451,89 @@ ALTER TABLE example_table Если вы выполните запрос `SELECT` между слияниями вы можете получить устаревшие данные. Чтобы избежать этого используйте запрос [OPTIMIZE](../../query_language/misc.md#misc_operations-optimize) перед `SELECT`. -[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) - - ## Хранение данных таблицы на нескольких блочных устройствах {#table_engine-mergetree-multiple-volumes} -### Общее -Данные таблиц семейства MergeTree могут храниться на нескольких блочных устройствах. Это может оказаться полезным, например, при неявном разделении данных одной таблицы на "горячие" и "холодные", когда наиболее свежая часть занимает малый объем и запрашивается регулярно, а большой хвост исторических данных запрашивается редко. При наличии в системе нескольких дисков, "горячая" часть данных может быть размещена на быстрых дисках (NVMe SSDs или даже в памяти), а холодная на более медленных (HDD). +### Введение -Минимальной перемещаемой единицей для MergeTree является кусок (part). Данные одного куска могут находится только на одном диске. Куски могут перемещаться между дисками в фоне, согласно пользовательским настройкам, а также с помощью [ALTER](../../query_language/alter.md#alter_move-partition) запросов. +Движки таблиц семейства `MergeTree` могут хранить данные на нескольких блочных устройствах. Это может оказаться полезным, например, при неявном разделении данных одной таблицы на "горячие" и "холодные". Наиболее свежая часть занимает малый объем и запрашивается регулярно, а большой хвост исторических данных запрашивается редко. При наличии в системе нескольких дисков, "горячая" часть данных может быть размещена на быстрых дисках (например, на NVMe SSD или в памяти), а холодная на более медленных (например, HDD). + +Минимальной перемещаемой единицей для `MergeTree` является кусок данных (data part). Данные одного куска могут находится только на одном диске. Куски могут перемещаться между дисками в фоне, согласно пользовательским настройкам, а также с помощью запросов [ALTER](../../query_language/alter.md# -partition). ### Термины -* Диск — примонтированное в файловой системе блочное устройство. -* Диск по умолчанию — диск, на котором находится путь, указанный в корне `config.xml` в теге ``. -* Том (Volume) — упорядоченный набор равноценных дисков (схоже с [JBOD](https://ru.wikipedia.org/wiki/JBOD)) -* Политика хранения (StoragePolicy) — множество томов с правилами перемещения данных между ними. -У всех описанных сущностей, при создании, указываются имена, которые будут отражены в системных таблицах [system.storage_policies](../system_tables.md#system_tables-storage_policies) и [system.disks](../system_tables.md#system_tables-disks). Имя политики хранения используется как настройка у таблиц семейства MergeTree. +- Диск — примонтированное в файловой системе блочное устройство. +- Диск по умолчанию — диск, на котором находится путь, указанный в конфигурационной настройке сервера [path](../server_settings/settings.md#server_settings-path). +- Том (Volume) — упорядоченный набор равноценных дисков (схоже с [JBOD](https://ru.wikipedia.org/wiki/JBOD)) +- Политика хранения (StoragePolicy) — множество томов с правилами перемещения данных между ними. + +У всех описанных сущностей при создании указываются имена, можно найти в системных таблицах [system.storage_policies](../system_tables.md#system_tables-storage_policies) и [system.disks](../system_tables.md#system_tables-disks). Имя политики хранения можно указать в настройке `storage_policy` движков таблиц семейства `MergeTree`. ### Конфигурация {#table_engine-mergetree-multiple-volumes_configure} -Диски, тома и политики хранения задаются внутри тега `` в основном файле `config.xml` или в отдельном файле в директории `config.d`. Правила составления данной секции конфигурации имеет следующую структуру: +Диски, тома и политики хранения задаются внутри тега `` в основном файле `config.xml` или в отдельном файле в директории `config.d`. + +Структура конфигурации: ```xml - + /mnt/fast_ssd/clickhouse - - + + /mnt/hdd1/clickhouse 10485760_ - - + + /mnt/hdd2/clickhouse 10485760_ - + ... ``` -Где, +Теги: -* имя диска задается внутри имени тега. -* `path` — путь по которому будут храниться данные сервера (каталоги `data` и `shadow`), должен быть терминирован `/`. -* `keep_free_space_bytes` — размер зарезервированного свободного места на диске. +- `` — имя диска. Имена должны быть разными для всех дисков. +- `path` — путь по которому будут храниться данные сервера (каталоги `data` и `shadow`), должен быть терминирован `/`. +- `keep_free_space_bytes` — размер зарезервированного свободного места на диске. Порядок задания дисков не имеет значения. -Конфигурация политик хранения: +Общий вид конфигурации политик хранения: + +```xml + + + + + disk_name_from_disks_configuration + 1073741824 + + + + + + + 0.2 + + + + + + + +``` + +Тэги: + +- `policy_name_N` — название политики. Названия политик должны быть уникальны. +- `volume_name_N` — название тома. Названия томов должны быть уникальны. +- `disk` — диск, находящийся внутри тома. +- `max_data_part_size_bytes` — максимальный размер куска данных, который может находится на любом из дисков этого тома. +- `move_factor` — доля свободного места, при превышении которого данные начинают перемещаться на следующий том, если он есть (по умолчанию 0.1). + +Примеры конфигураций: ```xml @@ -526,14 +561,9 @@ ALTER TABLE example_table ``` -Где, +В приведенном примере, политика `hdd_in_order` реализует прицип [round-robin](https://ru.wikipedia.org/wiki/Round-robin_(%D0%B0%D0%BB%D0%B3%D0%BE%D1%80%D0%B8%D1%82%D0%BC)). Так как в политике есть всего один том (`single`), то все записи производятся на его диски по круговому циклу. Такая политика может быть полезна при наличии в системе нескольких похожих дисков, но при этом не сконфигурирован RAID. Учтите, что каждый отдельный диск ненадёжен и чтобы не потерять важные данные это необходимо скомпенсировать за счет хранения данных в трёх копиях. -* имя политики и тома задаются внутри имен тегов. -* `disk` — диск, находящийся внутри тома. -* `max_data_part_size_bytes` — максимальный размер куска, который может находится на любом из дисков этого тома. -* `move_factor` — доля свободного места, при превышении которого данные начинают перемещаться на следующий том, если он есть (по умолчанию 0.1). - -В приведенном примере, политика `hdd_in_order` реализует прицип [round-robin](https://ru.wikipedia.org/wiki/Round-robin_(%D0%B0%D0%BB%D0%B3%D0%BE%D1%80%D0%B8%D1%82%D0%BC)). Так как в политике есть всего 1 том (`single`) все записи производятся на его диски по круговому циклу. Такая политика может быть полезна при наличии в системе нескольких похожих дисков. Политика `moving_from_ssd_to_hdd` полезна при наличии в разных типов дисков. В томе `hot` находится один SSD-диск (`fast_ssd`), а также задается ограничение на максимальный размер куска, который может храниться на этом томе (1GB). Все куски такой таблицы больше 1GB будут записываться сразу на том `cold`, в котором содержится один HDD-диск `disk1`. Также, при заполнении диска `fast_ssd` более чем на 80% данные будут переносится на диск `disk1` фоновым процессом. +Если система содержит диски различных типов, то может пригодиться политика `moving_from_ssd_to_hdd`. В томе `hot` находится один SSD-диск (`fast_ssd`), а также задается ограничение на максимальный размер куска, который может храниться на этом томе (1GB). Все куски такой таблицы больше 1GB будут записываться сразу на том `cold`, в котором содержится один HDD-диск `disk1`. Также, при заполнении диска `fast_ssd` более чем на 80% данные будут переносится на диск `disk1` фоновым процессом. Порядок томов в политиках хранения важен, при достижении условий на переполнение тома данные переносятся на следующий. Порядок дисков в томах так же важен, данные пишутся по очереди на каждый из них. @@ -555,12 +585,12 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' ### Особенности работы -В таблицах MergeTree данные попадают на диск несколькими способами: +В таблицах `MergeTree` данные попадают на диск несколькими способами: -* В результате вставки (запрос `INSERT`). -* В фоновых операциях слияний и [мутаций](../../query_language/alter.md#alter-mutations). -* При скачивании данных с другой реплики. -* В результате заморозки партиций [ALTER TABLE ... FREEZE PARTITION](../../query_language/alter.md#alter_freeze-partition). +- В результате вставки (запрос `INSERT`). +- В фоновых операциях слияний и [мутаций](../../query_language/alter.md#alter-mutations). +- При скачивании данных с другой реплики. +- В результате заморозки партиций [ALTER TABLE ... FREEZE PARTITION](../../query_language/alter.md#alter_freeze-partition). Во всех случаях, кроме мутаций и заморозки партиций, при записи куска выбирается том и диск в соответствии с указанной конфигурацией хранилища: @@ -576,3 +606,5 @@ SETTINGS storage_policy = 'moving_from_ssd_to_hdd' Перемещения данных не взаимодействуют с репликацией данных, поэтому на разных репликах одной и той же таблицы могут быть указаны разные политики хранения. После выполнения фоновых слияний или мутаций старые куски не удаляются сразу, а через некоторое время (табличная настройка `old_parts_lifetime`). Также они не перемещаются на другие тома или диски, поэтому до момента удаления они продолжают учитываться при подсчёте занятого дискового пространства. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/) diff --git a/docs/ru/query_language/alter.md b/docs/ru/query_language/alter.md index bb0f6530b63..b4f1622e2dd 100644 --- a/docs/ru/query_language/alter.md +++ b/docs/ru/query_language/alter.md @@ -361,17 +361,16 @@ ALTER TABLE users ATTACH PARTITION 201902; #### MOVE PARTITION|PART {#alter_move-partition} +Перемещает партицию или кусок данных на другой том или диск для таблиц с движком `MergeTree`. Смотрите [Хранение данных таблицы на нескольких блочных устройствах](../operations/table_engines/mergetree.md#table_engine-mergetree-multiple-volumes). + ```sql ALTER TABLE table_name MOVE PARTITION|PART partition_expr TO DISK|VOLUME 'disk_name' ``` +Запрос `ALTER TABLE t MOVE`: -Перемещает партицию или кусок на другой том или диск. Запрос работает только для движков семейства MergeTree. Подробнее о хранении данных на разных дисках читайте в разделе [Хранение данных таблицы на нескольких блочных устройствах](../operations/table_engines/mergetree.md#table_engine-mergetree-multiple-volumes). - -Следует иметь ввиду: - -- Запрос `ALTER TABLE t MOVE` не реплицируется, т.к. на разных репликах могут быть различные конфигурации политик хранения. -- Запрос `ALTER TABLE t MOVE` будет возвращать ошибку, если указан несуществующий том или диск, а также в случае невыполнения условий перемещения данных, которые указаны в конфигурации политики хранения. -- Запрос `ALTER TABLE t MOVE` может возвращать ошибку в случае, когда перемещаемые данные уже оказались перемещены в результате фонового процесса, конкурентного запроса `ALTER TABLE t MOVE` или как часть результата фоновой операции слияния. В данном случае никаких дополнительных действий от пользователя не требуется. +- Не реплицируется, т.к. на разных репликах могут быть различные конфигурации политик хранения. +- Возвращает ошибку, если указан несконфигурированный том или диск. Ошибка также возвращается в случае невыполнения условий перемещения данных, которые указаны в конфигурации политики хранения. +- Может возвращать ошибку в случае, когда перемещаемые данные уже оказались перемещены в результате фонового процесса, конкурентного запроса `ALTER TABLE t MOVE` или как часть результата фоновой операции слияния. В данном случае никаких дополнительных действий от пользователя не требуется. Примеры: From e9b9fcd7d2749016807ae9cd67efb1c522032fc7 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Thu, 7 Nov 2019 17:08:56 +0300 Subject: [PATCH 57/77] DOCS-7719: EN review, RU translation. Columns matcher (#7373) * Typo fix. * Links fix. * Fixed links in docs. * More fixes. * Link fixes. * Update select.md (#63) * DOCAPI-7719: EN review. RU translation. * DOCAPI-7719: Fixes. * Update docs/ru/query_language/select.md Co-Authored-By: Ivan Blinkov * Update docs/en/query_language/select.md Co-Authored-By: Ivan Blinkov * DOCAPI-7719: Clarifications. * Update of the SELECT clause description. --- docs/en/query_language/select.md | 16 +++--- docs/ru/operations/table_engines/mysql.md | 4 +- docs/ru/query_language/select.md | 64 +++++++++++++++++++++-- 3 files changed, 70 insertions(+), 14 deletions(-) diff --git a/docs/en/query_language/select.md b/docs/en/query_language/select.md index 8ffc4b8efdc..10569ee801c 100644 --- a/docs/en/query_language/select.md +++ b/docs/en/query_language/select.md @@ -964,11 +964,11 @@ External sorting works much less effectively than sorting in RAM. ### SELECT Clause {#select-select} -[Expressions](syntax.md#syntax-expressions) that specified in the `SELECT` clause are analyzed after the calculations for all the clauses listed above are completed. More specifically, expressions are analyzed that are above the aggregate functions, if there are any aggregate functions. The aggregate functions and everything below them are calculated during aggregation (`GROUP BY`). These expressions work as if they are applied to separate rows in the result. +[Expressions](syntax.md#syntax-expressions) specified in the `SELECT` clause are calculated after all the operations in the clauses described above are finished. These expressions work as if they apply to separate rows in the result. If expressions in the `SELECT` clause contain aggregate functions, then ClickHouse processes aggregate functions and expressions used as their arguments during the [GROUP BY](#select-group-by-clause) aggregation. -If you want to get all columns in the result, use the asterisk (`*`) symbol. For example, `SELECT * FROM ...`. +If you want to include all columns in the result, use the asterisk (`*`) symbol. For example, `SELECT * FROM ...`. -To match some columns in the result by a [re2](https://en.wikipedia.org/wiki/RE2_(software)) regular expression, you can use the `COLUMNS` expression. +To match some columns in the result with a [re2](https://en.wikipedia.org/wiki/RE2_(software)) regular expression, you can use the `COLUMNS` expression. ```sql COLUMNS('regexp') @@ -991,7 +991,9 @@ SELECT COLUMNS('a') FROM col_names └────┴────┘ ``` -You can use multiple `COLUMNS` expressions in a query, also you can apply functions to it. +The selected columns are returned not in the alphabetical order. + +You can use multiple `COLUMNS` expressions in a query and apply functions to them. For example: @@ -1004,7 +1006,7 @@ SELECT COLUMNS('a'), COLUMNS('c'), toTypeName(COLUMNS('c')) FROM col_names └────┴────┴────┴────────────────┘ ``` -Be careful when using functions because the `COLUMN` expression returns variable number of columns, and, if a function doesn't support this number of arguments, ClickHouse throws an exception. +Each column returned by the `COLUMNS` expression is passed to the function as a separate argument. Also you can pass other arguments to the function if it supports them. Be careful when using functions. If a function doesn't support the number of arguments you have passed to it, ClickHouse throws an exception. For example: @@ -1016,9 +1018,9 @@ Received exception from server (version 19.14.1): Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of arguments for function plus doesn't match: passed 3, should be 2. ``` -In this example, `COLUMNS('a')` returns two columns `aa`, `ab`, and `COLUMNS('c')` returns the `bc` column. The `+` operator can't apply to 3 arguments, so ClickHouse throws an exception with the message about it. +In this example, `COLUMNS('a')` returns two columns: `aa` and `ab`. `COLUMNS('c')` returns the `bc` column. The `+` operator can't apply to 3 arguments, so ClickHouse throws an exception with the relevant message. -Columns that matched by the `COLUMNS` expression can be in different types. If `COLUMNS` doesn't match any columns and it is the single expression in `SELECT`, ClickHouse throws an exception. +Columns that matched the `COLUMNS` expression can have different data types. If `COLUMNS` doesn't match any columns and is the only expression in `SELECT`, ClickHouse throws an exception. ### DISTINCT Clause {#select-distinct} diff --git a/docs/ru/operations/table_engines/mysql.md b/docs/ru/operations/table_engines/mysql.md index efd11d7e09b..ef7dd5b9d5d 100644 --- a/docs/ru/operations/table_engines/mysql.md +++ b/docs/ru/operations/table_engines/mysql.md @@ -63,7 +63,7 @@ mysql> select * from test; 1 row in set (0,00 sec) ``` -Таблица в ClickHouse, которая получает данные из таблицы MySQL: +Таблица в ClickHouse, которая получает данные из созданной ранее таблицы MySQL: ```sql CREATE TABLE mysql_table @@ -75,7 +75,7 @@ ENGINE = MySQL('localhost:3306', 'test', 'test', 'bayonet', '123') ``` ```sql -SELECT * FROM mysql_table6 +SELECT * FROM mysql_table ``` ```text diff --git a/docs/ru/query_language/select.md b/docs/ru/query_language/select.md index 49e2c0692ef..56549f21e53 100644 --- a/docs/ru/query_language/select.md +++ b/docs/ru/query_language/select.md @@ -984,12 +984,66 @@ WHERE и HAVING отличаются тем, что WHERE выполняется Внешняя сортировка работает существенно менее эффективно, чем сортировка в оперативке. -### Секция SELECT +### Секция SELECT {#select-select} -После вычислений, соответствующих всем перечисленным выше секциям, производится вычисление выражений, указанных в секции SELECT. -Вернее, вычисляются выражения, стоящие над агрегатными функциями, если есть агрегатные функции. -Сами агрегатные функции и то, что под ними, вычисляются при агрегации (GROUP BY). -Эти выражения работают так, как будто применяются к отдельным строкам результата. +[Выражения](syntax.md#syntax-expressions) указанные в секции `SELECT` анализируются после завершения всех вычислений из секций, описанных выше. Вернее, анализируются выражения, стоящие над агрегатными функциями, если есть агрегатные функции. +Сами агрегатные функции и то, что под ними, вычисляются при агрегации (`GROUP BY`). Эти выражения работают так, как будто применяются к отдельным строкам результата. + +Если в результат необходимо включить все столбцы, используйте символ звёздочка (`*`). Например, `SELECT * FROM ...`. + +Чтобы включить в результат несколько столбцов, выбрав их имена с помощью регулярных выражений [re2](https://en.wikipedia.org/wiki/RE2_(software)), используйте выражение `COLUMNS`. + +```sql +COLUMNS('regexp') +``` + +Например, рассмотрим таблицу: + +```sql +CREATE TABLE default.col_names (aa Int8, ab Int8, bc Int8) ENGINE = TinyLog +``` + +Следующий запрос выбирает данные из всех столбцов, содержащих в имени символ `a`. + +```sql +SELECT COLUMNS('a') FROM col_names +``` +```text +┌─aa─┬─ab─┐ +│ 1 │ 1 │ +└────┴────┘ +``` + +Выбранные стоблцы возвращаются не в алфавитном порядке. + +В запросе можно использовать несколько выражений `COLUMNS`, а также вызывать над ними функции. + +Например: + +```sql +SELECT COLUMNS('a'), COLUMNS('c'), toTypeName(COLUMNS('c')) FROM col_names +``` +```text +┌─aa─┬─ab─┬─bc─┬─toTypeName(bc)─┐ +│ 1 │ 1 │ 1 │ Int8 │ +└────┴────┴────┴────────────────┘ +``` + +Каждый столбец, возвращённый выражением `COLUMNS`, передаётся в функцию отдельным аргументом. Также можно передавать и другие аргументы, если функция их поддерживаем. Аккуратно используйте функции. Если функция не поддерживает переданное количество аргументов, то ClickHouse генерирует исключение. + +Например: + +```sql +SELECT COLUMNS('a') + COLUMNS('c') FROM col_names +``` +```text +Received exception from server (version 19.14.1): +Code: 42. DB::Exception: Received from localhost:9000. DB::Exception: Number of arguments for function plus doesn't match: passed 3, should be 2. +``` + +В этом примере, `COLUMNS('a')` возвращает два столбца: `aa` и `ab`. `COLUMNS('c')` возвращает столбец `bc`. Оператор `+` не работает с тремя аргументами, поэтому ClickHouse генерирует исключение с соответствущим сообщением. + +Столбцы, которые возвращаются выражением `COLUMNS` могут быть разных типов. Если `COLUMNS` не возвращает ни одного столбца и это единственное выражение в запросе `SELECT`, то ClickHouse генерирует исключение. ### Секция DISTINCT {#select-distinct} From 45a0243afd765a827277770a6f8974d3f2186246 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 7 Nov 2019 18:17:10 +0300 Subject: [PATCH 58/77] Add test for HTTPS replication failover --- .../test_https_replication/test.py | 32 +++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/dbms/tests/integration/test_https_replication/test.py b/dbms/tests/integration/test_https_replication/test.py index d9f9b1f23f4..a34c5faeccc 100644 --- a/dbms/tests/integration/test_https_replication/test.py +++ b/dbms/tests/integration/test_https_replication/test.py @@ -4,6 +4,9 @@ import pytest from helpers.cluster import ClickHouseCluster from helpers.test_tools import assert_eq_with_retry +from helpers.network import PartitionManager +from multiprocessing.dummy import Pool +import random """ Both ssl_conf.xml and no_ssl_conf.xml have the same port @@ -46,6 +49,35 @@ def test_both_https(both_https_cluster): assert_eq_with_retry(node1, "SELECT id FROM test_table order by id", '111\n222') assert_eq_with_retry(node2, "SELECT id FROM test_table order by id", '111\n222') + +def test_replication_after_partition(both_https_cluster): + node1.query("truncate table test_table") + node2.query("truncate table test_table") + + manager = PartitionManager() + + def close(num): + manager.partition_instances(node1, node2, port=9010) + time.sleep(1) + manager.heal_all() + + def insert_data_and_check(num): + node1.query("insert into test_table values('2019-10-15', {}, 888)".format(num)) + time.sleep(0.5) + + closing_pool = Pool(1) + inserting_pool = Pool(5) + cres = closing_pool.map_async(close, [random.randint(1, 3) for _ in range(10)]) + ires = inserting_pool.map_async(insert_data_and_check, range(100)) + + cres.wait() + ires.wait() + + assert_eq_with_retry(node1, "SELECT count() FROM test_table", '100') + assert_eq_with_retry(node2, "SELECT count() FROM test_table", '100') + + + node3 = cluster.add_instance('node3', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) node4 = cluster.add_instance('node4', config_dir="configs", main_configs=['configs/remote_servers.xml', 'configs/no_ssl_conf.xml'], with_zookeeper=True) From 8837155e37d1216e38438edd7fb84d7f2901ab6f Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 8 Nov 2019 09:05:51 +0800 Subject: [PATCH 59/77] String join perf test --- dbms/tests/performance/string_join.xml | 34 ++++++++++++++++++++++++++ 1 file changed, 34 insertions(+) create mode 100644 dbms/tests/performance/string_join.xml diff --git a/dbms/tests/performance/string_join.xml b/dbms/tests/performance/string_join.xml new file mode 100644 index 00000000000..6c0ad83d5b4 --- /dev/null +++ b/dbms/tests/performance/string_join.xml @@ -0,0 +1,34 @@ + + loop + + + + 10 + + + + + + + + + default.hits_10m_single + + + CREATE TABLE hits_10m_words (word String, UserID UInt64) ENGINE Memory + CREATE TABLE strings (short String, long String) ENGINE Memory + + INSERT INTO hits_10m_words SELECT DISTINCT arrayJoin(splitByString(' ', SearchPhrase)) AS word, UserID FROM hits_10m_single WHERE length(word) > 0 + INSERT INTO strings SELECT toString(rand()) a, a || a || a || a || a || a || a || a || a || a || a || a FROM numbers(1000000) + + + 1 + + + SELECT 1 FROM hits_10m_words AS l ANY LEFT JOIN hits_10m_words AS r USING (word) FORMAT Null + SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (short) FORMAT Null + SELECT 1 FROM strings AS l ANY LEFT JOIN strings AS r USING (long) FORMAT Null + + DROP TABLE IF EXISTS hits_10m_words + DROP TABLE IF EXISTS strings + From ecd8512529995a967e42c3aab6fbab54cf8bc70c Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 8 Nov 2019 11:36:07 +0300 Subject: [PATCH 60/77] Auto version update to [19.17.1.1639] [54428] --- dbms/cmake/version.cmake | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index a544274826e..5165c45cfcd 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -3,9 +3,9 @@ set(VERSION_REVISION 54428) set(VERSION_MAJOR 19) set(VERSION_MINOR 17) set(VERSION_PATCH 1) -set(VERSION_GITHASH 5286d0afb285a5fbf3d320af3daa6de6b1841374) -set(VERSION_DESCRIBE v19.17.1.1-prestable) -set(VERSION_STRING 19.17.1.1) +set(VERSION_GITHASH 4e68211879480b637683ae66dbcc89a2714682af) +set(VERSION_DESCRIBE v19.17.1.1639-prestable) +set(VERSION_STRING 19.17.1.1639) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") From 94c7719c73ffaf0af5f9c9a2ef5e131861e8274b Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 8 Nov 2019 11:36:41 +0300 Subject: [PATCH 61/77] Auto version update to [19.18.1.1] [54429] --- dbms/cmake/version.cmake | 8 ++++---- .../System/StorageSystemContributors.generated.cpp | 11 +++++++++++ debian/changelog | 4 ++-- docker/client/Dockerfile | 2 +- docker/server/Dockerfile | 2 +- docker/test/Dockerfile | 2 +- 6 files changed, 20 insertions(+), 9 deletions(-) diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 5165c45cfcd..3afa379fee2 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 54428) +set(VERSION_REVISION 54429) set(VERSION_MAJOR 19) -set(VERSION_MINOR 17) +set(VERSION_MINOR 18) set(VERSION_PATCH 1) set(VERSION_GITHASH 4e68211879480b637683ae66dbcc89a2714682af) -set(VERSION_DESCRIBE v19.17.1.1639-prestable) -set(VERSION_STRING 19.17.1.1639) +set(VERSION_DESCRIBE v19.18.1.1-prestable) +set(VERSION_STRING 19.18.1.1) # end of autochange set(VERSION_EXTRA "" CACHE STRING "") diff --git a/dbms/src/Storages/System/StorageSystemContributors.generated.cpp b/dbms/src/Storages/System/StorageSystemContributors.generated.cpp index 6261ab1c2b5..1a675c9b92e 100644 --- a/dbms/src/Storages/System/StorageSystemContributors.generated.cpp +++ b/dbms/src/Storages/System/StorageSystemContributors.generated.cpp @@ -11,6 +11,7 @@ const char * auto_contributors[] { "Alex Ryndin", "Alex Zatelepin", "Alexander Avdonkin", + "Alexander Burmak", "Alexander Ermolaev", "Alexander GQ Gerasiov", "Alexander Kazakov", @@ -49,6 +50,7 @@ const char * auto_contributors[] { "Andrew Grigorev", "Andrey", "Andrey Dudin", + "Andrey Konyaev", "Andrey M", "Andrey Mironov", "Andrey Urusov", @@ -86,6 +88,7 @@ const char * auto_contributors[] { "Chen Yufei", "Ciprian Hacman", "Clément Rodriguez", + "Colum", "Constantin S. Pan", "CurtizJ", "Daniel Bershatsky", @@ -95,6 +98,7 @@ const char * auto_contributors[] { "DarkWanderer", "Darío", "Denis Burlaka", + "Denis Glazachev", "Denis Zhuravlev", "Derek Perkins", "Dmitry Bilunov", @@ -110,6 +114,7 @@ const char * auto_contributors[] { "Elghazal Ahmed", "Emmanuel Donin de Rosière", "Eric", + "Ernest Poletaev", "Eugene Klimov", "Eugene Konkov", "Evgenii Pravda", @@ -145,6 +150,7 @@ const char * auto_contributors[] { "Ilya", "Ilya Breev", "Ilya Khomutov", + "Ilya Korol", "Ilya Korolev", "Ilya Kovalenko", "Ilya Shipitsin", @@ -341,6 +347,7 @@ const char * auto_contributors[] { "Zhichang Yu", "abdrakhmanov", "abyss7", + "achimbab", "achulkov2", "akazz", "akonyaev", @@ -350,6 +357,7 @@ const char * auto_contributors[] { "alex.lvxin", "alexander kozhikhov", "alexey-milovidov", + "andrei-karpliuk", "andrewsg", "anrodigina", "anton", @@ -411,6 +419,7 @@ const char * auto_contributors[] { "levysh", "liangqian", "linceyou", + "liu-bov", "liuyangkuan", "liuyimin", "lomberts", @@ -419,6 +428,7 @@ const char * auto_contributors[] { "malkfilipp", "maqroll", "maxkuzn", + "memo", "mf5137", "mfridental", "miha-g", @@ -439,6 +449,7 @@ const char * auto_contributors[] { "pyos", "qianlixiang", "quid", + "rainbowsysu", "robot-clickhouse", "robot-metrika-test", "root", diff --git a/debian/changelog b/debian/changelog index b61349fc41a..0ce071ce5e6 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (19.17.1.1) unstable; urgency=low +clickhouse (19.18.1.1) unstable; urgency=low * Modified source code - -- clickhouse-release Mon, 21 Oct 2019 15:47:56 +0300 + -- clickhouse-release Fri, 08 Nov 2019 11:36:37 +0300 diff --git a/docker/client/Dockerfile b/docker/client/Dockerfile index 38558228893..556aee41a24 100644 --- a/docker/client/Dockerfile +++ b/docker/client/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.17.1.* +ARG version=19.18.1.* RUN apt-get update \ && apt-get install --yes --no-install-recommends \ diff --git a/docker/server/Dockerfile b/docker/server/Dockerfile index 1bd22f1c95b..ba47bb3d842 100644 --- a/docker/server/Dockerfile +++ b/docker/server/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.17.1.* +ARG version=19.18.1.* ARG gosu_ver=1.10 RUN apt-get update \ diff --git a/docker/test/Dockerfile b/docker/test/Dockerfile index fd03a7e0b56..5708284b15f 100644 --- a/docker/test/Dockerfile +++ b/docker/test/Dockerfile @@ -1,7 +1,7 @@ FROM ubuntu:18.04 ARG repository="deb http://repo.yandex.ru/clickhouse/deb/stable/ main/" -ARG version=19.17.1.* +ARG version=19.18.1.* RUN apt-get update && \ apt-get install -y apt-transport-https dirmngr && \ From 5cf36f66eaef7ce62f1aafafbb3c465300693542 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Fri, 8 Nov 2019 11:51:01 +0300 Subject: [PATCH 62/77] Doc links fix (#7682) * Doc links fix. * More links fix. --- docs/en/operations/system_tables.md | 2 +- docs/ru/operations/system_tables.md | 2 +- docs/ru/operations/table_engines/mergetree.md | 2 +- docs/zh/operations/table_engines/mergetree.md | 3 +++ 4 files changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system_tables.md b/docs/en/operations/system_tables.md index c234ae0b464..ececb54e596 100644 --- a/docs/en/operations/system_tables.md +++ b/docs/en/operations/system_tables.md @@ -785,7 +785,7 @@ Columns: - `volume_priority` ([UInt64](../data_types/int_uint.md)) — Volume order number in the configuration. - `disks` ([Array(String)](../data_types/array.md)) — Disk names, defined in the storage policy. - `max_data_part_size` ([UInt64](../data_types/int_uint.md)) — Maximum size of a data part that can be stored on volume disks (0 — no limit). -- `move_factor` ([Float64](..data_types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. +- `move_factor` ([Float64](../data_types/float.md)) — Ratio of free disk space. When the ratio exceeds the value of configuration parameter, ClickHouse start to move data to the next volume in order. If the storage policy contains more then one volume, then information for each volume is stored in the individual row of the table. diff --git a/docs/ru/operations/system_tables.md b/docs/ru/operations/system_tables.md index 298a0b5ad90..fa6c80bfb05 100644 --- a/docs/ru/operations/system_tables.md +++ b/docs/ru/operations/system_tables.md @@ -778,7 +778,7 @@ Cодержит информацию о дисках, заданных в [ко - `volume_priority` ([UInt64](../data_types/int_uint.md)) — порядковый номер тома согласно конфигурации. - `disks` ([Array(String)](../data_types/array.md)) — имена дисков, содержащихся в политике хранения. - `max_data_part_size` ([UInt64](../data_types/int_uint.md)) — максимальный размер куска данных, который может храниться на дисках тома (0 — без ограничений). -- `move_factor` ([Float64](..data_types/float.md))` — доля свободного места, при превышении которой данные начинают перемещаться на следующий том. +- `move_factor` ([Float64](../data_types/float.md))` — доля свободного места, при превышении которой данные начинают перемещаться на следующий том. Если политика хранения содержит несколько томов, то каждому тому соответствует отдельная запись в таблице. diff --git a/docs/ru/operations/table_engines/mergetree.md b/docs/ru/operations/table_engines/mergetree.md index 303cd641038..187bc8ab6c6 100644 --- a/docs/ru/operations/table_engines/mergetree.md +++ b/docs/ru/operations/table_engines/mergetree.md @@ -457,7 +457,7 @@ ALTER TABLE example_table Движки таблиц семейства `MergeTree` могут хранить данные на нескольких блочных устройствах. Это может оказаться полезным, например, при неявном разделении данных одной таблицы на "горячие" и "холодные". Наиболее свежая часть занимает малый объем и запрашивается регулярно, а большой хвост исторических данных запрашивается редко. При наличии в системе нескольких дисков, "горячая" часть данных может быть размещена на быстрых дисках (например, на NVMe SSD или в памяти), а холодная на более медленных (например, HDD). -Минимальной перемещаемой единицей для `MergeTree` является кусок данных (data part). Данные одного куска могут находится только на одном диске. Куски могут перемещаться между дисками в фоне, согласно пользовательским настройкам, а также с помощью запросов [ALTER](../../query_language/alter.md# -partition). +Минимальной перемещаемой единицей для `MergeTree` является кусок данных (data part). Данные одного куска могут находится только на одном диске. Куски могут перемещаться между дисками в фоне, согласно пользовательским настройкам, а также с помощью запросов [ALTER](../../query_language/alter.md#alter_move-partition). ### Термины diff --git a/docs/zh/operations/table_engines/mergetree.md b/docs/zh/operations/table_engines/mergetree.md index 4c35f3cf6b9..6d8baea8cf2 100644 --- a/docs/zh/operations/table_engines/mergetree.md +++ b/docs/zh/operations/table_engines/mergetree.md @@ -296,5 +296,8 @@ INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 对表的读操作是自动并行的。 +## Using Multiple Block Devices for Data Storage {#table_engine-mergetree-multiple-volumes} + +### Configuration {#table_engine-mergetree-multiple-volumes_configure} [来源文章](https://clickhouse.yandex/docs/en/operations/table_engines/mergetree/) From 40a5cf4bb9ae218b5dcbe1a5ae791d3e71d8409c Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2019 12:57:32 +0300 Subject: [PATCH 63/77] Forbidden to use column name more than once in insert query. --- dbms/src/DataStreams/NativeBlockInputStream.cpp | 2 +- .../src/Interpreters/InterpreterInsertQuery.cpp | 3 +++ ...1032_duplicate_column_insert_query.reference | 1 + .../01032_duplicate_column_insert_query.sql | 17 +++++++++++++++++ 4 files changed, 22 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.reference create mode 100644 dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.sql diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 659cfcbdfca..246d1882a5d 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -188,7 +188,7 @@ Block NativeBlockInputStream::readImpl() for (auto & col : header) { if (res.has(col.name)) - tmp_res.insert(std::move(res.getByName(col.name))); + tmp_res.insert(res.getByName(col.name)); else tmp_res.insert({col.type->createColumn()->cloneResized(rows), col.type, col.name}); } diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 61c153b13be..b052bb82c1e 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -30,6 +30,7 @@ namespace ErrorCodes extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int READONLY; extern const int ILLEGAL_COLUMN; + extern const int DUPLICATE_COLUMN; } @@ -84,6 +85,8 @@ Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const if (!allow_materialized && !table_sample_non_materialized.has(current_name)) throw Exception("Cannot insert column " + current_name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); + if (res.has(current_name)) + throw Exception("Column " + current_name + " specified more than once", ErrorCodes::DUPLICATE_COLUMN); res.insert(ColumnWithTypeAndName(table_sample.getByName(current_name).type, current_name)); } diff --git a/dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.reference b/dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.sql b/dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.sql new file mode 100644 index 00000000000..ac1a2439c4b --- /dev/null +++ b/dbms/tests/queries/0_stateless/01032_duplicate_column_insert_query.sql @@ -0,0 +1,17 @@ +DROP TABLE IF EXISTS sometable; + +CREATE TABLE sometable ( + date Date, + time Int64, + value UInt64 +) ENGINE=MergeTree() +ORDER BY time; + + +INSERT INTO sometable (date, time, value) VALUES ('2019-11-08', 1573185600, 100); + +SELECT COUNT() from sometable; + +INSERT INTO sometable (date, time, value, time) VALUES ('2019-11-08', 1573185600, 100, 1573185600); -- {serverError 15} + +DROP TABLE IF EXISTS sometable; From e55a7b93c4bceef00d790211b70b213b5b3e4df9 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2019 13:50:26 +0300 Subject: [PATCH 64/77] Remove extra space --- dbms/src/Parsers/ASTOrderByElement.cpp | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Parsers/ASTOrderByElement.cpp b/dbms/src/Parsers/ASTOrderByElement.cpp index ac57cfa437b..c22118756b5 100644 --- a/dbms/src/Parsers/ASTOrderByElement.cpp +++ b/dbms/src/Parsers/ASTOrderByElement.cpp @@ -31,17 +31,17 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL " << (settings.hilite ? hilite_none : ""); if (fill_from) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "FROM " << (settings.hilite ? hilite_none : ""); fill_from->formatImpl(settings, state, frame); } if (fill_to) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "TO " << (settings.hilite ? hilite_none : ""); fill_to->formatImpl(settings, state, frame); } if (fill_step) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << "STEP " << (settings.hilite ? hilite_none : ""); fill_step->formatImpl(settings, state, frame); } } From 5624bb3abb265081cfb5d34de4f70c6dce0a5670 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Mon, 28 Oct 2019 14:01:09 +0300 Subject: [PATCH 65/77] odbc table function now respects external_table_functions_use_nulls setting * Passing setting value to ODBC-bridge on each request * Handling that setting value correctly in ODBC-bridge * Fixed issue with executing table-functions on context with no settings applied in `SELECT ... SETTINGS x=foo` query. * Added tests to verify fix. --- .../programs/odbc-bridge/ColumnInfoHandler.cpp | 18 +++++++++++++++--- dbms/src/Core/Settings.h | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 5 +++-- dbms/src/TableFunctions/ITableFunctionXDBC.cpp | 5 +++++ .../integration/test_odbc_interaction/test.py | 3 ++- 5 files changed, 26 insertions(+), 7 deletions(-) diff --git a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp index 594cddfd3db..95ac31af8c9 100644 --- a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -18,6 +18,7 @@ #include #include #include +#include #include #include #include @@ -62,6 +63,15 @@ namespace return factory.get("String"); } } + + bool parseBool(const std::string & s, bool default_value = false) + { + bool result; + if (Poco::NumberParser::tryParseBool(s, result)) + return result; + + return default_value; + } } namespace ErrorCodes @@ -95,6 +105,8 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques std::string schema_name = ""; std::string table_name = params.get("table"); std::string connection_string = params.get("connection_string"); + const bool external_table_functions_use_nulls = parseBool(params.get("external_table_functions_use_nulls", "false")); + if (params.has("schema")) { schema_name = params.get("schema"); @@ -160,13 +172,13 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques /// TODO Why 301? SQLCHAR column_name[301]; - SQLSMALLINT nullable; - const auto result = POCO_SQL_ODBC_CLASS::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, &nullable); + SQLSMALLINT is_nullable; + const auto result = POCO_SQL_ODBC_CLASS::SQLDescribeCol(hstmt, ncol, column_name, sizeof(column_name), nullptr, &type, nullptr, nullptr, &is_nullable); if (POCO_SQL_ODBC_CLASS::Utility::isError(result)) throw POCO_SQL_ODBC_CLASS::StatementException(hstmt); auto column_type = getDataType(type); - if (nullable == SQL_NULLABLE) + if (external_table_functions_use_nulls && is_nullable == SQL_NULLABLE) { column_type = std::make_shared(column_type); } diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index fb3021275be..e0d97fe62fb 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -357,7 +357,7 @@ struct Settings : public SettingsCollection M(SettingBool, allow_experimental_multiple_joins_emulation, true, "Emulate multiple joins using subselects") \ M(SettingBool, allow_experimental_cross_to_join_conversion, true, "Convert CROSS JOIN to INNER JOIN if possible") \ M(SettingBool, cancel_http_readonly_queries_on_client_close, false, "Cancel HTTP readonly queries when a client closes the connection without waiting for response.") \ - M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \ + M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only by 'mysql' and 'odbc' table functions.") \ M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \ \ M(SettingBool, experimental_use_processors, false, "Use processors pipeline.") \ diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index a2c7f1075ab..14d3f043811 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -286,8 +286,9 @@ InterpreterSelectQuery::InterpreterSelectQuery( { if (is_table_func) { - /// Read from table function. - storage = context.getQueryContext().executeTableFunction(table_expression); + /// Read from table function. propagate all settings from initSettings(), + /// alternative is to call on current `context`, but that can potentially pollute it. + storage = getSubqueryContext(context).executeTableFunction(table_expression); } else { diff --git a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp index 81e30d0917d..fe64b7a496e 100644 --- a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp @@ -16,6 +16,7 @@ #include #include #include +#include namespace DB @@ -70,6 +71,10 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co columns_info_uri.addQueryParameter("schema", schema_name); columns_info_uri.addQueryParameter("table", remote_table_name); + const auto use_nuls = context.getSettingsRef().external_table_functions_use_nulls; + columns_info_uri.addQueryParameter("external_table_functions_use_nulls", + Poco::NumberFormatter::format(use_nuls)); + ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, nullptr); std::string columns_info; diff --git a/dbms/tests/integration/test_odbc_interaction/test.py b/dbms/tests/integration/test_odbc_interaction/test.py index d8e9cc3fb1a..f4af04b5fbc 100644 --- a/dbms/tests/integration/test_odbc_interaction/test.py +++ b/dbms/tests/integration/test_odbc_interaction/test.py @@ -91,7 +91,8 @@ def test_mysql_simple_select_works(started_cluster): with conn.cursor() as cursor: cursor.execute("INSERT INTO clickhouse.{} VALUES(50, 'null-guy', 127, 255, NULL), (100, 'non-null-guy', 127, 255, 511);".format(table_name)) conn.commit() - assert node1.query("SELECT column_x FROM odbc('DSN={}', '{}')".format(mysql_setup["DSN"], table_name)) == '\\N\n511\n' + assert node1.query("SELECT column_x FROM odbc('DSN={}', '{}') SETTINGS external_table_functions_use_nulls=1".format(mysql_setup["DSN"], table_name)) == '\\N\n511\n' + assert node1.query("SELECT column_x FROM odbc('DSN={}', '{}') SETTINGS external_table_functions_use_nulls=0".format(mysql_setup["DSN"], table_name)) == '0\n511\n' node1.query(''' CREATE TABLE {}(id UInt32, name String, age UInt32, money UInt32, column_x Nullable(UInt32)) ENGINE = MySQL('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse'); From 4ac21ca6ba7e629c37b3867b159890c5e6c27d03 Mon Sep 17 00:00:00 2001 From: Vasily Nemkov Date: Thu, 7 Nov 2019 09:08:59 +0300 Subject: [PATCH 66/77] PR-review fixes: handling bad value of 'external_table_functions_use_nulls' param, fixed typo --- dbms/programs/odbc-bridge/ColumnInfoHandler.cpp | 12 ++---------- dbms/src/TableFunctions/ITableFunctionXDBC.cpp | 4 ++-- 2 files changed, 4 insertions(+), 12 deletions(-) diff --git a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp index 95ac31af8c9..3dadc7632de 100644 --- a/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp +++ b/dbms/programs/odbc-bridge/ColumnInfoHandler.cpp @@ -63,15 +63,6 @@ namespace return factory.get("String"); } } - - bool parseBool(const std::string & s, bool default_value = false) - { - bool result; - if (Poco::NumberParser::tryParseBool(s, result)) - return result; - - return default_value; - } } namespace ErrorCodes @@ -105,7 +96,6 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques std::string schema_name = ""; std::string table_name = params.get("table"); std::string connection_string = params.get("connection_string"); - const bool external_table_functions_use_nulls = parseBool(params.get("external_table_functions_use_nulls", "false")); if (params.has("schema")) { @@ -118,6 +108,8 @@ void ODBCColumnsInfoHandler::handleRequest(Poco::Net::HTTPServerRequest & reques try { + const bool external_table_functions_use_nulls = Poco::NumberParser::parseBool(params.get("external_table_functions_use_nulls", "false")); + POCO_SQL_ODBC_CLASS::SessionImpl session(validateODBCConnectionString(connection_string), DBMS_DEFAULT_CONNECT_TIMEOUT_SEC); SQLHDBC hdbc = session.dbc().handle(); diff --git a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp index fe64b7a496e..c90ed0f34fb 100644 --- a/dbms/src/TableFunctions/ITableFunctionXDBC.cpp +++ b/dbms/src/TableFunctions/ITableFunctionXDBC.cpp @@ -71,9 +71,9 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & ast_function, const Co columns_info_uri.addQueryParameter("schema", schema_name); columns_info_uri.addQueryParameter("table", remote_table_name); - const auto use_nuls = context.getSettingsRef().external_table_functions_use_nulls; + const auto use_nulls = context.getSettingsRef().external_table_functions_use_nulls; columns_info_uri.addQueryParameter("external_table_functions_use_nulls", - Poco::NumberFormatter::format(use_nuls)); + Poco::NumberFormatter::format(use_nulls)); ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, nullptr); From db10ef3129d7e6baa60b88bd1ad350ab31d36504 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Fri, 8 Nov 2019 17:26:25 +0300 Subject: [PATCH 67/77] [docs] update repo_name and repo_url --- docs/tools/build.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/tools/build.py b/docs/tools/build.py index 72a16839bef..729229fdee7 100755 --- a/docs/tools/build.py +++ b/docs/tools/build.py @@ -92,8 +92,8 @@ def build_for_lang(lang, args): theme=theme_cfg, copyright='©2016–2019 Yandex LLC', use_directory_urls=True, - repo_name='yandex/ClickHouse', - repo_url='https://github.com/yandex/ClickHouse/', + repo_name='ClickHouse/ClickHouse', + repo_url='https://github.com/ClickHouse/ClickHouse/', edit_uri='edit/master/docs/%s' % lang, extra_css=['assets/stylesheets/custom.css'], markdown_extensions=[ From 9305f04c9f34dc5fcf39e961c380a67422faa6b2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 8 Nov 2019 19:54:13 +0300 Subject: [PATCH 68/77] Fix for fix --- dbms/src/Parsers/ASTOrderByElement.cpp | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/dbms/src/Parsers/ASTOrderByElement.cpp b/dbms/src/Parsers/ASTOrderByElement.cpp index c22118756b5..b597c98eabe 100644 --- a/dbms/src/Parsers/ASTOrderByElement.cpp +++ b/dbms/src/Parsers/ASTOrderByElement.cpp @@ -28,20 +28,20 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState if (with_fill) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL" << (settings.hilite ? hilite_none : ""); if (fill_from) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "FROM " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : ""); fill_from->formatImpl(settings, state, frame); } if (fill_to) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "TO " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : ""); fill_to->formatImpl(settings, state, frame); } if (fill_step) { - settings.ostr << (settings.hilite ? hilite_keyword : "") << "STEP " << (settings.hilite ? hilite_none : ""); + settings.ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : ""); fill_step->formatImpl(settings, state, frame); } } From 6f7012579834d914dd7985947c2f006928c8094f Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 9 Nov 2019 15:16:00 +0300 Subject: [PATCH 69/77] Get constant values from remote source in RemoteBlockInputStream. --- .../DataStreams/RemoteBlockInputStream.cpp | 28 +++++++++++++++++-- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataStreams/RemoteBlockInputStream.cpp b/dbms/src/DataStreams/RemoteBlockInputStream.cpp index 61432939a95..aedbe676688 100644 --- a/dbms/src/DataStreams/RemoteBlockInputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockInputStream.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -173,9 +174,30 @@ static Block adaptBlockStructure(const Block & block, const Block & header, cons ColumnPtr column; if (elem.column && isColumnConst(*elem.column)) - /// TODO: check that column from block contains the same value. - /// TODO: serialize const columns. - column = elem.column->cloneResized(block.rows()); + { + /// We expect constant column in block. + /// If block is not empty, then get value for constant from it, + /// because it may be different for remote server for functions like version(), uptime(), ... + if (block.rows() > 0 && block.has(elem.name)) + { + /// Const column is passed as materialized. Get first value from it. + /// + /// TODO: check that column contains the same value. + /// TODO: serialize const columns. + auto col = block.getByName(elem.name); + col.column = block.getByName(elem.name).column->cut(0, 1); + + column = castColumn(col, elem.type, context); + + if (!isColumnConst(*column)) + column = ColumnConst::create(column, block.rows()); + else + /// It is not possible now. Just in case we support const columns serialization. + column = column->cloneResized(block.rows()); + } + else + column = elem.column->cloneResized(block.rows()); + } else column = castColumn(block.getByName(elem.name), elem.type, context); From 0ba523afa1aa91fe0fcdf7733874bf506d530a03 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Sat, 9 Nov 2019 15:16:34 +0300 Subject: [PATCH 70/77] Added integration test with different constants from remote. --- .../test_replicating_constants/__init__.py | 0 .../test_replicating_constants/test.py | 21 +++++++++++++++++++ 2 files changed, 21 insertions(+) create mode 100644 dbms/tests/integration/test_replicating_constants/__init__.py create mode 100644 dbms/tests/integration/test_replicating_constants/test.py diff --git a/dbms/tests/integration/test_replicating_constants/__init__.py b/dbms/tests/integration/test_replicating_constants/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_replicating_constants/test.py b/dbms/tests/integration/test_replicating_constants/test.py new file mode 100644 index 00000000000..f340817b584 --- /dev/null +++ b/dbms/tests/integration/test_replicating_constants/test.py @@ -0,0 +1,21 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) + +node1 = cluster.add_instance('node1', with_zookeeper=True) +node2 = cluster.add_instance('node2', with_zookeeper=True, image='yandex/clickhouse-server:19.1.14', with_installed_binary=True) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + + yield cluster + finally: + cluster.shutdown() + +def test_different_versions(start_cluster): + + assert node1.query("SELECT uniqExact(x) FROM (SELECT version() as x from remote('node{1,2}', system.one))") == "2\n" From 6a5f042a7f6b66def8f547d4e8e18d29ee1741bd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sun, 10 Nov 2019 17:04:29 +0300 Subject: [PATCH 71/77] Removed fixed index granularity from system logs --- dbms/src/Interpreters/SystemLog.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/SystemLog.cpp b/dbms/src/Interpreters/SystemLog.cpp index cb87990e4c0..6eb0ce69f4e 100644 --- a/dbms/src/Interpreters/SystemLog.cpp +++ b/dbms/src/Interpreters/SystemLog.cpp @@ -32,7 +32,7 @@ std::shared_ptr createSystemLog( String database = config.getString(config_prefix + ".database", default_database_name); String table = config.getString(config_prefix + ".table", default_table_name); String partition_by = config.getString(config_prefix + ".partition_by", "toYYYYMM(event_date)"); - String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time) SETTINGS index_granularity = 1024"; + String engine = "ENGINE = MergeTree PARTITION BY (" + partition_by + ") ORDER BY (event_date, event_time)"; size_t flush_interval_milliseconds = config.getUInt64(config_prefix + ".flush_interval_milliseconds", DEFAULT_SYSTEM_LOG_FLUSH_INTERVAL_MILLISECONDS); From f47415b7fa3731a129d97eba05820425bcd82f39 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Nov 2019 04:45:03 +0300 Subject: [PATCH 72/77] Allowed non-constant negative "size" argument for function "substring" --- dbms/src/Functions/substring.cpp | 6 ------ .../0_stateless/01033_substr_negative_size_arg.reference | 8 ++++++++ .../0_stateless/01033_substr_negative_size_arg.sql | 8 ++++++++ 3 files changed, 16 insertions(+), 6 deletions(-) create mode 100644 dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference create mode 100644 dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.sql diff --git a/dbms/src/Functions/substring.cpp b/dbms/src/Functions/substring.cpp index 0106f6945d5..6c753fc49cd 100644 --- a/dbms/src/Functions/substring.cpp +++ b/dbms/src/Functions/substring.cpp @@ -138,15 +138,9 @@ public: Int64 length_value = 0; if (column_start_const) - { start_value = column_start_const->getInt(0); - } if (column_length_const) - { length_value = column_length_const->getInt(0); - if (length_value < 0) - throw Exception("Third argument provided for function substring could not be negative.", ErrorCodes::ARGUMENT_OUT_OF_BOUND); - } if constexpr (is_utf8) { diff --git a/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference b/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference new file mode 100644 index 00000000000..98c07557034 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.reference @@ -0,0 +1,8 @@ +lickhous +lickhous +lickhous +lickhous +lickhous +lickhous +lickhous +lickhous diff --git a/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.sql b/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.sql new file mode 100644 index 00000000000..a0fba1a6eee --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_substr_negative_size_arg.sql @@ -0,0 +1,8 @@ +SELECT substr('clickhouse', 2, -2); +SELECT substr(materialize('clickhouse'), 2, -2); +SELECT substr('clickhouse', materialize(2), -2); +SELECT substr(materialize('clickhouse'), materialize(2), -2); +SELECT substr('clickhouse', 2, materialize(-2)); +SELECT substr(materialize('clickhouse'), 2, materialize(-2)); +SELECT substr('clickhouse', materialize(2), materialize(-2)); +SELECT substr(materialize('clickhouse'), materialize(2), materialize(-2)); From 0adaf48a5cb692e038f0bb45453160ff6bfe0403 Mon Sep 17 00:00:00 2001 From: achimbab <36371084+achimbab@users.noreply.github.com> Date: Mon, 11 Nov 2019 12:41:23 +0900 Subject: [PATCH 73/77] [docs] added 'strict' parameter in windowFunnel(). --- .../en/query_language/agg_functions/parametric_functions.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/query_language/agg_functions/parametric_functions.md b/docs/en/query_language/agg_functions/parametric_functions.md index c72bfe26f7b..8784811a311 100644 --- a/docs/en/query_language/agg_functions/parametric_functions.md +++ b/docs/en/query_language/agg_functions/parametric_functions.md @@ -219,17 +219,19 @@ SELECT sequenceCount('(?1).*(?2)')(time, number = 1, number = 2) FROM t - [sequenceMatch](#function-sequencematch) -## windowFunnel(window)(timestamp, cond1, cond2, cond3, ...) +## windowFunnel(window, [mode])(timestamp, cond1, cond2, cond3, ...) Searches for event chains in a sliding time window and calculates the maximum number of events that occurred from the chain. ``` -windowFunnel(window)(timestamp, cond1, cond2, cond3, ...) +windowFunnel(window, [mode])(timestamp, cond1, cond2, cond3, ...) ``` **Parameters:** - `window` — Length of the sliding window in seconds. +- `mode` - It is an optional argument. + * `'strict'` - When the `'strict'` is set, the windowFunnel() applies conditions only for the unique values. - `timestamp` — Name of the column containing the timestamp. Data types supported: `Date`, `DateTime`, and other unsigned integer types (note that even though timestamp supports the `UInt64` type, it's value can't exceed the Int64 maximum, which is 2^63 - 1). - `cond1`, `cond2`... — Conditions or data describing the chain of events. Data type: `UInt8`. Values can be 0 or 1. From c74538153f23b7a53861d28cfeed8e6a93618315 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Nov 2019 11:40:00 +0300 Subject: [PATCH 74/77] Fix use of destroyed variable captured in lambda --- dbms/src/Storages/StorageXDBC.cpp | 2 +- .../01033_storage_odbc_parsing_exception_check.reference | 1 + .../01033_storage_odbc_parsing_exception_check.sql | 9 +++++++++ 3 files changed, 11 insertions(+), 1 deletion(-) create mode 100644 dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.reference create mode 100644 dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.sql diff --git a/dbms/src/Storages/StorageXDBC.cpp b/dbms/src/Storages/StorageXDBC.cpp index 915146e1409..fc9bb776da0 100644 --- a/dbms/src/Storages/StorageXDBC.cpp +++ b/dbms/src/Storages/StorageXDBC.cpp @@ -105,7 +105,7 @@ namespace template void registerXDBCStorage(StorageFactory & factory, const std::string & name) { - factory.registerStorage(name, [&name](const StorageFactory::Arguments & args) + factory.registerStorage(name, [name](const StorageFactory::Arguments & args) { ASTs & engine_args = args.engine_args; diff --git a/dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.reference b/dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.reference new file mode 100644 index 00000000000..7e6d9355882 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.reference @@ -0,0 +1 @@ +CREATE TABLE default.BannerDict (`BannerID` UInt64, `CompaignID` UInt64) ENGINE = ODBC(\'DSN=pgconn;Database=postgres\', somedb, bannerdict) diff --git a/dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.sql b/dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.sql new file mode 100644 index 00000000000..7a3d2914111 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01033_storage_odbc_parsing_exception_check.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS BannerDict; + +CREATE TABLE BannerDict (`BannerID` UInt64, `CompaignID` UInt64) ENGINE = ODBC('DSN=pgconn;Database=postgres', bannerdict); -- {serverError 42} + +CREATE TABLE BannerDict (`BannerID` UInt64, `CompaignID` UInt64) ENGINE = ODBC('DSN=pgconn;Database=postgres', somedb, bannerdict); + +SHOW CREATE TABLE BannerDict; + +DROP TABLE IF EXISTS BannerDict; From 013435bdedbb686a17b443f3214c34def938eda5 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 11 Nov 2019 11:55:07 +0300 Subject: [PATCH 75/77] Updated test #7703 --- .../0_stateless/00970_substring_arg_validation.reference | 1 + .../queries/0_stateless/00970_substring_arg_validation.sql | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/tests/queries/0_stateless/00970_substring_arg_validation.reference b/dbms/tests/queries/0_stateless/00970_substring_arg_validation.reference index e69de29bb2d..13e7564ea0c 100644 --- a/dbms/tests/queries/0_stateless/00970_substring_arg_validation.reference +++ b/dbms/tests/queries/0_stateless/00970_substring_arg_validation.reference @@ -0,0 +1 @@ +o diff --git a/dbms/tests/queries/0_stateless/00970_substring_arg_validation.sql b/dbms/tests/queries/0_stateless/00970_substring_arg_validation.sql index 7d8320a1d64..43d73bc2cda 100644 --- a/dbms/tests/queries/0_stateless/00970_substring_arg_validation.sql +++ b/dbms/tests/queries/0_stateless/00970_substring_arg_validation.sql @@ -1,4 +1,4 @@ SELECT substring('hello', []); -- { serverError 43 } SELECT substring('hello', 1, []); -- { serverError 43 } -SELECT substring(materialize('hello'), -1, -1); -- { serverError 69 } +SELECT substring(materialize('hello'), -1, -1); SELECT substring(materialize('hello'), 0); -- { serverError 135 } \ No newline at end of file From 9dd37bd252fe49f8296a90a5de297b992eea3aaa Mon Sep 17 00:00:00 2001 From: Aliaksandr Shylau Date: Mon, 11 Nov 2019 12:23:06 +0300 Subject: [PATCH 76/77] Replace unmaintainable Ruby client by a newer one --- docs/en/interfaces/third-party/client_libraries.md | 2 +- docs/fa/interfaces/third-party/client_libraries.md | 2 +- docs/ru/interfaces/third-party/client_libraries.md | 2 +- docs/zh/interfaces/third-party/client_libraries.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/interfaces/third-party/client_libraries.md b/docs/en/interfaces/third-party/client_libraries.md index 9dc47807729..e0842ab36ef 100644 --- a/docs/en/interfaces/third-party/client_libraries.md +++ b/docs/en/interfaces/third-party/client_libraries.md @@ -27,7 +27,7 @@ - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) - [AnyEvent-ClickHouse](https://metacpan.org/release/AnyEvent-ClickHouse) - Ruby - - [clickhouse (Ruby)](https://github.com/archan937/clickhouse) + - [ClickHouse (Ruby)](https://github.com/shlima/click_house) - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) diff --git a/docs/fa/interfaces/third-party/client_libraries.md b/docs/fa/interfaces/third-party/client_libraries.md index e5c0d1bf888..c31998191e5 100644 --- a/docs/fa/interfaces/third-party/client_libraries.md +++ b/docs/fa/interfaces/third-party/client_libraries.md @@ -27,7 +27,7 @@ - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) - [AnyEvent-ClickHouse](https://metacpan.org/release/AnyEvent-ClickHouse) - Ruby - - [clickhouse (Ruby)](https://github.com/archan937/clickhouse) + - [ClickHouse (Ruby)](https://github.com/shlima/click_house) - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) diff --git a/docs/ru/interfaces/third-party/client_libraries.md b/docs/ru/interfaces/third-party/client_libraries.md index 8633cfc13ee..13b7b9d243e 100644 --- a/docs/ru/interfaces/third-party/client_libraries.md +++ b/docs/ru/interfaces/third-party/client_libraries.md @@ -26,7 +26,7 @@ - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) - [AnyEvent-ClickHouse](https://metacpan.org/release/AnyEvent-ClickHouse) - Ruby - - [clickhouse (Ruby)](https://github.com/archan937/clickhouse) + - [ClickHouse (Ruby)](https://github.com/shlima/click_house) - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) diff --git a/docs/zh/interfaces/third-party/client_libraries.md b/docs/zh/interfaces/third-party/client_libraries.md index b5df45c3d6b..a8625c0d4ac 100644 --- a/docs/zh/interfaces/third-party/client_libraries.md +++ b/docs/zh/interfaces/third-party/client_libraries.md @@ -26,7 +26,7 @@ - [HTTP-ClickHouse](https://metacpan.org/release/HTTP-ClickHouse) - [AnyEvent-ClickHouse](https://metacpan.org/release/AnyEvent-ClickHouse) - Ruby - - [clickhouse (Ruby)](https://github.com/archan937/clickhouse) + - [ClickHouse (Ruby)](https://github.com/shlima/click_house) - R - [clickhouse-r](https://github.com/hannesmuehleisen/clickhouse-r) - [RClickhouse](https://github.com/IMSMWU/RClickhouse) From 9d0126297dba29f0da775967ce4d1f5bf1e6b2b2 Mon Sep 17 00:00:00 2001 From: alesapin Date: Mon, 11 Nov 2019 18:57:33 +0300 Subject: [PATCH 77/77] Move client-option option from shard group to common group in clickhouse-test --- dbms/tests/clickhouse-test | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/tests/clickhouse-test b/dbms/tests/clickhouse-test index 10743cd5acc..eab850d3e48 100755 --- a/dbms/tests/clickhouse-test +++ b/dbms/tests/clickhouse-test @@ -502,13 +502,13 @@ if __name__ == '__main__': parser.add_argument('--no-stateful', action='store_true', help='Disable all stateful tests') parser.add_argument('--skip', nargs='+', help="Skip these tests") parser.add_argument('--no-long', action='store_false', dest='no_long', help='Do not run long tests') + parser.add_argument('--client-option', nargs='+', help='Specify additional client argument') group=parser.add_mutually_exclusive_group(required=False) group.add_argument('--zookeeper', action='store_true', default=None, dest='zookeeper', help='Run zookeeper related tests') group.add_argument('--no-zookeeper', action='store_false', default=None, dest='zookeeper', help='Do not run zookeeper related tests') group=parser.add_mutually_exclusive_group(required=False) group.add_argument('--shard', action='store_true', default=None, dest='shard', help='Run sharding related tests (required to clickhouse-server listen 127.0.0.2 127.0.0.3)') group.add_argument('--no-shard', action='store_false', default=None, dest='shard', help='Do not run shard related tests') - group.add_argument('--client-option', nargs='+', help='Specify additional client argument') args = parser.parse_args()