From c442f89fbd5f27f7e26c2a22382d8e19d1cc4434 Mon Sep 17 00:00:00 2001 From: "chou.fan" Date: Fri, 15 Jan 2021 15:07:43 +0000 Subject: [PATCH 01/82] update --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 45 ++++++++++++++++++- 1 file changed, 44 insertions(+), 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 4ad7743151a..7257445f469 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -7,11 +7,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -229,6 +231,30 @@ namespace DB } } + static void fillColumnWithArrayData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + { + ColumnArray & column_array = assert_cast(*internal_column); + ColumnArray::Offsets & offsets = column_array.getOffsets(); + IColumn & nested_column = column_array.getData(); + + column_array.reserve(arrow_column->length()); + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const auto & chunk_data = chunk.data(); + for (int i = 0; i < chunk_data->length; ++i) + { + const auto & array_val = chunk_data->GetValues(i); + for (int64_t array_i = 0; array_i < array_val->length(); ++array_i) + { + auto string_val = array_val->GetString(array_i); + nested_column.insertData(string_val.c_str(), string_val.length()); + } + offsets.push_back(offsets.back(), array_val->length()); + } + } + } + /// Creates a null bytemap from arrow's null bitmap static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) { @@ -291,6 +317,21 @@ namespace DB internal_nested_type = std::make_shared>(decimal_type->precision(), decimal_type->scale()); } + else if (arrow_type == arrow::Type::LIST) { + const auto * list_type = static_cast(arrow_column->type().get()); + const auto * nested_list_type = list_type->value_type().get(); + + if (nested_list_type->id() == arrow::Type::STRING) + { + const auto internal_nested_nested_type = DataTypeFactory::instance().get("String"); + internal_nested_type = std::make_shared(internal_nested_nested_type); + } + else { + throw Exception{"The internal type \"" + nested_list_type->name() + "\" of an array column \"" + header_column.name + + "\" is not supported for conversion from a " + format_name + " data format", + ErrorCodes::CANNOT_CONVERT_TYPE}; + } + } else if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), [=](auto && elem) { return elem.first == arrow_type; }); internal_type_it != arrow_type_to_internal_type.end()) @@ -335,7 +376,9 @@ namespace DB case arrow::Type::DECIMAL: //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster fillColumnWithDecimalData(arrow_column, read_column /*, internal_nested_type*/); - + break; + case arrow::Type::LIST: + fillColumnWithArrayData(arrow_column, read_column); break; # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ From 99e594e704b8fb10d72a4990d7b92b51851f5c1b Mon Sep 17 00:00:00 2001 From: "chou.fan" Date: Tue, 16 Mar 2021 15:29:21 +0000 Subject: [PATCH 02/82] parquet data type support array format read and write --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 351 ++++++++++++++++-- .../Formats/Impl/CHColumnToArrowColumn.cpp | 233 ++++++++---- .../Formats/Impl/CHColumnToArrowColumn.h | 10 + .../0_stateless/00900_parquet_load.reference | 33 ++ .../data_parquet/array_float.parquet | Bin 0 -> 520 bytes .../data_parquet/array_float.parquet.columns | 1 + .../data_parquet/array_int.parquet | Bin 0 -> 517 bytes .../data_parquet/array_int.parquet.columns | 1 + .../data_parquet/array_string.parquet | Bin 0 -> 725 bytes .../data_parquet/array_string.parquet.columns | 1 + 10 files changed, 536 insertions(+), 94 deletions(-) create mode 100755 tests/queries/0_stateless/data_parquet/array_float.parquet create mode 100644 tests/queries/0_stateless/data_parquet/array_float.parquet.columns create mode 100755 tests/queries/0_stateless/data_parquet/array_int.parquet create mode 100644 tests/queries/0_stateless/data_parquet/array_int.parquet.columns create mode 100755 tests/queries/0_stateless/data_parquet/array_string.parquet create mode 100644 tests/queries/0_stateless/data_parquet/array_string.parquet.columns diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 7257445f469..85cb7563b60 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -4,7 +4,6 @@ #if USE_ARROW || USE_ORC || USE_PARQUET #include #include -#include #include #include #include @@ -59,6 +58,24 @@ namespace DB // Full list of types: contrib/arrow/cpp/src/arrow/type.h }; + template + static void reserveArrayColumn(std::shared_ptr & arrow_column, ColumnArray & array_column, NestedColumnVector & nested_column) + { + size_t nested_column_length = 0; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != chunk.length(); ++array_idx) + { + const std::shared_ptr array = chunk.value_slice(array_idx); + nested_column_length += array->length(); + } + } + array_column.reserve(arrow_column->length()); + nested_column.reserve(nested_column_length); + } + /// Inserts numeric data right into internal column data to reduce an overhead template > static void fillColumnWithNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) @@ -77,6 +94,35 @@ namespace DB } } + template > + static void fillColumnWithArrayNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + { + ColumnArray & array_column = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + + VectorType & nested_column = static_cast(array_column.getData()); + auto & nested_column_data = nested_column.getData(); + + reserveArrayColumn(arrow_column, array_column, nested_column); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) + { + const std::shared_ptr chunk = list_chunk.value_slice(array_idx); + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk->data()->buffers[1]; + + const auto * raw_data = reinterpret_cast(buffer->data()); + nested_column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); + + column_array_offsets.emplace_back(column_array_offsets.back() + chunk->length()); + } + } + } + /// Inserts chars and offsets right into internal column data to reduce an overhead. /// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. /// Also internal strings are null terminated. @@ -118,6 +164,63 @@ namespace DB } } + static void fillColumnWithArrayStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + { + ColumnArray & column_array = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); + + ColumnString & nested_column = typeid_cast(column_array.getData()); + PaddedPODArray & nested_column_chars = nested_column.getChars(); + PaddedPODArray & nested_column_offsets = nested_column.getOffsets(); + + size_t chars_t_size = 0; + size_t number_size = 0; + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != chunk.length(); ++array_idx) + { + const std::shared_ptr array = chunk.value_slice(array_idx); + arrow::BinaryArray & binary_array = static_cast(*(array)); + const size_t binary_array_length = binary_array.length(); + + chars_t_size += binary_array.value_offset(binary_array_length - 1) + binary_array.value_length(binary_array_length - 1); + chars_t_size += binary_array_length; /// additional space for null bytes + number_size += binary_array_length; + } + } + column_array.reserve(arrow_column->length()); + + nested_column_chars.reserve(chars_t_size); + nested_column_offsets.reserve(number_size); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) + { + const std::shared_ptr array = list_chunk.value_slice(array_idx); + arrow::BinaryArray & chunk = static_cast(*(array)); + std::shared_ptr buffer = chunk.value_data(); + const size_t chunk_length = chunk.length(); + + for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + { + if (!chunk.IsNull(offset_i) && buffer) + { + const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); + nested_column_chars.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); + } + nested_column_chars.emplace_back('\0'); + nested_column_offsets.emplace_back(nested_column_chars.size()); + } + column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + } + } + } + static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = assert_cast &>(*internal_column).getData(); @@ -134,6 +237,38 @@ namespace DB } } + static void fillColumnWithArrayBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + { + ColumnArray & array_column = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + + ColumnVector & nested_column = assert_cast &>(array_column.getData()); + auto & nested_column_data = nested_column.getData(); + + reserveArrayColumn(arrow_column, array_column, nested_column); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) + { + const std::shared_ptr array = list_chunk.value_slice(array_idx); + + auto & chunk = static_cast(*(array)); + const size_t chunk_length = chunk.length(); + + /// buffers[0] is a null bitmap and buffers[1] are actual values + std::shared_ptr buffer = chunk.data()->buffers[1]; + + for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) + nested_column_data.emplace_back(chunk.Value(bool_i)); + + column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + } + } + } + /// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { @@ -162,6 +297,47 @@ namespace DB } } + static void fillColumnWithArrayDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + { + ColumnArray & array_column = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + + ColumnVector & nested_column = assert_cast &>(array_column.getData()); + auto & nested_column_data = nested_column.getData(); + + reserveArrayColumn(arrow_column, array_column, nested_column); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) + { + const std::shared_ptr array = list_chunk.value_slice(array_idx); + + auto & chunk = static_cast(*(array)); + const size_t chunk_length = chunk.length(); + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + UInt32 days_num = static_cast(chunk.Value(value_i)); + if (days_num > DATE_LUT_MAX_DAY_NUM) + { + // TODO: will it rollback correctly? + throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + internal_column->getName() + + "\" is greater than " + "max allowed Date value, which is " + + std::to_string(DATE_LUT_MAX_DAY_NUM), + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; + } + + nested_column_data.emplace_back(days_num); + } + column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + } + } + } + /// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { @@ -179,6 +355,37 @@ namespace DB } } + static void fillColumnWithArrayDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + { + ColumnArray & array_column = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + + ColumnVector & nested_column = typeid_cast &>(array_column.getData()); + auto & nested_column_data = nested_column.getData(); + + reserveArrayColumn(arrow_column, array_column, nested_column); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) + { + const std::shared_ptr array = list_chunk.value_slice(array_idx); + + auto & chunk = static_cast(*(array)); + const size_t chunk_length = chunk.length(); + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms + nested_column_data.emplace_back(timestamp); + } + column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + } + } + } + static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column_data = assert_cast &>(*internal_column).getData(); @@ -215,6 +422,56 @@ namespace DB } } + static void fillColumnWithArrayTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + { + ColumnArray & array_column = assert_cast(*internal_column); + ColumnArray::Offsets & array_column_offsets = array_column.getOffsets(); + + ColumnVector & nested_column = typeid_cast &>(array_column.getData()); + auto & nested_column_data = nested_column.getData(); + + reserveArrayColumn(arrow_column, array_column, nested_column); + + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) + { + const std::shared_ptr array = list_chunk.value_slice(array_idx); + + auto & chunk = static_cast(*array); + const auto & type = static_cast(*chunk.type()); + const size_t chunk_length = chunk.length(); + + UInt32 divide = 1; + const auto unit = type.unit(); + switch (unit) + { + case arrow::TimeUnit::SECOND: + divide = 1; + break; + case arrow::TimeUnit::MILLI: + divide = 1000; + break; + case arrow::TimeUnit::MICRO: + divide = 1000000; + break; + case arrow::TimeUnit::NANO: + divide = 1000000000; + break; + } + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... + nested_column_data.emplace_back(timestamp); + } + array_column_offsets.emplace_back(array_column_offsets.back() + chunk_length); + } + } + } + static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { auto & column = assert_cast &>(*internal_column); @@ -231,26 +488,31 @@ namespace DB } } - static void fillColumnWithArrayData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithArrayDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - ColumnArray & column_array = assert_cast(*internal_column); - ColumnArray::Offsets & offsets = column_array.getOffsets(); - IColumn & nested_column = column_array.getData(); + ColumnArray & array_column = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + + ColumnDecimal & nested_column = typeid_cast &>(array_column.getData()); + auto & nested_column_data = nested_column.getData(); + + reserveArrayColumn(arrow_column, array_column, nested_column); - column_array.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { - auto & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const auto & chunk_data = chunk.data(); - for (int i = 0; i < chunk_data->length; ++i) + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) { - const auto & array_val = chunk_data->GetValues(i); - for (int64_t array_i = 0; array_i < array_val->length(); ++array_i) + const std::shared_ptr array = list_chunk.value_slice(array_idx); + arrow::DecimalArray & chunk = static_cast(*(array)); + const size_t chunk_length = chunk.length(); + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) { - auto string_val = array_val->GetString(array_i); - nested_column.insertData(string_val.c_str(), string_val.length()); + nested_column_data.emplace_back(chunk.IsNull(value_i) ? Decimal128(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column } - offsets.push_back(offsets.back(), array_val->length()); + column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); } } } @@ -299,6 +561,7 @@ namespace DB std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; arrow::Type::type arrow_type = arrow_column->type()->id(); + arrow::Type::type list_nested_type; // TODO: check if a column is const? if (!column_type->isNullable() && arrow_column->null_count()) @@ -310,6 +573,7 @@ namespace DB const bool target_column_is_nullable = column_type->isNullable() || arrow_column->null_count(); DataTypePtr internal_nested_type; + DataTypePtr array_nested_type; if (arrow_type == arrow::Type::DECIMAL) { @@ -317,17 +581,21 @@ namespace DB internal_nested_type = std::make_shared>(decimal_type->precision(), decimal_type->scale()); } - else if (arrow_type == arrow::Type::LIST) { + else if (arrow_type == arrow::Type::LIST) + { const auto * list_type = static_cast(arrow_column->type().get()); - const auto * nested_list_type = list_type->value_type().get(); + list_nested_type = list_type->value_type()->id(); - if (nested_list_type->id() == arrow::Type::STRING) + if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), + [=](auto && elem) { return elem.first == list_nested_type; }); + internal_type_it != arrow_type_to_internal_type.end()) { - const auto internal_nested_nested_type = DataTypeFactory::instance().get("String"); - internal_nested_type = std::make_shared(internal_nested_nested_type); + array_nested_type = DataTypeFactory::instance().get(internal_type_it->second); + internal_nested_type = std::make_shared(array_nested_type); } - else { - throw Exception{"The internal type \"" + nested_list_type->name() + "\" of an array column \"" + header_column.name + else + { + throw Exception{"The internal type \"" + list_type->value_type()->name() + "\" of an array column \"" + header_column.name + "\" is not supported for conversion from a " + format_name + " data format", ErrorCodes::CANNOT_CONVERT_TYPE}; } @@ -378,7 +646,46 @@ namespace DB fillColumnWithDecimalData(arrow_column, read_column /*, internal_nested_type*/); break; case arrow::Type::LIST: - fillColumnWithArrayData(arrow_column, read_column); + if (array_nested_type) { + switch (list_nested_type) + { + case arrow::Type::STRING: + case arrow::Type::BINARY: + //case arrow::Type::FIXED_SIZE_BINARY: + fillColumnWithArrayStringData(arrow_column, read_column); + break; + case arrow::Type::BOOL: + fillColumnWithArrayBooleanData(arrow_column, read_column); + break; + case arrow::Type::DATE32: + fillColumnWithArrayDate32Data(arrow_column, read_column); + break; + case arrow::Type::DATE64: + fillColumnWithArrayDate64Data(arrow_column, read_column); + break; + case arrow::Type::TIMESTAMP: + fillColumnWithArrayTimestampData(arrow_column, read_column); + break; + case arrow::Type::DECIMAL: + //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster + fillColumnWithArrayDecimalData(arrow_column, read_column /*, internal_nested_type*/); + break; + # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ + case ARROW_NUMERIC_TYPE: \ + fillColumnWithArrayNumericData(arrow_column, read_column); \ + break; + + FOR_ARROW_NUMERIC_TYPES(DISPATCH) + # undef DISPATCH + default: + throw Exception + { + "Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" + + header_column.name + "\"", + ErrorCodes::UNKNOWN_TYPE + }; + } + } break; # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ case ARROW_NUMERIC_TYPE: \ diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index c1fce04f3b2..12acae89598 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -5,16 +5,19 @@ #include #include #include +#include #include #include #include #include +#include #include #include +#include +#include #include #include - namespace DB { namespace ErrorCodes @@ -63,12 +66,12 @@ namespace DB template static void fillArrowArrayWithNumericColumnData( ColumnPtr write_column, - std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap, - const String & format_name) + const String & format_name, + arrow::ArrayBuilder* abuilder) { const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); - ArrowBuilderType builder; + ArrowBuilderType & builder = assert_cast(*abuilder); arrow::Status status; const UInt8 * arrow_null_bytemap_raw_ptr = nullptr; @@ -91,20 +94,70 @@ namespace DB else status = builder.AppendValues(internal_data.data(), internal_data.size(), reinterpret_cast(arrow_null_bytemap_raw_ptr)); checkStatus(status, write_column->getName(), format_name); + } - status = builder.Finish(&arrow_array); - checkStatus(status, write_column->getName(), format_name); + static void fillArrowArrayWithArrayColumnData( + const String & column_name, + ColumnPtr & nested_column, + const std::shared_ptr & column_nested_type, + std::shared_ptr arrow_array, + const PaddedPODArray * null_bytemap, + arrow::ArrayBuilder * array_builder, + String format_name) + { + const auto * nested_array = typeid_cast(column_nested_type.get()); + const DataTypePtr & nested_data_type = nested_array->getNestedType(); + const String array_nested_type_name = nested_data_type->getFamilyName(); + const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), + [=](auto && elem) { return elem.first == array_nested_type_name; }); + if (arrow_type_it != internal_type_to_arrow_type.end()) + { + std::shared_ptr list_type = arrow::list(arrow_type_it->second); + + const auto & internal_column = assert_cast(*nested_column); + + arrow::ListBuilder & builder = assert_cast(*array_builder); + arrow::ArrayBuilder * value_builder = builder.value_builder(); + arrow::Status components_status; + + const auto & offsets = internal_column.getOffsets(); + const IColumn & data = internal_column.getData(); + + size_t offset_start = 0; + size_t offset_length = 0; + + for (size_t idx = 0, size = internal_column.size(); idx < size; ++idx) + { + if (null_bytemap && (*null_bytemap)[idx]) + { + components_status = builder.AppendNull(); + checkStatus(components_status, nested_column->getName(), format_name); + } + else + { + components_status = builder.Append(); + checkStatus(components_status, nested_column->getName(), format_name); + offset_length = offsets[idx] - offset_start; + if (offset_length > 0) + { + auto cut_data = data.cut(offset_start, offset_length); + CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, nested_data_type, array_nested_type_name, arrow_array, null_bytemap, value_builder, format_name); + } + offset_start = offsets[idx]; + } + } + } } template static void fillArrowArrayWithStringColumnData( ColumnPtr write_column, - std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap, - const String & format_name) + const String & format_name, + arrow::ArrayBuilder* abuilder) { const auto & internal_column = assert_cast(*write_column); - arrow::StringBuilder builder; + arrow::StringBuilder & builder = assert_cast(*abuilder); arrow::Status status; for (size_t string_i = 0, size = internal_column.size(); string_i < size; ++string_i) @@ -121,20 +174,17 @@ namespace DB checkStatus(status, write_column->getName(), format_name); } - - status = builder.Finish(&arrow_array); - checkStatus(status, write_column->getName(), format_name); } static void fillArrowArrayWithDateColumnData( ColumnPtr write_column, - std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap, - const String & format_name) + const String & format_name, + arrow::ArrayBuilder* abuilder) { const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); //arrow::Date32Builder date_builder; - arrow::UInt16Builder builder; + arrow::UInt16Builder & builder = assert_cast(*abuilder); arrow::Status status; for (size_t value_i = 0, size = internal_data.size(); value_i < size; ++value_i) @@ -146,20 +196,17 @@ namespace DB status = builder.Append(internal_data[value_i]); checkStatus(status, write_column->getName(), format_name); } - - status = builder.Finish(&arrow_array); - checkStatus(status, write_column->getName(), format_name); } static void fillArrowArrayWithDateTimeColumnData( ColumnPtr write_column, - std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap, - const String & format_name) + const String & format_name, + arrow::ArrayBuilder* abuilder) { const auto & internal_data = assert_cast &>(*write_column).getData(); //arrow::Date64Builder builder; - arrow::UInt32Builder builder; + arrow::UInt32Builder & builder = assert_cast(*abuilder); arrow::Status status; for (size_t value_i = 0, size = internal_data.size(); value_i < size; ++value_i) @@ -173,9 +220,71 @@ namespace DB checkStatus(status, write_column->getName(), format_name); } + } - status = builder.Finish(&arrow_array); - checkStatus(status, write_column->getName(), format_name); + void CHColumnToArrowColumn::fillArrowArray( + const String & column_name, + ColumnPtr & nested_column, + const std::shared_ptr & column_nested_type, + const String column_nested_type_name, + std::shared_ptr arrow_array, + const PaddedPODArray * null_bytemap, + arrow::ArrayBuilder * array_builder, + String format_name) + { + if ("String" == column_nested_type_name) + { + fillArrowArrayWithStringColumnData(nested_column, null_bytemap, format_name, array_builder); + } + else if ("FixedString" == column_nested_type_name) + { + fillArrowArrayWithStringColumnData(nested_column, null_bytemap, format_name, array_builder); + } + else if ("Date" == column_nested_type_name) + { + fillArrowArrayWithDateColumnData(nested_column, null_bytemap, format_name, array_builder); + } + else if ("DateTime" == column_nested_type_name) + { + fillArrowArrayWithDateTimeColumnData(nested_column, null_bytemap, format_name, array_builder); + } + else if ("Array" == column_nested_type_name) + { + fillArrowArrayWithArrayColumnData(column_name, nested_column, column_nested_type, arrow_array, null_bytemap, + array_builder, format_name); + } + else if (isDecimal(column_nested_type)) + { + auto fill_decimal = [&](const auto & types) -> bool + { + using Types = std::decay_t; + using ToDataType = typename Types::LeftType; + if constexpr ( + std::is_same_v> + || std::is_same_v> + || std::is_same_v>) + { + const auto & decimal_type = static_cast(column_nested_type.get()); + fillArrowArrayWithDecimalColumnData(nested_column, arrow_array, null_bytemap, decimal_type, format_name); + } + return false; + }; + callOnIndexAndDataType(column_nested_type->getTypeId(), fill_decimal); + } + #define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \ + else if (#CPP_NUMERIC_TYPE == column_nested_type_name) \ + { \ + fillArrowArrayWithNumericColumnData(nested_column, null_bytemap, format_name, array_builder); \ + } + + FOR_INTERNAL_NUMERIC_TYPES(DISPATCH) + #undef DISPATCH + else + { + throw Exception{"Internal type \"" + column_nested_type_name + "\" of a column \"" + column_name + "\"" + " is not supported for conversion into a " + format_name + " data format", + ErrorCodes::UNKNOWN_TYPE}; + } } template @@ -231,6 +340,7 @@ namespace DB if (isDecimal(column_nested_type)) { + const auto add_decimal_field = [&](const auto & types) -> bool { using Types = std::decay_t; using ToDataType = typename Types::LeftType; @@ -249,6 +359,24 @@ namespace DB }; callOnIndexAndDataType(column_nested_type->getTypeId(), add_decimal_field); } + else if (isArray(column_nested_type)) + { + const auto * nested_array = typeid_cast(column_nested_type.get()); + const DataTypePtr & nested_data_type = nested_array->getNestedType(); + const String array_nested_type_name = nested_data_type->getFamilyName(); + if (const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), + [=](auto && elem) { return elem.first == array_nested_type_name; }); + arrow_type_it != internal_type_to_arrow_type.end()) + { + arrow_fields.emplace_back(std::make_shared( + column.name, arrow::list(arrow_type_it->second), is_column_nullable)); + } else + { + throw Exception{"The type \"" + array_nested_type_name + "\" of a array column \"" + column.name + "\"" + " is not supported for conversion into a " + format_name + " data format", + ErrorCodes::UNKNOWN_TYPE}; + } + } else { if (const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), @@ -268,57 +396,17 @@ namespace DB = is_column_nullable ? assert_cast(*column.column).getNestedColumnPtr() : column.column; const PaddedPODArray * null_bytemap = is_column_nullable ? extractNullBytemapPtr(column.column) : nullptr; + arrow::MemoryPool* pool = arrow::default_memory_pool(); + std::unique_ptr array_builder; + arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder); + checkStatus(status, nested_column->getName(), format_name); + std::shared_ptr arrow_array; - if ("String" == column_nested_type_name) - { - fillArrowArrayWithStringColumnData(nested_column, arrow_array, null_bytemap, format_name); - } - else if ("FixedString" == column_nested_type_name) - { - fillArrowArrayWithStringColumnData(nested_column, arrow_array, null_bytemap, format_name); - } - else if ("Date" == column_nested_type_name) - { - fillArrowArrayWithDateColumnData(nested_column, arrow_array, null_bytemap, format_name); - } - else if ("DateTime" == column_nested_type_name) - { - fillArrowArrayWithDateTimeColumnData(nested_column, arrow_array, null_bytemap, format_name); - } - else if (isDecimal(column_nested_type)) - { - auto fill_decimal = [&](const auto & types) -> bool - { - using Types = std::decay_t; - using ToDataType = typename Types::LeftType; - if constexpr ( - std::is_same_v> - || std::is_same_v> - || std::is_same_v>) - { - const auto & decimal_type = static_cast(column_nested_type.get()); - fillArrowArrayWithDecimalColumnData(nested_column, arrow_array, null_bytemap, decimal_type, format_name); - } - return false; - }; - callOnIndexAndDataType(column_nested_type->getTypeId(), fill_decimal); - } -#define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \ - else if (#CPP_NUMERIC_TYPE == column_nested_type_name) \ - { \ - fillArrowArrayWithNumericColumnData(nested_column, arrow_array, null_bytemap, format_name); \ - } - - FOR_INTERNAL_NUMERIC_TYPES(DISPATCH) -#undef DISPATCH - else - { - throw Exception{"Internal type \"" + column_nested_type_name + "\" of a column \"" + column.name + "\"" - " is not supported for conversion into a " + format_name + " data format", - ErrorCodes::UNKNOWN_TYPE}; - } + fillArrowArray(column.name, nested_column, column_nested_type, column_nested_type_name, arrow_array, null_bytemap, array_builder.get(), format_name); + status = array_builder->Finish(&arrow_array); + checkStatus(status, nested_column->getName(), format_name); arrow_arrays.emplace_back(std::move(arrow_array)); } @@ -326,6 +414,7 @@ namespace DB res = arrow::Table::Make(arrow_schema, arrow_arrays); } -} + + } #endif diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h index de594389c25..07bceb6266c 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h @@ -30,6 +30,16 @@ private: public: static void chChunkToArrowTable(std::shared_ptr & res, const Block & header, const Chunk & chunk, size_t columns_num, String format_name); + + static void fillArrowArray( + const String & column_name, + ColumnPtr & nested_column, + const std::shared_ptr & column_nested_type, + const String column_nested_type_name, + std::shared_ptr arrow_array, + const PaddedPODArray * null_bytemap, + arrow::ArrayBuilder * array_builder, + String format_name); }; } #endif diff --git a/tests/queries/0_stateless/00900_parquet_load.reference b/tests/queries/0_stateless/00900_parquet_load.reference index 1c890119486..65973e05c24 100644 --- a/tests/queries/0_stateless/00900_parquet_load.reference +++ b/tests/queries/0_stateless/00900_parquet_load.reference @@ -13,6 +13,39 @@ === Try load data from alltypes_plain.snappy.parquet 6 1 0 0 0 0 0 0 04/01/09 0 1238544000 7 0 1 1 1 10 1.1 10.1 04/01/09 1 1238544060 +=== Try load data from array_float.parquet +idx1 [] +idx2 [10.2,8.2,7.2] +idx3 [10.2,8.2] +idx4 [10.2] +idx5 [10.2,8.2] +idx6 [10.2] +idx7 [10.2,8.2] +idx8 [10.2,8.2] +idx9 [10.2] +idx10 [10.2,8.2] +=== Try load data from array_int.parquet +idx1 [100,101,102] +idx2 [100,101] +idx3 [100,101,102,101] +idx4 [100] +idx5 [100,101] +idx6 [100,101] +idx7 [100,101] +idx8 [100,101] +idx9 [100,101,102] +idx10 [100,101,102] +=== Try load data from array_string.parquet +idx1 ['This','is','a','test'] +idx2 ['cigarette','smoke'] +idx3 ['the','grocery','clerks'] +idx4 [] +idx5 ['wants','to','get','out'] +idx6 ['me','up?'] +idx7 ['then','I','put','him','back'] +idx8 ['make','a','man'] +idx9 ['Which','Heaven','to','gaudy','day','denies'] +idx10 ['This','is','a','test'] === Try load data from binary.parquet \0  diff --git a/tests/queries/0_stateless/data_parquet/array_float.parquet b/tests/queries/0_stateless/data_parquet/array_float.parquet new file mode 100755 index 0000000000000000000000000000000000000000..10c693c211673e02e86833544dd47db2ee5328ec GIT binary patch literal 520 zcmZWny-or_5Z?RYyhJo$mn5{~6JG0JtCP!J0%4B6xMpOE zgoL=7E|wsILPFV)QhE&{0TC;9XjRe237fJRn=gs3nU*=|sTQ=%P*I7wkpSG+zXXh) zdh+iHd*6bE4!VkL#&w%o{&(-udN!LjK?2cCVE4DnERrZ=yi}H4E zyTf?s+WmA@PIEh;DGhSUqBM2#faPgcM2Y89=K6k8cyW>YMO0+PA=woO8RG}|0T44{ A?EnA( literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_parquet/array_int.parquet.columns b/tests/queries/0_stateless/data_parquet/array_int.parquet.columns new file mode 100644 index 00000000000..3b9395715d7 --- /dev/null +++ b/tests/queries/0_stateless/data_parquet/array_int.parquet.columns @@ -0,0 +1 @@ +`idx` String, `lst` Array(Int32) \ No newline at end of file diff --git a/tests/queries/0_stateless/data_parquet/array_string.parquet b/tests/queries/0_stateless/data_parquet/array_string.parquet new file mode 100755 index 0000000000000000000000000000000000000000..556c15fda8c72bb2c63c045ff8852337a44459f5 GIT binary patch literal 725 zcmZuv&5F}d6i$;gRyu-6Z)l*4BD7SfL({a=1_q=s!W3K#BFsjZxk*l%OPjQl+*FG% zAU=X`APlo};l`aCA3|{B1BgpS&q=cp4JYTj-1D99yMJf1r#4gAO$DRR8aOmqt<}^; z@X{@^Z7TSw_%|v#W-Qnx4oh@O9F;gO@ubwjQQR$<_T$^N5YTD?iBbIim3(9gB=M3yjVy8 ziGU2x^pain*$M8}U`DH1Etb_Z_Kha3g>>pSAHIClXfO9P?bq-7TD9JOJ$vK!-hF!e zYVdjf`2G3U>)Bn>SD0!&Kdnb%3m2ImLCo>^*g;2aYIr{#MH%C0l}V;DLmBH$&EpHE z?iqJ@n)2NRtRRL&BG+tKEvw(Hwb+3*R*afeh%lTte^fBKwc~%@L-%Ke=pEg#res70 ze+y&(iTPc}K<@GX_H2GL%!bmNZ+Lo@gM1jL?JcC4SSM}!aC&IBdtNS9L3_kKJ_ Date: Fri, 2 Apr 2021 10:59:32 +0100 Subject: [PATCH 03/82] fix list_nested_type uninitialized compile error --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 85cb7563b60..9d5a551eb48 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -561,7 +561,7 @@ namespace DB std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; arrow::Type::type arrow_type = arrow_column->type()->id(); - arrow::Type::type list_nested_type; + std::shared_ptr list_nested_type; // TODO: check if a column is const? if (!column_type->isNullable() && arrow_column->null_count()) @@ -584,10 +584,10 @@ namespace DB else if (arrow_type == arrow::Type::LIST) { const auto * list_type = static_cast(arrow_column->type().get()); - list_nested_type = list_type->value_type()->id(); + list_nested_type = list_type->value_type(); if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), - [=](auto && elem) { return elem.first == list_nested_type; }); + [=](auto && elem) { return elem.first == list_nested_type->id(); }); internal_type_it != arrow_type_to_internal_type.end()) { array_nested_type = DataTypeFactory::instance().get(internal_type_it->second); @@ -646,8 +646,8 @@ namespace DB fillColumnWithDecimalData(arrow_column, read_column /*, internal_nested_type*/); break; case arrow::Type::LIST: - if (array_nested_type) { - switch (list_nested_type) + if (array_nested_type && list_nested_type) { + switch (list_nested_type->id()) { case arrow::Type::STRING: case arrow::Type::BINARY: From 4a6345c210becdc27e3a71451a15810bc45efdde Mon Sep 17 00:00:00 2001 From: "chou.fan" Date: Sun, 11 Apr 2021 15:05:51 +0100 Subject: [PATCH 04/82] add array nullable datatype support --- .../Formats/Impl/CHColumnToArrowColumn.cpp | 60 ++++++++++++++----- 1 file changed, 46 insertions(+), 14 deletions(-) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 12acae89598..9ed1c5eb1a7 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -99,17 +99,26 @@ namespace DB static void fillArrowArrayWithArrayColumnData( const String & column_name, ColumnPtr & nested_column, - const std::shared_ptr & column_nested_type, + const std::shared_ptr & column_type, std::shared_ptr arrow_array, const PaddedPODArray * null_bytemap, arrow::ArrayBuilder * array_builder, String format_name) { - const auto * nested_array = typeid_cast(column_nested_type.get()); - const DataTypePtr & nested_data_type = nested_array->getNestedType(); - const String array_nested_type_name = nested_data_type->getFamilyName(); + const auto * column_array = static_cast(nested_column.get()); + const bool is_column_array_nullable = column_array->getData().isNullable(); + const IColumn & array_nested_column = + is_column_array_nullable ? static_cast(column_array->getData()).getNestedColumn() : + column_array->getData(); + const String column_array_nested_type_name = array_nested_column.getFamilyName(); + + const auto * column_array_type = static_cast(column_type.get()); + const DataTypePtr & array_nested_type = + is_column_array_nullable ? static_cast(column_array_type->getNestedType().get())->getNestedType() : + column_array_type->getNestedType(); + const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), - [=](auto && elem) { return elem.first == array_nested_type_name; }); + [=](auto && elem) { return elem.first == column_array_nested_type_name; }); if (arrow_type_it != internal_type_to_arrow_type.end()) { std::shared_ptr list_type = arrow::list(arrow_type_it->second); @@ -141,7 +150,23 @@ namespace DB if (offset_length > 0) { auto cut_data = data.cut(offset_start, offset_length); - CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, nested_data_type, array_nested_type_name, arrow_array, null_bytemap, value_builder, format_name); + if (null_bytemap == nullptr) + { + CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, + column_array_nested_type_name, arrow_array, + nullptr, value_builder, format_name); + } + else + { + PaddedPODArray * array_nested_null_bytemap = new PaddedPODArray(); + for (size_t pidx = offset_start; pidx < offset_start + offset_length; ++pidx) + { + array_nested_null_bytemap->push_back((*null_bytemap)[pidx]); + } + CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, + column_array_nested_type_name, arrow_array, array_nested_null_bytemap, value_builder, format_name); + delete array_nested_null_bytemap; + } } offset_start = offsets[idx]; } @@ -334,13 +359,13 @@ namespace DB column.type = recursiveRemoveLowCardinality(column.type); const bool is_column_nullable = column.type->isNullable(); + bool is_column_array_nullable = false; const auto & column_nested_type = is_column_nullable ? static_cast(column.type.get())->getNestedType() : column.type; const String column_nested_type_name = column_nested_type->getFamilyName(); if (isDecimal(column_nested_type)) { - const auto add_decimal_field = [&](const auto & types) -> bool { using Types = std::decay_t; using ToDataType = typename Types::LeftType; @@ -361,18 +386,22 @@ namespace DB } else if (isArray(column_nested_type)) { - const auto * nested_array = typeid_cast(column_nested_type.get()); - const DataTypePtr & nested_data_type = nested_array->getNestedType(); - const String array_nested_type_name = nested_data_type->getFamilyName(); + const auto * column_array_type = static_cast(column_nested_type.get()); + is_column_array_nullable = column_array_type->getNestedType()->isNullable(); + const DataTypePtr & column_array_nested_type = + is_column_array_nullable ? static_cast(column_array_type->getNestedType().get())->getNestedType() : + column_array_type->getNestedType(); + const String column_array_nested_type_name = column_array_nested_type->getFamilyName(); + if (const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), - [=](auto && elem) { return elem.first == array_nested_type_name; }); + [=](auto && elem) { return elem.first == column_array_nested_type_name; }); arrow_type_it != internal_type_to_arrow_type.end()) { arrow_fields.emplace_back(std::make_shared( - column.name, arrow::list(arrow_type_it->second), is_column_nullable)); + column.name, arrow::list(arrow_type_it->second), is_column_array_nullable)); } else { - throw Exception{"The type \"" + array_nested_type_name + "\" of a array column \"" + column.name + "\"" + throw Exception{"The type \"" + column_array_nested_type_name + "\" of a array column \"" + column.name + "\"" " is not supported for conversion into a " + format_name + " data format", ErrorCodes::UNKNOWN_TYPE}; } @@ -394,7 +423,10 @@ namespace DB ColumnPtr nested_column = is_column_nullable ? assert_cast(*column.column).getNestedColumnPtr() : column.column; - const PaddedPODArray * null_bytemap = is_column_nullable ? extractNullBytemapPtr(column.column) : nullptr; + + const PaddedPODArray * null_bytemap = + is_column_nullable ? extractNullBytemapPtr(column.column) : + is_column_array_nullable ? extractNullBytemapPtr(assert_cast(*nested_column).getDataPtr()) : nullptr; arrow::MemoryPool* pool = arrow::default_memory_pool(); std::unique_ptr array_builder; From 306bfeb576519efe62208126917a09208d8cd786 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 25 Apr 2021 09:58:57 +0300 Subject: [PATCH 05/82] clickhouse-format utility and links in index file --- .../operations/utilities/clickhouse-format.md | 96 +++++++++++++++++++ docs/en/operations/utilities/index.md | 5 +- .../operations/utilities/clickhouse-format.md | 96 +++++++++++++++++++ docs/ru/operations/utilities/index.md | 7 +- 4 files changed, 202 insertions(+), 2 deletions(-) create mode 100644 docs/en/operations/utilities/clickhouse-format.md create mode 100644 docs/ru/operations/utilities/clickhouse-format.md diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md new file mode 100644 index 00000000000..e80a22dd5c9 --- /dev/null +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -0,0 +1,96 @@ +--- +toc_priority: 65 +toc_title: clickhouse-format +--- + +# clickhouse-format {#clickhouse-format} + +The `clickhouse-format` program enables formatting input queries. + +Keys: + +- `--help` or`-h` — Produce help message +- `--hilite` — Add syntax highlight with ANSI terminal escape sequences +- `--oneline` — Format in single line +- `--quiet` or `-q` — Just check syntax, no output on success +- `--multiquery` or `-n` — Allow multiple queries in the same file +- `--obfuscate` — Obfuscate instead of formatting +- `--seed ` — Seed arbitrary string that determines the result of obfuscation +- `--backslash` — Add a backslash at the end of each line of the formatted query. Can be useful when you copy a query from web or somewhere else with multiple lines, and want to execute it in command line. + +## Examples {#examples} + +1. Example with highlighting and single line: + + ```bash + $clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" + ``` + + Result: + + ```sql + SELECT sum(number) FROM numbers(5) + ``` + +2. Example with multiqueries: + + ```bash + $clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" + ``` + + Result: + + ```text + SELECT 1 + ; + + SELECT 1 + UNION ALL + ( + SELECT 1 + UNION DISTINCT + SELECT 3 + ) + ; + ``` +3. Example with obfuscating: + + ```bash + $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + Result: + + ```text + SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; + ``` + + Another seed string: + + ```bash + $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + Result: + + ```text + SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; + ``` + +4. Example with backslash: + + ```bash + $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" + ``` + + Result: + + ```text + SELECT * \ + FROM \ + ( \ + SELECT 1 AS x \ + UNION ALL \ + SELECT 1 \ + UNION DISTINCT \ + SELECT 3 \ + ) + ``` \ No newline at end of file diff --git a/docs/en/operations/utilities/index.md b/docs/en/operations/utilities/index.md index fe5048f7044..4adbb299b1d 100644 --- a/docs/en/operations/utilities/index.md +++ b/docs/en/operations/utilities/index.md @@ -9,5 +9,8 @@ toc_title: Overview - [clickhouse-local](../../operations/utilities/clickhouse-local.md) — Allows running SQL queries on data without stopping the ClickHouse server, similar to how `awk` does this. - [clickhouse-copier](../../operations/utilities/clickhouse-copier.md) — Copies (and reshards) data from one cluster to another cluster. - [clickhouse-benchmark](../../operations/utilities/clickhouse-benchmark.md) — Loads server with the custom queries and settings. +- [clickhouse-format](../../operations/utilities/clickhouse-format.md) — Enables formatting input queries. +- [ClickHouse obfuscator](../../operations/utilities/clickhouse-obfuscator.md) — Obfuscates data. +- [ClickHouse compressor](../../operations/utilities/clickhouse-compressor.md) — Compresses and decompresses data. +- [clickhouse-odbc-bridge](../../operations/utilities/odbc-bridge.md) — A proxy server for ODBC driver. -[Original article](https://clickhouse.tech/docs/en/operations/utils/) diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md new file mode 100644 index 00000000000..e28bf9d40d0 --- /dev/null +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -0,0 +1,96 @@ +--- +toc_priority: 65 +toc_title: clickhouse-format +--- + +# clickhouse-format {#clickhouse-format} + +Утилита `clickhouse-format` позволяет форматировать входящие запросы. + +Ключи: + +- `--help` или`-h` — выводит описание ключей +- `--hilite` — добавляет подсветку синтаксиса с экранированием символов +- `--oneline` — форматирование в одну строку +- `--quiet` или `-q` — проверяет синтаксис без вывода результата +- `--multiquery` or `-n` — поддерживает несколько запросов в одной строке +- `--obfuscate` — обфускирует вместо форматирования +- `--seed <строка>` — задает строку, которая определяет результат обфускации +- `--backslash` — добавляет обратный слеш в конце каждой строки отформатированного запроса. Удобно использовать, если вы скопировали многострочный запрос из интернета или другого источника и хотите выполнить его из командной строки. + +## Примеры {#examples} + +1. Пример с подсветкой синтаксиса и форматированием в одну строку: + + ```bash + $clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" + ``` + + Результат: + + ```sql + SELECT sum(number) FROM numbers(5) + ``` + +2. Пример с несколькими запросами в одной строке: + + ```bash + $clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" + ``` + + Результат: + + ```text + SELECT 1 + ; + + SELECT 1 + UNION ALL + ( + SELECT 1 + UNION DISTINCT + SELECT 3 + ) + ; + ``` +3. Пример с обфуксацией: + + ```bash + $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + Результат: + + ```text + SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; + ``` + + Другая строка для обфускации: + + ```bash + $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + Результат: + + ```text + SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; + ``` + +4. Пример с обратным слешем: + + ```bash + $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" + ``` + + Результат: + + ```text + SELECT * \ + FROM \ + ( \ + SELECT 1 AS x \ + UNION ALL \ + SELECT 1 \ + UNION DISTINCT \ + SELECT 3 \ + ) + ``` \ No newline at end of file diff --git a/docs/ru/operations/utilities/index.md b/docs/ru/operations/utilities/index.md index fa257fb4b1a..66f096d687f 100644 --- a/docs/ru/operations/utilities/index.md +++ b/docs/ru/operations/utilities/index.md @@ -6,6 +6,11 @@ toc_title: "Обзор" # Утилиты ClickHouse {#utility-clickhouse} -- [clickhouse-local](clickhouse-local.md) +- [clickhouse-local](clickhouse-local.md) - позволяет выполнять SQL-запросы над данными без остановки сервера ClickHouse server, подобно утилите `awk`. - [clickhouse-copier](clickhouse-copier.md) - копирует (и перешардирует) данные с одного кластера на другой. +- [clickhouse-benchmark](../../operations/utilities/clickhouse-benchmark.md) — устанавливает соединение с сервером ClickHouse и запускает циклическое выполнение указанных запросов. +- [clickhouse-format](../../operations/utilities/clickhouse-format.md) — позволяет форматировать входящие запросы. +- [ClickHouse obfuscator](../../operations/utilities/clickhouse-obfuscator.md) — обфускирует данные. +- [ClickHouse compressor](https://clickhouse.tech/docs/en/operations/utilities/clickhouse-compressor) — упаковывает и распаковывает данные. +- [clickhouse-odbc-bridge](https://clickhouse.tech/docs/en/operations/utilities/odbc-bridge) — прокси-сервер для ODBC. From e815f90b5382c7001b85c0117201c94602848bae Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 25 Apr 2021 10:09:43 +0300 Subject: [PATCH 06/82] Fix --- .../operations/utilities/clickhouse-format.md | 63 ++++++++++--------- .../operations/utilities/clickhouse-format.md | 61 +++++++++--------- 2 files changed, 63 insertions(+), 61 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index e80a22dd5c9..1124724166d 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -55,42 +55,43 @@ Keys: ``` 3. Example with obfuscating: - ```bash - $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` - Result: + ```bash + $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + + Result: - ```text - SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; - ``` + ```text + SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; + ``` - Another seed string: + Another seed string: - ```bash - $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` - Result: - - ```text - SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; - ``` + ```bash + $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + Result: + + ```text + SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; + ``` 4. Example with backslash: - ```bash - $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" - ``` + ```bash + $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" + ``` - Result: + Result: - ```text - SELECT * \ - FROM \ - ( \ - SELECT 1 AS x \ - UNION ALL \ - SELECT 1 \ - UNION DISTINCT \ - SELECT 3 \ - ) - ``` \ No newline at end of file + ```text + SELECT * \ + FROM \ + ( \ + SELECT 1 AS x \ + UNION ALL \ + SELECT 1 \ + UNION DISTINCT \ + SELECT 3 \ + ) + ``` \ No newline at end of file diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index e28bf9d40d0..56e260d4b60 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -55,42 +55,43 @@ toc_title: clickhouse-format ``` 3. Пример с обфуксацией: - ```bash - $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` - Результат: + ```bash + $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + Результат: - ```text - SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; - ``` + ```text + SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; + ``` - Другая строка для обфускации: + Другая строка для обфускации: - ```bash - $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` - Результат: + ```bash + $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" + ``` + + Результат: - ```text - SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; - ``` + ```text + SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; + ``` 4. Пример с обратным слешем: - ```bash - $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" - ``` + ```bash + $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" + ``` - Результат: + Результат: - ```text - SELECT * \ - FROM \ - ( \ - SELECT 1 AS x \ - UNION ALL \ - SELECT 1 \ - UNION DISTINCT \ - SELECT 3 \ - ) - ``` \ No newline at end of file + ```text + SELECT * \ + FROM \ + ( \ + SELECT 1 AS x \ + UNION ALL \ + SELECT 1 \ + UNION DISTINCT \ + SELECT 3 \ + ) + ``` \ No newline at end of file From 64e85c3deb626558975eb97de0c84c6d207dbeb8 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 25 Apr 2021 10:17:14 +0300 Subject: [PATCH 07/82] Another fix in md --- docs/en/operations/utilities/clickhouse-format.md | 1 + docs/ru/operations/utilities/clickhouse-format.md | 1 + 2 files changed, 2 insertions(+) diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index 1124724166d..eacef5191f7 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -53,6 +53,7 @@ Keys: ) ; ``` + 3. Example with obfuscating: ```bash diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index 56e260d4b60..24e63802328 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -53,6 +53,7 @@ toc_title: clickhouse-format ) ; ``` + 3. Пример с обфуксацией: ```bash From 4d2fd45660edc4fdf3a73f00ce5150e89eec8421 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 25 Apr 2021 10:21:46 +0300 Subject: [PATCH 08/82] Formatting --- .../operations/utilities/clickhouse-format.md | 107 +++++++++--------- .../operations/utilities/clickhouse-format.md | 107 +++++++++--------- 2 files changed, 108 insertions(+), 106 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index eacef5191f7..70b24b027af 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -22,77 +22,78 @@ Keys: 1. Example with highlighting and single line: - ```bash - $clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" - ``` +```bash +$clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" +``` - Result: +Result: - ```sql - SELECT sum(number) FROM numbers(5) - ``` +```sql +SELECT sum(number) FROM numbers(5) +``` 2. Example with multiqueries: - ```bash - $clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" - ``` +```bash +$clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +``` - Result: +Result: - ```text - SELECT 1 - ; +```text +SELECT 1 +; +SELECT 1 +UNION ALL +( SELECT 1 - UNION ALL - ( - SELECT 1 - UNION DISTINCT - SELECT 3 - ) - ; - ``` - + UNION DISTINCT + SELECT 3 +) +; +``` + 3. Example with obfuscating: - ```bash - $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` +```bash +$clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" +``` - Result: +Result: - ```text - SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; - ``` +```text +SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; +``` - Another seed string: +Another seed string: - ```bash - $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` - Result: +```bash +$ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" +``` - ```text - SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; - ``` +Result: + +```text +SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; +``` 4. Example with backslash: - ```bash - $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" - ``` +```bash +$clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +``` - Result: +Result: - ```text - SELECT * \ - FROM \ - ( \ - SELECT 1 AS x \ - UNION ALL \ - SELECT 1 \ - UNION DISTINCT \ - SELECT 3 \ - ) - ``` \ No newline at end of file +```text +SELECT * \ +FROM \ +( \ + SELECT 1 AS x \ + UNION ALL \ + SELECT 1 \ + UNION DISTINCT \ + SELECT 3 \ +) +``` \ No newline at end of file diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index 24e63802328..91a921dacd5 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -22,77 +22,78 @@ toc_title: clickhouse-format 1. Пример с подсветкой синтаксиса и форматированием в одну строку: - ```bash - $clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" - ``` +```bash +$clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" +``` - Результат: +Результат: - ```sql - SELECT sum(number) FROM numbers(5) - ``` +```sql +SELECT sum(number) FROM numbers(5) +``` 2. Пример с несколькими запросами в одной строке: - ```bash - $clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" - ``` +```bash +$clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +``` - Результат: +Результат: - ```text - SELECT 1 - ; +```text +SELECT 1 +; +SELECT 1 +UNION ALL +( SELECT 1 - UNION ALL - ( - SELECT 1 - UNION DISTINCT - SELECT 3 - ) - ; - ``` - + UNION DISTINCT + SELECT 3 +) +; +``` + 3. Пример с обфуксацией: - ```bash - $clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` - Результат: +```bash +$clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" +``` - ```text - SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; - ``` +Результат: + +```text +SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; +``` - Другая строка для обфускации: +Другая строка для обфускации: - ```bash - $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" - ``` +```bash +$ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" +``` - Результат: +Результат: - ```text - SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; - ``` +```text +SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; +``` 4. Пример с обратным слешем: - ```bash - $clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" - ``` +```bash +$clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +``` - Результат: +Результат: - ```text - SELECT * \ - FROM \ - ( \ - SELECT 1 AS x \ - UNION ALL \ - SELECT 1 \ - UNION DISTINCT \ - SELECT 3 \ - ) - ``` \ No newline at end of file +```text +SELECT * \ +FROM \ +( \ + SELECT 1 AS x \ + UNION ALL \ + SELECT 1 \ + UNION DISTINCT \ + SELECT 3 \ +) +``` \ No newline at end of file From 7b0cea45736ebcc059901b82ba856f83d6d57330 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Sun, 25 Apr 2021 11:47:05 +0300 Subject: [PATCH 09/82] Minor fixes --- docs/en/operations/utilities/clickhouse-format.md | 8 ++++---- docs/ru/operations/utilities/clickhouse-format.md | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index 70b24b027af..daf7c5ee943 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -23,7 +23,7 @@ Keys: 1. Example with highlighting and single line: ```bash -$clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" +$ clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" ``` Result: @@ -35,7 +35,7 @@ SELECT sum(number) FROM numbers(5) 2. Example with multiqueries: ```bash -$clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +$ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" ``` Result: @@ -57,7 +57,7 @@ UNION ALL 3. Example with obfuscating: ```bash -$clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" +$ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" ``` Result: @@ -81,7 +81,7 @@ SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 4. Example with backslash: ```bash -$clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +$ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" ``` Result: diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index 91a921dacd5..08a14bbe7d0 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -23,7 +23,7 @@ toc_title: clickhouse-format 1. Пример с подсветкой синтаксиса и форматированием в одну строку: ```bash -$clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" +$ clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" ``` Результат: @@ -35,7 +35,7 @@ SELECT sum(number) FROM numbers(5) 2. Пример с несколькими запросами в одной строке: ```bash -$clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +$ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" ``` Результат: @@ -57,7 +57,7 @@ UNION ALL 3. Пример с обфуксацией: ```bash -$clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" +$ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" ``` Результат: @@ -81,7 +81,7 @@ SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 4. Пример с обратным слешем: ```bash -$clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" +$ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" ``` Результат: From 1a3557d6a2fc0219e3610ff58015fc6867e6adda Mon Sep 17 00:00:00 2001 From: adevyatova Date: Mon, 26 Apr 2021 17:17:20 +0000 Subject: [PATCH 10/82] update --- .../functions/ext-dict-functions.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 5fc146f603f..2f270d49643 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -100,6 +100,23 @@ LIMIT 3 - [External Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) +## dictGetOrNull {#dictgetornull } + +Retrieves a value from an external dictionary. It works like `dictGet`, but return `NULL` in case key was not found in dictionary. + +``` sql +dictGetOrNull('dict_name', 'attr_name', id_expr) +``` + +**Arguments** + + + +**Returned value** + + +Type: + ## dictHas {#dicthas} Checks whether a key is present in a dictionary. From 96f739025cf66c0860f45b7f13b34176eb0a1168 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Thu, 29 Apr 2021 06:27:21 +0300 Subject: [PATCH 11/82] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../operations/utilities/clickhouse-format.md | 26 +++++++++---------- .../operations/utilities/clickhouse-format.md | 12 ++++----- docs/ru/operations/utilities/index.md | 3 +-- 3 files changed, 20 insertions(+), 21 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index daf7c5ee943..25e2c400bf5 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -5,22 +5,22 @@ toc_title: clickhouse-format # clickhouse-format {#clickhouse-format} -The `clickhouse-format` program enables formatting input queries. +Allows formatting input queries. Keys: -- `--help` or`-h` — Produce help message -- `--hilite` — Add syntax highlight with ANSI terminal escape sequences -- `--oneline` — Format in single line -- `--quiet` or `-q` — Just check syntax, no output on success -- `--multiquery` or `-n` — Allow multiple queries in the same file -- `--obfuscate` — Obfuscate instead of formatting -- `--seed ` — Seed arbitrary string that determines the result of obfuscation +- `--help` or`-h` — Produce help message. +- `--hilite` — Add syntax highlight with ANSI terminal escape sequences. +- `--oneline` — Format in single line. +- `--quiet` or `-q` — Just check syntax, no output on success. +- `--multiquery` or `-n` — Allow multiple queries in the same file. +- `--obfuscate` — Obfuscate instead of formatting. +- `--seed ` — Seed arbitrary string that determines the result of obfuscation. - `--backslash` — Add a backslash at the end of each line of the formatted query. Can be useful when you copy a query from web or somewhere else with multiple lines, and want to execute it in command line. ## Examples {#examples} -1. Example with highlighting and single line: +1. Highlighting and single line: ```bash $ clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" @@ -32,7 +32,7 @@ Result: SELECT sum(number) FROM numbers(5) ``` -2. Example with multiqueries: +2. Multiqueries: ```bash $ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" @@ -54,7 +54,7 @@ UNION ALL ; ``` -3. Example with obfuscating: +3. Obfuscating: ```bash $ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" @@ -78,7 +78,7 @@ Result: SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; ``` -4. Example with backslash: +4. Adding backslash: ```bash $ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" @@ -96,4 +96,4 @@ FROM \ UNION DISTINCT \ SELECT 3 \ ) -``` \ No newline at end of file +``` diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index 08a14bbe7d0..9ef7c8d402c 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -5,7 +5,7 @@ toc_title: clickhouse-format # clickhouse-format {#clickhouse-format} -Утилита `clickhouse-format` позволяет форматировать входящие запросы. +Позволяет форматировать входящие запросы. Ключи: @@ -20,7 +20,7 @@ toc_title: clickhouse-format ## Примеры {#examples} -1. Пример с подсветкой синтаксиса и форматированием в одну строку: +1. Подсветка синтаксиса и форматирование в одну строку: ```bash $ clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" @@ -32,7 +32,7 @@ $ clickhouse-format --oneline --hilite <<< "SELECT sum(number) FROM numbers(5);" SELECT sum(number) FROM numbers(5) ``` -2. Пример с несколькими запросами в одной строке: +2. Несколько запросов в одной строке: ```bash $ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" @@ -54,7 +54,7 @@ UNION ALL ; ``` -3. Пример с обфуксацией: +3. Обфуксация: ```bash $ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" @@ -78,7 +78,7 @@ $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWE SELECT horse_tape_summer BETWEEN folklore AND moccasins, CASE WHEN intestine >= 116 THEN nonconformist ELSE FORESTRY END; ``` -4. Пример с обратным слешем: +4. Добавление обратного слеша: ```bash $ clickhouse-format --backslash <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNION DISTINCT SELECT 3);" @@ -96,4 +96,4 @@ FROM \ UNION DISTINCT \ SELECT 3 \ ) -``` \ No newline at end of file +``` diff --git a/docs/ru/operations/utilities/index.md b/docs/ru/operations/utilities/index.md index 66f096d687f..00604e4b6ed 100644 --- a/docs/ru/operations/utilities/index.md +++ b/docs/ru/operations/utilities/index.md @@ -6,11 +6,10 @@ toc_title: "Обзор" # Утилиты ClickHouse {#utility-clickhouse} -- [clickhouse-local](clickhouse-local.md) - позволяет выполнять SQL-запросы над данными без остановки сервера ClickHouse server, подобно утилите `awk`. +- [clickhouse-local](clickhouse-local.md) - позволяет выполнять SQL-запросы над данными без остановки сервера ClickHouse, подобно утилите `awk`. - [clickhouse-copier](clickhouse-copier.md) - копирует (и перешардирует) данные с одного кластера на другой. - [clickhouse-benchmark](../../operations/utilities/clickhouse-benchmark.md) — устанавливает соединение с сервером ClickHouse и запускает циклическое выполнение указанных запросов. - [clickhouse-format](../../operations/utilities/clickhouse-format.md) — позволяет форматировать входящие запросы. - [ClickHouse obfuscator](../../operations/utilities/clickhouse-obfuscator.md) — обфускирует данные. - [ClickHouse compressor](https://clickhouse.tech/docs/en/operations/utilities/clickhouse-compressor) — упаковывает и распаковывает данные. - [clickhouse-odbc-bridge](https://clickhouse.tech/docs/en/operations/utilities/odbc-bridge) — прокси-сервер для ODBC. - From b9e5a288294466d8037715b1c55497e31fa1f7fe Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Thu, 29 Apr 2021 06:28:00 +0300 Subject: [PATCH 12/82] Update docs/ru/operations/utilities/clickhouse-format.md --- docs/ru/operations/utilities/clickhouse-format.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index 9ef7c8d402c..2ce5ec924de 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -9,13 +9,13 @@ toc_title: clickhouse-format Ключи: -- `--help` или`-h` — выводит описание ключей -- `--hilite` — добавляет подсветку синтаксиса с экранированием символов -- `--oneline` — форматирование в одну строку -- `--quiet` или `-q` — проверяет синтаксис без вывода результата -- `--multiquery` or `-n` — поддерживает несколько запросов в одной строке -- `--obfuscate` — обфускирует вместо форматирования -- `--seed <строка>` — задает строку, которая определяет результат обфускации +- `--help` или`-h` — выводит описание ключей. +- `--hilite` — добавляет подсветку синтаксиса с экранированием символов. +- `--oneline` — форматирование в одну строку. +- `--quiet` или `-q` — проверяет синтаксис без вывода результата. +- `--multiquery` or `-n` — поддерживает несколько запросов в одной строке. +- `--obfuscate` — обфускирует вместо форматирования. +- `--seed <строка>` — задает строку, которая определяет результат обфускации. - `--backslash` — добавляет обратный слеш в конце каждой строки отформатированного запроса. Удобно использовать, если вы скопировали многострочный запрос из интернета или другого источника и хотите выполнить его из командной строки. ## Примеры {#examples} From 4c40b7a9df3328b178520227ac44661345d0b857 Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Thu, 29 Apr 2021 06:39:01 +0300 Subject: [PATCH 13/82] Apply suggestions from code review Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/operations/utilities/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/utilities/index.md b/docs/ru/operations/utilities/index.md index 00604e4b6ed..4397953073f 100644 --- a/docs/ru/operations/utilities/index.md +++ b/docs/ru/operations/utilities/index.md @@ -10,6 +10,6 @@ toc_title: "Обзор" - [clickhouse-copier](clickhouse-copier.md) - копирует (и перешардирует) данные с одного кластера на другой. - [clickhouse-benchmark](../../operations/utilities/clickhouse-benchmark.md) — устанавливает соединение с сервером ClickHouse и запускает циклическое выполнение указанных запросов. - [clickhouse-format](../../operations/utilities/clickhouse-format.md) — позволяет форматировать входящие запросы. -- [ClickHouse obfuscator](../../operations/utilities/clickhouse-obfuscator.md) — обфускирует данные. +- [ClickHouse obfuscator](../../operations/utilities/clickhouse-obfuscator.md) — обфусцирует данные. - [ClickHouse compressor](https://clickhouse.tech/docs/en/operations/utilities/clickhouse-compressor) — упаковывает и распаковывает данные. - [clickhouse-odbc-bridge](https://clickhouse.tech/docs/en/operations/utilities/odbc-bridge) — прокси-сервер для ODBC. From 1f1ec13ac5269f2c9db5ad852027fd3772212c18 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Fri, 30 Apr 2021 10:00:53 +0300 Subject: [PATCH 14/82] Links updated --- .../utilities/clickhouse-compressor.md | 27 +++++++++++++ docs/ru/operations/utilities/index.md | 4 +- docs/ru/operations/utilities/odbc-bridge.md | 38 +++++++++++++++++++ 3 files changed, 67 insertions(+), 2 deletions(-) create mode 100644 docs/ru/operations/utilities/clickhouse-compressor.md create mode 100644 docs/ru/operations/utilities/odbc-bridge.md diff --git a/docs/ru/operations/utilities/clickhouse-compressor.md b/docs/ru/operations/utilities/clickhouse-compressor.md new file mode 100644 index 00000000000..44a1f052824 --- /dev/null +++ b/docs/ru/operations/utilities/clickhouse-compressor.md @@ -0,0 +1,27 @@ +## ClickHouse compressor + +Simple program for data compression and decompression. + +### Examples + +Compress data with LZ4: +``` +$ ./clickhouse-compressor < input_file > output_file +``` + +Decompress data from LZ4 format: +``` +$ ./clickhouse-compressor --decompress < input_file > output_file +``` + +Compress data with ZSTD at level 5: + +``` +$ ./clickhouse-compressor --codec 'ZSTD(5)' < input_file > output_file +``` + +Compress data with Delta of four bytes and ZSTD level 10. + +``` +$ ./clickhouse-compressor --codec 'Delta(4)' --codec 'ZSTD(10)' < input_file > output_file +``` diff --git a/docs/ru/operations/utilities/index.md b/docs/ru/operations/utilities/index.md index 4397953073f..88bd0305386 100644 --- a/docs/ru/operations/utilities/index.md +++ b/docs/ru/operations/utilities/index.md @@ -11,5 +11,5 @@ toc_title: "Обзор" - [clickhouse-benchmark](../../operations/utilities/clickhouse-benchmark.md) — устанавливает соединение с сервером ClickHouse и запускает циклическое выполнение указанных запросов. - [clickhouse-format](../../operations/utilities/clickhouse-format.md) — позволяет форматировать входящие запросы. - [ClickHouse obfuscator](../../operations/utilities/clickhouse-obfuscator.md) — обфусцирует данные. -- [ClickHouse compressor](https://clickhouse.tech/docs/en/operations/utilities/clickhouse-compressor) — упаковывает и распаковывает данные. -- [clickhouse-odbc-bridge](https://clickhouse.tech/docs/en/operations/utilities/odbc-bridge) — прокси-сервер для ODBC. +- [ClickHouse compressor](../../operations/utilities/clickhouse-compressor.md) — упаковывает и распаковывает данные. +- [clickhouse-odbc-bridge](../../operations/utilities/odbc-bridge.md) — прокси-сервер для ODBC. diff --git a/docs/ru/operations/utilities/odbc-bridge.md b/docs/ru/operations/utilities/odbc-bridge.md new file mode 100644 index 00000000000..70b413c9c1f --- /dev/null +++ b/docs/ru/operations/utilities/odbc-bridge.md @@ -0,0 +1,38 @@ +# clickhouse-odbc-bridge + +Simple HTTP-server which works like a proxy for ODBC driver. The main motivation +was possible segfaults or another faults in ODBC implementations, which can +crash whole clickhouse-server process. + +This tool works via HTTP, not via pipes, shared memory, or TCP because: +- It's simpler to implement +- It's simpler to debug +- jdbc-bridge can be implemented in the same way + +## Usage + +`clickhouse-server` use this tool inside odbc table function and StorageODBC. +However it can be used as standalone tool from command line with the following +parameters in POST-request URL: +- `connection_string` -- ODBC connection string. +- `columns` -- columns in ClickHouse NamesAndTypesList format, name in backticks, + type as string. Name and type are space separated, rows separated with + newline. +- `max_block_size` -- optional parameter, sets maximum size of single block. +Query is send in post body. Response is returned in RowBinary format. + +## Example: + +```bash +$ clickhouse-odbc-bridge --http-port 9018 --daemon + +$ curl -d "query=SELECT PageID, ImpID, AdType FROM Keys ORDER BY PageID, ImpID" --data-urlencode "connection_string=DSN=ClickHouse;DATABASE=stat" --data-urlencode "columns=columns format version: 1 +3 columns: +\`PageID\` String +\`ImpID\` String +\`AdType\` String +" "http://localhost:9018/" > result.txt + +$ cat result.txt +12246623837185725195925621517 +``` From a74111c5b5ef9c08b054ddfa1f4568ecb1ef5d09 Mon Sep 17 00:00:00 2001 From: Olga Revyakina Date: Fri, 30 Apr 2021 10:05:33 +0300 Subject: [PATCH 15/82] Additional fixes --- docs/ru/operations/settings/settings.md | 93 +++++++++++++++++++++++++ 1 file changed, 93 insertions(+) diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 4951be49629..2046cda2e38 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -2755,4 +2755,97 @@ SELECT * FROM test2; Значение по умолчанию: `0`. +## prefer_column_name_to_alias {#prefer-column-name-to-alias} + +Включает или отключает замену названий столбцов на псевдонимы (alias) в выражениях и секциях запросов, см. [Примечания по использованию синонимов](../../sql-reference/syntax.md#syntax-expression_aliases). Включите эту настройку, чтобы синтаксис псевдонимов в ClickHouse был более совместим с большинством других СУБД. + +Возможные значения: + +- 0 — псевдоним подставляется вместо имени столбца. +- 1 — псевдоним не подставляется вместо имени столбца. + +Значение по умолчанию: `0`. + +**Пример** + +Какие изменения привносит включение и выключение настройки: + +Запрос: + +```sql +SET prefer_column_name_to_alias = 0; +SELECT avg(number) AS number, max(number) FROM numbers(10); +``` + +Результат: + +```text +Received exception from server (version 21.5.1): +Code: 184. DB::Exception: Received from localhost:9000. DB::Exception: Aggregate function avg(number) is found inside another aggregate function in query: While processing avg(number) AS number. +``` + +Запрос: + +```sql +SET prefer_column_name_to_alias = 1; +SELECT avg(number) AS number, max(number) FROM numbers(10); +``` + +Результат: + +```text +┌─number─┬─max(number)─┐ +│ 4.5 │ 9 │ +└────────┴─────────────┘ +``` + +## limit {#limit} + +Устанавливает максимальное количество строк, возвращаемых запросом. Ограничивает сверху значение, установленное в запросе в секции [LIMIT](../../sql-reference/statements/select/limit.md#limit-clause). + +Возможные значения: + +- 0 — число строк не ограничено. +- Положительное целое число. + +Значение по умолчанию: `0`. + +## offset {#offset} + +Устанавливает количество строк, которые необходимо пропустить перед началом возврата строк из запроса. Суммируется со значением, установленным в запросе в секции [OFFSET](../../sql-reference/statements/select/offset.md#offset-fetch). + +Возможные значения: + +- 0 — строки не пропускаются. +- Положительное целое число. + +Значение по умолчанию: `0`. + +**Пример** + +Исходная таблица: + +``` sql +CREATE TABLE test (i UInt64) ENGINE = MergeTree() ORDER BY i; +INSERT INTO test SELECT number FROM numbers(500); +``` + +Запрос: + +``` sql +SET limit = 5; +SET offset = 7; +SELECT * FROM test LIMIT 10 OFFSET 100; +``` + +Результат: + +``` text +┌───i─┐ +│ 107 │ +│ 108 │ +│ 109 │ +└─────┘ +``` + [Оригинальная статья](https://clickhouse.tech/docs/ru/operations/settings/settings/) From 557afc86f4ff749e3d40893eb17376ee027fde95 Mon Sep 17 00:00:00 2001 From: filimonov <1549571+filimonov@users.noreply.github.com> Date: Mon, 3 May 2021 10:56:32 +0200 Subject: [PATCH 16/82] Ability to configure the log level for every log channel (main log, error log, stdout, syslog) individually. --- base/loggers/Loggers.cpp | 39 +++++++++++++++++++++++++---- base/loggers/OwnFormattingChannel.h | 3 +++ 2 files changed, 37 insertions(+), 5 deletions(-) diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index ed806741895..9003ca2b0f4 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -52,6 +52,11 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log split = new DB::OwnSplitChannel(); auto log_level = config.getString("logger.level", "trace"); + + // if some of the loggers (like console or syslog) will have more + // verbose log level - we should to use it for all loggers as default + auto max_log_level = Poco::Logger::parseLevel(log_level); + const auto log_path = config.getString("logger.log", ""); if (!log_path.empty()) { @@ -79,6 +84,15 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log if (!errorlog_path.empty()) { createDirectory(errorlog_path); + + // NOTE: we don't use notice & critical in the code, so in practice error log collects fatal & error & warning. + // (!) Warnings are important, they require attention and should never be silenced / ignored. + auto errorlog_level = Poco::Logger::parseLevel(config.getString("logger.errorlog_level", "notice")); + if (errorlog_level > max_log_level) + { + max_log_level = errorlog_level; + } + std::cerr << "Logging errors to " << errorlog_path << std::endl; error_log_file = new Poco::FileChannel; @@ -93,7 +107,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr pf = new OwnPatternFormatter(this); Poco::AutoPtr errorlog = new DB::OwnFormattingChannel(pf, error_log_file); - errorlog->setLevel(Poco::Message::PRIO_NOTICE); + errorlog->setLevel(errorlog_level); errorlog->open(); split->addChannel(errorlog); } @@ -104,6 +118,11 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log if (config.getBool("logger.use_syslog", false) || config.getBool("dynamic_layer_selection", false)) { //const std::string & cmd_name = commandName(); + auto syslog_level = Poco::Logger::parseLevel(config.getString("logger.syslog_level", log_level)); + if (syslog_level > max_log_level) + { + max_log_level = syslog_level; + } if (config.has("logger.syslog.address")) { @@ -130,6 +149,8 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr pf = new OwnPatternFormatter(this, OwnPatternFormatter::ADD_LAYER_TAG); Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, syslog_channel); + log->setLevel(syslog_level); + split->addChannel(log); } @@ -141,9 +162,17 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log { bool color_enabled = config.getBool("logger.color_terminal", color_logs_by_default); + auto console_loglevel_string = config.getString("logger.console_loglevel", log_level); + auto console_loglevel = Poco::Logger::parseLevel(console_loglevel_string); + if (console_loglevel > max_log_level) + { + max_log_level = console_loglevel; + } + Poco::AutoPtr pf = new OwnPatternFormatter(this, OwnPatternFormatter::ADD_NOTHING, color_enabled); Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel); - logger.warning("Logging " + log_level + " to console"); + logger.warning("Logging " + console_loglevel_string + " to console"); + log->setLevel(console_loglevel); split->addChannel(log); } @@ -152,17 +181,17 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log logger.setChannel(split); // Global logging level (it can be overridden for specific loggers). - logger.setLevel(log_level); + logger.setLevel(max_log_level); // Set level to all already created loggers std::vector names; //logger_root = Logger::root(); logger.root().names(names); for (const auto & name : names) - logger.root().get(name).setLevel(log_level); + logger.root().get(name).setLevel(max_log_level); // Attach to the root logger. - logger.root().setLevel(log_level); + logger.root().setLevel(max_log_level); logger.root().setChannel(logger.getChannel()); // Explicitly specified log levels for specific loggers. diff --git a/base/loggers/OwnFormattingChannel.h b/base/loggers/OwnFormattingChannel.h index cd2e66279d7..2336dacad04 100644 --- a/base/loggers/OwnFormattingChannel.h +++ b/base/loggers/OwnFormattingChannel.h @@ -22,6 +22,9 @@ public: void setLevel(Poco::Message::Priority priority_) { priority = priority_; } + // Poco::Logger::parseLevel returns ints + void setLevel(int level) { priority = static_cast(level); } + void open() override { if (pChannel) From 25b9d26a4771c0d85dda2ea536a90b6e68da656b Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Mon, 3 May 2021 18:29:50 +0200 Subject: [PATCH 17/82] Allow main log channel to be less verbose than other log channels --- base/loggers/Loggers.cpp | 20 +++++++++++++------- 1 file changed, 13 insertions(+), 7 deletions(-) diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index 9003ca2b0f4..3c004740e4c 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -51,17 +51,22 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log /// Use extended interface of Channel for more comprehensive logging. split = new DB::OwnSplitChannel(); - auto log_level = config.getString("logger.level", "trace"); + auto log_level_string = config.getString("logger.level", "trace"); - // if some of the loggers (like console or syslog) will have more - // verbose log level - we should to use it for all loggers as default - auto max_log_level = Poco::Logger::parseLevel(log_level); + /// different channels (log, console, syslog) may have different loglevels configured + /// The maximum (the most verbose) of those will be used as default for Poco loggers + int max_log_level = 0; const auto log_path = config.getString("logger.log", ""); if (!log_path.empty()) { createDirectory(log_path); - std::cerr << "Logging " << log_level << " to " << log_path << std::endl; + std::cerr << "Logging " << log_level_string << " to " << log_path << std::endl; + auto log_level = Poco::Logger::parseLevel(log_level_string); + if (log_level > max_log_level) + { + max_log_level = log_level; + } // Set up two channel chains. log_file = new Poco::FileChannel; @@ -77,6 +82,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log Poco::AutoPtr pf = new OwnPatternFormatter(this); Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, log_file); + log->setLevel(log_level); split->addChannel(log); } @@ -118,7 +124,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log if (config.getBool("logger.use_syslog", false) || config.getBool("dynamic_layer_selection", false)) { //const std::string & cmd_name = commandName(); - auto syslog_level = Poco::Logger::parseLevel(config.getString("logger.syslog_level", log_level)); + auto syslog_level = Poco::Logger::parseLevel(config.getString("logger.syslog_level", log_level_string)); if (syslog_level > max_log_level) { max_log_level = syslog_level; @@ -162,7 +168,7 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log { bool color_enabled = config.getBool("logger.color_terminal", color_logs_by_default); - auto console_loglevel_string = config.getString("logger.console_loglevel", log_level); + auto console_loglevel_string = config.getString("logger.console_loglevel", log_level_string); auto console_loglevel = Poco::Logger::parseLevel(console_loglevel_string); if (console_loglevel > max_log_level) { From 8990120eb72b372fbe78a427c0a6c0bb82512bb4 Mon Sep 17 00:00:00 2001 From: "chou.fan" Date: Mon, 10 May 2021 14:17:52 +0100 Subject: [PATCH 18/82] add array nullable datatype support in ArrowColumnToCHColumn and fix select from array datatype misplaced bug --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 382 ++++++++++++------ .../Formats/Impl/CHColumnToArrowColumn.cpp | 51 ++- 2 files changed, 285 insertions(+), 148 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 9d5a551eb48..144ccef1fe3 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -17,7 +17,6 @@ #include #include - namespace DB { namespace ErrorCodes @@ -76,6 +75,21 @@ namespace DB nested_column.reserve(nested_column_length); } + /// Creates a null bytemap from arrow's null bitmap + static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) + { + PaddedPODArray & bytemap_data = assert_cast &>(*bytemap).getData(); + bytemap_data.reserve(arrow_column->length()); + + for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) + { + std::shared_ptr chunk = arrow_column->chunk(chunk_i); + + for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) + bytemap_data.emplace_back(chunk->IsNull(value_i)); + } + } + /// Inserts numeric data right into internal column data to reduce an overhead template > static void fillColumnWithNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) @@ -97,35 +111,63 @@ namespace DB template > static void fillColumnWithArrayNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - ColumnArray & array_column = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + ColumnArray & column_array = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - VectorType & nested_column = static_cast(array_column.getData()); + const bool is_column_array_nullable = column_array.getData().isNullable(); + ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; + + IColumn & array_nested_column = + is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : + column_array.getData(); + VectorType & nested_column = static_cast(array_nested_column); auto & nested_column_data = nested_column.getData(); - reserveArrayColumn(arrow_column, array_column, nested_column); + reserveArrayColumn(arrow_column, column_array, nested_column); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const std::shared_ptr array = list_chunk.values(); + std::shared_ptr buffer = array->data()->buffers[1]; + const auto * raw_data = reinterpret_cast(buffer->data()); + nested_column_data.insert_assume_reserved(raw_data, raw_data + array->length()); + for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) { const std::shared_ptr chunk = list_chunk.value_slice(array_idx); /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk->data()->buffers[1]; + std::shared_ptr inner_buffer = chunk->data()->buffers[1]; - const auto * raw_data = reinterpret_cast(buffer->data()); - nested_column_data.insert_assume_reserved(raw_data, raw_data + chunk->length()); + const size_t chunk_length = list_chunk.value_length(array_idx); + + for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) + { + if (!chunk->IsNull(offset_i) && inner_buffer) + { + if (is_column_array_nullable && column_nullable) + { + column_nullable->getNullMapData().push_back(0); + } + } + else + { + if (is_column_array_nullable && column_nullable) + { + column_nullable->getNullMapData().push_back(1); + } + } + } column_array_offsets.emplace_back(column_array_offsets.back() + chunk->length()); } } } -/// Inserts chars and offsets right into internal column data to reduce an overhead. -/// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. -/// Also internal strings are null terminated. + /// Inserts chars and offsets right into internal column data to reduce an overhead. + /// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. + /// Also internal strings are null terminated. static void fillColumnWithStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); @@ -169,7 +211,13 @@ namespace DB ColumnArray & column_array = assert_cast(*internal_column); ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - ColumnString & nested_column = typeid_cast(column_array.getData()); + const bool is_column_array_nullable = column_array.getData().isNullable(); + ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; + + IColumn & array_nested_column = + is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : + column_array.getData(); + ColumnString & nested_column = static_cast(array_nested_column); PaddedPODArray & nested_column_chars = nested_column.getChars(); PaddedPODArray & nested_column_offsets = nested_column.getOffsets(); @@ -212,6 +260,17 @@ namespace DB { const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); nested_column_chars.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); + if (is_column_array_nullable && column_nullable) + { + column_nullable->getNullMapData().push_back(0); + } + } + else + { + if (is_column_array_nullable && column_nullable) + { + column_nullable->getNullMapData().push_back(1); + } } nested_column_chars.emplace_back('\0'); nested_column_offsets.emplace_back(nested_column_chars.size()); @@ -239,32 +298,48 @@ namespace DB static void fillColumnWithArrayBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - ColumnArray & array_column = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + ColumnArray & column_array = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - ColumnVector & nested_column = assert_cast &>(array_column.getData()); + const bool is_column_array_nullable = column_array.getData().isNullable(); + ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; + + IColumn & array_nested_column = + is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : + column_array.getData(); + ColumnVector & nested_column = assert_cast &>(array_nested_column); auto & nested_column_data = nested_column.getData(); - reserveArrayColumn(arrow_column, array_column, nested_column); + reserveArrayColumn(arrow_column, column_array, nested_column); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const std::shared_ptr list_array = list_chunk.values(); + auto & chunk = static_cast(*(list_array)); + for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) + { + nested_column_data.emplace_back(chunk.Value(bool_i)); + } + + if (is_column_array_nullable && column_nullable) + { + for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) + { + if (!chunk.IsNull(bool_i)) + { + column_nullable->getNullMapData().push_back(0); + } + else + { + column_nullable->getNullMapData().push_back(1); + } + } + } for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) { - const std::shared_ptr array = list_chunk.value_slice(array_idx); - - auto & chunk = static_cast(*(array)); - const size_t chunk_length = chunk.length(); - - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr buffer = chunk.data()->buffers[1]; - - for (size_t bool_i = 0; bool_i != static_cast(chunk.length()); ++bool_i) - nested_column_data.emplace_back(chunk.Value(bool_i)); - - column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); } } } @@ -302,7 +377,13 @@ namespace DB ColumnArray & array_column = assert_cast(*internal_column); ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); - ColumnVector & nested_column = assert_cast &>(array_column.getData()); + const bool is_column_array_nullable = array_column.getData().isNullable(); + ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&array_column.getData()) : nullptr; + + IColumn & array_nested_column = + is_column_array_nullable ? static_cast(array_column.getData()).getNestedColumn() : + array_column.getData(); + ColumnVector & nested_column = assert_cast &>(array_nested_column); auto & nested_column_data = nested_column.getData(); reserveArrayColumn(arrow_column, array_column, nested_column); @@ -310,30 +391,42 @@ namespace DB for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const std::shared_ptr list_array = list_chunk.values(); + auto & chunk = static_cast(*(list_array)); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + UInt32 days_num = static_cast(chunk.Value(value_i)); + if (days_num > DATE_LUT_MAX_DAY_NUM) + { + // TODO: will it rollback correctly? + throw Exception{ + "Input value " + std::to_string(days_num) + " of a column \"" + internal_column->getName() + + "\" is greater than " + "max allowed Date value, which is " + + std::to_string(DATE_LUT_MAX_DAY_NUM), + ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; + } + nested_column_data.emplace_back(days_num); + } + + if (is_column_array_nullable && column_nullable) + { + for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) + { + if (!chunk.IsNull(bool_i)) + { + column_nullable->getNullMapData().push_back(0); + } + else + { + column_nullable->getNullMapData().push_back(1); + } + } + } for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) { - const std::shared_ptr array = list_chunk.value_slice(array_idx); - - auto & chunk = static_cast(*(array)); - const size_t chunk_length = chunk.length(); - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - UInt32 days_num = static_cast(chunk.Value(value_i)); - if (days_num > DATE_LUT_MAX_DAY_NUM) - { - // TODO: will it rollback correctly? - throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + internal_column->getName() - + "\" is greater than " - "max allowed Date value, which is " - + std::to_string(DATE_LUT_MAX_DAY_NUM), - ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; - } - - nested_column_data.emplace_back(days_num); - } - column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); } } } @@ -360,7 +453,13 @@ namespace DB ColumnArray & array_column = assert_cast(*internal_column); ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); - ColumnVector & nested_column = typeid_cast &>(array_column.getData()); + const bool is_column_array_nullable = array_column.getData().isNullable(); + ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&array_column.getData()) : nullptr; + + IColumn & array_nested_column = + is_column_array_nullable ? static_cast(array_column.getData()).getNestedColumn() : + array_column.getData(); + ColumnVector & nested_column = assert_cast &>(array_nested_column); auto & nested_column_data = nested_column.getData(); reserveArrayColumn(arrow_column, array_column, nested_column); @@ -368,20 +467,32 @@ namespace DB for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const std::shared_ptr list_array = list_chunk.values(); + auto & chunk = static_cast(*(list_array)); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms + nested_column_data.emplace_back(timestamp); + } + + if (is_column_array_nullable && column_nullable) + { + for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) + { + if (!chunk.IsNull(bool_i)) + { + column_nullable->getNullMapData().push_back(0); + } + else + { + column_nullable->getNullMapData().push_back(1); + } + } + } for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) { - const std::shared_ptr array = list_chunk.value_slice(array_idx); - - auto & chunk = static_cast(*(array)); - const size_t chunk_length = chunk.length(); - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms - nested_column_data.emplace_back(timestamp); - } - column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); } } } @@ -424,50 +535,68 @@ namespace DB static void fillColumnWithArrayTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) { - ColumnArray & array_column = assert_cast(*internal_column); - ColumnArray::Offsets & array_column_offsets = array_column.getOffsets(); + ColumnArray & column_array = assert_cast(*internal_column); + ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - ColumnVector & nested_column = typeid_cast &>(array_column.getData()); + const bool is_column_array_nullable = column_array.getData().isNullable(); + ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; + + IColumn & array_nested_column = + is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : + column_array.getData(); + ColumnVector & nested_column = assert_cast &>(array_nested_column); auto & nested_column_data = nested_column.getData(); - reserveArrayColumn(arrow_column, array_column, nested_column); + reserveArrayColumn(arrow_column, column_array, nested_column); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const auto & type = static_cast(*list_chunk.type()); + const std::shared_ptr list_array = list_chunk.values(); + auto & chunk = static_cast(*(list_array)); + UInt32 divide = 1; + const auto unit = type.unit(); + switch (unit) + { + case arrow::TimeUnit::SECOND: + divide = 1; + break; + case arrow::TimeUnit::MILLI: + divide = 1000; + break; + case arrow::TimeUnit::MICRO: + divide = 1000000; + break; + case arrow::TimeUnit::NANO: + divide = 1000000000; + break; + } + + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... + nested_column_data.emplace_back(timestamp); + } + + if (is_column_array_nullable && column_nullable) + { + for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) + { + if (!chunk.IsNull(bool_i)) + { + column_nullable->getNullMapData().push_back(0); + } + else + { + column_nullable->getNullMapData().push_back(1); + } + } + } for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) { - const std::shared_ptr array = list_chunk.value_slice(array_idx); - - auto & chunk = static_cast(*array); - const auto & type = static_cast(*chunk.type()); - const size_t chunk_length = chunk.length(); - - UInt32 divide = 1; - const auto unit = type.unit(); - switch (unit) - { - case arrow::TimeUnit::SECOND: - divide = 1; - break; - case arrow::TimeUnit::MILLI: - divide = 1000; - break; - case arrow::TimeUnit::MICRO: - divide = 1000000; - break; - case arrow::TimeUnit::NANO: - divide = 1000000000; - break; - } - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... - nested_column_data.emplace_back(timestamp); - } - array_column_offsets.emplace_back(array_column_offsets.back() + chunk_length); + column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); } } } @@ -493,7 +622,13 @@ namespace DB ColumnArray & array_column = assert_cast(*internal_column); ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); - ColumnDecimal & nested_column = typeid_cast &>(array_column.getData()); + const bool is_column_array_nullable = array_column.getData().isNullable(); + ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&array_column.getData()) : nullptr; + + IColumn & array_nested_column = + is_column_array_nullable ? static_cast(array_column.getData()).getNestedColumn() : + array_column.getData(); + ColumnDecimal & nested_column = assert_cast &>(array_nested_column); auto & nested_column_data = nested_column.getData(); reserveArrayColumn(arrow_column, array_column, nested_column); @@ -501,37 +636,35 @@ namespace DB for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + const std::shared_ptr list_array = list_chunk.values(); + auto & chunk = static_cast(*(list_array)); + for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) + { + nested_column_data.emplace_back(*reinterpret_cast(chunk.Value(value_i))); // TODO: copy column + } + + if (is_column_array_nullable && column_nullable) + { + for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) + { + if (!chunk.IsNull(bool_i)) + { + column_nullable->getNullMapData().push_back(0); + } + else + { + column_nullable->getNullMapData().push_back(1); + } + } + } for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) { - const std::shared_ptr array = list_chunk.value_slice(array_idx); - arrow::DecimalArray & chunk = static_cast(*(array)); - const size_t chunk_length = chunk.length(); - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - nested_column_data.emplace_back(chunk.IsNull(value_i) ? Decimal128(0) : *reinterpret_cast(chunk.Value(value_i))); // TODO: copy column - } - column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); + column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); } } } -/// Creates a null bytemap from arrow's null bitmap - static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) - { - PaddedPODArray & bytemap_data = assert_cast &>(*bytemap).getData(); - bytemap_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) - { - std::shared_ptr chunk = arrow_column->chunk(chunk_i); - - for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) - bytemap_data.emplace_back(chunk->IsNull(value_i)); - } - } - void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, const Block & header, std::string format_name) { @@ -586,11 +719,16 @@ namespace DB const auto * list_type = static_cast(arrow_column->type().get()); list_nested_type = list_type->value_type(); + const auto * column_array_type = static_cast(column_type.get()); + const bool is_column_array_nullable = column_array_type->getNestedType()->isNullable(); + if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), [=](auto && elem) { return elem.first == list_nested_type->id(); }); internal_type_it != arrow_type_to_internal_type.end()) { - array_nested_type = DataTypeFactory::instance().get(internal_type_it->second); + DataTypePtr array_instance_type = DataTypeFactory::instance().get(internal_type_it->second); + array_nested_type = is_column_array_nullable ? makeNullable(array_instance_type) + : array_instance_type; internal_nested_type = std::make_shared(array_nested_type); } else diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 9ed1c5eb1a7..be98bf68bbd 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -117,6 +117,9 @@ namespace DB is_column_array_nullable ? static_cast(column_array_type->getNestedType().get())->getNestedType() : column_array_type->getNestedType(); + const PaddedPODArray * array_null_bytemap = + is_column_array_nullable ? extractNullBytemapPtr(assert_cast(*nested_column).getDataPtr()) : nullptr; + const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), [=](auto && elem) { return elem.first == column_array_nested_type_name; }); if (arrow_type_it != internal_type_to_arrow_type.end()) @@ -130,10 +133,12 @@ namespace DB arrow::Status components_status; const auto & offsets = internal_column.getOffsets(); - const IColumn & data = internal_column.getData(); + ColumnPtr & data = is_column_array_nullable ? + const_cast(static_cast(internal_column.getData()).getNestedColumnPtr()) : + const_cast(internal_column.getDataPtr()); - size_t offset_start = 0; - size_t offset_length = 0; + size_t array_start = 0; + size_t array_length = 0; for (size_t idx = 0, size = internal_column.size(); idx < size; ++idx) { @@ -146,29 +151,24 @@ namespace DB { components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); - offset_length = offsets[idx] - offset_start; - if (offset_length > 0) + array_length = offsets[idx] - array_start; + auto cut_data = data->cut(array_start, array_length); + if (array_null_bytemap == nullptr) { - auto cut_data = data.cut(offset_start, offset_length); - if (null_bytemap == nullptr) - { - CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, - column_array_nested_type_name, arrow_array, - nullptr, value_builder, format_name); - } - else - { - PaddedPODArray * array_nested_null_bytemap = new PaddedPODArray(); - for (size_t pidx = offset_start; pidx < offset_start + offset_length; ++pidx) - { - array_nested_null_bytemap->push_back((*null_bytemap)[pidx]); - } - CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, - column_array_nested_type_name, arrow_array, array_nested_null_bytemap, value_builder, format_name); - delete array_nested_null_bytemap; - } + CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, + column_array_nested_type_name, arrow_array, + nullptr, value_builder, format_name); } - offset_start = offsets[idx]; + else + { + PaddedPODArray array_nested_null_bytemap; + array_nested_null_bytemap.insertByOffsets(*array_null_bytemap, array_start, array_start + array_length); + + CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, + column_array_nested_type_name, arrow_array, + &array_nested_null_bytemap, value_builder, format_name); + } + array_start = offsets[idx]; } } } @@ -425,8 +425,7 @@ namespace DB = is_column_nullable ? assert_cast(*column.column).getNestedColumnPtr() : column.column; const PaddedPODArray * null_bytemap = - is_column_nullable ? extractNullBytemapPtr(column.column) : - is_column_array_nullable ? extractNullBytemapPtr(assert_cast(*nested_column).getDataPtr()) : nullptr; + is_column_nullable ? extractNullBytemapPtr(column.column) : nullptr; arrow::MemoryPool* pool = arrow::default_memory_pool(); std::unique_ptr array_builder; From 32cfb29edf3c4f3955873af7562e2190001f1ef1 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Mon, 10 May 2021 23:04:35 +0300 Subject: [PATCH 19/82] Edit and translate to Russian MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Выполнил перевод на русский язык. --- .../engines/table-engines/integrations/s3.md | 2 +- .../aggregate-functions/reference/deltasum.md | 6 +-- .../reference/deltasumtimestamp.md | 32 +++++++------ .../engines/table-engines/integrations/s3.md | 5 ++- .../aggregate-functions/reference/deltasum.md | 6 ++- .../reference/deltasumtimestamp.md | 45 +++++++++++++++++++ 6 files changed, 76 insertions(+), 20 deletions(-) create mode 100644 docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index eb0d92b7738..11af18e6de6 100644 --- a/docs/en/engines/table-engines/integrations/s3.md +++ b/docs/en/engines/table-engines/integrations/s3.md @@ -139,7 +139,7 @@ The following settings can be specified in configuration file for given endpoint - `endpoint` — Specifies prefix of an endpoint. Mandatory. - `access_key_id` and `secret_access_key` — Specifies credentials to use with given endpoint. Optional. -- `use_environment_credentials` — If set to `true`, S3 client will try to obtain credentials from environment variables and Amazon EC2 metadata for given endpoint. Optional, default value is `false`. +- `use_environment_credentials` — If set to `true`, S3 client will try to obtain credentials from environment variables and [Amazon EC2](https://en.wikipedia.org/wiki/Amazon_Elastic_Compute_Cloud) metadata for given endpoint. Optional, default value is `false`. - `use_insecure_imds_request` — If set to `true`, S3 client will use insecure IMDS request while obtaining credentials from Amazon EC2 metadata. Optional, default value is `false`. - `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be speficied multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md index c40f2372033..84259230aff 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md @@ -6,9 +6,9 @@ toc_priority: 141 Sums the arithmetic difference between consecutive rows. If the difference is negative, it is ignored. -Note that the underlying data must be sorted in order for this function to work properly. -If you would like to use this function in a materialized view, you most likely want to use the -[deltaSumTimestamp](deltasumtimestamp.md) method instead. +Note that the underlying data must be sorted for this function to work properly. + +If you would like to use this function in a [materialized view](../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](deltasumtimestamp.md) method instead. **Syntax** diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 2bfafdc81d1..6e613c54d4b 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -4,36 +4,40 @@ toc_priority: 141 # deltaSumTimestamp {#agg_functions-deltasum} -Syntax: `deltaSumTimestamp(value, timestamp)` +Adds the difference between consecutive rows. If the difference is negative, it is ignored. Uses `timestamp` to order values. -Adds the differences between consecutive rows. If the difference is negative, it is ignored. -Uses `timestamp` to order values. +This function is primarily for [materialized views](../../sql-reference/statements/create/view.md#materialized) that are ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the "right" order. This function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states correctly during merging. -This function is primarily for materialized views that are ordered by some time bucket aligned -timestamp, for example a `toStartOfMinute` bucket. Because the rows in such a materialized view -will all have the same timestamp, it is impossible for them to be merged in the "right" order. This -function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states -correctly during merging. +To calculate the delta sum across an ordered collection you can simply use the [deltaSum](./deltasum.md) function. -To calculate the delta sum across an ordered collection you can simply use the -[deltaSum](./deltasum.md) function. +**Syntax** + +``` sql +deltaSumTimestamp(value, timestamp) +``` **Arguments** -- `value` must be some [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md). -- `timestamp` must be some [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md). +- `value` — Input values, must be some [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md). +- `timestamp` — The parameter for order values, must be some [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md). **Returned value** -- Accumulated differences between consecutive values, ordered by the `timestamp` parameter. +- Accumulated differences between consecutive values, ordered by the `timestamp` parameter. + +Type: [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md). **Example** +Query: + ```sql SELECT deltaSumTimestamp(value, timestamp) -FROM (select number as timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] as value from numbers(1, 10)) +FROM (select number as timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] as value from numbers(1, 10)); ``` +Result: + ``` text ┌─deltaSumTimestamp(value, timestamp)─┐ │ 13 │ diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index bee1fc1318a..f84beab2767 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -82,7 +82,8 @@ SELECT * FROM s3_engine_table LIMIT 2; Необязательные настройки: - `access_key_id` и `secret_access_key` — указывают учетные данные для использования с данной точкой приема запроса. -- `use_environment_credentials` — если `true`, S3-клиент будет пытаться получить учетные данные из переменных среды и метаданных Amazon EC2 для данной точки приема запроса. Значение по умолчанию - `false`. +- `use_environment_credentials` — если `true`, S3-клиент будет пытаться получить учетные данные из переменных среды и метаданных [Amazon EC2](https://ru.wikipedia.org/wiki/Amazon_EC2) для данной точки приема запроса. Значение по умолчанию — `false`. +- `use_insecure_imds_request` — признак, нужно ли использовать менее безопасное соединение при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию — `false`. - `header` — добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса. Может быть определен несколько раз. - `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. @@ -95,6 +96,7 @@ SELECT * FROM s3_engine_table LIMIT 2; + @@ -141,6 +143,7 @@ ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/{some,another}_p CREATE TABLE big_table (name String, value UInt32) ENGINE = S3('https://storage.yandexcloud.net/my-test-bucket-768/big_prefix/file-{000..999}.csv', 'CSV'); ``` + **Смотрите также** - [Табличная функция S3](../../../sql-reference/table-functions/s3.md) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index b025a248f3c..124ee51632f 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -6,6 +6,10 @@ toc_priority: 141 Суммирует арифметическую разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. +Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. + +Если вы хотите использовать эту функцию в [материализованном представлении](../../sql-reference/statements/create/view.md#materialized), вместо нее лучше используйте функцию [deltaSumTimestamp](deltasumtimestamp.md). + **Синтаксис** ``` sql @@ -18,7 +22,7 @@ deltaSum(value) **Возвращаемое значение** -- накопленная арифметическая разница, типа `Integer` или `Float`. +- Накопленная арифметическая разница, типа `Integer` или `Float`. **Примеры** diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md new file mode 100644 index 00000000000..37d0da1cd4f --- /dev/null +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -0,0 +1,45 @@ +--- +toc_priority: 141 +--- + +# deltaSumTimestamp {#agg_functions-deltasum} + +Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. Параметр `timestamp` используется для упорядочивания значений. + +Эта функция предназначена в первую очередь для [материализованных представлений](../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например по бакету `toStartOfMinute`. Поэтому строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. + +Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](./deltasum.md) вместо функции `deltaSumTimestamp`. + +**Синтаксис** + +``` sql +deltaSumTimestamp(value, timestamp) +``` + +**Аргументы** + +- `value` — входные значения, должны быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). +- `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). + +**Возвращаемое значение** + +- Накопленная разница между последовательными значениями, упорядоченными по параметру `timestamp`. + +Тип: [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). + +**Пример** + +Запрос: + +```sql +SELECT deltaSumTimestamp(value, timestamp) +FROM (select number as timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] as value from numbers(1, 10)); +``` + +Результат: + +``` text +┌─deltaSumTimestamp(value, timestamp)─┐ +│ 13 │ +└─────────────────────────────────────┘ +``` From c80042682deec8c779e366dfb366c0526c24a4d3 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Tue, 11 May 2021 00:06:24 +0300 Subject: [PATCH 20/82] Fix links MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поправил ссылки. --- docs/en/sql-reference/aggregate-functions/reference/deltasum.md | 2 +- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- docs/ru/sql-reference/aggregate-functions/reference/deltasum.md | 2 +- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md index 84259230aff..59ecb968207 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md @@ -8,7 +8,7 @@ Sums the arithmetic difference between consecutive rows. If the difference is ne Note that the underlying data must be sorted for this function to work properly. -If you would like to use this function in a [materialized view](../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](deltasumtimestamp.md) method instead. +If you would like to use this function in a [materialized view](../../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](deltasumtimestamp.md) method instead. **Syntax** diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 6e613c54d4b..7ffb69b8e5f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -6,7 +6,7 @@ toc_priority: 141 Adds the difference between consecutive rows. If the difference is negative, it is ignored. Uses `timestamp` to order values. -This function is primarily for [materialized views](../../sql-reference/statements/create/view.md#materialized) that are ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the "right" order. This function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states correctly during merging. +This function is primarily for [materialized views](../../../sql-reference/statements/create/view.md#materialized) that are ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the "right" order. This function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states correctly during merging. To calculate the delta sum across an ordered collection you can simply use the [deltaSum](./deltasum.md) function. diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index 124ee51632f..9dec611eb83 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -8,7 +8,7 @@ toc_priority: 141 Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. -Если вы хотите использовать эту функцию в [материализованном представлении](../../sql-reference/statements/create/view.md#materialized), вместо нее лучше используйте функцию [deltaSumTimestamp](deltasumtimestamp.md). +Если вы хотите использовать эту функцию в [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized), вместо нее лучше используйте функцию [deltaSumTimestamp](deltasumtimestamp.md). **Синтаксис** diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 37d0da1cd4f..4bf60fbc2d0 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -6,7 +6,7 @@ toc_priority: 141 Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. Параметр `timestamp` используется для упорядочивания значений. -Эта функция предназначена в первую очередь для [материализованных представлений](../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например по бакету `toStartOfMinute`. Поэтому строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. +Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например по бакету `toStartOfMinute`. Поэтому строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](./deltasum.md) вместо функции `deltaSumTimestamp`. From 447c73ce147fe43233a26b7d7f7a4500986cf4cf Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Wed, 12 May 2021 15:11:44 +0300 Subject: [PATCH 21/82] Apply suggestions from code review Co-authored-by: alesapin --- docs/en/operations/utilities/clickhouse-format.md | 2 +- docs/ru/operations/utilities/clickhouse-compressor.md | 2 +- docs/ru/operations/utilities/clickhouse-format.md | 6 +++--- docs/ru/operations/utilities/odbc-bridge.md | 2 +- 4 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index 25e2c400bf5..33b528983b6 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -66,7 +66,7 @@ Result: SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; ``` -Another seed string: +Same query and another seed string: ```bash $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" diff --git a/docs/ru/operations/utilities/clickhouse-compressor.md b/docs/ru/operations/utilities/clickhouse-compressor.md index 44a1f052824..d7f6862a62c 100644 --- a/docs/ru/operations/utilities/clickhouse-compressor.md +++ b/docs/ru/operations/utilities/clickhouse-compressor.md @@ -1,6 +1,6 @@ ## ClickHouse compressor -Simple program for data compression and decompression. +Simple program for data compression and decompression in ClickHouse way. ### Examples diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index 2ce5ec924de..8474d94800c 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -16,7 +16,7 @@ toc_title: clickhouse-format - `--multiquery` or `-n` — поддерживает несколько запросов в одной строке. - `--obfuscate` — обфускирует вместо форматирования. - `--seed <строка>` — задает строку, которая определяет результат обфускации. -- `--backslash` — добавляет обратный слеш в конце каждой строки отформатированного запроса. Удобно использовать, если вы скопировали многострочный запрос из интернета или другого источника и хотите выполнить его из командной строки. +- `--backslash` — добавляет обратный слеш в конце каждой строки отформатированного запроса. Удобно использовать если многострочный запрос скопирован из интернета или другого источника и его нужно выполнить из командной строки. ## Примеры {#examples} @@ -54,7 +54,7 @@ UNION ALL ; ``` -3. Обфуксация: +3. Обфускация: ```bash $ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" @@ -66,7 +66,7 @@ $ clickhouse-format --seed Hello --obfuscate <<< "SELECT cost_first_screen BETWE SELECT treasury_mammoth_hazelnut BETWEEN nutmeg AND span, CASE WHEN chive >= 116 THEN switching ELSE ANYTHING END; ``` -Другая строка для обфускации: +Тот же запрос с другой инициализацией обфускатора: ```bash $ clickhouse-format --seed World --obfuscate <<< "SELECT cost_first_screen BETWEEN a AND b, CASE WHEN x >= 123 THEN y ELSE NULL END;" diff --git a/docs/ru/operations/utilities/odbc-bridge.md b/docs/ru/operations/utilities/odbc-bridge.md index 70b413c9c1f..39c796c10c1 100644 --- a/docs/ru/operations/utilities/odbc-bridge.md +++ b/docs/ru/operations/utilities/odbc-bridge.md @@ -33,6 +33,6 @@ $ curl -d "query=SELECT PageID, ImpID, AdType FROM Keys ORDER BY PageID, ImpID" \`AdType\` String " "http://localhost:9018/" > result.txt -$ cat result.txt +$ cat result.txt # Result in RowBinary format 12246623837185725195925621517 ``` From 46f52ca211f8bea26aa697bbc51c15c8b2a7748e Mon Sep 17 00:00:00 2001 From: olgarev <56617294+olgarev@users.noreply.github.com> Date: Wed, 12 May 2021 15:15:02 +0300 Subject: [PATCH 22/82] Update docs/ru/operations/utilities/clickhouse-format.md Co-authored-by: alesapin --- docs/ru/operations/utilities/clickhouse-format.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index 8474d94800c..ca58b4a3b93 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -14,7 +14,7 @@ toc_title: clickhouse-format - `--oneline` — форматирование в одну строку. - `--quiet` или `-q` — проверяет синтаксис без вывода результата. - `--multiquery` or `-n` — поддерживает несколько запросов в одной строке. -- `--obfuscate` — обфускирует вместо форматирования. +- `--obfuscate` — обфусцирует вместо форматирования. - `--seed <строка>` — задает строку, которая определяет результат обфускации. - `--backslash` — добавляет обратный слеш в конце каждой строки отформатированного запроса. Удобно использовать если многострочный запрос скопирован из интернета или другого источника и его нужно выполнить из командной строки. From f6cb9f95df2debf3269cf436ecf6f0a4044a45a7 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov Date: Wed, 12 May 2021 16:01:48 +0300 Subject: [PATCH 23/82] lower two-level aggregation threshold for uniq test to avoid jitter --- src/Core/Settings.h | 2 +- tests/performance/uniq.xml | 10 ++++++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index eaa5e2d34f8..f25393c9502 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -104,7 +104,7 @@ class IColumn; M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ - M(UInt64, group_by_two_level_threshold_bytes, 100000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ + M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ \ diff --git a/tests/performance/uniq.xml b/tests/performance/uniq.xml index b4e73733769..0ddd2321ca6 100644 --- a/tests/performance/uniq.xml +++ b/tests/performance/uniq.xml @@ -9,6 +9,16 @@ 30000000000 + + 10000000 From 8ed6ad7c55d248768cd696af2f97eb7c246ad3ef Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 12 May 2021 19:06:08 +0300 Subject: [PATCH 24/82] Refactor, support all types in array, add nested arrays support, more tests --- src/Columns/IColumn.cpp | 6 + src/Columns/IColumn.h | 2 + .../Formats/Impl/ArrowColumnToCHColumn.cpp | 760 ++++-------------- .../Formats/Impl/CHColumnToArrowColumn.cpp | 338 ++++---- .../Formats/Impl/CHColumnToArrowColumn.h | 10 +- .../00900_orc_arrays_load.reference | 4 + .../0_stateless/00900_orc_arrays_load.sh | 15 + .../00900_orc_nested_arrays_load.reference | 2 + .../00900_orc_nested_arrays_load.sh | 15 + .../00900_orc_nullable_arrays_load.reference | 6 + .../00900_orc_nullable_arrays_load.sh | 15 + .../0_stateless/00900_parquet.reference | 12 + tests/queries/0_stateless/00900_parquet.sh | 31 + .../0_stateless/00900_parquet_load.reference | 20 +- .../0_stateless/data_orc/array_test.orc | Bin 0 -> 3870 bytes .../data_orc/nested_array_test.orc | Bin 0 -> 1344 bytes .../data_orc/nullable_array_test.orc | Bin 0 -> 714 bytes tests/queries/0_stateless/data_orc/tmp | Bin 0 -> 714 bytes .../data_parquet/00900_parquet_load.reference | 0 .../data_parquet/alltypes_list.parquet | Bin 0 -> 13405 bytes .../alltypes_list.parquet.columns | 1 + .../data_parquet/array_float.parquet | Bin 520 -> 668 bytes .../data_parquet/array_int.parquet | Bin 517 -> 724 bytes .../data_parquet/list_columns.parquet.columns | 2 +- .../data_parquet/nested_lists.parquet | Bin 0 -> 1755 bytes .../data_parquet/nested_lists.parquet.columns | 1 + .../nested_lists.snappy.parquet.columns | 2 +- .../data_parquet/nullable_list.parquet | Bin 0 -> 2138 bytes .../nullable_list.parquet.columns | 1 + 29 files changed, 441 insertions(+), 802 deletions(-) create mode 100644 tests/queries/0_stateless/00900_orc_arrays_load.reference create mode 100755 tests/queries/0_stateless/00900_orc_arrays_load.sh create mode 100644 tests/queries/0_stateless/00900_orc_nested_arrays_load.reference create mode 100755 tests/queries/0_stateless/00900_orc_nested_arrays_load.sh create mode 100644 tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference create mode 100755 tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh create mode 100644 tests/queries/0_stateless/data_orc/array_test.orc create mode 100644 tests/queries/0_stateless/data_orc/nested_array_test.orc create mode 100644 tests/queries/0_stateless/data_orc/nullable_array_test.orc create mode 100644 tests/queries/0_stateless/data_orc/tmp create mode 100644 tests/queries/0_stateless/data_parquet/00900_parquet_load.reference create mode 100644 tests/queries/0_stateless/data_parquet/alltypes_list.parquet create mode 100644 tests/queries/0_stateless/data_parquet/alltypes_list.parquet.columns create mode 100644 tests/queries/0_stateless/data_parquet/nested_lists.parquet create mode 100644 tests/queries/0_stateless/data_parquet/nested_lists.parquet.columns create mode 100644 tests/queries/0_stateless/data_parquet/nullable_list.parquet create mode 100644 tests/queries/0_stateless/data_parquet/nullable_list.parquet.columns diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index a3ed0885651..2e61dc50fa4 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include @@ -40,4 +41,9 @@ bool isColumnConst(const IColumn & column) return checkColumn(column); } +bool isColumnArray(const IColumn & column) +{ + return checkColumn(column); +} + } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 1dedd191e1d..ac8e9f9872a 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -530,4 +530,6 @@ bool isColumnConst(const IColumn & column); /// True if column's an ColumnNullable instance. It's just a syntax sugar for type check. bool isColumnNullable(const IColumn & column); +bool isColumnArray(const IColumn & column); + } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 144ccef1fe3..cef9bd79b2a 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -17,6 +17,7 @@ #include #include + namespace DB { namespace ErrorCodes @@ -57,44 +58,11 @@ namespace DB // Full list of types: contrib/arrow/cpp/src/arrow/type.h }; - template - static void reserveArrayColumn(std::shared_ptr & arrow_column, ColumnArray & array_column, NestedColumnVector & nested_column) - { - size_t nested_column_length = 0; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - - for (int64_t array_idx = 0; array_idx != chunk.length(); ++array_idx) - { - const std::shared_ptr array = chunk.value_slice(array_idx); - nested_column_length += array->length(); - } - } - array_column.reserve(arrow_column->length()); - nested_column.reserve(nested_column_length); - } - - /// Creates a null bytemap from arrow's null bitmap - static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, MutableColumnPtr & bytemap) - { - PaddedPODArray & bytemap_data = assert_cast &>(*bytemap).getData(); - bytemap_data.reserve(arrow_column->length()); - - for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) - { - std::shared_ptr chunk = arrow_column->chunk(chunk_i); - - for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) - bytemap_data.emplace_back(chunk->IsNull(value_i)); - } - } - /// Inserts numeric data right into internal column data to reduce an overhead template > - static void fillColumnWithNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithNumericData(std::shared_ptr & arrow_column, IColumn & internal_column) { - auto & column_data = static_cast(*internal_column).getData(); + auto & column_data = static_cast(internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -108,70 +76,13 @@ namespace DB } } - template > - static void fillColumnWithArrayNumericData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) +/// Inserts chars and offsets right into internal column data to reduce an overhead. +/// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. +/// Also internal strings are null terminated. + static void fillColumnWithStringData(std::shared_ptr & arrow_column, IColumn & internal_column) { - ColumnArray & column_array = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - - const bool is_column_array_nullable = column_array.getData().isNullable(); - ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; - - IColumn & array_nested_column = - is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : - column_array.getData(); - VectorType & nested_column = static_cast(array_nested_column); - auto & nested_column_data = nested_column.getData(); - - reserveArrayColumn(arrow_column, column_array, nested_column); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - - const std::shared_ptr array = list_chunk.values(); - std::shared_ptr buffer = array->data()->buffers[1]; - const auto * raw_data = reinterpret_cast(buffer->data()); - nested_column_data.insert_assume_reserved(raw_data, raw_data + array->length()); - - for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) - { - const std::shared_ptr chunk = list_chunk.value_slice(array_idx); - /// buffers[0] is a null bitmap and buffers[1] are actual values - std::shared_ptr inner_buffer = chunk->data()->buffers[1]; - - const size_t chunk_length = list_chunk.value_length(array_idx); - - for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) - { - if (!chunk->IsNull(offset_i) && inner_buffer) - { - if (is_column_array_nullable && column_nullable) - { - column_nullable->getNullMapData().push_back(0); - } - } - else - { - if (is_column_array_nullable && column_nullable) - { - column_nullable->getNullMapData().push_back(1); - } - } - } - - column_array_offsets.emplace_back(column_array_offsets.back() + chunk->length()); - } - } - } - - /// Inserts chars and offsets right into internal column data to reduce an overhead. - /// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars. - /// Also internal strings are null terminated. - static void fillColumnWithStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) - { - PaddedPODArray & column_chars_t = assert_cast(*internal_column).getChars(); - PaddedPODArray & column_offsets = assert_cast(*internal_column).getOffsets(); + PaddedPODArray & column_chars_t = assert_cast(internal_column).getChars(); + PaddedPODArray & column_offsets = assert_cast(internal_column).getOffsets(); size_t chars_t_size = 0; for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -206,83 +117,9 @@ namespace DB } } - static void fillColumnWithArrayStringData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, IColumn & internal_column) { - ColumnArray & column_array = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - - const bool is_column_array_nullable = column_array.getData().isNullable(); - ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; - - IColumn & array_nested_column = - is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : - column_array.getData(); - ColumnString & nested_column = static_cast(array_nested_column); - PaddedPODArray & nested_column_chars = nested_column.getChars(); - PaddedPODArray & nested_column_offsets = nested_column.getOffsets(); - - size_t chars_t_size = 0; - size_t number_size = 0; - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & chunk = static_cast(*(arrow_column->chunk(chunk_i))); - - for (int64_t array_idx = 0; array_idx != chunk.length(); ++array_idx) - { - const std::shared_ptr array = chunk.value_slice(array_idx); - arrow::BinaryArray & binary_array = static_cast(*(array)); - const size_t binary_array_length = binary_array.length(); - - chars_t_size += binary_array.value_offset(binary_array_length - 1) + binary_array.value_length(binary_array_length - 1); - chars_t_size += binary_array_length; /// additional space for null bytes - number_size += binary_array_length; - } - } - column_array.reserve(arrow_column->length()); - - nested_column_chars.reserve(chars_t_size); - nested_column_offsets.reserve(number_size); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - - for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) - { - const std::shared_ptr array = list_chunk.value_slice(array_idx); - arrow::BinaryArray & chunk = static_cast(*(array)); - std::shared_ptr buffer = chunk.value_data(); - const size_t chunk_length = chunk.length(); - - for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i) - { - if (!chunk.IsNull(offset_i) && buffer) - { - const auto * raw_data = buffer->data() + chunk.value_offset(offset_i); - nested_column_chars.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i)); - if (is_column_array_nullable && column_nullable) - { - column_nullable->getNullMapData().push_back(0); - } - } - else - { - if (is_column_array_nullable && column_nullable) - { - column_nullable->getNullMapData().push_back(1); - } - } - nested_column_chars.emplace_back('\0'); - nested_column_offsets.emplace_back(nested_column_chars.size()); - } - column_array_offsets.emplace_back(column_array_offsets.back() + chunk_length); - } - } - } - - static void fillColumnWithBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) - { - auto & column_data = assert_cast &>(*internal_column).getData(); + auto & column_data = assert_cast &>(internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -296,58 +133,10 @@ namespace DB } } - static void fillColumnWithArrayBooleanData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) - { - ColumnArray & column_array = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - - const bool is_column_array_nullable = column_array.getData().isNullable(); - ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; - - IColumn & array_nested_column = - is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : - column_array.getData(); - ColumnVector & nested_column = assert_cast &>(array_nested_column); - auto & nested_column_data = nested_column.getData(); - - reserveArrayColumn(arrow_column, column_array, nested_column); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const std::shared_ptr list_array = list_chunk.values(); - auto & chunk = static_cast(*(list_array)); - for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) - { - nested_column_data.emplace_back(chunk.Value(bool_i)); - } - - if (is_column_array_nullable && column_nullable) - { - for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) - { - if (!chunk.IsNull(bool_i)) - { - column_nullable->getNullMapData().push_back(0); - } - else - { - column_nullable->getNullMapData().push_back(1); - } - } - } - - for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) - { - column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); - } - } - } - /// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving - static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithDate32Data(std::shared_ptr & arrow_column, IColumn & internal_column) { - PaddedPODArray & column_data = assert_cast &>(*internal_column).getData(); + PaddedPODArray & column_data = assert_cast &>(internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -360,7 +149,7 @@ namespace DB if (days_num > DATE_LUT_MAX_DAY_NUM) { // TODO: will it rollback correctly? - throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + internal_column->getName() + throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + internal_column.getName() + "\" is greater than " "max allowed Date value, which is " + std::to_string(DATE_LUT_MAX_DAY_NUM), @@ -372,69 +161,10 @@ namespace DB } } - static void fillColumnWithArrayDate32Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) - { - ColumnArray & array_column = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); - - const bool is_column_array_nullable = array_column.getData().isNullable(); - ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&array_column.getData()) : nullptr; - - IColumn & array_nested_column = - is_column_array_nullable ? static_cast(array_column.getData()).getNestedColumn() : - array_column.getData(); - ColumnVector & nested_column = assert_cast &>(array_nested_column); - auto & nested_column_data = nested_column.getData(); - - reserveArrayColumn(arrow_column, array_column, nested_column); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const std::shared_ptr list_array = list_chunk.values(); - auto & chunk = static_cast(*(list_array)); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - UInt32 days_num = static_cast(chunk.Value(value_i)); - if (days_num > DATE_LUT_MAX_DAY_NUM) - { - // TODO: will it rollback correctly? - throw Exception{ - "Input value " + std::to_string(days_num) + " of a column \"" + internal_column->getName() - + "\" is greater than " - "max allowed Date value, which is " - + std::to_string(DATE_LUT_MAX_DAY_NUM), - ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE}; - } - nested_column_data.emplace_back(days_num); - } - - if (is_column_array_nullable && column_nullable) - { - for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) - { - if (!chunk.IsNull(bool_i)) - { - column_nullable->getNullMapData().push_back(0); - } - else - { - column_nullable->getNullMapData().push_back(1); - } - } - } - - for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) - { - column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); - } - } - } - /// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving - static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithDate64Data(std::shared_ptr & arrow_column, IColumn & internal_column) { - auto & column_data = assert_cast &>(*internal_column).getData(); + auto & column_data = assert_cast &>(internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -448,58 +178,9 @@ namespace DB } } - static void fillColumnWithArrayDate64Data(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, IColumn & internal_column) { - ColumnArray & array_column = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); - - const bool is_column_array_nullable = array_column.getData().isNullable(); - ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&array_column.getData()) : nullptr; - - IColumn & array_nested_column = - is_column_array_nullable ? static_cast(array_column.getData()).getNestedColumn() : - array_column.getData(); - ColumnVector & nested_column = assert_cast &>(array_nested_column); - auto & nested_column_data = nested_column.getData(); - - reserveArrayColumn(arrow_column, array_column, nested_column); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const std::shared_ptr list_array = list_chunk.values(); - auto & chunk = static_cast(*(list_array)); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / 1000); // Always? in ms - nested_column_data.emplace_back(timestamp); - } - - if (is_column_array_nullable && column_nullable) - { - for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) - { - if (!chunk.IsNull(bool_i)) - { - column_nullable->getNullMapData().push_back(0); - } - else - { - column_nullable->getNullMapData().push_back(1); - } - } - } - - for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) - { - column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); - } - } - } - - static void fillColumnWithTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) - { - auto & column_data = assert_cast &>(*internal_column).getData(); + auto & column_data = assert_cast &>(internal_column).getData(); column_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) @@ -533,77 +214,9 @@ namespace DB } } - static void fillColumnWithArrayTimestampData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) + static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, IColumn & internal_column) { - ColumnArray & column_array = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = column_array.getOffsets(); - - const bool is_column_array_nullable = column_array.getData().isNullable(); - ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&column_array.getData()) : nullptr; - - IColumn & array_nested_column = - is_column_array_nullable ? static_cast(column_array.getData()).getNestedColumn() : - column_array.getData(); - ColumnVector & nested_column = assert_cast &>(array_nested_column); - auto & nested_column_data = nested_column.getData(); - - reserveArrayColumn(arrow_column, column_array, nested_column); - - for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) - { - arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const auto & type = static_cast(*list_chunk.type()); - const std::shared_ptr list_array = list_chunk.values(); - auto & chunk = static_cast(*(list_array)); - UInt32 divide = 1; - const auto unit = type.unit(); - switch (unit) - { - case arrow::TimeUnit::SECOND: - divide = 1; - break; - case arrow::TimeUnit::MILLI: - divide = 1000; - break; - case arrow::TimeUnit::MICRO: - divide = 1000000; - break; - case arrow::TimeUnit::NANO: - divide = 1000000000; - break; - } - - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - auto timestamp = static_cast(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ... - nested_column_data.emplace_back(timestamp); - } - - if (is_column_array_nullable && column_nullable) - { - for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) - { - if (!chunk.IsNull(bool_i)) - { - column_nullable->getNullMapData().push_back(0); - } - else - { - column_nullable->getNullMapData().push_back(1); - } - } - } - - for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) - { - column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); - } - } - } - - static void fillColumnWithDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) - { - auto & column = assert_cast &>(*internal_column); + auto & column = assert_cast &>(internal_column); auto & column_data = column.getData(); column_data.reserve(arrow_column->length()); @@ -617,54 +230,155 @@ namespace DB } } - static void fillColumnWithArrayDecimalData(std::shared_ptr & arrow_column, MutableColumnPtr & internal_column) +/// Creates a null bytemap from arrow's null bitmap + static void fillByteMapFromArrowColumn(std::shared_ptr & arrow_column, IColumn & bytemap) { - ColumnArray & array_column = assert_cast(*internal_column); - ColumnArray::Offsets & column_array_offsets = array_column.getOffsets(); + PaddedPODArray & bytemap_data = assert_cast &>(bytemap).getData(); + bytemap_data.reserve(arrow_column->length()); - const bool is_column_array_nullable = array_column.getData().isNullable(); - ColumnNullable * column_nullable = is_column_array_nullable ? static_cast(&array_column.getData()) : nullptr; + for (size_t chunk_i = 0; chunk_i != static_cast(arrow_column->num_chunks()); ++chunk_i) + { + std::shared_ptr chunk = arrow_column->chunk(chunk_i); - IColumn & array_nested_column = - is_column_array_nullable ? static_cast(array_column.getData()).getNestedColumn() : - array_column.getData(); - ColumnDecimal & nested_column = assert_cast &>(array_nested_column); - auto & nested_column_data = nested_column.getData(); + for (size_t value_i = 0; value_i != static_cast(chunk->length()); ++value_i) + bytemap_data.emplace_back(chunk->IsNull(value_i)); + } + } - reserveArrayColumn(arrow_column, array_column, nested_column); + static void fillOffsetsFromArrowListColumn(std::shared_ptr & arrow_column, IColumn & offsets) + { + ColumnArray::Offsets & offsets_data = assert_cast &>(offsets).getData(); + offsets_data.reserve(arrow_column->length()); for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) { arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); - const std::shared_ptr list_array = list_chunk.values(); - auto & chunk = static_cast(*(list_array)); - for (size_t value_i = 0, length = static_cast(chunk.length()); value_i < length; ++value_i) - { - nested_column_data.emplace_back(*reinterpret_cast(chunk.Value(value_i))); // TODO: copy column - } - - if (is_column_array_nullable && column_nullable) - { - for (size_t bool_i = 0; bool_i != static_cast(list_array->length()); ++bool_i) - { - if (!chunk.IsNull(bool_i)) - { - column_nullable->getNullMapData().push_back(0); - } - else - { - column_nullable->getNullMapData().push_back(1); - } - } - } - - for (int64_t array_idx = 0; array_idx != list_chunk.length(); ++array_idx) - { - column_array_offsets.emplace_back(column_array_offsets.back() + list_chunk.value_offset(array_idx)); - } + auto arrow_offsets_array = list_chunk.offsets(); + auto & arrow_offsets = static_cast(*arrow_offsets_array); + auto start = offsets_data.back(); + for (int64_t i = 1; i < arrow_offsets.length(); ++i) + offsets_data.emplace_back(start + arrow_offsets.Value(i)); } } + static void readColumnFromArrowColumn(std::shared_ptr & arrow_column, IColumn & internal_column, const std::string & column_name, const std::string format_name, bool is_nullable) + { + if (internal_column.isNullable()) + { + ColumnNullable & column_nullable = typeid_cast(internal_column); + readColumnFromArrowColumn(arrow_column, column_nullable.getNestedColumn(), column_name, format_name, true); + fillByteMapFromArrowColumn(arrow_column, column_nullable.getNullMapColumn()); + return; + } + + // TODO: check if a column is const? + if (!is_nullable && !isColumnArray(internal_column) && arrow_column->null_count()) + { + throw Exception{ + "Can not insert NULL data into non-nullable column \"" + column_name + "\"", + ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN}; + } + + switch (arrow_column->type()->id()) + { + case arrow::Type::STRING: + case arrow::Type::BINARY: + //case arrow::Type::FIXED_SIZE_BINARY: + fillColumnWithStringData(arrow_column, internal_column); + break; + case arrow::Type::BOOL: + fillColumnWithBooleanData(arrow_column, internal_column); + break; + case arrow::Type::DATE32: + fillColumnWithDate32Data(arrow_column, internal_column); + break; + case arrow::Type::DATE64: + fillColumnWithDate64Data(arrow_column, internal_column); + break; + case arrow::Type::TIMESTAMP: + fillColumnWithTimestampData(arrow_column, internal_column); + break; + case arrow::Type::DECIMAL: + //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster + fillColumnWithDecimalData(arrow_column, internal_column /*, internal_nested_type*/); + break; + case arrow::Type::LIST: + { + const auto * list_type = static_cast(arrow_column->type().get()); + auto list_nested_type = list_type->value_type(); + arrow::ArrayVector array_vector; + array_vector.reserve(arrow_column->num_chunks()); + for (size_t chunk_i = 0, num_chunks = static_cast(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i) + { + arrow::ListArray & list_chunk = static_cast(*(arrow_column->chunk(chunk_i))); + std::shared_ptr chunk = list_chunk.values(); + array_vector.emplace_back(std::move(chunk)); + } + auto arrow_nested_column = std::make_shared(array_vector); + + ColumnArray & column_array = typeid_cast(internal_column); + readColumnFromArrowColumn(arrow_nested_column, column_array.getData(), column_name, format_name, false); + fillOffsetsFromArrowListColumn(arrow_column, column_array.getOffsetsColumn()); + break; + } +# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ + case ARROW_NUMERIC_TYPE: \ + fillColumnWithNumericData(arrow_column, internal_column); \ + break; + + FOR_ARROW_NUMERIC_TYPES(DISPATCH) +# undef DISPATCH + // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? + // TODO: read JSON as a string? + // TODO: read UUID as a string? + default: + throw Exception + { + "Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" + + column_name + "\"", + ErrorCodes::UNKNOWN_TYPE + }; + } + } + + static DataTypePtr getInternalType(std::shared_ptr arrow_type, const DataTypePtr & column_type, const std::string & column_name, const std::string & format_name) + { + if (column_type->isNullable()) + { + DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); + return makeNullable(getInternalType(arrow_type, nested_type, column_name, format_name)); + } + + if (arrow_type->id() == arrow::Type::DECIMAL) + { + const auto * decimal_type = static_cast(arrow_type.get()); + return std::make_shared>(decimal_type->precision(), decimal_type->scale()); + } + + if (arrow_type->id() == arrow::Type::LIST) + { + const auto * list_type = static_cast(arrow_type.get()); + auto list_nested_type = list_type->value_type(); + + const DataTypeArray * array_type = typeid_cast(column_type.get()); + if (!array_type) + throw Exception{"Cannot convert arrow LIST type to a not Array ClickHouse type " + column_type->getName(), ErrorCodes::CANNOT_CONVERT_TYPE}; + + return std::make_shared(getInternalType(list_nested_type, array_type->getNestedType(), column_name, format_name)); + } + + if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), + [=](auto && elem) { return elem.first == arrow_type->id(); }); + internal_type_it != arrow_type_to_internal_type.end()) + { + return DataTypeFactory::instance().get(internal_type_it->second); + } + throw Exception{ + "The type \"" + arrow_type->name() + "\" of an input column \"" + column_name + "\" is not supported for conversion from a " + + format_name + " data format", + ErrorCodes::CANNOT_CONVERT_TYPE}; + } + void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, const Block & header, std::string format_name) { @@ -693,166 +407,16 @@ namespace DB ErrorCodes::THERE_IS_NO_COLUMN}; std::shared_ptr arrow_column = name_to_column_ptr[header_column.name]; - arrow::Type::type arrow_type = arrow_column->type()->id(); - std::shared_ptr list_nested_type; - // TODO: check if a column is const? - if (!column_type->isNullable() && arrow_column->null_count()) - { - throw Exception{"Can not insert NULL data into non-nullable column \"" + header_column.name + "\"", - ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN}; - } + DataTypePtr internal_type = getInternalType(arrow_column->type(), column_type, header_column.name, format_name); - const bool target_column_is_nullable = column_type->isNullable() || arrow_column->null_count(); - - DataTypePtr internal_nested_type; - DataTypePtr array_nested_type; - - if (arrow_type == arrow::Type::DECIMAL) - { - const auto * decimal_type = static_cast(arrow_column->type().get()); - internal_nested_type = std::make_shared>(decimal_type->precision(), - decimal_type->scale()); - } - else if (arrow_type == arrow::Type::LIST) - { - const auto * list_type = static_cast(arrow_column->type().get()); - list_nested_type = list_type->value_type(); - - const auto * column_array_type = static_cast(column_type.get()); - const bool is_column_array_nullable = column_array_type->getNestedType()->isNullable(); - - if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), - [=](auto && elem) { return elem.first == list_nested_type->id(); }); - internal_type_it != arrow_type_to_internal_type.end()) - { - DataTypePtr array_instance_type = DataTypeFactory::instance().get(internal_type_it->second); - array_nested_type = is_column_array_nullable ? makeNullable(array_instance_type) - : array_instance_type; - internal_nested_type = std::make_shared(array_nested_type); - } - else - { - throw Exception{"The internal type \"" + list_type->value_type()->name() + "\" of an array column \"" + header_column.name - + "\" is not supported for conversion from a " + format_name + " data format", - ErrorCodes::CANNOT_CONVERT_TYPE}; - } - } - else if (const auto * internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), - [=](auto && elem) { return elem.first == arrow_type; }); - internal_type_it != arrow_type_to_internal_type.end()) - { - internal_nested_type = DataTypeFactory::instance().get(internal_type_it->second); - } - else - { - throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + header_column.name - + "\" is not supported for conversion from a " + format_name + " data format", - ErrorCodes::CANNOT_CONVERT_TYPE}; - } - - const DataTypePtr internal_type = target_column_is_nullable ? makeNullable(internal_nested_type) - : internal_nested_type; + MutableColumnPtr read_column = internal_type->createColumn(); + readColumnFromArrowColumn(arrow_column, *read_column, header_column.name, format_name, false); ColumnWithTypeAndName column; column.name = header_column.name; column.type = internal_type; - - /// Data - MutableColumnPtr read_column = internal_nested_type->createColumn(); - switch (arrow_type) - { - case arrow::Type::STRING: - case arrow::Type::BINARY: - //case arrow::Type::FIXED_SIZE_BINARY: - fillColumnWithStringData(arrow_column, read_column); - break; - case arrow::Type::BOOL: - fillColumnWithBooleanData(arrow_column, read_column); - break; - case arrow::Type::DATE32: - fillColumnWithDate32Data(arrow_column, read_column); - break; - case arrow::Type::DATE64: - fillColumnWithDate64Data(arrow_column, read_column); - break; - case arrow::Type::TIMESTAMP: - fillColumnWithTimestampData(arrow_column, read_column); - break; - case arrow::Type::DECIMAL: - //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster - fillColumnWithDecimalData(arrow_column, read_column /*, internal_nested_type*/); - break; - case arrow::Type::LIST: - if (array_nested_type && list_nested_type) { - switch (list_nested_type->id()) - { - case arrow::Type::STRING: - case arrow::Type::BINARY: - //case arrow::Type::FIXED_SIZE_BINARY: - fillColumnWithArrayStringData(arrow_column, read_column); - break; - case arrow::Type::BOOL: - fillColumnWithArrayBooleanData(arrow_column, read_column); - break; - case arrow::Type::DATE32: - fillColumnWithArrayDate32Data(arrow_column, read_column); - break; - case arrow::Type::DATE64: - fillColumnWithArrayDate64Data(arrow_column, read_column); - break; - case arrow::Type::TIMESTAMP: - fillColumnWithArrayTimestampData(arrow_column, read_column); - break; - case arrow::Type::DECIMAL: - //fillColumnWithNumericData>(arrow_column, read_column); // Have problems with trash values under NULL, but faster - fillColumnWithArrayDecimalData(arrow_column, read_column /*, internal_nested_type*/); - break; - # define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ - case ARROW_NUMERIC_TYPE: \ - fillColumnWithArrayNumericData(arrow_column, read_column); \ - break; - - FOR_ARROW_NUMERIC_TYPES(DISPATCH) - # undef DISPATCH - default: - throw Exception - { - "Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" - + header_column.name + "\"", - ErrorCodes::UNKNOWN_TYPE - }; - } - } - break; -# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \ - case ARROW_NUMERIC_TYPE: \ - fillColumnWithNumericData(arrow_column, read_column); \ - break; - - FOR_ARROW_NUMERIC_TYPES(DISPATCH) -# undef DISPATCH - // TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds? - // TODO: read JSON as a string? - // TODO: read UUID as a string? - default: - throw Exception - { - "Unsupported " + format_name + " type \"" + arrow_column->type()->name() + "\" of an input column \"" - + header_column.name + "\"", - ErrorCodes::UNKNOWN_TYPE - }; - } - - - if (column.type->isNullable()) - { - MutableColumnPtr null_bytemap = DataTypeUInt8().createColumn(); - fillByteMapFromArrowColumn(arrow_column, null_bytemap); - column.column = ColumnNullable::create(std::move(read_column), std::move(null_bytemap)); - } - else - column.column = std::move(read_column); + column.column = std::move(read_column); column.column = castColumn(column, header_column.type); column.type = header_column.type; diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index be98bf68bbd..522a3927bef 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -18,6 +18,8 @@ #include #include +#include + namespace DB { namespace ErrorCodes @@ -50,12 +52,6 @@ namespace DB {"FixedString", arrow::utf8()}, }; - static const PaddedPODArray * extractNullBytemapPtr(ColumnPtr column) - { - ColumnPtr null_column = assert_cast(*column).getNullMapColumnPtr(); - const PaddedPODArray & null_bytemap = assert_cast &>(*null_column).getData(); - return &null_bytemap; - } static void checkStatus(const arrow::Status & status, const String & column_name, const String & format_name) { @@ -68,10 +64,12 @@ namespace DB ColumnPtr write_column, const PaddedPODArray * null_bytemap, const String & format_name, - arrow::ArrayBuilder* abuilder) + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) { const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); - ArrowBuilderType & builder = assert_cast(*abuilder); + ArrowBuilderType & builder = assert_cast(*array_builder); arrow::Status status; const UInt8 * arrow_null_bytemap_raw_ptr = nullptr; @@ -79,98 +77,48 @@ namespace DB if (null_bytemap) { /// Invert values since Arrow interprets 1 as a non-null value, while CH as a null - arrow_null_bytemap.reserve(null_bytemap->size()); - for (auto is_null : *null_bytemap) - arrow_null_bytemap.emplace_back(!is_null); + arrow_null_bytemap.reserve(end - start); + for (size_t i = start; i < end; ++i) + arrow_null_bytemap.template emplace_back(!(*null_bytemap)[i]); arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data(); } if constexpr (std::is_same_v) status = builder.AppendValues( - reinterpret_cast(internal_data.data()), - internal_data.size(), + reinterpret_cast(internal_data.data() + start), + end - start, reinterpret_cast(arrow_null_bytemap_raw_ptr)); else - status = builder.AppendValues(internal_data.data(), internal_data.size(), reinterpret_cast(arrow_null_bytemap_raw_ptr)); + status = builder.AppendValues(internal_data.data() + start, end - start, reinterpret_cast(arrow_null_bytemap_raw_ptr)); checkStatus(status, write_column->getName(), format_name); } static void fillArrowArrayWithArrayColumnData( const String & column_name, - ColumnPtr & nested_column, + ColumnPtr & column, const std::shared_ptr & column_type, - std::shared_ptr arrow_array, const PaddedPODArray * null_bytemap, arrow::ArrayBuilder * array_builder, - String format_name) + String format_name, + size_t start, + size_t end) { - const auto * column_array = static_cast(nested_column.get()); - const bool is_column_array_nullable = column_array->getData().isNullable(); - const IColumn & array_nested_column = - is_column_array_nullable ? static_cast(column_array->getData()).getNestedColumn() : - column_array->getData(); - const String column_array_nested_type_name = array_nested_column.getFamilyName(); + const auto * column_array = static_cast(column.get()); + ColumnPtr nested_column = column_array->getDataPtr(); + DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); + const auto & offsets = column_array->getOffsets(); - const auto * column_array_type = static_cast(column_type.get()); - const DataTypePtr & array_nested_type = - is_column_array_nullable ? static_cast(column_array_type->getNestedType().get())->getNestedType() : - column_array_type->getNestedType(); + arrow::ListBuilder & builder = assert_cast(*array_builder); + arrow::ArrayBuilder * value_builder = builder.value_builder(); + arrow::Status components_status; - const PaddedPODArray * array_null_bytemap = - is_column_array_nullable ? extractNullBytemapPtr(assert_cast(*nested_column).getDataPtr()) : nullptr; - - const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), - [=](auto && elem) { return elem.first == column_array_nested_type_name; }); - if (arrow_type_it != internal_type_to_arrow_type.end()) + for (size_t array_idx = start; array_idx < end; ++array_idx) { - std::shared_ptr list_type = arrow::list(arrow_type_it->second); - - const auto & internal_column = assert_cast(*nested_column); - - arrow::ListBuilder & builder = assert_cast(*array_builder); - arrow::ArrayBuilder * value_builder = builder.value_builder(); - arrow::Status components_status; - - const auto & offsets = internal_column.getOffsets(); - ColumnPtr & data = is_column_array_nullable ? - const_cast(static_cast(internal_column.getData()).getNestedColumnPtr()) : - const_cast(internal_column.getDataPtr()); - - size_t array_start = 0; - size_t array_length = 0; - - for (size_t idx = 0, size = internal_column.size(); idx < size; ++idx) - { - if (null_bytemap && (*null_bytemap)[idx]) - { - components_status = builder.AppendNull(); - checkStatus(components_status, nested_column->getName(), format_name); - } - else - { - components_status = builder.Append(); - checkStatus(components_status, nested_column->getName(), format_name); - array_length = offsets[idx] - array_start; - auto cut_data = data->cut(array_start, array_length); - if (array_null_bytemap == nullptr) - { - CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, - column_array_nested_type_name, arrow_array, - nullptr, value_builder, format_name); - } - else - { - PaddedPODArray array_nested_null_bytemap; - array_nested_null_bytemap.insertByOffsets(*array_null_bytemap, array_start, array_start + array_length); - - CHColumnToArrowColumn::fillArrowArray(column_name, cut_data, array_nested_type, - column_array_nested_type_name, arrow_array, - &array_nested_null_bytemap, value_builder, format_name); - } - array_start = offsets[idx]; - } - } + /// Start new array + components_status = builder.Append(); + checkStatus(components_status, nested_column->getName(), format_name); + CHColumnToArrowColumn::fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx]); } } @@ -179,13 +127,15 @@ namespace DB ColumnPtr write_column, const PaddedPODArray * null_bytemap, const String & format_name, - arrow::ArrayBuilder* abuilder) + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) { const auto & internal_column = assert_cast(*write_column); - arrow::StringBuilder & builder = assert_cast(*abuilder); + arrow::StringBuilder & builder = assert_cast(*array_builder); arrow::Status status; - for (size_t string_i = 0, size = internal_column.size(); string_i < size; ++string_i) + for (size_t string_i = start; string_i < end; ++string_i) { if (null_bytemap && (*null_bytemap)[string_i]) { @@ -205,14 +155,16 @@ namespace DB ColumnPtr write_column, const PaddedPODArray * null_bytemap, const String & format_name, - arrow::ArrayBuilder* abuilder) + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) { const PaddedPODArray & internal_data = assert_cast &>(*write_column).getData(); //arrow::Date32Builder date_builder; - arrow::UInt16Builder & builder = assert_cast(*abuilder); + arrow::UInt16Builder & builder = assert_cast(*array_builder); arrow::Status status; - for (size_t value_i = 0, size = internal_data.size(); value_i < size; ++value_i) + for (size_t value_i = start; value_i < end; ++value_i) { if (null_bytemap && (*null_bytemap)[value_i]) status = builder.AppendNull(); @@ -227,14 +179,16 @@ namespace DB ColumnPtr write_column, const PaddedPODArray * null_bytemap, const String & format_name, - arrow::ArrayBuilder* abuilder) + arrow::ArrayBuilder* array_builder, + size_t start, + size_t end) { const auto & internal_data = assert_cast &>(*write_column).getData(); //arrow::Date64Builder builder; - arrow::UInt32Builder & builder = assert_cast(*abuilder); + arrow::UInt32Builder & builder = assert_cast(*array_builder); arrow::Status status; - for (size_t value_i = 0, size = internal_data.size(); value_i < size; ++value_i) + for (size_t value_i = start; value_i < end; ++value_i) { if (null_bytemap && (*null_bytemap)[value_i]) status = builder.AppendNull(); @@ -249,36 +203,46 @@ namespace DB void CHColumnToArrowColumn::fillArrowArray( const String & column_name, - ColumnPtr & nested_column, - const std::shared_ptr & column_nested_type, - const String column_nested_type_name, - std::shared_ptr arrow_array, + ColumnPtr & column, + const std::shared_ptr & column_type, const PaddedPODArray * null_bytemap, arrow::ArrayBuilder * array_builder, - String format_name) + String format_name, + size_t start, + size_t end) { - if ("String" == column_nested_type_name) + const String column_type_name = column_type->getFamilyName(); + + if ("Nullable" == column_type_name) { - fillArrowArrayWithStringColumnData(nested_column, null_bytemap, format_name, array_builder); + const ColumnNullable * column_nullable = checkAndGetColumn(column.get()); + ColumnPtr nested_column = column_nullable->getNestedColumnPtr(); + DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); + ColumnPtr null_column = column_nullable->getNullMapColumnPtr(); + const PaddedPODArray & bytemap = assert_cast &>(*null_column).getData(); + fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end); } - else if ("FixedString" == column_nested_type_name) + else if ("String" == column_type_name) { - fillArrowArrayWithStringColumnData(nested_column, null_bytemap, format_name, array_builder); + fillArrowArrayWithStringColumnData(column, null_bytemap, format_name, array_builder, start, end); } - else if ("Date" == column_nested_type_name) + else if ("FixedString" == column_type_name) { - fillArrowArrayWithDateColumnData(nested_column, null_bytemap, format_name, array_builder); + fillArrowArrayWithStringColumnData(column, null_bytemap, format_name, array_builder, start, end); } - else if ("DateTime" == column_nested_type_name) + else if ("Date" == column_type_name) { - fillArrowArrayWithDateTimeColumnData(nested_column, null_bytemap, format_name, array_builder); + fillArrowArrayWithDateColumnData(column, null_bytemap, format_name, array_builder, start, end); } - else if ("Array" == column_nested_type_name) + else if ("DateTime" == column_type_name) { - fillArrowArrayWithArrayColumnData(column_name, nested_column, column_nested_type, arrow_array, null_bytemap, - array_builder, format_name); + fillArrowArrayWithDateTimeColumnData(column, null_bytemap, format_name, array_builder, start, end); } - else if (isDecimal(column_nested_type)) + else if ("Array" == column_type_name) + { + fillArrowArrayWithArrayColumnData(column_name, column, column_type, null_bytemap, array_builder, format_name, start, end); + } + else if (isDecimal(column_type)) { auto fill_decimal = [&](const auto & types) -> bool { @@ -289,24 +253,23 @@ namespace DB || std::is_same_v> || std::is_same_v>) { - const auto & decimal_type = static_cast(column_nested_type.get()); - fillArrowArrayWithDecimalColumnData(nested_column, arrow_array, null_bytemap, decimal_type, format_name); + fillArrowArrayWithDecimalColumnData(column, null_bytemap, array_builder, format_name, start, end); } return false; }; - callOnIndexAndDataType(column_nested_type->getTypeId(), fill_decimal); + callOnIndexAndDataType(column_type->getTypeId(), fill_decimal); } #define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \ - else if (#CPP_NUMERIC_TYPE == column_nested_type_name) \ + else if (#CPP_NUMERIC_TYPE == column_type_name) \ { \ - fillArrowArrayWithNumericColumnData(nested_column, null_bytemap, format_name, array_builder); \ + fillArrowArrayWithNumericColumnData(column, null_bytemap, format_name, array_builder, start, end); \ } FOR_INTERNAL_NUMERIC_TYPES(DISPATCH) #undef DISPATCH else { - throw Exception{"Internal type \"" + column_nested_type_name + "\" of a column \"" + column_name + "\"" + throw Exception{"Internal type \"" + column_type_name + "\" of a column \"" + column_name + "\"" " is not supported for conversion into a " + format_name + " data format", ErrorCodes::UNKNOWN_TYPE}; } @@ -315,16 +278,17 @@ namespace DB template static void fillArrowArrayWithDecimalColumnData( ColumnPtr write_column, - std::shared_ptr & arrow_array, const PaddedPODArray * null_bytemap, - const DataType * decimal_type, - const String & format_name) + arrow::ArrayBuilder * array_builder, + const String & format_name, + size_t start, + size_t end) { const auto & column = static_cast(*write_column); - arrow::DecimalBuilder builder(arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale())); + arrow::DecimalBuilder & builder = assert_cast(*array_builder); arrow::Status status; - for (size_t value_i = 0, size = column.size(); value_i < size; ++value_i) + for (size_t value_i = start; value_i < end; ++value_i) { if (null_bytemap && (*null_bytemap)[value_i]) status = builder.AppendNull(); @@ -334,10 +298,65 @@ namespace DB checkStatus(status, write_column->getName(), format_name); } - status = builder.Finish(&arrow_array); checkStatus(status, write_column->getName(), format_name); } + static std::shared_ptr getArrowType(DataTypePtr column_type, const std::string & column_name, const std::string & format_name, bool * is_column_nullable) + { + if (column_type->isNullable()) + { + DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); + auto arrow_type = getArrowType(nested_type, column_name, format_name, is_column_nullable); + *is_column_nullable = true; + return arrow_type; + } + + if (isDecimal(column_type)) + { + std::shared_ptr arrow_type; + const auto create_arrow_type = [&](const auto & types) -> bool { + using Types = std::decay_t; + using ToDataType = typename Types::LeftType; + + if constexpr ( + std::is_same_v> + || std::is_same_v> + || std::is_same_v>) + { + const auto & decimal_type = static_cast(column_type.get()); + arrow_type = arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale()); + } + + return false; + }; + callOnIndexAndDataType(column_type->getTypeId(), create_arrow_type); + return arrow_type; + } + + if (isArray(column_type)) + { + auto nested_type = typeid_cast(column_type.get())->getNestedType(); + auto nested_arrow_type = getArrowType(nested_type, column_name, format_name, is_column_nullable); + return arrow::list(nested_arrow_type); + } + + const std::string type_name = column_type->getFamilyName(); + if (const auto * arrow_type_it = std::find_if( + internal_type_to_arrow_type.begin(), + internal_type_to_arrow_type.end(), + [=](auto && elem) { return elem.first == type_name; }); + arrow_type_it != internal_type_to_arrow_type.end()) + { + return arrow_type_it->second; + } + throw Exception{ + "The type \"" + type_name + "\" of a column \"" + column_name + + "\"" + " is not supported for conversion into a " + + format_name + " data format", + ErrorCodes::UNKNOWN_TYPE}; + } + void CHColumnToArrowColumn::chChunkToArrowTable( std::shared_ptr & res, const Block & header, @@ -358,86 +377,20 @@ namespace DB column.column = recursiveRemoveLowCardinality(chunk.getColumns()[column_i]); column.type = recursiveRemoveLowCardinality(column.type); - const bool is_column_nullable = column.type->isNullable(); - bool is_column_array_nullable = false; - const auto & column_nested_type - = is_column_nullable ? static_cast(column.type.get())->getNestedType() : column.type; - const String column_nested_type_name = column_nested_type->getFamilyName(); - - if (isDecimal(column_nested_type)) - { - const auto add_decimal_field = [&](const auto & types) -> bool { - using Types = std::decay_t; - using ToDataType = typename Types::LeftType; - - if constexpr ( - std::is_same_v> - || std::is_same_v> - || std::is_same_v>) - { - const auto & decimal_type = static_cast(column_nested_type.get()); - arrow_fields.emplace_back(std::make_shared( - column.name, arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale()), is_column_nullable)); - } - - return false; - }; - callOnIndexAndDataType(column_nested_type->getTypeId(), add_decimal_field); - } - else if (isArray(column_nested_type)) - { - const auto * column_array_type = static_cast(column_nested_type.get()); - is_column_array_nullable = column_array_type->getNestedType()->isNullable(); - const DataTypePtr & column_array_nested_type = - is_column_array_nullable ? static_cast(column_array_type->getNestedType().get())->getNestedType() : - column_array_type->getNestedType(); - const String column_array_nested_type_name = column_array_nested_type->getFamilyName(); - - if (const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), - [=](auto && elem) { return elem.first == column_array_nested_type_name; }); - arrow_type_it != internal_type_to_arrow_type.end()) - { - arrow_fields.emplace_back(std::make_shared( - column.name, arrow::list(arrow_type_it->second), is_column_array_nullable)); - } else - { - throw Exception{"The type \"" + column_array_nested_type_name + "\" of a array column \"" + column.name + "\"" - " is not supported for conversion into a " + format_name + " data format", - ErrorCodes::UNKNOWN_TYPE}; - } - } - else - { - if (const auto * arrow_type_it = std::find_if(internal_type_to_arrow_type.begin(), internal_type_to_arrow_type.end(), - [=](auto && elem) { return elem.first == column_nested_type_name; }); - arrow_type_it != internal_type_to_arrow_type.end()) - { - arrow_fields.emplace_back(std::make_shared(column.name, arrow_type_it->second, is_column_nullable)); - } else - { - throw Exception{"The type \"" + column_nested_type_name + "\" of a column \"" + column.name + "\"" - " is not supported for conversion into a " + format_name + " data format", - ErrorCodes::UNKNOWN_TYPE}; - } - } - - ColumnPtr nested_column - = is_column_nullable ? assert_cast(*column.column).getNestedColumnPtr() : column.column; - - const PaddedPODArray * null_bytemap = - is_column_nullable ? extractNullBytemapPtr(column.column) : nullptr; + bool is_column_nullable = false; + auto arrow_type = getArrowType(column.type, column.name, format_name, &is_column_nullable); + arrow_fields.emplace_back(std::make_shared(column.name, arrow_type, is_column_nullable)); arrow::MemoryPool* pool = arrow::default_memory_pool(); std::unique_ptr array_builder; arrow::Status status = MakeBuilder(pool, arrow_fields[column_i]->type(), &array_builder); - checkStatus(status, nested_column->getName(), format_name); + checkStatus(status, column.column->getName(), format_name); + + fillArrowArray(column.name, column.column, column.type, nullptr, array_builder.get(), format_name, 0, column.column->size()); std::shared_ptr arrow_array; - - fillArrowArray(column.name, nested_column, column_nested_type, column_nested_type_name, arrow_array, null_bytemap, array_builder.get(), format_name); - status = array_builder->Finish(&arrow_array); - checkStatus(status, nested_column->getName(), format_name); + checkStatus(status, column.column->getName(), format_name); arrow_arrays.emplace_back(std::move(arrow_array)); } @@ -445,7 +398,6 @@ namespace DB res = arrow::Table::Make(arrow_schema, arrow_arrays); } - - } +} #endif diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h index 07bceb6266c..9740063f110 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h @@ -33,13 +33,13 @@ public: static void fillArrowArray( const String & column_name, - ColumnPtr & nested_column, - const std::shared_ptr & column_nested_type, - const String column_nested_type_name, - std::shared_ptr arrow_array, + ColumnPtr & column, + const std::shared_ptr & column_type, const PaddedPODArray * null_bytemap, arrow::ArrayBuilder * array_builder, - String format_name); + String format_name, + size_t start, + size_t end); }; } #endif diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.reference b/tests/queries/0_stateless/00900_orc_arrays_load.reference new file mode 100644 index 00000000000..9b20ef98164 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_arrays_load.reference @@ -0,0 +1,4 @@ +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.sh b/tests/queries/0_stateless/00900_orc_arrays_load.sh new file mode 100755 index 00000000000..ff3d4596b61 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_arrays_load.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_FILE=$CUR_DIR/data_orc/array_test.orc + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) ENGINE=Memory()" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE +${CLICKHOUSE_CLIENT} --query="select * from orc_load" + +${CLICKHOUSE_CLIENT} --query="drop table orc_load" diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference new file mode 100644 index 00000000000..dd9c9900684 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference @@ -0,0 +1,2 @@ +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh new file mode 100755 index 00000000000..d9d5e5f1c57 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_FILE=$CUR_DIR/data_orc/nested_array_test.orc + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory()" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE +${CLICKHOUSE_CLIENT} --query="select * from orc_load" + +${CLICKHOUSE_CLIENT} --query="drop table orc_load" diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference new file mode 100644 index 00000000000..62e95652040 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference @@ -0,0 +1,6 @@ +[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] +[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh new file mode 100755 index 00000000000..ec2a8be2d07 --- /dev/null +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +DATA_FILE=$CUR_DIR/data_orc/nullable_array_test.orc + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE +${CLICKHOUSE_CLIENT} --query="select * from orc_load" + +${CLICKHOUSE_CLIENT} --query="drop table orc_load" diff --git a/tests/queries/0_stateless/00900_parquet.reference b/tests/queries/0_stateless/00900_parquet.reference index 0f4be2c74a0..230d1f5ca48 100644 --- a/tests/queries/0_stateless/00900_parquet.reference +++ b/tests/queries/0_stateless/00900_parquet.reference @@ -60,3 +60,15 @@ dest from null: -108 108 -1016 1116 -1032 1132 -1064 1164 -1.032 -1.064 string-0 fixedstring\0\0\0\0 2001-02-03 2002-02-03 04:05:06 127 255 32767 65535 2147483647 4294967295 9223372036854775807 9223372036854775807 -1.032 -1.064 string-2 fixedstring-2\0\0 2004-06-07 2004-02-03 04:05:06 \N \N \N \N \N \N \N \N \N \N \N \N \N \N +1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +2 [NULL] [NULL] [NULL] +2 [NULL] [NULL] [NULL] +3 [] [] [] +3 [] [] [] +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] diff --git a/tests/queries/0_stateless/00900_parquet.sh b/tests/queries/0_stateless/00900_parquet.sh index 4b06001429f..8c19c7cecab 100755 --- a/tests/queries/0_stateless/00900_parquet.sh +++ b/tests/queries/0_stateless/00900_parquet.sh @@ -127,6 +127,7 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT echo dest from null: ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types6 ORDER BY int8" + ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types5" ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types6" @@ -135,3 +136,33 @@ ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types1" ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types2" ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types3" ${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types4" + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_arrays" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory()" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001])" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays FORMAT Parquet" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays ORDER BY id" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_arrays" + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nullable_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory()" +${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], [])" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays FORMAT Parquet" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays ORDER BY id" +${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nullable_arrays" + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nested_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory() " +${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []])" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays FORMAT Parquet" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays" +${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nested_arrays" diff --git a/tests/queries/0_stateless/00900_parquet_load.reference b/tests/queries/0_stateless/00900_parquet_load.reference index 65973e05c24..4bd699f40fe 100644 --- a/tests/queries/0_stateless/00900_parquet_load.reference +++ b/tests/queries/0_stateless/00900_parquet_load.reference @@ -1,6 +1,10 @@ === Try load data from alltypes_dictionary.parquet 0 1 0 0 0 0 0 0 01/01/09 0 1230768000 1 0 1 1 1 10 1.1 10.1 01/01/09 1 1230768060 +=== Try load data from alltypes_list.parquet +[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] === Try load data from alltypes_plain.parquet 4 1 0 0 0 0 0 0 03/01/09 0 1235865600 5 0 1 1 1 10 1.1 10.1 03/01/09 1 1235865660 @@ -258,8 +262,9 @@ Code: 33. DB::ParsingEx---tion: Error while reading Parquet data: IOError: Not y 23.00 24.00 === Try load data from list_columns.parquet -Code: 70. DB::Ex---tion: The type "list" of an input column "int64_list" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin - +[1,2,3] ['abc','efg','hij'] +[NULL,1] [] +[4] ['efg',NULL,'hij','xyz'] === Try load data from nation.dict-malformed.parquet 0 ALGERIA 0 haggle. carefully final deposits detect slyly agai 1 ARGENTINA 1 al foxes promise slyly according to the regular accounts. bold requests alon @@ -286,9 +291,12 @@ Code: 70. DB::Ex---tion: The type "list" of an input column "int64_list" is not 22 RUSSIA 3 requests against the platelets use never according to the quickly regular pint 23 UNITED KINGDOM 3 eans boost carefully special requests. accounts are. carefull 24 UNITED STATES 1 y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be +=== Try load data from nested_lists.parquet +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] === Try load data from nested_lists.snappy.parquet -Code: 70. DB::Ex---tion: The type "list" of an input column "a" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin - +[[['a','b'],['c']],[[],['d']]] 1 +[[['a','b'],['c','d']],[[],['e']]] 1 +[[['a','b'],['c','d'],['e']],[[],['f']]] 1 === Try load data from nested_maps.snappy.parquet Code: 70. DB::Ex---tion: The type "map" of an input column "a" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin @@ -301,6 +309,10 @@ Code: 70. DB::Ex---tion: The type "map" of an input column "a" is not supported ../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id()) === Try load data from nullable.impala.parquet ../contrib/arrow/cpp/src/arrow/array/array_nested.cc:192: Check failed: (self->list_type_->value_type()->id()) == (data->child_data[0]->type->id()) +=== Try load data from nullable_list.parquet +[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] === Try load data from nulls.snappy.parquet Code: 70. DB::Ex---tion: The type "struct" of an input column "b_struct" is not supported for conversion from a Parquet data format: data for INSERT was parsed from stdin diff --git a/tests/queries/0_stateless/data_orc/array_test.orc b/tests/queries/0_stateless/data_orc/array_test.orc new file mode 100644 index 0000000000000000000000000000000000000000..8de2ebd7054c8586c60f918bb268525bf6594664 GIT binary patch literal 3870 zcmds3+iM(E7@wIlGkbRSHaokUJv-TKb{CC_)iAqB8j}azrW8fcmRkRSf>H!8pgt9p zAtE8t8|qW55CkbGgrZd+l$zo#zSIze)@ww4P=q!}t@vQ$nKQqc>@C^$)nU(>^SgX! zcFs9`^X=Wg+lU!Nk|ax|%09_R7#as5%mpw`iKSCw>P+qFQn)ML>0-fLDLz+}9w+7{ zm7O3xjmxrUJQM9SmVy~`e8v>rcy~Tyy2maQnR|9V)6?$0d{s8xZ$FJPckXnu(>@f; zKEP+s(~XZWA6t!?uDd$-^GA0w*Ztw*+S+Jm*lj^riH9-jF8%Z4SBv~XLEO?`XFvOU zvNO&ybg}Z1MJbt4=uB@5>$Hp*s@T2?C3b(550~;o-`$hugT~kkhZ_Y+TF<@bJlD8^ zNFKANkzPMP_j%&fm0g7c59j{;o)0=B^T8V@DVe79xrR4r3 zb7wEUadfimuDWpG(N#xswi3ZB>N@{^0_+Vic^jGH(^WP@p zE_^WeyS8#fe(Bl@Kgg<~Q{9wULfmDR5_g5=#hn1Fz#6d5Y@tVhDR30nU}d4lfaAak z;3OLsdJ5QNBLb&^EtVBH1DplU0r#;TLeB#;-~zDC3PSG(F0zWi1HdIVB=8_`ncX39 z1-Qy4IhOdkDB>=QyJE_W2&@P!n+o&RPcjK~0$qi!Lf4>c&~@lKi-`CL^aykcogzL8 zJqq1`Zis!8%oy|-^f>f5^aS(-^d$5o%ZcZwpr@dl&`s7S^fdG|bPL!LST-%OzLJ>% zpMft6oP{sT;$prW_#AwF0?TF}^gQ%DbOxOquB6Gy-Ew;wOL0rW3OoN@eR}n zZAPBHwk}G)y!B!Ei(qq1^xy*wMld5FmuaAuam>BfbFsVE>2^ zk?kjJKmgLRP1#*-+@l2|G~IrFb-ZfVln)N~ZD|z%v>L?*Y;6l7mr*tL7DTTN!4+1m zLFtTNjiYuXcQiBwx7Ngh&?2fmiFi8(9@lPkm*SQqB}b$Gjyi_<Ud-PgAH90}^71XEX-ZI})V)5JT51+jq;PUk z{z1VSG!A?nQTQtnC8J{0{zkMH%njp&5&hNZpE9?MCFfPlYsMzV1J#w32TnwkjNl}5 z>2>rg=E|GMwO53~(Vq}`!TuhBMk45j(dhFh%qPBx@yHxm2mP&RpR&D2aF)5{yfSj( zO6C>&djwZAueol+yx}^HzfL1h=l&n-Og()3%pTePP7Q0AevRScjPJ|XA4dH5RTzB1 zT46r#gD0pW@=T+0U(Kv0_#yl!G5@TVVsVphrxwMun#t#P=)FttJ$moYVF@c(lxSNa zbE0yhnb1e&W|NaiPB0}5yI8^IF=dJwiPm{clVVtBCD!rOX{EE%o;tc7b#%v1y<4Y_ oCSS0QI+{E|HFfmC$0v^jevx!Rwp+cBP%e*G=EFCo4i5MIFLpbKG5`Po literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_orc/nullable_array_test.orc b/tests/queries/0_stateless/data_orc/nullable_array_test.orc new file mode 100644 index 0000000000000000000000000000000000000000..8a1ad838b502edc42110bf2bbb547c1425a74db6 GIT binary patch literal 714 zcma)(u};G<5Qfh_+d4TdL{R~gsj3j_RGH9;scHuppsfloz<>|~AOR0hv9d5Tb>Kmm zcnx?6RvL`shE|m-2H)@R_xTdv4Msg3X#{{o5Dr0CwRBM8#tbXO6Gc20MlEM0TKP+m z5V0<&&E=GxG9@LR)wHlJYZKYdTJ0FxDJ{r=(Z>gIBG z4zJy*5L%In!Bo96v+k9fEw30@n3`u9SeaeN;O=p+%qx$X`K<>r_spE4XAI{U&PCut znd?a8^JMU4$Wx(?@^xhF)b-eX_dhYk!;>?8tc#%(ad8BN_t)=&S2UL}e!;Zjndv0ZJ?bo3R78()hgz*4GIB$#6rg>YH_L|!X%Pk)G-wJWz0l_OD f5O~D{I?w$hM(@l`ZafwZ4pFmj)#Df9VAL}|T|~AOR0hv9d5Tb>Kmm zcnx?6RvL`shE|m-2H)@R_xTdv4Msg3X#{{o5Dr0CwRBM8#tbXO6Gc20MlEM0TKP+m z5V0<&&E=GxG9@LR)wHlJYZKYdTJ0FxDJ{r=(Z>gIBG z4zJy*5L%In!Bo96v+k9fEw30@n3`u9SeaeN;O=p+%qx$X`K<>r_spE4XAI{U&PCut znd?a8^JMU4$Wx(?@^xhF)b-eX_dhYk!;>?8tc#%(ad8BN_t)=&S2UL}e!;Zjndv0ZJ?bo3R78()hgz*4GIB$#6rg>YH_L|!X%Pk)G-wJWz0l_OD f5O~D{I?w$hM(@l`ZafwZ4pFmj)#Df9VAL}|TfG=^y3?6J2WH^Il$YK6_F1$ znLG+$^<;SDV54c7Vgy`F0eln-U9rwtSU1%O=%)a7#X(ct=m4?jgmuFI;uSG2Eh!2zWmWqh9aC1Hhs*;>T?D8sN zyzW(6ygBKGUfDUs=GRP!w1D$)CfuCq9Ae!p6Cy3*bScoJL`0NDC4fUN8HM9AS?moj zUcP+k!aEM0DPmeCsZ)n!nJ1+(PwFKm!|;T~3HhX1y9DwpaGWcRfzH`_3$qT_f^>kIXhy5r zhNiXy5&h^b;#Mm)nJ1+(PwI&`^(Q*b2Wdu{=I*L_vv)2u%|&nOx-6Q9NBKH3piw6F z2UQcp@S=FL<#i)Xb062d*#%Gssy7YVhNf0gcZOkp?|M=3%MCW%o1|3cNj$Wn>k0t!G1$Fsm?7xN0 znDIUSrWLzlxOtkS?ABz7dEup~i_)h4@)^xDM>y5IJ{H>EguVr6UbxN(NFy+B`QK-K zBu9y0UY^C2pcCk(JY8rs#XMURq&8OSV4*(;F6NN#VKQbwE0uP#m>M9N6Ox?fpMuCm z`%6nw%E}Mz-nSD!23G4)9Q95yRLjN(d~=%L7{^(GfxCJ7^ouJt&-Vn3;m8{9=0bfg zp={Jt>?{gTT#Og%&dkP(Jm%nKZB2;6lX96S!YD*(G1r#poWrJ!YX5Fwq1z9q z{Aw67sH4&}jW?c4^H$V*@2qU|{rR?)o6M6MnJ0A;^3Gka`uUao_guT-sT%U-hwATf8R0nr^f}z2Cr4i!}z;33G7J&G6+T~#Y&3bo8B2@ z8mo2H+P9(oE$phphN9YM`X;~5XP9M~4AY45)oK%IFQi=Ni4>jpxfS|w5H=ashOO2D zs4P(1i$NWgo5}Zu-cmn-Eo@l7l-z&2<;y2`D^J<2)yQt`9pd}&Qhb@>`^HszXbGEM zYre037g|@M@57ZwfTG$LCnjj4fZ$;~u3ok#%o@(sD|3to)eiiN&ieFf)AZFk64$a) zJqs7!hv)Apa=1z=JC7&_aLP)eVQdNZU*vEuliZvg}ItU ze7uM>BM&?<3`(L=5$o^;N+W7sXR^>b*4A)RH3$6xXbdPC(g$yD8`9Hj3((}2|JZoIU6bo~Zj$(MWZQ|!3&r6J^&~uQg5@^{$y2r%GLGJC+=kt+rkb67f z{7yo7g5)6AztlNL%t7w&fzx}`Ff?vC$Qj&Ki@4aWbUZ7<2Wk3B4=oXMklG{Aet06~AZL%6CcV~?IGBT6ItrCX zCqNGJ>}wNQdJgjRIJ6yqRdbNm3f5$zb+8R4AywZ((+LNY5YF4bD3_6ivhBM{N{hu* z1|KkEz-2WdK=SomURF0A$*BS^os-pr2kFUyi;3>(q?$knY^UrGo_Ws6xDETu^B7aD@DW3#BVaZh45@5~_5ZhMUd=l%S8uusH%* z`p0r>dWk)#k^YflEmA{$9HYBQq@&U_xe}%5UnQQLQ5k4YvG{Ly=2r3>asmA_#f!0~ zw>;`wp3qzL?-nmgrnhp`w{q|;N$}=-`={GDV*|Dh%z&-aFJS8^3)niB0=5pBfUOfF zVC(n?*gDGr_6}x%y;Bxo@5lt$JO2Ro4l{tglL=t&m;u;3Gaz+Ll{nakH;gtUWJGDB z9~|77hbV2IgM<6s5T#9N7^CfB7^8h!h{7gn^maMH$-PL3)`lUB)y^F_xzz?^wdVy+ zZ&O8eD+$J97f0&%;;7I$8}A-GfsMmLl=fc0!Oaqg(vAi=xCH@G+9!Z9TCs;QS~`a) ztZPHm7M{V$Bb5S`mw&eK3b$j!Q!TtWD_U~&+T^AHjXud^VELSbP;5F!aE0uq0PrLi^? z#>yXH=?}3o7XAol(ZU#GV=QK8C-ZjRdy{QM2R?GV)$pW_7V;FpzQX~ae|+gDOeiNz z9`I!e8&rf86^c!YYQoIbVxPcu1bQqzH31DYoC*{;O{htA`h;4^@mR9ln%&UdebpR{ z+aU+TmO25D2}@iv@vOO5nrnW4ZF@A8BLiYl2~?8k)QL(7j(b6njegHg1-J5V z<{532G=Q*d$M*id$$#@H=DM&*A7u-mkQued=NH{+u`?Q_XWj9ne|DPoOJT_`Hg}^& Tb7!w*dt_g2Agcj1=wH4A2TOA7 literal 520 zcmZWny-or_5Z?RYyhJo$mn5{~6JG0JtCP!J0%4B6xMUIgToH|sbT75!oqh)V^&TZsN?d8aDGUdYoErV%z zN>g~^Lld8x^AvI3*>4EfjL59x0Hi+Y_QTT-a==6e7H(yrVUs2*Dt=_BH>zsSpOE zgoL=7E|wsILPFV)QhE&{0TC;9XjRe237fJRn=gs3nU*=|sTQ=%P*I7wkpSG+zXXh) zdh+iHd*6bE4!VkL#&w%o{&(-udN!LjK?2cCVE4DnERrZ=yi}H4E zyTf?s+WmA@PIEh;DGhSUqBM2#faPgcM2Y89=K6k8cyW>YMO0+PA=woO8RG}|0T44{ A?EnA( diff --git a/tests/queries/0_stateless/data_parquet/list_columns.parquet.columns b/tests/queries/0_stateless/data_parquet/list_columns.parquet.columns index 86745c2f074..1a488ee6e00 100644 --- a/tests/queries/0_stateless/data_parquet/list_columns.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/list_columns.parquet.columns @@ -1 +1 @@ -`int64_list` Nullable(Int64), `utf8_list` Nullable(String) +`int64_list` Array(Nullable(Int64)), `utf8_list` Array(Nullable(String)) diff --git a/tests/queries/0_stateless/data_parquet/nested_lists.parquet b/tests/queries/0_stateless/data_parquet/nested_lists.parquet new file mode 100644 index 0000000000000000000000000000000000000000..0e9b9998ab60d9dbb5c9a3144060f8d0eb81a1fe GIT binary patch literal 1755 zcmc(gL5tHs6vt;K)7jY8DvNK(EJ4cB9dhUv+uam}UKTtkL@iqn-n1-XTWEJ{Tg9^< z!Gi}+euzEE9{mi0A_yY*4P+4yzG<5#NGO|24I%Gk-sHXa%YPnib3@(hu7Tsx1W|kXH!^bR|(5i z?!O`WHFJS0pA(^8v2L9Y`jaW|Ps3rF2TX3s3o3t8eW>j$O{gk1&VdmGF@_j(6Sf0j zGwf@(3pmy}9fsw}blg8V#(5R+s*uNd2DnMQLL%%WZw%92^FFKr<;FB05GsiNkaMB9 zWNV}m#bd~pm*-S|r235OU)Ji1sXS$<1lX>T@p>y#B~mg1DlLEqvhWT;N53yF zB}1nDm?ZWRLcJ^KuX(q*s!j?8Q-73C95FfSg~Kk`1aENZ<4P1k3tzU+omn+prap6K zn;~lq_|D?VdGbcq1gi=|F-pEe0@pOmu>ZZRC6Yaw1pAp2{M(0Q%2mPt`=qRQ!>@n; ztz&n{A3`#e5bX^@yzNnU{QN~Y-8~wO$}hw5q?uaCwra K_#tlNZ|yJJxFX~L literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_parquet/nested_lists.parquet.columns b/tests/queries/0_stateless/data_parquet/nested_lists.parquet.columns new file mode 100644 index 00000000000..c4ca4443236 --- /dev/null +++ b/tests/queries/0_stateless/data_parquet/nested_lists.parquet.columns @@ -0,0 +1 @@ +`a1` Array(Array(Array(UInt32))), `a2` Array(Array(Array(String))), `a3` Array(Array(Nullable(UInt32))), `a4` Array(Array(Nullable(String))) diff --git a/tests/queries/0_stateless/data_parquet/nested_lists.snappy.parquet.columns b/tests/queries/0_stateless/data_parquet/nested_lists.snappy.parquet.columns index 6d55d46dd5b..99310769309 100644 --- a/tests/queries/0_stateless/data_parquet/nested_lists.snappy.parquet.columns +++ b/tests/queries/0_stateless/data_parquet/nested_lists.snappy.parquet.columns @@ -1 +1 @@ -`a` Nullable(String), `b` Nullable(Int32) +`a` Array(Array(Array(Nullable(String)))), `b` Nullable(Int32) diff --git a/tests/queries/0_stateless/data_parquet/nullable_list.parquet b/tests/queries/0_stateless/data_parquet/nullable_list.parquet new file mode 100644 index 0000000000000000000000000000000000000000..f42cf1beb466e640778d6e0809e2e3d199c693bf GIT binary patch literal 2138 zcmcguF>KR76ump24mWXKx9Xj2WK~f`P8rgOv~pAlFWov=5D`WOqRK!d(uO23GIU^M z=)l~e>eQ)XU_e#Xp+kp`OchgwIwD3U{>4qz8ixpMu&w*H@8AFby?_7P^_7QJPI-af z2|&;pge)JU=pdJpBQ;;<3^W*LuB{JeY<;LJNGahh7_fY$tT9cE_Yl^I{EA&p9giaB zER$xHzG(IOwAByWoCqd9WAO#cRjL1pAE&IM=LA8lN+2cX_r2c9#F*L+Uph$+DqR76@ z=gNxX!N#a5;wSH_k-!Nf@N_gWI@?4%5l`h0U7qPKEY8N^PJVP^B8O)=d63J_;oHg4 zBu5Sp47qJ&W$S0j~pM$BD&BU_|n(mui_`5l_Z=1 literal 0 HcmV?d00001 diff --git a/tests/queries/0_stateless/data_parquet/nullable_list.parquet.columns b/tests/queries/0_stateless/data_parquet/nullable_list.parquet.columns new file mode 100644 index 00000000000..2f308030da6 --- /dev/null +++ b/tests/queries/0_stateless/data_parquet/nullable_list.parquet.columns @@ -0,0 +1 @@ +`a1` Array(Nullable(UInt32)), `a2` Array(Nullable(String)), `a3` Array(Nullable(Decimal(4, 2))) From 8bad7f6112b7911ebf5cf3bb694e9fc94a674c39 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:21:26 +0300 Subject: [PATCH 25/82] Update docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 7ffb69b8e5f..ff177c1ea43 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -4,7 +4,7 @@ toc_priority: 141 # deltaSumTimestamp {#agg_functions-deltasum} -Adds the difference between consecutive rows. If the difference is negative, it is ignored. Uses `timestamp` to order values. +Adds the difference between consecutive rows. If the difference is negative, it is ignored. This function is primarily for [materialized views](../../../sql-reference/statements/create/view.md#materialized) that are ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the "right" order. This function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states correctly during merging. From de13b20f11097db7387234804cc8191ea55ef27a Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:21:45 +0300 Subject: [PATCH 26/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 4bf60fbc2d0..78cbf13cd43 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -4,7 +4,7 @@ toc_priority: 141 # deltaSumTimestamp {#agg_functions-deltasum} -Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. Параметр `timestamp` используется для упорядочивания значений. +Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например по бакету `toStartOfMinute`. Поэтому строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. From 8e3d1975340de90ccb6b7e3fbbfb94eecde32885 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:22:15 +0300 Subject: [PATCH 27/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 78cbf13cd43..22471322bec 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -33,7 +33,7 @@ deltaSumTimestamp(value, timestamp) ```sql SELECT deltaSumTimestamp(value, timestamp) -FROM (select number as timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] as value from numbers(1, 10)); +FROM (SELECT number AS timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] AS value FROM numbers(1, 10)); ``` Результат: From 11111d49f6539eff857af89843d57547a91e793f Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:25:16 +0300 Subject: [PATCH 28/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasum.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/aggregate-functions/reference/deltasum.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index 9dec611eb83..2bcfac94b68 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -8,7 +8,7 @@ toc_priority: 141 Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. -Если вы хотите использовать эту функцию в [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized), вместо нее лучше используйте функцию [deltaSumTimestamp](deltasumtimestamp.md). +В [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized), вместо этой функции рекомендуется использовать [deltaSumTimestamp](deltasumtimestamp.md). **Синтаксис** From 303d6b145d34f9bb89eade9f50d24ad0070791ee Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:26:00 +0300 Subject: [PATCH 29/82] Update docs/ru/engines/table-engines/integrations/s3.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/engines/table-engines/integrations/s3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/engines/table-engines/integrations/s3.md b/docs/ru/engines/table-engines/integrations/s3.md index aea711a7aca..ea4b579ead8 100644 --- a/docs/ru/engines/table-engines/integrations/s3.md +++ b/docs/ru/engines/table-engines/integrations/s3.md @@ -83,7 +83,7 @@ SELECT * FROM s3_engine_table LIMIT 2; Необязательные настройки: - `access_key_id` и `secret_access_key` — указывают учетные данные для использования с данной точкой приема запроса. - `use_environment_credentials` — если `true`, S3-клиент будет пытаться получить учетные данные из переменных среды и метаданных [Amazon EC2](https://ru.wikipedia.org/wiki/Amazon_EC2) для данной точки приема запроса. Значение по умолчанию — `false`. -- `use_insecure_imds_request` — признак, нужно ли использовать менее безопасное соединение при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию — `false`. +- `use_insecure_imds_request` — признак использования менее безопасного соединения при выполнении запроса к IMDS при получении учётных данных из метаданных Amazon EC2. Значение по умолчанию — `false`. - `region` — название региона S3. - `header` — добавляет указанный HTTP-заголовок к запросу на заданную точку приема запроса. Может быть определен несколько раз. - `server_side_encryption_customer_key_base64` — устанавливает необходимые заголовки для доступа к объектам S3 с шифрованием SSE-C. From 3ff1c87d1ab71a1981169baf302f24e629e95e6d Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:26:23 +0300 Subject: [PATCH 30/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasum.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../ru/sql-reference/aggregate-functions/reference/deltasum.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index 2bcfac94b68..e63f6409c83 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -22,7 +22,8 @@ deltaSum(value) **Возвращаемое значение** -- Накопленная арифметическая разница, типа `Integer` или `Float`. +- Накопленная арифметическая разница. +Тип: `Integer` или `Float`. **Примеры** From 38fb75fa039c24ae085315271949c2c6503ba0a3 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:26:34 +0300 Subject: [PATCH 31/82] Update docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index ff177c1ea43..d266ccc7c7a 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -25,7 +25,7 @@ deltaSumTimestamp(value, timestamp) - Accumulated differences between consecutive values, ordered by the `timestamp` parameter. -Type: [Integer](../../data-types/int-uint.md) type or [Float](../../data-types/float.md) type or a [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md). +Type: [Integer](../../data-types/int-uint.md) or [Float](../../data-types/float.md) or [Date](../../data-types/date.md) or [DateTime](../../data-types/datetime.md). **Example** From 402674ab86e24143dc6ab080af81cf567d053465 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:26:51 +0300 Subject: [PATCH 32/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 22471322bec..940f990138d 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -6,7 +6,7 @@ toc_priority: 141 Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. -Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например по бакету `toStartOfMinute`. Поэтому строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. +Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например, по бакету `toStartOfMinute`. Строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](./deltasum.md) вместо функции `deltaSumTimestamp`. From 3c1aaba9f3f0831d27e8dd701532f01992cc43fa Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:27:10 +0300 Subject: [PATCH 33/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 940f990138d..1f62e42e543 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -19,7 +19,7 @@ deltaSumTimestamp(value, timestamp) **Аргументы** - `value` — входные значения, должны быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). -- `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). +- `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). **Возвращаемое значение** From cde5009ee8793a77f4385b052751e66e44040f0c Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 21:27:22 +0300 Subject: [PATCH 34/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 1f62e42e543..9f8e09b07ec 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -25,7 +25,7 @@ deltaSumTimestamp(value, timestamp) - Накопленная разница между последовательными значениями, упорядоченными по параметру `timestamp`. -Тип: [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). +Тип: [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). **Пример** From 86f061976e0b818cca3b13d93c94c2f6ab5e5380 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Wed, 12 May 2021 22:02:56 +0300 Subject: [PATCH 35/82] Some fixes --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 17 +++++++++------ .../Formats/Impl/CHColumnToArrowColumn.cpp | 20 +++++++++--------- .../0_stateless/00900_orc_arrays_load.sh | 10 ++++----- .../00900_orc_nested_arrays_load.sh | 12 +++++------ .../00900_orc_nullable_arrays_load.sh | 12 +++++------ tests/queries/0_stateless/data_orc/tmp | Bin 714 -> 0 bytes .../data_parquet/00900_parquet_load.reference | 0 7 files changed, 37 insertions(+), 34 deletions(-) delete mode 100644 tests/queries/0_stateless/data_orc/tmp delete mode 100644 tests/queries/0_stateless/data_parquet/00900_parquet_load.reference diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index cef9bd79b2a..3134fe7d919 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -274,9 +274,11 @@ namespace DB // TODO: check if a column is const? if (!is_nullable && !isColumnArray(internal_column) && arrow_column->null_count()) { - throw Exception{ - "Can not insert NULL data into non-nullable column \"" + column_name + "\"", - ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN}; + throw Exception + { + "Can not insert NULL data into non-nullable column \"" + column_name + "\"", + ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN + }; } switch (arrow_column->type()->id()) @@ -373,10 +375,11 @@ namespace DB { return DataTypeFactory::instance().get(internal_type_it->second); } - throw Exception{ - "The type \"" + arrow_type->name() + "\" of an input column \"" + column_name + "\" is not supported for conversion from a " - + format_name + " data format", - ErrorCodes::CANNOT_CONVERT_TYPE}; + throw Exception + { + "The type \"" + arrow_type->name() + "\" of an input column \"" + column_name + "\" is not supported for conversion from a " + format_name + " data format", + ErrorCodes::CANNOT_CONVERT_TYPE + }; } void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk & res, std::shared_ptr & table, diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 522a3927bef..3d0b16e3b0a 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -218,7 +218,7 @@ namespace DB const ColumnNullable * column_nullable = checkAndGetColumn(column.get()); ColumnPtr nested_column = column_nullable->getNestedColumnPtr(); DataTypePtr nested_type = typeid_cast(column_type.get())->getNestedType(); - ColumnPtr null_column = column_nullable->getNullMapColumnPtr(); + const ColumnPtr & null_column = column_nullable->getNullMapColumnPtr(); const PaddedPODArray & bytemap = assert_cast &>(*null_column).getData(); fillArrowArray(column_name, nested_column, nested_type, &bytemap, array_builder, format_name, start, end); } @@ -269,9 +269,11 @@ namespace DB #undef DISPATCH else { - throw Exception{"Internal type \"" + column_type_name + "\" of a column \"" + column_name + "\"" - " is not supported for conversion into a " + format_name + " data format", - ErrorCodes::UNKNOWN_TYPE}; + throw Exception + { + "Internal type \"" + column_type_name + "\" of a column \"" + column_name + "\" is not supported for conversion into a " + format_name + " data format", + ErrorCodes::UNKNOWN_TYPE + }; } } @@ -349,12 +351,10 @@ namespace DB { return arrow_type_it->second; } - throw Exception{ - "The type \"" + type_name + "\" of a column \"" + column_name - + "\"" - " is not supported for conversion into a " - + format_name + " data format", - ErrorCodes::UNKNOWN_TYPE}; + + throw Exception{"The type \"" + column_name + "\" of a column \"" + column_name + "\"" + " is not supported for conversion into a " + format_name + " data format", + ErrorCodes::UNKNOWN_TYPE}; } void CHColumnToArrowColumn::chChunkToArrowTable( diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.sh b/tests/queries/0_stateless/00900_orc_arrays_load.sh index ff3d4596b61..0b3dfd62526 100755 --- a/tests/queries/0_stateless/00900_orc_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_arrays_load.sh @@ -7,9 +7,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=$CUR_DIR/data_orc/array_test.orc ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) ENGINE=Memory()" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE -${CLICKHOUSE_CLIENT} --query="select * from orc_load" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) ENGINE=Memory()" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_arrays format ORC" +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_arrays format ORC" < $DATA_FILE +${CLICKHOUSE_CLIENT} --query="select * from orc_arrays" -${CLICKHOUSE_CLIENT} --query="drop table orc_load" +${CLICKHOUSE_CLIENT} --query="drop table orc_arrays" diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh index d9d5e5f1c57..9c40efd6676 100755 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh @@ -6,10 +6,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=$CUR_DIR/data_orc/nested_array_test.orc -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory()" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE -${CLICKHOUSE_CLIENT} --query="select * from orc_load" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nested_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory()" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nested_arrays format ORC" +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_nested_arrays format ORC" < $DATA_FILE +${CLICKHOUSE_CLIENT} --query="select * from orc_nested_arrays" -${CLICKHOUSE_CLIENT} --query="drop table orc_load" +${CLICKHOUSE_CLIENT} --query="drop table orc_nested_arrays" diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh index ec2a8be2d07..b3de283033a 100755 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh @@ -6,10 +6,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=$CUR_DIR/data_orc/nullable_array_test.orc -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_load (a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_load format ORC" < $DATA_FILE -${CLICKHOUSE_CLIENT} --query="select * from orc_load" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nullable_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" +cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nullable_arrays format ORC" +timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_nullable_arrays format ORC" < $DATA_FILE +${CLICKHOUSE_CLIENT} --query="select * from orc_nullable_arrays" -${CLICKHOUSE_CLIENT} --query="drop table orc_load" +${CLICKHOUSE_CLIENT} --query="drop table orc_nullable_arrays" diff --git a/tests/queries/0_stateless/data_orc/tmp b/tests/queries/0_stateless/data_orc/tmp deleted file mode 100644 index 8a1ad838b502edc42110bf2bbb547c1425a74db6..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 714 zcma)(u};G<5Qfh_+d4TdL{R~gsj3j_RGH9;scHuppsfloz<>|~AOR0hv9d5Tb>Kmm zcnx?6RvL`shE|m-2H)@R_xTdv4Msg3X#{{o5Dr0CwRBM8#tbXO6Gc20MlEM0TKP+m z5V0<&&E=GxG9@LR)wHlJYZKYdTJ0FxDJ{r=(Z>gIBG z4zJy*5L%In!Bo96v+k9fEw30@n3`u9SeaeN;O=p+%qx$X`K<>r_spE4XAI{U&PCut znd?a8^JMU4$Wx(?@^xhF)b-eX_dhYk!;>?8tc#%(ad8BN_t)=&S2UL}e!;Zjndv0ZJ?bo3R78()hgz*4GIB$#6rg>YH_L|!X%Pk)G-wJWz0l_OD f5O~D{I?w$hM(@l`ZafwZ4pFmj)#Df9VAL}|T Date: Wed, 12 May 2021 22:52:52 +0300 Subject: [PATCH 36/82] Update deltaSum and deltaSumTimestamp functions MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Внес небольшие правки. --- .../sql-reference/aggregate-functions/reference/deltasum.md | 3 ++- .../aggregate-functions/reference/deltasumtimestamp.md | 6 +++--- .../sql-reference/aggregate-functions/reference/deltasum.md | 3 ++- .../aggregate-functions/reference/deltasumtimestamp.md | 6 +++--- 4 files changed, 10 insertions(+), 8 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md index 59ecb968207..f42007fc857 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md @@ -8,7 +8,8 @@ Sums the arithmetic difference between consecutive rows. If the difference is ne Note that the underlying data must be sorted for this function to work properly. -If you would like to use this function in a [materialized view](../../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](deltasumtimestamp.md) method instead. +!!! info "Note" + If you would like to use this function in a [materialized view](../../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp) method instead. **Syntax** diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index d266ccc7c7a..241010c4761 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -2,13 +2,13 @@ toc_priority: 141 --- -# deltaSumTimestamp {#agg_functions-deltasum} +# deltaSumTimestamp {#agg_functions-deltasumtimestamp} Adds the difference between consecutive rows. If the difference is negative, it is ignored. This function is primarily for [materialized views](../../../sql-reference/statements/create/view.md#materialized) that are ordered by some time bucket-aligned timestamp, for example, a `toStartOfMinute` bucket. Because the rows in such a materialized view will all have the same timestamp, it is impossible for them to be merged in the "right" order. This function keeps track of the `timestamp` of the values it's seen, so it's possible to order the states correctly during merging. -To calculate the delta sum across an ordered collection you can simply use the [deltaSum](./deltasum.md) function. +To calculate the delta sum across an ordered collection you can simply use the [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) function. **Syntax** @@ -33,7 +33,7 @@ Query: ```sql SELECT deltaSumTimestamp(value, timestamp) -FROM (select number as timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] as value from numbers(1, 10)); +FROM (SELECT number AS timestamp, [0, 4, 8, 3, 0, 0, 0, 1, 3, 5][number] AS value FROM numbers(1, 10)); ``` Result: diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index e63f6409c83..71fb1926488 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -8,7 +8,8 @@ toc_priority: 141 Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. -В [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized), вместо этой функции рекомендуется использовать [deltaSumTimestamp](deltasumtimestamp.md). +!!! info "Примечание" + В [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized) вместо этой функции рекомендуется использовать [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp). **Синтаксис** diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 9f8e09b07ec..08fa2bf2bb0 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -2,13 +2,13 @@ toc_priority: 141 --- -# deltaSumTimestamp {#agg_functions-deltasum} +# deltaSumTimestamp {#agg_functions-deltasumtimestamp} Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например, по бакету `toStartOfMinute`. Строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. -Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](./deltasum.md) вместо функции `deltaSumTimestamp`. +Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вместо функции `deltaSumTimestamp`. **Синтаксис** @@ -18,7 +18,7 @@ deltaSumTimestamp(value, timestamp) **Аргументы** -- `value` — входные значения, должны быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). +- `value` — входные значения, должны быть типа [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). - `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). **Возвращаемое значение** From 3bdc52eae0985590e08bcc73529758d2e59604fb Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 22:55:25 +0300 Subject: [PATCH 37/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 08fa2bf2bb0..76178546cb4 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -6,7 +6,7 @@ toc_priority: 141 Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. -Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например, по бакету `toStartOfMinute`. Строки в таком материализованном представлении будут иметь одинаковый timestamp. Невозможно, чтобы они были объединены в "правом" порядке. Эта функция отслеживает `timestamp` значений, которые она видит. Поэтому можно правильно упорядочить состояния во время слияния. +Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, невозможно объединить их в "правом" порядке. Функция отслеживает `timestamp` наблюдаемых значений, поэтому возможно правильно упорядочить состояния во время слияния. Чтобы вычислить разницу между упорядоченными последовательными строками, вы можете использовать функцию [deltaSum](../../../sql-reference/aggregate-functions/reference/deltasum.md#agg_functions-deltasum) вместо функции `deltaSumTimestamp`. From 6747151c6d0f946f57656d19c9507ac33080600f Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 22:55:45 +0300 Subject: [PATCH 38/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 76178546cb4..e41a8f76a09 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -19,7 +19,7 @@ deltaSumTimestamp(value, timestamp) **Аргументы** - `value` — входные значения, должны быть типа [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). -- `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). +- `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). **Возвращаемое значение** From 42152a4cc09823508393b7f7b6b0986fc5af75d2 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Wed, 12 May 2021 22:55:57 +0300 Subject: [PATCH 39/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index e41a8f76a09..6f90fd371bc 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -25,7 +25,7 @@ deltaSumTimestamp(value, timestamp) - Накопленная разница между последовательными значениями, упорядоченными по параметру `timestamp`. -Тип: [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). +Тип: [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). **Пример** From 107054c9cd58fafad06ae07cfe2fe81c2365bf57 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Wed, 12 May 2021 22:58:04 +0300 Subject: [PATCH 40/82] Update deltasumtimestamp.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Поставил недостающие запятые. --- .../aggregate-functions/reference/deltasumtimestamp.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index 6f90fd371bc..b01f290591c 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -18,8 +18,8 @@ deltaSumTimestamp(value, timestamp) **Аргументы** -- `value` — входные значения, должны быть типа [Integer](../../data-types/int-uint.md) или [Float](../../data-types/float.md) или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). -- `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md) или [DateTime](../../data-types/datetime.md). +- `value` — входные значения, должны быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). +- `timestamp` — параметр для упорядочивания значений, должен быть типа [Integer](../../data-types/int-uint.md), или [Float](../../data-types/float.md), или [Date](../../data-types/date.md), или [DateTime](../../data-types/datetime.md). **Возвращаемое значение** From cef7c8700bd9cbea0aecdb9097d1377556f124c5 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 13 May 2021 00:48:06 +0300 Subject: [PATCH 41/82] Try to fix tests --- .../00900_orc_arrays_load.reference | 2 -- .../0_stateless/00900_orc_arrays_load.sh | 3 +-- .../00900_orc_nested_arrays_load.reference | 1 - .../00900_orc_nested_arrays_load.sh | 3 +-- .../00900_orc_nullable_arrays_load.reference | 3 --- .../00900_orc_nullable_arrays_load.sh | 3 +-- ...cimal.sh => 00900_parquet_decimal_long.sh} | 0 ...uet_load.sh => 00900_parquet_load_long.sh} | 0 ...00900_parquet.sh => 00900_parquet_long.sh} | 0 .../0_stateless/data_orc/array_test.orc | Bin 3870 -> 3905 bytes .../data_orc/nested_array_test.orc | Bin 1344 -> 1352 bytes .../data_orc/nullable_array_test.orc | Bin 714 -> 720 bytes 12 files changed, 3 insertions(+), 12 deletions(-) rename tests/queries/0_stateless/{00900_parquet_decimal.sh => 00900_parquet_decimal_long.sh} (100%) rename tests/queries/0_stateless/{00900_parquet_load.sh => 00900_parquet_load_long.sh} (100%) rename tests/queries/0_stateless/{00900_parquet.sh => 00900_parquet_long.sh} (100%) diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.reference b/tests/queries/0_stateless/00900_orc_arrays_load.reference index 9b20ef98164..de31dfebd77 100644 --- a/tests/queries/0_stateless/00900_orc_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_arrays_load.reference @@ -1,4 +1,2 @@ [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] -[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] -[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.sh b/tests/queries/0_stateless/00900_orc_arrays_load.sh index 0b3dfd62526..8aa8e05bbc2 100755 --- a/tests/queries/0_stateless/00900_orc_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_arrays_load.sh @@ -7,9 +7,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=$CUR_DIR/data_orc/array_test.orc ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(Datetime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) ENGINE=Memory()" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime), arr15 Array(Decimal(4, 2)), arr16 Array(Decimal(10, 2)), arr17 Array(Decimal(25, 2))) ENGINE=Memory()" cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_arrays format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_arrays format ORC" < $DATA_FILE ${CLICKHOUSE_CLIENT} --query="select * from orc_arrays" ${CLICKHOUSE_CLIENT} --query="drop table orc_arrays" diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference index dd9c9900684..aef31fc1b60 100644 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference @@ -1,2 +1 @@ [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] -[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh index 9c40efd6676..685e380d3f8 100755 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh @@ -7,9 +7,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=$CUR_DIR/data_orc/nested_array_test.orc ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nested_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory()" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (arr1 Array(Array(Array(UInt32))), arr2 Array(Array(Array(String))), arr3 Array(Array(Nullable(UInt32))), arr4 Array(Array(Nullable(String)))) engine=Memory()" cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nested_arrays format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_nested_arrays format ORC" < $DATA_FILE ${CLICKHOUSE_CLIENT} --query="select * from orc_nested_arrays" ${CLICKHOUSE_CLIENT} --query="drop table orc_nested_arrays" diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference index 62e95652040..19379b16138 100644 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference @@ -1,6 +1,3 @@ [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] [NULL] [NULL] [NULL] [] [] [] -[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] -[NULL] [NULL] [NULL] -[] [] [] diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh index b3de283033a..40312c2e2cf 100755 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh @@ -7,9 +7,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) DATA_FILE=$CUR_DIR/data_orc/nullable_array_test.orc ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nullable_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (arr1 Array(Nullable(UInt32)), arr2 Array(Nullable(String)), arr3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nullable_arrays format ORC" -timeout 3 ${CLICKHOUSE_CLIENT} -q "insert into orc_nullable_arrays format ORC" < $DATA_FILE ${CLICKHOUSE_CLIENT} --query="select * from orc_nullable_arrays" ${CLICKHOUSE_CLIENT} --query="drop table orc_nullable_arrays" diff --git a/tests/queries/0_stateless/00900_parquet_decimal.sh b/tests/queries/0_stateless/00900_parquet_decimal_long.sh similarity index 100% rename from tests/queries/0_stateless/00900_parquet_decimal.sh rename to tests/queries/0_stateless/00900_parquet_decimal_long.sh diff --git a/tests/queries/0_stateless/00900_parquet_load.sh b/tests/queries/0_stateless/00900_parquet_load_long.sh similarity index 100% rename from tests/queries/0_stateless/00900_parquet_load.sh rename to tests/queries/0_stateless/00900_parquet_load_long.sh diff --git a/tests/queries/0_stateless/00900_parquet.sh b/tests/queries/0_stateless/00900_parquet_long.sh similarity index 100% rename from tests/queries/0_stateless/00900_parquet.sh rename to tests/queries/0_stateless/00900_parquet_long.sh diff --git a/tests/queries/0_stateless/data_orc/array_test.orc b/tests/queries/0_stateless/data_orc/array_test.orc index 8de2ebd7054c8586c60f918bb268525bf6594664..1a468e3489cf0b21bff0bf38e84d7e5d05108573 100644 GIT binary patch delta 386 zcmbOycTjGF8*{{gf6NU3AN*zf$NV40WP&qKF!nRwV1Y_7q?}-cscE=%g!BLAPT_{W zb27+c4U@hwHq3s<^#8!*dCby`2PW@gmI0F2nYFYarZX_G7yyAG5EuagRIkW7O{ZCs zSDg15+_VE~)n<_sf;wi_YQ}FH7%yxC+B`X$MN$bOtZ{*V0UOXLsP+YHQ#z}F2rROB z8cP{leJ3LakB}fEGb=kMH!r`Su&B7Cw5+_M6iZ@Jks+8g0+YsI(gaMJf=M$lX$~eW zz@()VD@d0CgffIsMi9yvLYY7)QwRk#0;FSe4);uE$qgI=3=$0uEJ{oa+#0Pc25e1V Kn3?^9oFxIV*^t}- delta 333 zcmX>oH&1SZ8*@a%KW2vi5B@U#WBw0gGQpWA82g!Tus|gkQcf_!)HK{W!ufx5r*K2x zIT>WJhDl!-8)m;_`rj~l9J?e1=`>67 zit|2$n|460+AMNHP{+(#&G>Bt9S< QN(|VVzA!WU2RTas0CK|-UjP6A delta 43 ucmX@Xb%1NbQ)US{4jv&EMpjN9DW*h2AT|PGV<6tl%A(E4czv=Vt2hAAZV1`{ diff --git a/tests/queries/0_stateless/data_orc/nullable_array_test.orc b/tests/queries/0_stateless/data_orc/nullable_array_test.orc index 8a1ad838b502edc42110bf2bbb547c1425a74db6..2f37530f63a8303176f7fb5e93a111c281c1c8f1 100644 GIT binary patch delta 192 zcmX@bdVzI{xA5)!~$hH{O1nN&rMY*E-A{)OaK31 z<0dvn2}Uj!4(0#`E)EVxCngTD02qJrL?$UcMh<3ySCqt-$74!Off8#iS_D=rF<0gOTG5qX2_MLj#Kv69cz~ Pp8;Fb7iMPvAZG~xl{GH4 delta 179 zcmcb>dWv;|DI>!~GbJDqX`;&T|H1zQf0-E>{xA5)!~$hH{O1nN&rMY*E-A{)OaFgh z<0dvnK?W`s4(0#`E)EVxCngTD0EWpY85K1cIG6=yF-!3PscuFI1BLsH8q7+P96UnI zjLfW3Oo@g-Yy`xUC799$8LYrYSRss Date: Thu, 13 May 2021 06:15:44 +0300 Subject: [PATCH 42/82] try even smaller value --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index f25393c9502..c49e3884c31 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -104,7 +104,7 @@ class IColumn; M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ - M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ + M(UInt64, group_by_two_level_threshold_bytes, 10000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ \ From 42d7621aeeb0f5cae0e58bfed0e8165afbb9a005 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 13 May 2021 10:20:24 +0300 Subject: [PATCH 43/82] Change tests --- ...et.reference => 00900_long_parquet.reference} | 0 ...900_parquet_long.sh => 00900_long_parquet.sh} | 0 ...ence => 00900_long_parquet_decimal.reference} | 0 ...mal_long.sh => 00900_long_parquet_decimal.sh} | 0 ...ference => 00900_long_parquet_load.reference} | 0 ...t_load_long.sh => 00900_long_parquet_load.sh} | 0 .../0_stateless/00900_orc_arrays_load.reference | 2 ++ .../queries/0_stateless/00900_orc_arrays_load.sh | 13 ++++++++----- .../00900_orc_nested_arrays_load.reference | 1 + .../0_stateless/00900_orc_nested_arrays_load.sh | 13 ++++++++----- .../00900_orc_nullable_arrays_load.reference | 3 +++ .../00900_orc_nullable_arrays_load.sh | 11 ++++++----- .../queries/0_stateless/data_orc/array_test.orc | Bin 3905 -> 0 bytes .../0_stateless/data_orc/nested_array_test.orc | Bin 1352 -> 0 bytes .../0_stateless/data_orc/nullable_array_test.orc | Bin 720 -> 0 bytes 15 files changed, 28 insertions(+), 15 deletions(-) rename tests/queries/0_stateless/{00900_parquet.reference => 00900_long_parquet.reference} (100%) rename tests/queries/0_stateless/{00900_parquet_long.sh => 00900_long_parquet.sh} (100%) rename tests/queries/0_stateless/{00900_parquet_decimal.reference => 00900_long_parquet_decimal.reference} (100%) rename tests/queries/0_stateless/{00900_parquet_decimal_long.sh => 00900_long_parquet_decimal.sh} (100%) rename tests/queries/0_stateless/{00900_parquet_load.reference => 00900_long_parquet_load.reference} (100%) rename tests/queries/0_stateless/{00900_parquet_load_long.sh => 00900_long_parquet_load.sh} (100%) delete mode 100644 tests/queries/0_stateless/data_orc/array_test.orc delete mode 100644 tests/queries/0_stateless/data_orc/nested_array_test.orc delete mode 100644 tests/queries/0_stateless/data_orc/nullable_array_test.orc diff --git a/tests/queries/0_stateless/00900_parquet.reference b/tests/queries/0_stateless/00900_long_parquet.reference similarity index 100% rename from tests/queries/0_stateless/00900_parquet.reference rename to tests/queries/0_stateless/00900_long_parquet.reference diff --git a/tests/queries/0_stateless/00900_parquet_long.sh b/tests/queries/0_stateless/00900_long_parquet.sh similarity index 100% rename from tests/queries/0_stateless/00900_parquet_long.sh rename to tests/queries/0_stateless/00900_long_parquet.sh diff --git a/tests/queries/0_stateless/00900_parquet_decimal.reference b/tests/queries/0_stateless/00900_long_parquet_decimal.reference similarity index 100% rename from tests/queries/0_stateless/00900_parquet_decimal.reference rename to tests/queries/0_stateless/00900_long_parquet_decimal.reference diff --git a/tests/queries/0_stateless/00900_parquet_decimal_long.sh b/tests/queries/0_stateless/00900_long_parquet_decimal.sh similarity index 100% rename from tests/queries/0_stateless/00900_parquet_decimal_long.sh rename to tests/queries/0_stateless/00900_long_parquet_decimal.sh diff --git a/tests/queries/0_stateless/00900_parquet_load.reference b/tests/queries/0_stateless/00900_long_parquet_load.reference similarity index 100% rename from tests/queries/0_stateless/00900_parquet_load.reference rename to tests/queries/0_stateless/00900_long_parquet_load.reference diff --git a/tests/queries/0_stateless/00900_parquet_load_long.sh b/tests/queries/0_stateless/00900_long_parquet_load.sh similarity index 100% rename from tests/queries/0_stateless/00900_parquet_load_long.sh rename to tests/queries/0_stateless/00900_long_parquet_load.sh diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.reference b/tests/queries/0_stateless/00900_orc_arrays_load.reference index de31dfebd77..9b20ef98164 100644 --- a/tests/queries/0_stateless/00900_orc_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_arrays_load.reference @@ -1,2 +1,4 @@ [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.sh b/tests/queries/0_stateless/00900_orc_arrays_load.sh index 8aa8e05bbc2..270bc0037e9 100755 --- a/tests/queries/0_stateless/00900_orc_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_arrays_load.sh @@ -4,11 +4,14 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -DATA_FILE=$CUR_DIR/data_orc/array_test.orc - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime), arr15 Array(Decimal(4, 2)), arr16 Array(Decimal(10, 2)), arr17 Array(Decimal(25, 2))) ENGINE=Memory()" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_arrays format ORC" -${CLICKHOUSE_CLIENT} --query="select * from orc_arrays" -${CLICKHOUSE_CLIENT} --query="drop table orc_arrays" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_arrays VALUES ([1,-2,3],[1,2,3],[100,-200,300],[100,200,300],[10000000,-20000000,30000000],[10000000,2000000,3000000],[100000000000000,-200000000000,3000000000000],[100000000000000,20000000000000,3000000000000],['Some string','Some string','Some string'],['0000','1111','2222'],[42.42,424.2,0.4242],[424242.424242,4242042420.242424,42],['2000-01-01','2001-01-01','2002-01-01'],['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'],[0.20,10.00,4.00],[4.00,10000.10,10000.10],[1000000000.00,90.00,101001.01]),([],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/arrays.orc + +cat "${CLICKHOUSE_TMP}"/arrays.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_arrays FORMAT ORC" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_arrays" +${CLICKHOUSE_CLIENT} --query="DROP TABLE orc_arrays" diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference index aef31fc1b60..dd9c9900684 100644 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference @@ -1 +1,2 @@ [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh index 685e380d3f8..81d2aeed4bd 100755 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh @@ -4,11 +4,14 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -DATA_FILE=$CUR_DIR/data_orc/nested_array_test.orc - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nested_arrays" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (arr1 Array(Array(Array(UInt32))), arr2 Array(Array(Array(String))), arr3 Array(Array(Nullable(UInt32))), arr4 Array(Array(Nullable(String)))) engine=Memory()" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nested_arrays format ORC" -${CLICKHOUSE_CLIENT} --query="select * from orc_nested_arrays" -${CLICKHOUSE_CLIENT} --query="drop table orc_nested_arrays" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nested_arrays VALUES ([[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]],[[['Some string','Some string'],[]],[['Some string']],[[]]],[[NULL,1,2],[NULL],[1,2],[]],[['Some string',NULL,'Some string'],[NULL],[]])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nested_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/nested_arrays.orc + +cat "${CLICKHOUSE_TMP}"/nested_arrays.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_nested_arrays FORMAT ORC" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nested_arrays" +${CLICKHOUSE_CLIENT} --query="DROP table orc_nested_arrays" diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference index 19379b16138..62e95652040 100644 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference @@ -1,3 +1,6 @@ [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] [NULL] [NULL] [NULL] [] [] [] +[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh index 40312c2e2cf..f1217df735a 100755 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh @@ -4,11 +4,12 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -DATA_FILE=$CUR_DIR/data_orc/nullable_array_test.orc - ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nullable_arrays" ${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (arr1 Array(Nullable(UInt32)), arr2 Array(Nullable(String)), arr3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" -cat "$DATA_FILE" | ${CLICKHOUSE_CLIENT} -q "insert into orc_nullable_arrays format ORC" -${CLICKHOUSE_CLIENT} --query="select * from orc_nullable_arrays" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nullable_arrays VALUES ([1,NULL,2],[NULL,'Some string',NULL],[0.00,NULL,42.42]),([NULL],[NULL],[NULL]),([],[],[])" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nullable_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/nullable_arrays.orc -${CLICKHOUSE_CLIENT} --query="drop table orc_nullable_arrays" +cat "${CLICKHOUSE_TMP}"/nullable_arrays.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_nullable_arrays FORMAT ORC" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nullable_arrays" +${CLICKHOUSE_CLIENT} --query="DROP TABLE orc_nullable_arrays" diff --git a/tests/queries/0_stateless/data_orc/array_test.orc b/tests/queries/0_stateless/data_orc/array_test.orc deleted file mode 100644 index 1a468e3489cf0b21bff0bf38e84d7e5d05108573..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 3905 zcmds3+iM(E7@wIlvwL>-HapqPo}FwqyNlM;YM4#a#`HncgrW%AQtKZeC?a@)>Qg}( zA`&9CE$UON5CkbGgrHR)l$zqLzSIze)@ww4P=q!}t@vQ$nQwkG*;~5ptHW}>^SgX! zcFr8W`S$LgHj^e(6vfuLwoftBrXfKD3n5H#X2&?QVnXlf(xfZZ=@Q{uS-Dn%A7$1B zT^wUQjb~NgxR~fPR>Bnva>XoPe|N6W@{XJ<3-8oiUr)RD@@3WXzWp>IyxEhPPWzT{ z^+CCMk*|Mz@yJTj^1PMVpFg@?c-{}^S64?m({2gVDl$!QZ|R>Ozgmzt3iFo!I`!G# zMOiDn=RP}m z9hov`&tz`x%yRCPT!Cj_y3!`>^x`p zW`s9${*9N$tKQ0MfBdqm(|=R6BMxVyc~f^sMqS%zxFfFX7>zNBI+JdVlBzPvG4(O& zwt2&q*GR*4_^6z8Pu&rvZj-4uo5qBHCp%5Kem>opV8d;=6O9qKu}@iFT3@=#l=Ywf zRj;yWszuYY>QQzy7J*CZ1wcXH`(gi>$85Lfik?DotFnb+?e^;t8<6s zZ-epXKbZa9SU#-2cx720WZjJMn5Bp`MO9=eYNAMy0qej9a7;L)$ALL;0@xH)(v!d` z;52YX43nM(w!~J#Ibd7l3HJf#feXO>VjJm2U;$hLc0`Hv0pPNz5gr7th%JPNfUDv* z!ZqN!7?)U)`=U`)DQcD~7-5aDYH7mXKgD9u8FU@G4&8umK#xI>i8$rQp~sV{PY zKM3C-a0R}KFvwQ{KLpAA)mj9ujXt6o`%OcDucJq3 zGs^P4NlOqI+Z%v@?f$V3v}=9Y14r-Ac;LH{Z?Hdf|FeA5!=@G)iqM5ElwEqOUs|6cdZK2JOuD8%7S97&p zbha?Il!_J(r2<@gsc4}$N(JcdrJ_Y_FBK54T`FMeQ7YhyN<|Bnr2@Ejsc1KDS}I^{ zD-~e7mI`RsC>3z@EEVwGNU4}ScKA0nX$e5r%V3<2)%cW diff --git a/tests/queries/0_stateless/data_orc/nested_array_test.orc b/tests/queries/0_stateless/data_orc/nested_array_test.orc deleted file mode 100644 index c0cde3db0572953d4964470233e8e775e544c0ac..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1352 zcmcIjy>1gh5T5pZ42te!jUI&m0{exZEiKXtWh8aHY$T+(gyTn<2f`2N$+z zjFq+Vi(%B3XVC>a%__IJphF*l4OM(cOUKW6S2bIvQ6my8v~1J;$82elxSj8IGF z(rfZ7=E@u5+ABm+j!l1(fN?hM^l)?0%o~& zjgTo(DbY;u82UnhevL!AnJl5 Date: Thu, 13 May 2021 12:06:59 +0300 Subject: [PATCH 44/82] fix the tests --- tests/queries/0_stateless/00106_totals_after_having.sql | 7 +++++++ tests/queries/0_stateless/00107_totals_after_having.sql | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/tests/queries/0_stateless/00106_totals_after_having.sql b/tests/queries/0_stateless/00106_totals_after_having.sql index 5ec2bbb9d02..b9e5b793383 100644 --- a/tests/queries/0_stateless/00106_totals_after_having.sql +++ b/tests/queries/0_stateless/00106_totals_after_having.sql @@ -1,6 +1,13 @@ SET max_rows_to_group_by = 100000; SET group_by_overflow_mode = 'any'; +-- 'any' overflow mode might select different values for two-level and +-- single-level GROUP BY, so we set a big enough threshold here to ensure that +-- the switch doesn't happen, we only use single-level GROUP BY and get a +-- predictable result. +SET group_by_two_level_threshold_bytes = 100000000; +SET group_by_two_level_threshold = 1000000; + SET totals_mode = 'after_having_auto'; SELECT dummy, count() GROUP BY dummy WITH TOTALS; diff --git a/tests/queries/0_stateless/00107_totals_after_having.sql b/tests/queries/0_stateless/00107_totals_after_having.sql index 2f63ef4123c..40a598a194d 100644 --- a/tests/queries/0_stateless/00107_totals_after_having.sql +++ b/tests/queries/0_stateless/00107_totals_after_having.sql @@ -4,6 +4,13 @@ SET max_rows_to_group_by = 100000; SET max_block_size = 100001; SET group_by_overflow_mode = 'any'; +-- 'any' overflow mode might select different values for two-level and +-- single-level GROUP BY, so we set a big enough threshold here to ensure that +-- the switch doesn't happen, we only use single-level GROUP BY and get a +-- predictable result. +SET group_by_two_level_threshold_bytes = 100000000; +SET group_by_two_level_threshold = 1000000; + SELECT '**** totals_mode = after_having_auto'; SET totals_mode = 'after_having_auto'; SELECT intDiv(number, 2) AS k, count(), argMax(toString(number), number) FROM (SELECT number FROM system.numbers LIMIT 500000) GROUP BY k WITH TOTALS ORDER BY k LIMIT 10; From ec214bb04c9dbc58b6bd4eb181f197e61a1afd7e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 13 May 2021 12:23:59 +0300 Subject: [PATCH 45/82] Try fix tests again --- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 1 + .../0_stateless/00900_orc_arrays_load.reference | 8 ++++---- tests/queries/0_stateless/00900_orc_arrays_load.sh | 4 ++-- .../00900_orc_nested_arrays_load.reference | 4 ++-- .../0_stateless/00900_orc_nested_arrays_load.sh | 4 ++-- .../00900_orc_nullable_arrays_load.reference | 12 ++++++------ .../0_stateless/00900_orc_nullable_arrays_load.sh | 4 ++-- 7 files changed, 19 insertions(+), 18 deletions(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 3134fe7d919..a27c9dd016b 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -395,6 +395,7 @@ namespace DB NameToColumnPtr name_to_column_ptr; for (const auto& column_name : table->ColumnNames()) { + std::cout << column_name << "\n"; std::shared_ptr arrow_column = table->GetColumnByName(column_name); name_to_column_ptr[column_name] = arrow_column; } diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.reference b/tests/queries/0_stateless/00900_orc_arrays_load.reference index 9b20ef98164..35df4fa923c 100644 --- a/tests/queries/0_stateless/00900_orc_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_arrays_load.reference @@ -1,4 +1,4 @@ -[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] -[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] -[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] -[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.sh b/tests/queries/0_stateless/00900_orc_arrays_load.sh index 270bc0037e9..b1b0e897156 100755 --- a/tests/queries/0_stateless/00900_orc_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_arrays_load.sh @@ -5,9 +5,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime), arr15 Array(Decimal(4, 2)), arr16 Array(Decimal(10, 2)), arr17 Array(Decimal(25, 2))) ENGINE=Memory()" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (id UInt32, arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime), arr15 Array(Decimal(4, 2)), arr16 Array(Decimal(10, 2)), arr17 Array(Decimal(25, 2))) ENGINE=MergeTree() order by id" -${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_arrays VALUES ([1,-2,3],[1,2,3],[100,-200,300],[100,200,300],[10000000,-20000000,30000000],[10000000,2000000,3000000],[100000000000000,-200000000000,3000000000000],[100000000000000,20000000000000,3000000000000],['Some string','Some string','Some string'],['0000','1111','2222'],[42.42,424.2,0.4242],[424242.424242,4242042420.242424,42],['2000-01-01','2001-01-01','2002-01-01'],['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'],[0.20,10.00,4.00],[4.00,10000.10,10000.10],[1000000000.00,90.00,101001.01]),([],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[])" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_arrays VALUES (1, [1,-2,3],[1,2,3],[100,-200,300],[100,200,300],[10000000,-20000000,30000000],[10000000,2000000,3000000],[100000000000000,-200000000000,3000000000000],[100000000000000,20000000000000,3000000000000],['Some string','Some string','Some string'],['0000','1111','2222'],[42.42,424.2,0.4242],[424242.424242,4242042420.242424,42],['2000-01-01','2001-01-01','2002-01-01'],['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'],[0.20,10.00,4.00],[4.00,10000.10,10000.10],[1000000000.00,90.00,101001.01]),(2, [],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[])" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/arrays.orc diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference index dd9c9900684..78cf96eb37c 100644 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference @@ -1,2 +1,2 @@ -[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] -[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +1 [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +1 [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh index 81d2aeed4bd..8875a4686dd 100755 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh @@ -5,9 +5,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nested_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (arr1 Array(Array(Array(UInt32))), arr2 Array(Array(Array(String))), arr3 Array(Array(Nullable(UInt32))), arr4 Array(Array(Nullable(String)))) engine=Memory()" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (id UInt32, arr1 Array(Array(Array(UInt32))), arr2 Array(Array(Array(String))), arr3 Array(Array(Nullable(UInt32))), arr4 Array(Array(Nullable(String)))) engine=MergeTree() ORDER BY id" -${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nested_arrays VALUES ([[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]],[[['Some string','Some string'],[]],[['Some string']],[[]]],[[NULL,1,2],[NULL],[1,2],[]],[['Some string',NULL,'Some string'],[NULL],[]])" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nested_arrays VALUES (1, [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]],[[['Some string','Some string'],[]],[['Some string']],[[]]],[[NULL,1,2],[NULL],[1,2],[]],[['Some string',NULL,'Some string'],[NULL],[]])" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nested_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/nested_arrays.orc diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference index 62e95652040..d548488a196 100644 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference @@ -1,6 +1,6 @@ -[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] -[NULL] [NULL] [NULL] -[] [] [] -[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] -[NULL] [NULL] [NULL] -[] [] [] +1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +2 [NULL] [NULL] [NULL] +3 [] [] [] +1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +2 [NULL] [NULL] [NULL] +3 [] [] [] diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh index f1217df735a..59fb7874515 100755 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh @@ -5,8 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nullable_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (arr1 Array(Nullable(UInt32)), arr2 Array(Nullable(String)), arr3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" -${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nullable_arrays VALUES ([1,NULL,2],[NULL,'Some string',NULL],[0.00,NULL,42.42]),([NULL],[NULL],[NULL]),([],[],[])" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (id UInt32, arr1 Array(Nullable(UInt32)), arr2 Array(Nullable(String)), arr3 Array(Nullable(Decimal(4, 2)))) ENGINE=MergeTree() order by id" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nullable_arrays VALUES (1, [1,NULL,2],[NULL,'Some string',NULL],[0.00,NULL,42.42]), (2, [NULL],[NULL],[NULL]), (3, [],[],[])" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nullable_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/nullable_arrays.orc cat "${CLICKHOUSE_TMP}"/nullable_arrays.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_nullable_arrays FORMAT ORC" From ff6e4bf91a3e40f80b9d4cb52e1833a7a3c2bc9d Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 13 May 2021 15:21:39 +0300 Subject: [PATCH 46/82] Remove debug output --- src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index a27c9dd016b..3134fe7d919 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -395,7 +395,6 @@ namespace DB NameToColumnPtr name_to_column_ptr; for (const auto& column_name : table->ColumnNames()) { - std::cout << column_name << "\n"; std::shared_ptr arrow_column = table->GetColumnByName(column_name); name_to_column_ptr[column_name] = arrow_column; } From 7b26483080337a4fc22f2687c81f48d5b40e50f6 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 13 May 2021 16:37:44 +0300 Subject: [PATCH 47/82] Rename param --- base/loggers/Loggers.cpp | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/base/loggers/Loggers.cpp b/base/loggers/Loggers.cpp index d9e87e0e2c3..913deaf1eb8 100644 --- a/base/loggers/Loggers.cpp +++ b/base/loggers/Loggers.cpp @@ -165,17 +165,17 @@ void Loggers::buildLoggers(Poco::Util::AbstractConfiguration & config, Poco::Log { bool color_enabled = config.getBool("logger.color_terminal", color_logs_by_default); - auto console_loglevel_string = config.getString("logger.console_loglevel", log_level_string); - auto console_loglevel = Poco::Logger::parseLevel(console_loglevel_string); - if (console_loglevel > max_log_level) + auto console_log_level_string = config.getString("logger.console_log_level", log_level_string); + auto console_log_level = Poco::Logger::parseLevel(console_log_level_string); + if (console_log_level > max_log_level) { - max_log_level = console_loglevel; + max_log_level = console_log_level; } Poco::AutoPtr pf = new OwnPatternFormatter(color_enabled); Poco::AutoPtr log = new DB::OwnFormattingChannel(pf, new Poco::ConsoleChannel); - logger.warning("Logging " + console_loglevel_string + " to console"); - log->setLevel(console_loglevel); + logger.warning("Logging " + console_log_level_string + " to console"); + log->setLevel(console_log_level); split->addChannel(log); } From d78be51fb49420dbefbec9b66254340520a0f275 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 May 2021 16:38:18 +0300 Subject: [PATCH 48/82] Calculate header for ActionsDAG --- src/Interpreters/ActionsDAG.cpp | 159 ++++++++++++++++++ src/Interpreters/ActionsDAG.h | 8 + src/Processors/QueryPlan/ExpressionStep.cpp | 7 +- src/Processors/QueryPlan/ExpressionStep.h | 1 - src/Processors/QueryPlan/FilterStep.cpp | 4 +- src/Processors/QueryPlan/TotalsHavingStep.cpp | 2 +- .../Transforms/ExpressionTransform.cpp | 8 +- .../Transforms/ExpressionTransform.h | 4 +- src/Processors/Transforms/FilterTransform.cpp | 10 +- src/Processors/Transforms/FilterTransform.h | 4 +- .../Transforms/TotalsHavingTransform.cpp | 8 +- .../Transforms/TotalsHavingTransform.h | 4 +- 12 files changed, 194 insertions(+), 25 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 814b085e367..0065aea0fc3 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -26,6 +26,7 @@ namespace ErrorCodes extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; extern const int THERE_IS_NO_COLUMN; extern const int ILLEGAL_COLUMN; + extern const int NOT_FOUND_COLUMN_IN_BLOCK; } const char * ActionsDAG::typeToString(ActionsDAG::ActionType type) @@ -439,6 +440,164 @@ void ActionsDAG::removeUnusedActions(bool allow_remove_inputs) inputs.erase(it, inputs.end()); } +static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * node, ColumnsWithTypeAndName arguments) +{ + ColumnWithTypeAndName res_column; + res_column.type = node->result_type; + res_column.name = node->result_name; + + switch (node->type) + { + case ActionsDAG::ActionType::FUNCTION: + { + bool all_args_are_const = true; + + for (size_t i = 0; i < arguments.size(); ++i) + if (typeid_cast(arguments[i].column.get()) == nullptr) + all_args_are_const = false; + + res_column.column = node->function->execute(arguments, res_column.type, 0, true); + + if (!all_args_are_const) + res_column.column = res_column.column->convertToFullColumnIfConst(); + + break; + } + + case ActionsDAG::ActionType::ARRAY_JOIN: + { + auto key = arguments.at(0); + key.column = key.column->convertToFullColumnIfConst(); + + const ColumnArray * array = typeid_cast(key.column.get()); + if (!array) + throw Exception(ErrorCodes::TYPE_MISMATCH, + "ARRAY JOIN of not array: {}", node->result_name); + + res_column.column = array->getDataPtr()->cloneEmpty(); + break; + } + + case ActionsDAG::ActionType::COLUMN: + { + res_column.column = node->column->cloneResized(0); + break; + } + + case ActionsDAG::ActionType::ALIAS: + { + res_column.column = arguments.at(0).column; + break; + } + + case ActionsDAG::ActionType::INPUT: + { + break; + } + } + + return res_column; +} + +Block ActionsDAG::updateHeader(Block header) const +{ + std::unordered_map result_cache; + std::vector pos_to_remove; + + { + std::unordered_map> input_positions; + + for (size_t pos = 0; pos < inputs.size(); ++pos) + input_positions[inputs[pos]->result_name].emplace_back(pos); + + pos_to_remove.reserve(inputs.size()); + + for (size_t pos = 0; pos < header.columns(); ++pos) + { + const auto & col = header.getByPosition(pos); + auto it = input_positions.find(col.name); + if (it != input_positions.end() && !it->second.empty()) + { + auto & list = it->second; + pos_to_remove.push_back(list.front()); + result_cache[inputs[list.front()]] = std::move(col); + list.pop_front(); + } + } + } + + ColumnsWithTypeAndName result_columns; + result_columns.reserve(index.size()); + + { + for (const auto * output : index) + { + if (result_cache.count(output) == 0) + { + std::stack stack; + stack.push(output); + + while (!stack.empty()) + { + const Node * node = stack.top(); + + bool all_children_calculated = true; + for (const auto * child : node->children) + { + if (result_cache.count(child) == 0) + { + stack.push(child); + all_children_calculated = false; + break; + } + } + + if (!all_children_calculated) + continue; + + stack.pop(); + + ColumnsWithTypeAndName arguments(node->children.size()); + for (size_t i = 0; i < arguments.size(); ++i) + arguments[i] = result_cache[node->children[i]]; + + if (node->type == ActionsDAG::ActionType::INPUT) + { + if (result_cache.find(node) == result_cache.end()) + throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, + "Not found column {} in block", node->result_name); + } + else + result_cache[node] = executeActionForHeader(node, std::move(arguments)); + } + } + + result_columns.push_back(result_cache[output]); + } + } + + if (isInputProjected()) + { + header.clear(); + } + else + { + std::sort(pos_to_remove.rbegin(), pos_to_remove.rend()); + for (auto pos : pos_to_remove) + header.erase(pos); + } + + Block res; + + for (auto & col : result_columns) + res.insert(std::move(col)); + + for (const auto & item : header) + res.insert(std::move(item)); + + return res; +} + NameSet ActionsDAG::foldActionsByProjection( const NameSet & required_columns, const Block & projection_block_for_keys, const String & predicate_column_name, bool add_missing_keys) { diff --git a/src/Interpreters/ActionsDAG.h b/src/Interpreters/ActionsDAG.h index 9862cb8708e..d8e2505f5b3 100644 --- a/src/Interpreters/ActionsDAG.h +++ b/src/Interpreters/ActionsDAG.h @@ -186,6 +186,14 @@ public: ActionsDAGPtr clone() const; + /// Execute actions for header. Input block must have empty columns. + /// Result should be equal to the execution of ExpressionActions build form this DAG. + /// Actions are not changed, no expressions are compiled. + /// + /// In addition, check that result constants are constants according to DAG. + /// In case if function return constant, but arguments are not constant, materialize it. + Block updateHeader(Block header) const; + /// For apply materialize() function for every output. /// Also add aliases so the result names remain unchanged. void addMaterializingOutputActions(); diff --git a/src/Processors/QueryPlan/ExpressionStep.cpp b/src/Processors/QueryPlan/ExpressionStep.cpp index ddf6ed00c3f..eb0c5abe669 100644 --- a/src/Processors/QueryPlan/ExpressionStep.cpp +++ b/src/Processors/QueryPlan/ExpressionStep.cpp @@ -31,7 +31,7 @@ static ITransformingStep::Traits getTraits(const ActionsDAGPtr & actions) ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_) : ITransformingStep( input_stream_, - Transform::transformHeader(input_stream_.header, std::make_shared(actions_dag_, ExpressionActionsSettings{})), + ExpressionTransform::transformHeader(input_stream_.header, *actions_dag_), getTraits(actions_dag_)) , actions_dag(std::move(actions_dag_)) { @@ -42,8 +42,7 @@ ExpressionStep::ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr a void ExpressionStep::updateInputStream(DataStream input_stream, bool keep_header) { Block out_header = keep_header ? std::move(output_stream->header) - : Transform::transformHeader(input_stream.header, - std::make_shared(actions_dag, ExpressionActionsSettings{})); + : ExpressionTransform::transformHeader(input_stream.header, *actions_dag); output_stream = createOutputStream( input_stream, std::move(out_header), @@ -58,7 +57,7 @@ void ExpressionStep::transformPipeline(QueryPipeline & pipeline, const BuildQuer auto expression = std::make_shared(actions_dag, settings.getActionsSettings()); pipeline.addSimpleTransform([&](const Block & header) { - return std::make_shared(header, expression); + return std::make_shared(header, expression); }); if (!blocksHaveEqualStructure(pipeline.getHeader(), output_stream->header)) diff --git a/src/Processors/QueryPlan/ExpressionStep.h b/src/Processors/QueryPlan/ExpressionStep.h index 753d446f1f3..43272e19536 100644 --- a/src/Processors/QueryPlan/ExpressionStep.h +++ b/src/Processors/QueryPlan/ExpressionStep.h @@ -14,7 +14,6 @@ class JoiningTransform; class ExpressionStep : public ITransformingStep { public: - using Transform = ExpressionTransform; explicit ExpressionStep(const DataStream & input_stream_, ActionsDAGPtr actions_dag_); String getName() const override { return "Expression"; } diff --git a/src/Processors/QueryPlan/FilterStep.cpp b/src/Processors/QueryPlan/FilterStep.cpp index 522e7dabba8..49c9326087b 100644 --- a/src/Processors/QueryPlan/FilterStep.cpp +++ b/src/Processors/QueryPlan/FilterStep.cpp @@ -34,7 +34,7 @@ FilterStep::FilterStep( input_stream_, FilterTransform::transformHeader( input_stream_.header, - std::make_shared(actions_dag_, ExpressionActionsSettings{}), + *actions_dag_, filter_column_name_, remove_filter_column_), getTraits(actions_dag_)) @@ -52,7 +52,7 @@ void FilterStep::updateInputStream(DataStream input_stream, bool keep_header) if (keep_header) out_header = FilterTransform::transformHeader( input_stream.header, - std::make_shared(actions_dag, ExpressionActionsSettings{}), + *actions_dag, filter_column_name, remove_filter_column); diff --git a/src/Processors/QueryPlan/TotalsHavingStep.cpp b/src/Processors/QueryPlan/TotalsHavingStep.cpp index 4966c04dee7..ce073db4daa 100644 --- a/src/Processors/QueryPlan/TotalsHavingStep.cpp +++ b/src/Processors/QueryPlan/TotalsHavingStep.cpp @@ -37,7 +37,7 @@ TotalsHavingStep::TotalsHavingStep( input_stream_, TotalsHavingTransform::transformHeader( input_stream_.header, - (actions_dag_ ? std::make_shared(actions_dag_, ExpressionActionsSettings{}) : nullptr), + actions_dag_.get(), final_), getTraits(!filter_column_.empty())) , overflow_row(overflow_row_) diff --git a/src/Processors/Transforms/ExpressionTransform.cpp b/src/Processors/Transforms/ExpressionTransform.cpp index e60fc65e96e..efdff086efa 100644 --- a/src/Processors/Transforms/ExpressionTransform.cpp +++ b/src/Processors/Transforms/ExpressionTransform.cpp @@ -3,16 +3,14 @@ namespace DB { -Block ExpressionTransform::transformHeader(Block header, const ExpressionActionsPtr & expression) +Block ExpressionTransform::transformHeader(Block header, const ActionsDAG & expression) { - size_t num_rows = header.rows(); - expression->execute(header, num_rows, true); - return header; + return expression.updateHeader(std::move(header)); } ExpressionTransform::ExpressionTransform(const Block & header_, ExpressionActionsPtr expression_) - : ISimpleTransform(header_, transformHeader(header_, expression_), false) + : ISimpleTransform(header_, transformHeader(header_, expression_->getActionsDAG()), false) , expression(std::move(expression_)) { } diff --git a/src/Processors/Transforms/ExpressionTransform.h b/src/Processors/Transforms/ExpressionTransform.h index 525273bad81..d4cc1c8f78a 100644 --- a/src/Processors/Transforms/ExpressionTransform.h +++ b/src/Processors/Transforms/ExpressionTransform.h @@ -7,6 +7,8 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; + /** Executes a certain expression over the block. * The expression consists of column identifiers from the block, constants, common functions. * For example: hits * 2 + 3, url LIKE '%yandex%' @@ -21,7 +23,7 @@ public: String getName() const override { return "ExpressionTransform"; } - static Block transformHeader(Block header, const ExpressionActionsPtr & expression); + static Block transformHeader(Block header, const ActionsDAG & expression); protected: void transform(Chunk & chunk) override; diff --git a/src/Processors/Transforms/FilterTransform.cpp b/src/Processors/Transforms/FilterTransform.cpp index 23b1bf8a984..364fb8e1958 100644 --- a/src/Processors/Transforms/FilterTransform.cpp +++ b/src/Processors/Transforms/FilterTransform.cpp @@ -29,12 +29,11 @@ static void replaceFilterToConstant(Block & block, const String & filter_column_ Block FilterTransform::transformHeader( Block header, - const ExpressionActionsPtr & expression, + const ActionsDAG & expression, const String & filter_column_name, bool remove_filter_column) { - size_t num_rows = header.rows(); - expression->execute(header, num_rows); + header = expression.updateHeader(std::move(header)); if (remove_filter_column) header.erase(filter_column_name); @@ -50,7 +49,10 @@ FilterTransform::FilterTransform( String filter_column_name_, bool remove_filter_column_, bool on_totals_) - : ISimpleTransform(header_, transformHeader(header_, expression_, filter_column_name_, remove_filter_column_), true) + : ISimpleTransform( + header_, + transformHeader(header_, expression_->getActionsDAG(), filter_column_name_, remove_filter_column_), + true) , expression(std::move(expression_)) , filter_column_name(std::move(filter_column_name_)) , remove_filter_column(remove_filter_column_) diff --git a/src/Processors/Transforms/FilterTransform.h b/src/Processors/Transforms/FilterTransform.h index c0ccf0fd072..39f1f1c42db 100644 --- a/src/Processors/Transforms/FilterTransform.h +++ b/src/Processors/Transforms/FilterTransform.h @@ -8,6 +8,8 @@ namespace DB class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; + /** Implements WHERE, HAVING operations. * Takes an expression, which adds to the block one ColumnUInt8 column containing the filtering conditions. * The expression is evaluated and result chunks contain only the filtered rows. @@ -22,7 +24,7 @@ public: static Block transformHeader( Block header, - const ExpressionActionsPtr & expression, + const ActionsDAG & expression, const String & filter_column_name, bool remove_filter_column); diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index c81c389b6fa..6ebaadf6e01 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -28,15 +28,13 @@ void finalizeChunk(Chunk & chunk) chunk.setColumns(std::move(columns), num_rows); } -Block TotalsHavingTransform::transformHeader(Block block, const ExpressionActionsPtr & expression, bool final) +Block TotalsHavingTransform::transformHeader(Block block, const ActionsDAG * expression, bool final) { if (final) finalizeBlock(block); - size_t num_rows = block.rows(); - if (expression) - expression->execute(block, num_rows); + block = expression->updateHeader(std::move(block)); return block; } @@ -49,7 +47,7 @@ TotalsHavingTransform::TotalsHavingTransform( TotalsMode totals_mode_, double auto_include_threshold_, bool final_) - : ISimpleTransform(header, transformHeader(header, expression_, final_), true) + : ISimpleTransform(header, transformHeader(header, expression_->getActionsDAG(), final_), true) , overflow_row(overflow_row_) , expression(expression_) , filter_column_name(filter_column_) diff --git a/src/Processors/Transforms/TotalsHavingTransform.h b/src/Processors/Transforms/TotalsHavingTransform.h index 8191ec763b0..5809f382e0e 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.h +++ b/src/Processors/Transforms/TotalsHavingTransform.h @@ -12,6 +12,8 @@ using ArenaPtr = std::shared_ptr; class ExpressionActions; using ExpressionActionsPtr = std::shared_ptr; +class ActionsDAG; + enum class TotalsMode; /** Takes blocks after grouping, with non-finalized aggregate functions. @@ -37,7 +39,7 @@ public: Status prepare() override; void work() override; - static Block transformHeader(Block block, const ExpressionActionsPtr & expression, bool final); + static Block transformHeader(Block block, const ActionsDAG * expression, bool final); protected: void transform(Chunk & chunk) override; From 25ceb1df658d0676069676a04a4a2bde15666d3a Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 13 May 2021 16:50:15 +0300 Subject: [PATCH 49/82] Final fixes --- src/Columns/IColumn.cpp | 5 ----- src/Columns/IColumn.h | 2 -- .../Formats/Impl/ArrowColumnToCHColumn.cpp | 2 +- .../Formats/Impl/CHColumnToArrowColumn.cpp | 1 - .../Formats/Impl/ORCBlockInputFormat.cpp | 2 +- .../0_stateless/00900_orc_arrays_load.reference | 8 ++++---- .../queries/0_stateless/00900_orc_arrays_load.sh | 6 +++--- .../00900_orc_nested_arrays_load.reference | 4 ++-- .../0_stateless/00900_orc_nested_arrays_load.sh | 4 ++-- .../00900_orc_nullable_arrays_load.reference | 12 ++++++------ .../00900_orc_nullable_arrays_load.sh | 4 ++-- .../0_stateless/test_6pbz8e/nested_arrays.orc | Bin 0 -> 1454 bytes .../0_stateless/test_6pbz8e/nullable_arrays.orc | Bin 0 -> 821 bytes 13 files changed, 21 insertions(+), 29 deletions(-) create mode 100644 tests/queries/0_stateless/test_6pbz8e/nested_arrays.orc create mode 100644 tests/queries/0_stateless/test_6pbz8e/nullable_arrays.orc diff --git a/src/Columns/IColumn.cpp b/src/Columns/IColumn.cpp index 2e61dc50fa4..2fe19d5d9fd 100644 --- a/src/Columns/IColumn.cpp +++ b/src/Columns/IColumn.cpp @@ -41,9 +41,4 @@ bool isColumnConst(const IColumn & column) return checkColumn(column); } -bool isColumnArray(const IColumn & column) -{ - return checkColumn(column); -} - } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index a788191e49f..23acc81e63d 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -533,6 +533,4 @@ bool isColumnConst(const IColumn & column); /// True if column's an ColumnNullable instance. It's just a syntax sugar for type check. bool isColumnNullable(const IColumn & column); -bool isColumnArray(const IColumn & column); - } diff --git a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 3134fe7d919..f5a4ed89387 100644 --- a/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -272,7 +272,7 @@ namespace DB } // TODO: check if a column is const? - if (!is_nullable && !isColumnArray(internal_column) && arrow_column->null_count()) + if (!is_nullable && !checkColumn(internal_column) && arrow_column->null_count()) { throw Exception { diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 3d0b16e3b0a..9772f6c5c2f 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -18,7 +18,6 @@ #include #include -#include namespace DB { diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 6f43addc4ed..07edaa551fb 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -80,7 +80,7 @@ void ORCBlockInputFormat::prepareReader() { if (getPort().getHeader().has(schema->field(i)->name())) { - include_indices.push_back(i+1); + include_indices.push_back(i); } } } diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.reference b/tests/queries/0_stateless/00900_orc_arrays_load.reference index 35df4fa923c..9b20ef98164 100644 --- a/tests/queries/0_stateless/00900_orc_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_arrays_load.reference @@ -1,4 +1,4 @@ -1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] -2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] -1 [1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] -2 [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] [0.20,10.00,4.00] [4.00,10000.10,10000.10] [1000000000.00,90.00,101001.01] +[] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] [] diff --git a/tests/queries/0_stateless/00900_orc_arrays_load.sh b/tests/queries/0_stateless/00900_orc_arrays_load.sh index b1b0e897156..410945b31eb 100755 --- a/tests/queries/0_stateless/00900_orc_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_arrays_load.sh @@ -4,10 +4,10 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) # shellcheck source=../shell_config.sh . "$CUR_DIR"/../shell_config.sh -${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_load" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (id UInt32, arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime), arr15 Array(Decimal(4, 2)), arr16 Array(Decimal(10, 2)), arr17 Array(Decimal(25, 2))) ENGINE=MergeTree() order by id" +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_arrays (arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime), arr15 Array(Decimal(4, 2)), arr16 Array(Decimal(10, 2)), arr17 Array(Decimal(25, 2))) ENGINE=Memory()" -${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_arrays VALUES (1, [1,-2,3],[1,2,3],[100,-200,300],[100,200,300],[10000000,-20000000,30000000],[10000000,2000000,3000000],[100000000000000,-200000000000,3000000000000],[100000000000000,20000000000000,3000000000000],['Some string','Some string','Some string'],['0000','1111','2222'],[42.42,424.2,0.4242],[424242.424242,4242042420.242424,42],['2000-01-01','2001-01-01','2002-01-01'],['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'],[0.20,10.00,4.00],[4.00,10000.10,10000.10],[1000000000.00,90.00,101001.01]),(2, [],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[])" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_arrays VALUES ([1,-2,3],[1,2,3],[100,-200,300],[100,200,300],[10000000,-20000000,30000000],[10000000,2000000,3000000],[100000000000000,-200000000000,3000000000000],[100000000000000,20000000000000,3000000000000],['Some string','Some string','Some string'],['0000','1111','2222'],[42.42,424.2,0.4242],[424242.424242,4242042420.242424,42],['2000-01-01','2001-01-01','2002-01-01'],['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'],[0.20,10.00,4.00],[4.00,10000.10,10000.10],[1000000000.00,90.00,101001.01]),([],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[],[])" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/arrays.orc diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference index 78cf96eb37c..dd9c9900684 100644 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.reference @@ -1,2 +1,2 @@ -1 [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] -1 [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] diff --git a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh index 8875a4686dd..81d2aeed4bd 100755 --- a/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nested_arrays_load.sh @@ -5,9 +5,9 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nested_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (id UInt32, arr1 Array(Array(Array(UInt32))), arr2 Array(Array(Array(String))), arr3 Array(Array(Nullable(UInt32))), arr4 Array(Array(Nullable(String)))) engine=MergeTree() ORDER BY id" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nested_arrays (arr1 Array(Array(Array(UInt32))), arr2 Array(Array(Array(String))), arr3 Array(Array(Nullable(UInt32))), arr4 Array(Array(Nullable(String)))) engine=Memory()" -${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nested_arrays VALUES (1, [[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]],[[['Some string','Some string'],[]],[['Some string']],[[]]],[[NULL,1,2],[NULL],[1,2],[]],[['Some string',NULL,'Some string'],[NULL],[]])" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nested_arrays VALUES ([[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]],[[['Some string','Some string'],[]],[['Some string']],[[]]],[[NULL,1,2],[NULL],[1,2],[]],[['Some string',NULL,'Some string'],[NULL],[]])" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nested_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/nested_arrays.orc diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference index d548488a196..62e95652040 100644 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.reference @@ -1,6 +1,6 @@ -1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] -2 [NULL] [NULL] [NULL] -3 [] [] [] -1 [1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] -2 [NULL] [NULL] [NULL] -3 [] [] [] +[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] +[1,NULL,2] [NULL,'Some string',NULL] [0.00,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] diff --git a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh index 59fb7874515..61829d0709b 100755 --- a/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh +++ b/tests/queries/0_stateless/00900_orc_nullable_arrays_load.sh @@ -5,8 +5,8 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . "$CUR_DIR"/../shell_config.sh ${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_nullable_arrays" -${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (id UInt32, arr1 Array(Nullable(UInt32)), arr2 Array(Nullable(String)), arr3 Array(Nullable(Decimal(4, 2)))) ENGINE=MergeTree() order by id" -${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nullable_arrays VALUES (1, [1,NULL,2],[NULL,'Some string',NULL],[0.00,NULL,42.42]), (2, [NULL],[NULL],[NULL]), (3, [],[],[])" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE orc_nullable_arrays (arr1 Array(Nullable(UInt32)), arr2 Array(Nullable(String)), arr3 Array(Nullable(Decimal(4, 2)))) ENGINE=Memory()" +${CLICKHOUSE_CLIENT} --query="INSERT INTO orc_nullable_arrays VALUES ([1,NULL,2],[NULL,'Some string',NULL],[0.00,NULL,42.42]), ([NULL],[NULL],[NULL]), ([],[],[])" ${CLICKHOUSE_CLIENT} --query="SELECT * FROM orc_nullable_arrays FORMAT ORC" > "${CLICKHOUSE_TMP}"/nullable_arrays.orc cat "${CLICKHOUSE_TMP}"/nullable_arrays.orc | ${CLICKHOUSE_CLIENT} -q "INSERT INTO orc_nullable_arrays FORMAT ORC" diff --git a/tests/queries/0_stateless/test_6pbz8e/nested_arrays.orc b/tests/queries/0_stateless/test_6pbz8e/nested_arrays.orc new file mode 100644 index 0000000000000000000000000000000000000000..d538b7ded10449d4a2255037996767aedea57a4d GIT binary patch literal 1454 zcmcIjOODe(5Uu{W-D%sgW$|gU3Ry*hMfj|;LNg24z(fNVKmwtW_(*{RM9c+7NbEV< zTmWenELemw)#Zxmj%Jo9PCvhT^{T8=*U|BzE1Uv=R$H+GchhA^kd{`o#>z%|iQ#G& z?z1AQQZ-9!lvKm`MVb{lU0*|^+B!qu0j;&_K-z=Uu~#5CqA%CfRr_)!;x>Dx@a1tE ziEZw$M7P18#J}p!|41Rb-vH_opY8&L9G)RTbQfC;Lw62Py!`ax?ez1P^~uL~SHGf7 zqd{S%?)_G3skxBC3O_E3YbrQnO#{9n6#h1$^fjSk)QlPHhI!6t8AtvG`5g0tv1DJx ze9Tx=K45=|K4=7?V1#f*Fiuy6bj4hGOgwU(E_s^(ihsSwVj|1PFK)m+T( zVYf_3z~kWSMn0h&%{2qb-*Q|8J} z8k|4g-&UWV$D@b4yiY#5bH*eWToGNi2zt*V)!P=)apt(ED^}-@`+DDSV(wFmU=b8( zw47Q_71h7rAqrg|U7xu=bA8Y83_*E1u=lvrx%2sMj&J99%Am=`yyO38gvX~B@=|6& zEvJ}ki-dCOERrZKkdTiZ)J7TUs&PAhhFe8Y~N*RvB4UtoTK$%NG? P2^v>1fB8o24}1Cts>@YL literal 0 HcmV?d00001 From 0b66aa0e85cca084b795acd31998a208b43bb118 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Thu, 13 May 2021 20:05:58 +0300 Subject: [PATCH 50/82] edited EN and RU --- docs/en/getting-started/install.md | 2 +- .../external-dicts-dict-layout.md | 7 ++++--- docs/ru/getting-started/install.md | 4 +++- docs/ru/operations/system-tables/dictionaries.md | 1 + .../external-dicts-dict-layout.md | 15 ++++++++++++--- 5 files changed, 21 insertions(+), 8 deletions(-) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 2134de9d0f3..c444264b71f 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -104,7 +104,7 @@ For non-Linux operating systems and for AArch64 CPU arhitecture, ClickHouse buil After downloading, you can use the `clickhouse client` to connect to the server, or `clickhouse local` to process local data. -Run `sudo ./clickhouse install` if you want to install clickhouse system-wide (also with needed condiguration files, configuring users etc.). After that run `clickhouse start` commands to start the clickhouse-server and `clickhouse-client` to connect to it. +Run `sudo ./clickhouse install` if you want to install clickhouse system-wide (also with needed configuration files, configuring users etc.). After that run `clickhouse start` commands to start the clickhouse-server and `clickhouse-client` to connect to it. These builds are not recommended for use in production environments because they are less thoroughly tested, but you can do so on your own risk. They also have only a subset of ClickHouse features available. diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 3c6eb5a0d62..214b1e0d58c 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -95,9 +95,10 @@ LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000)) The dictionary is completely stored in memory in the form of a hash table. The dictionary can contain any number of elements with any identifiers In practice, the number of keys can reach tens of millions of items. -If `preallocate` is `true` (default is `false`) the hash table will be preallocated (this will make dictionary load faster). But note that you should use it only if: -- the source support approximate number of elements (for now it is supported only by the `ClickHouse` source) -- there is no duplicates in the data (otherwise it may increase memory usage for the hashtable) +If `preallocate` is `true` (default is `false`) the hash table will be preallocated (this will make the dictionary load faster). But note that you should use it only if: + +- the source support an approximate number of elements (for now it is supported only by the `ClickHouse` source) +- there are no duplicates in the data (otherwise it may increase memory usage for the hashtable) All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index 4ae27a910ea..ec3059db561 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -95,7 +95,9 @@ sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh - [AArch64](https://builds.clickhouse.tech/master/aarch64/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/aarch64/clickhouse' && chmod a+x ./clickhouse` - [FreeBSD](https://builds.clickhouse.tech/master/freebsd/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/freebsd/clickhouse' && chmod a+x ./clickhouse` -После скачивания, можно воспользоваться `clickhouse client` для подключения к серверу, или `clickhouse local` для обработки локальных данных. Для запуска `clickhouse server` необходимо скачать конфигурационные файлы [сервера](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/config.xml) и [пользователей](https://github.com/ClickHouse/ClickHouse/blob/master/programs/server/users.xml) с GitHub. +После скачивания, можно воспользоваться `clickhouse client` для подключения к серверу, или `clickhouse local` для обработки локальных данных. + +Чтобы установить ClickHouse в рамках всей системы (с необходимыми конфигурационными файлами, настройками пользователей и т.д.), выполните `sudo ./clickhouse install`. Затем выполните команды `clickhouse start` (чтобы запустить сервер Clickhouse) и `clickhouse-client` (чтобы подключиться к нему). Данные сборки не рекомендуются для использования в продакшене, так как они недостаточно тщательно протестированны. Также, в них присутствуют не все возможности ClickHouse. diff --git a/docs/ru/operations/system-tables/dictionaries.md b/docs/ru/operations/system-tables/dictionaries.md index 6a49904aae9..940fa094065 100644 --- a/docs/ru/operations/system-tables/dictionaries.md +++ b/docs/ru/operations/system-tables/dictionaries.md @@ -21,6 +21,7 @@ - `bytes_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Объем оперативной памяти, используемый словарем. - `query_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Количество запросов с момента загрузки словаря или с момента последней успешной перезагрузки. - `hit_rate` ([Float64](../../sql-reference/data-types/float.md)) — Для cache-словарей — процент закэшированных значений. +- `found_rate` ([Float64](../../sql-reference/data-types/float.md)) — Процент использований словаря, для которых было найдено значение. - `element_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Количество элементов, хранящихся в словаре. - `load_factor` ([Float64](../../sql-reference/data-types/float.md)) — Процент заполнения словаря (для хэшированного словаря — процент заполнения хэш-таблицы). - `source` ([String](../../sql-reference/data-types/string.md)) — Текст, описывающий [источник данных](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) для словаря. diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 40ea4ba7d26..9b65db484cb 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -94,26 +94,35 @@ LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000)) Словарь полностью хранится в оперативной памяти в виде хэш-таблиц. Словарь может содержать произвольное количество элементов с произвольными идентификаторами. На практике количество ключей может достигать десятков миллионов элементов. +Если `preallocate` имеет значение `true` (по умолчанию `false`), хэш-таблица будет предварительно определена (это ускорит загрузку словаря). Используйте этот метод только в случае, если: + +- Источник поддерживает произвольное количество элементов (пока поддерживается только источником `ClickHouse`). +- В данных нет дубликатов (иначе это может увеличить объем используемой памяти хэш-таблицы). + Поддерживаются все виды источников. При обновлении данные (из файла, из таблицы) читаются целиком. Пример конфигурации: ``` xml - + + 0 + ``` или ``` sql -LAYOUT(HASHED()) +LAYOUT(HASHED(PREALLOCATE 0)) ``` ### sparse_hashed {#dicts-external_dicts_dict_layout-sparse_hashed} Аналогичен `hashed`, но при этом занимает меньше места в памяти и генерирует более высокую загрузку CPU. +Для этого типа размещения также можно задать `preallocate` в значении `true`. В данном случае это более важно, чем для типа `hashed`. + Пример конфигурации: ``` xml @@ -125,7 +134,7 @@ LAYOUT(HASHED()) или ``` sql -LAYOUT(SPARSE_HASHED()) +LAYOUT(SPARSE_HASHED([PREALLOCATE 0])) ``` ### complex_key_hashed {#complex-key-hashed} From 00d963d40fecea295add4caa48ab75c88913a383 Mon Sep 17 00:00:00 2001 From: Evgeniia Sudarikova Date: Thu, 13 May 2021 20:26:03 +0300 Subject: [PATCH 51/82] minor changes --- .../external-dictionaries/external-dicts-dict-layout.md | 4 ++-- docs/ru/getting-started/install.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 214b1e0d58c..84ac45d2f35 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -97,8 +97,8 @@ The dictionary is completely stored in memory in the form of a hash table. The d If `preallocate` is `true` (default is `false`) the hash table will be preallocated (this will make the dictionary load faster). But note that you should use it only if: -- the source support an approximate number of elements (for now it is supported only by the `ClickHouse` source) -- there are no duplicates in the data (otherwise it may increase memory usage for the hashtable) +- The source support an approximate number of elements (for now it is supported only by the `ClickHouse` source). +- There are no duplicates in the data (otherwise it may increase memory usage for the hashtable). All types of sources are supported. When updating, data (from a file or from a table) is read in its entirety. diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index ec3059db561..e355959b290 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -97,7 +97,7 @@ sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh После скачивания, можно воспользоваться `clickhouse client` для подключения к серверу, или `clickhouse local` для обработки локальных данных. -Чтобы установить ClickHouse в рамках всей системы (с необходимыми конфигурационными файлами, настройками пользователей и т.д.), выполните `sudo ./clickhouse install`. Затем выполните команды `clickhouse start` (чтобы запустить сервер Clickhouse) и `clickhouse-client` (чтобы подключиться к нему). +Чтобы установить ClickHouse в рамках всей системы (с необходимыми конфигурационными файлами, настройками пользователей и т.д.), выполните `sudo ./clickhouse install`. Затем выполните команды `clickhouse start` (чтобы запустить сервер) и `clickhouse-client` (чтобы подключиться к нему). Данные сборки не рекомендуются для использования в продакшене, так как они недостаточно тщательно протестированны. Также, в них присутствуют не все возможности ClickHouse. From bc4ccdece6b02580bf07cb402a8bad37d6393ce7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 13 May 2021 21:07:47 +0300 Subject: [PATCH 52/82] Fix tests. --- src/Interpreters/ActionsDAG.cpp | 31 ++++++++++--------- .../Transforms/TotalsHavingTransform.cpp | 2 +- 2 files changed, 18 insertions(+), 15 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 0065aea0fc3..917e9889972 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -450,16 +450,16 @@ static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * nod { case ActionsDAG::ActionType::FUNCTION: { - bool all_args_are_const = true; + // bool all_args_are_const = true; - for (size_t i = 0; i < arguments.size(); ++i) - if (typeid_cast(arguments[i].column.get()) == nullptr) - all_args_are_const = false; + // for (const auto & argument : arguments) + // if (typeid_cast(argument.column.get()) == nullptr) + // all_args_are_const = false; res_column.column = node->function->execute(arguments, res_column.type, 0, true); - if (!all_args_are_const) - res_column.column = res_column.column->convertToFullColumnIfConst(); + // if (!all_args_are_const) + // res_column.column = res_column.column->convertToFullColumnIfConst(); break; } @@ -519,7 +519,7 @@ Block ActionsDAG::updateHeader(Block header) const if (it != input_positions.end() && !it->second.empty()) { auto & list = it->second; - pos_to_remove.push_back(list.front()); + pos_to_remove.push_back(pos); result_cache[inputs[list.front()]] = std::move(col); list.pop_front(); } @@ -559,20 +559,23 @@ Block ActionsDAG::updateHeader(Block header) const ColumnsWithTypeAndName arguments(node->children.size()); for (size_t i = 0; i < arguments.size(); ++i) - arguments[i] = result_cache[node->children[i]]; - - if (node->type == ActionsDAG::ActionType::INPUT) { - if (result_cache.find(node) == result_cache.end()) + arguments[i] = result_cache[node->children[i]]; + if (!arguments[i].column) throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, - "Not found column {} in block", node->result_name); + "Not found column {} in block", node->children[i]->result_name); } - else + + if (node->type != ActionsDAG::ActionType::INPUT) result_cache[node] = executeActionForHeader(node, std::move(arguments)); + else + result_cache[node] = {}; } } - result_columns.push_back(result_cache[output]); + auto & column = result_cache[output]; + if (column.column) + result_columns.push_back(result_cache[output]); } } diff --git a/src/Processors/Transforms/TotalsHavingTransform.cpp b/src/Processors/Transforms/TotalsHavingTransform.cpp index 6ebaadf6e01..9724d332f15 100644 --- a/src/Processors/Transforms/TotalsHavingTransform.cpp +++ b/src/Processors/Transforms/TotalsHavingTransform.cpp @@ -47,7 +47,7 @@ TotalsHavingTransform::TotalsHavingTransform( TotalsMode totals_mode_, double auto_include_threshold_, bool final_) - : ISimpleTransform(header, transformHeader(header, expression_->getActionsDAG(), final_), true) + : ISimpleTransform(header, transformHeader(header, expression_ ? &expression_->getActionsDAG() : nullptr, final_), true) , overflow_row(overflow_row_) , expression(expression_) , filter_column_name(filter_column_) From bbe470a670473780e2e33da143723728464bc0f4 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Thu, 13 May 2021 21:15:55 +0300 Subject: [PATCH 53/82] Remove fillArrowArray from .h file --- .../Formats/Impl/CHColumnToArrowColumn.cpp | 14 ++++++++++++-- .../Formats/Impl/CHColumnToArrowColumn.h | 10 ---------- 2 files changed, 12 insertions(+), 12 deletions(-) diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp index 9772f6c5c2f..0e9968bec17 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.cpp @@ -93,6 +93,16 @@ namespace DB checkStatus(status, write_column->getName(), format_name); } + static void fillArrowArray( + const String & column_name, + ColumnPtr & column, + const std::shared_ptr & column_type, + const PaddedPODArray * null_bytemap, + arrow::ArrayBuilder * array_builder, + String format_name, + size_t start, + size_t end); + static void fillArrowArrayWithArrayColumnData( const String & column_name, ColumnPtr & column, @@ -117,7 +127,7 @@ namespace DB /// Start new array components_status = builder.Append(); checkStatus(components_status, nested_column->getName(), format_name); - CHColumnToArrowColumn::fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx]); + fillArrowArray(column_name, nested_column, nested_type, null_bytemap, value_builder, format_name, offsets[array_idx - 1], offsets[array_idx]); } } @@ -200,7 +210,7 @@ namespace DB } } - void CHColumnToArrowColumn::fillArrowArray( + static void fillArrowArray( const String & column_name, ColumnPtr & column, const std::shared_ptr & column_type, diff --git a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h index 9740063f110..de594389c25 100644 --- a/src/Processors/Formats/Impl/CHColumnToArrowColumn.h +++ b/src/Processors/Formats/Impl/CHColumnToArrowColumn.h @@ -30,16 +30,6 @@ private: public: static void chChunkToArrowTable(std::shared_ptr & res, const Block & header, const Chunk & chunk, size_t columns_num, String format_name); - - static void fillArrowArray( - const String & column_name, - ColumnPtr & column, - const std::shared_ptr & column_type, - const PaddedPODArray * null_bytemap, - arrow::ArrayBuilder * array_builder, - String format_name, - size_t start, - size_t end); }; } #endif From 6e3a1d4e99a113ebd435ccfd85a6e7e5f9ec0241 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Thu, 13 May 2021 18:18:26 +0000 Subject: [PATCH 54/82] update --- .../functions/ext-dict-functions.md | 19 ++----------------- 1 file changed, 2 insertions(+), 17 deletions(-) diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 2f270d49643..e46d6d1873a 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -17,6 +17,7 @@ Retrieves a value from an external dictionary. ``` sql dictGet('dict_name', 'attr_name', id_expr) dictGetOrDefault('dict_name', 'attr_name', id_expr, default_value_expr) +dictGetOrNull('dict_name', 'attr_name', id_expr) ``` **Arguments** @@ -34,6 +35,7 @@ dictGetOrDefault('dict_name', 'attr_name', id_expr, default_value_expr) - `dictGet` returns the content of the `` element specified for the attribute in the dictionary configuration. - `dictGetOrDefault` returns the value passed as the `default_value_expr` parameter. + - `dictGetOrNull` returns `NULL` in case key was not found in dictionary. ClickHouse throws an exception if it cannot parse the value of the attribute or the value doesn’t match the attribute data type. @@ -100,23 +102,6 @@ LIMIT 3 - [External Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) -## dictGetOrNull {#dictgetornull } - -Retrieves a value from an external dictionary. It works like `dictGet`, but return `NULL` in case key was not found in dictionary. - -``` sql -dictGetOrNull('dict_name', 'attr_name', id_expr) -``` - -**Arguments** - - - -**Returned value** - - -Type: - ## dictHas {#dicthas} Checks whether a key is present in a dictionary. From 7a0ec69f37f1aab12c69c865714d8f89c03292a3 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Thu, 13 May 2021 19:24:41 +0000 Subject: [PATCH 55/82] add example --- .../functions/ext-dict-functions.md | 65 ++++++++++++++++++- 1 file changed, 62 insertions(+), 3 deletions(-) diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 906905c8538..0d1642011a7 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -39,9 +39,9 @@ dictGetOrNull('dict_name', attr_name, id_expr) ClickHouse throws an exception if it cannot parse the value of the attribute or the value doesn’t match the attribute data type. -**Example for single attribute** +**Example for simple key dictionary** -Create a text file `ext-dict-text.csv` containing the following: +Create a text file `ext-dict-test.csv` containing the following: ``` text 1,1 @@ -98,7 +98,7 @@ LIMIT 3 └─────┴────────┘ ``` -**Example for multiple attributes** +**Example for complex key dictionary** Create a text file `ext-dict-mult.csv` containing the following: @@ -163,6 +163,65 @@ LIMIT 3; └─────────┴───────────────────────┘ ``` +**Example for range key dictionary** + +Input table: + +```sql +CREATE TABLE range_key_dictionary_source_table +( + key UInt64, + start_date Date, + end_date Date, + value String, + value_nullable Nullable(String) +) +ENGINE = TinyLog(); + +INSERT INTO range_key_dictionary_source_table VALUES(1, toDate('2019-05-20'), toDate('2019-05-20'), 'First', 'First'); +INSERT INTO range_key_dictionary_source_table VALUES(2, toDate('2019-05-20'), toDate('2019-05-20'), 'Second', NULL); +INSERT INTO range_key_dictionary_source_table VALUES(3, toDate('2019-05-20'), toDate('2019-05-20'), 'Third', 'Third'); +``` + +Create the external dictionary: + +```sql +CREATE DICTIONARY range_key_dictionary +( + key UInt64, + start_date Date, + end_date Date, + value String, + value_nullable Nullable(String) +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'range_key_dictionary_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(RANGE_HASHED()) +RANGE(MIN start_date MAX end_date); +``` + +Perform the query: + +``` sql +SELECT + (number, toDate('2019-05-20')), + dictHas('range_key_dictionary', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', 'value', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', 'value_nullable', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', ('value', 'value_nullable'), number, toDate('2019-05-20')) +FROM system.numbers LIMIT 5 FORMAT TabSeparated; +``` +Result: + +``` text +(0,'2019-05-20') 0 \N \N (NULL,NULL) +(1,'2019-05-20') 1 First First ('First','First') +(2,'2019-05-20') 0 \N \N (NULL,NULL) +(3,'2019-05-20') 0 \N \N (NULL,NULL) +(4,'2019-05-20') 0 \N \N (NULL,NULL) +``` + **See Also** - [External Dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md) From 9cd33b58373e500b2b665ea943d4916b945deb18 Mon Sep 17 00:00:00 2001 From: adevyatova Date: Thu, 13 May 2021 19:32:28 +0000 Subject: [PATCH 56/82] add ru docs --- .../functions/ext-dict-functions.md | 2 +- .../functions/ext-dict-functions.md | 63 ++++++++++++++++++- 2 files changed, 63 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/functions/ext-dict-functions.md b/docs/en/sql-reference/functions/ext-dict-functions.md index 0d1642011a7..8eb10bd0208 100644 --- a/docs/en/sql-reference/functions/ext-dict-functions.md +++ b/docs/en/sql-reference/functions/ext-dict-functions.md @@ -10,7 +10,7 @@ toc_title: External Dictionaries For information on connecting and configuring external dictionaries, see [External dictionaries](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). -## dictGet, dictGetOrDefault {#dictget} +## dictGet, dictGetOrDefault, dictGetOrNull {#dictget} Retrieves values from an external dictionary. diff --git a/docs/ru/sql-reference/functions/ext-dict-functions.md b/docs/ru/sql-reference/functions/ext-dict-functions.md index 4770f17789a..be91145659e 100644 --- a/docs/ru/sql-reference/functions/ext-dict-functions.md +++ b/docs/ru/sql-reference/functions/ext-dict-functions.md @@ -7,13 +7,14 @@ toc_title: "Функции для работы с внешними словар Информацию о подключении и настройке внешних словарей смотрите в разделе [Внешние словари](../../sql-reference/dictionaries/external-dictionaries/external-dicts.md). -## dictGet, dictGetOrDefault {#dictget} +## dictGet, dictGetOrDefault, dictGetOrNull {#dictget} Извлекает значение из внешнего словаря. ``` sql dictGet('dict_name', attr_names, id_expr) dictGetOrDefault('dict_name', attr_names, id_expr, default_value_expr) +dictGetOrNull('dict_name', attr_name, id_expr) ``` **Аргументы** @@ -31,6 +32,7 @@ dictGetOrDefault('dict_name', attr_names, id_expr, default_value_expr) - `dictGet` возвращает содержимое элемента ``, указанного для атрибута в конфигурации словаря. - `dictGetOrDefault` возвращает атрибут `default_value_expr`. + - `dictGetOrNull` возвращает `NULL` в случае, если ключ не найден в словаре. Если значение атрибута не удалось обработать или оно не соответствует типу данных атрибута, то ClickHouse генерирует исключение. @@ -158,6 +160,65 @@ LIMIT 3; └─────────┴───────────────────────┘ ``` +**Пример для словаря с диапазоном ключей** + +Создадим таблицу: + +```sql +CREATE TABLE range_key_dictionary_source_table +( + key UInt64, + start_date Date, + end_date Date, + value String, + value_nullable Nullable(String) +) +ENGINE = TinyLog(); + +INSERT INTO range_key_dictionary_source_table VALUES(1, toDate('2019-05-20'), toDate('2019-05-20'), 'First', 'First'); +INSERT INTO range_key_dictionary_source_table VALUES(2, toDate('2019-05-20'), toDate('2019-05-20'), 'Second', NULL); +INSERT INTO range_key_dictionary_source_table VALUES(3, toDate('2019-05-20'), toDate('2019-05-20'), 'Third', 'Third'); +``` + +Создадим внешний словарь: + +```sql +CREATE DICTIONARY range_key_dictionary +( + key UInt64, + start_date Date, + end_date Date, + value String, + value_nullable Nullable(String) +) +PRIMARY KEY key +SOURCE(CLICKHOUSE(HOST 'localhost' PORT tcpPort() TABLE 'range_key_dictionary_source_table')) +LIFETIME(MIN 1 MAX 1000) +LAYOUT(RANGE_HASHED()) +RANGE(MIN start_date MAX end_date); +``` + +Выполним запрос: + +``` sql +SELECT + (number, toDate('2019-05-20')), + dictHas('range_key_dictionary', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', 'value', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', 'value_nullable', number, toDate('2019-05-20')), + dictGetOrNull('range_key_dictionary', ('value', 'value_nullable'), number, toDate('2019-05-20')) +FROM system.numbers LIMIT 5 FORMAT TabSeparated; +``` +Результат: + +``` text +(0,'2019-05-20') 0 \N \N (NULL,NULL) +(1,'2019-05-20') 1 First First ('First','First') +(2,'2019-05-20') 0 \N \N (NULL,NULL) +(3,'2019-05-20') 0 \N \N (NULL,NULL) +(4,'2019-05-20') 0 \N \N (NULL,NULL) +``` + **Смотрите также** - [Внешние словари](../../sql-reference/functions/ext-dict-functions.md) From fc0a36f40464beb9eea733476b06dafe98b3b8ad Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 13 May 2021 23:11:16 +0300 Subject: [PATCH 57/82] Update docs/en/sql-reference/aggregate-functions/reference/deltasum.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/aggregate-functions/reference/deltasum.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md index f42007fc857..7af99dc690f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md @@ -9,7 +9,7 @@ Sums the arithmetic difference between consecutive rows. If the difference is ne Note that the underlying data must be sorted for this function to work properly. !!! info "Note" - If you would like to use this function in a [materialized view](../../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp) method instead. + The underlying data must be sorted for this function to work properly. If you would like to use this function in a [materialized view](../../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp) method instead. **Syntax** From fbca7fdfa829cd603dc5d514b24f76e2e7a083c4 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 13 May 2021 23:11:28 +0300 Subject: [PATCH 58/82] Update docs/en/sql-reference/aggregate-functions/reference/deltasum.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/en/sql-reference/aggregate-functions/reference/deltasum.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md index 7af99dc690f..a5c268a881f 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md @@ -6,7 +6,6 @@ toc_priority: 141 Sums the arithmetic difference between consecutive rows. If the difference is negative, it is ignored. -Note that the underlying data must be sorted for this function to work properly. !!! info "Note" The underlying data must be sorted for this function to work properly. If you would like to use this function in a [materialized view](../../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp) method instead. From c1983e2e8ce43715d42378fe902955839f07655b Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 13 May 2021 23:11:37 +0300 Subject: [PATCH 59/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasum.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/aggregate-functions/reference/deltasum.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index 71fb1926488..26cc1396270 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -9,7 +9,7 @@ toc_priority: 141 Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. !!! info "Примечание" - В [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized) вместо этой функции рекомендуется использовать [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp). + Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. В [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized) вместо нее рекомендуется использовать [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp). **Синтаксис** From 975d49bf637555cf71ddebb93edfec645059a433 Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 13 May 2021 23:11:51 +0300 Subject: [PATCH 60/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasum.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- docs/ru/sql-reference/aggregate-functions/reference/deltasum.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index 26cc1396270..d3a96980dfd 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -6,7 +6,6 @@ toc_priority: 141 Суммирует арифметическую разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. -Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. !!! info "Примечание" Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. В [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized) вместо нее рекомендуется использовать [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp). From 4cd4b94ef63879f2021ce374fcbd9c0f663b704a Mon Sep 17 00:00:00 2001 From: sevirov <72220289+sevirov@users.noreply.github.com> Date: Thu, 13 May 2021 23:12:00 +0300 Subject: [PATCH 61/82] Update docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md Co-authored-by: Anna <42538400+adevyatova@users.noreply.github.com> --- .../aggregate-functions/reference/deltasumtimestamp.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md index b01f290591c..10294eb9e6d 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasumtimestamp.md @@ -4,7 +4,7 @@ toc_priority: 141 # deltaSumTimestamp {#agg_functions-deltasumtimestamp} -Прибавляет разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. +Суммирует разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. Эта функция предназначена в первую очередь для [материализованных представлений](../../../sql-reference/statements/create/view.md#materialized), упорядоченных по некоторому временному бакету согласно timestamp, например, по бакету `toStartOfMinute`. Поскольку строки в таком материализованном представлении будут иметь одинаковый timestamp, невозможно объединить их в "правом" порядке. Функция отслеживает `timestamp` наблюдаемых значений, поэтому возможно правильно упорядочить состояния во время слияния. From 9b9d69278244ccb0daf2bcc5ad9ed17b37a61d83 Mon Sep 17 00:00:00 2001 From: Dmitriy Date: Thu, 13 May 2021 23:27:33 +0300 Subject: [PATCH 62/82] Update deltasum.md MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Убрал лишний отступ. --- docs/en/sql-reference/aggregate-functions/reference/deltasum.md | 1 - docs/ru/sql-reference/aggregate-functions/reference/deltasum.md | 1 - 2 files changed, 2 deletions(-) diff --git a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md index a5c268a881f..2945084db77 100644 --- a/docs/en/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/en/sql-reference/aggregate-functions/reference/deltasum.md @@ -6,7 +6,6 @@ toc_priority: 141 Sums the arithmetic difference between consecutive rows. If the difference is negative, it is ignored. - !!! info "Note" The underlying data must be sorted for this function to work properly. If you would like to use this function in a [materialized view](../../../sql-reference/statements/create/view.md#materialized), you most likely want to use the [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp) method instead. diff --git a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md index d3a96980dfd..6825847f256 100644 --- a/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md +++ b/docs/ru/sql-reference/aggregate-functions/reference/deltasum.md @@ -6,7 +6,6 @@ toc_priority: 141 Суммирует арифметическую разницу между последовательными строками. Если разница отрицательна — она будет проигнорирована. - !!! info "Примечание" Чтобы эта функция работала должным образом, исходные данные должны быть отсортированы. В [материализованном представлении](../../../sql-reference/statements/create/view.md#materialized) вместо нее рекомендуется использовать [deltaSumTimestamp](../../../sql-reference/aggregate-functions/reference/deltasumtimestamp.md#agg_functions-deltasumtimestamp). From 46a7cc5f1d0125cf7a0b66409f9f1c335b622e4e Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 14 May 2021 00:52:29 +0300 Subject: [PATCH 63/82] Remove include_indices --- .../Formats/Impl/ORCBlockInputFormat.cpp | 14 +------------- src/Processors/Formats/Impl/ORCBlockInputFormat.h | 3 --- 2 files changed, 1 insertion(+), 16 deletions(-) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index 07edaa551fb..f669f6695d1 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -42,7 +42,7 @@ Chunk ORCBlockInputFormat::generate() return res; std::shared_ptr batch_result; - arrow::Status batch_status = file_reader->ReadStripe(stripe_current, include_indices, &batch_result); + arrow::Status batch_status = file_reader->ReadStripe(stripe_current, &batch_result); if (!batch_status.ok()) throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", batch_status.ToString()); @@ -63,7 +63,6 @@ void ORCBlockInputFormat::resetParser() IInputFormat::resetParser(); file_reader.reset(); - include_indices.clear(); stripe_current = 0; } @@ -72,17 +71,6 @@ void ORCBlockInputFormat::prepareReader() THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(in), arrow::default_memory_pool(), &file_reader)); stripe_total = file_reader->NumberOfStripes(); stripe_current = 0; - - std::shared_ptr schema; - THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); - - for (int i = 0; i < schema->num_fields(); ++i) - { - if (getPort().getHeader().has(schema->field(i)->name())) - { - include_indices.push_back(i); - } - } } void registerInputFormatProcessorORC(FormatFactory &factory) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 0c78290f3cc..5aec330a0c8 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -30,9 +30,6 @@ private: int stripe_current = 0; - // indices of columns to read from ORC file - std::vector include_indices; - void prepareReader(); }; From baa0078c0d69b4f24fca7a89c2514c5fe36b9e2e Mon Sep 17 00:00:00 2001 From: Evgenia Sudarikova <56156889+otrazhenia@users.noreply.github.com> Date: Fri, 14 May 2021 09:12:09 +0300 Subject: [PATCH 64/82] Update docs/ru/getting-started/install.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/getting-started/install.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/getting-started/install.md b/docs/ru/getting-started/install.md index e355959b290..d0a54d9043a 100644 --- a/docs/ru/getting-started/install.md +++ b/docs/ru/getting-started/install.md @@ -95,7 +95,7 @@ sudo clickhouse-client-$LATEST_VERSION/install/doinst.sh - [AArch64](https://builds.clickhouse.tech/master/aarch64/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/aarch64/clickhouse' && chmod a+x ./clickhouse` - [FreeBSD](https://builds.clickhouse.tech/master/freebsd/clickhouse) — `curl -O 'https://builds.clickhouse.tech/master/freebsd/clickhouse' && chmod a+x ./clickhouse` -После скачивания, можно воспользоваться `clickhouse client` для подключения к серверу, или `clickhouse local` для обработки локальных данных. +После скачивания можно воспользоваться `clickhouse client` для подключения к серверу или `clickhouse local` для обработки локальных данных. Чтобы установить ClickHouse в рамках всей системы (с необходимыми конфигурационными файлами, настройками пользователей и т.д.), выполните `sudo ./clickhouse install`. Затем выполните команды `clickhouse start` (чтобы запустить сервер) и `clickhouse-client` (чтобы подключиться к нему). @@ -174,4 +174,3 @@ SELECT 1 **Поздравляем, система работает!** Для дальнейших экспериментов можно попробовать загрузить один из тестовых наборов данных или пройти [пошаговое руководство для начинающих](https://clickhouse.tech/tutorial.html). - From c2d75e7f45e3c5fa9a499a25b80e85f669e70c05 Mon Sep 17 00:00:00 2001 From: Evgenia Sudarikova <56156889+otrazhenia@users.noreply.github.com> Date: Fri, 14 May 2021 09:12:19 +0300 Subject: [PATCH 65/82] Update docs/ru/operations/system-tables/dictionaries.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- docs/ru/operations/system-tables/dictionaries.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/operations/system-tables/dictionaries.md b/docs/ru/operations/system-tables/dictionaries.md index 940fa094065..b865fea736f 100644 --- a/docs/ru/operations/system-tables/dictionaries.md +++ b/docs/ru/operations/system-tables/dictionaries.md @@ -21,7 +21,7 @@ - `bytes_allocated` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Объем оперативной памяти, используемый словарем. - `query_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Количество запросов с момента загрузки словаря или с момента последней успешной перезагрузки. - `hit_rate` ([Float64](../../sql-reference/data-types/float.md)) — Для cache-словарей — процент закэшированных значений. -- `found_rate` ([Float64](../../sql-reference/data-types/float.md)) — Процент использований словаря, для которых было найдено значение. +- `found_rate` ([Float64](../../sql-reference/data-types/float.md)) — Процент обращений к словарю, при которых значение было найдено. - `element_count` ([UInt64](../../sql-reference/data-types/int-uint.md#uint-ranges)) — Количество элементов, хранящихся в словаре. - `load_factor` ([Float64](../../sql-reference/data-types/float.md)) — Процент заполнения словаря (для хэшированного словаря — процент заполнения хэш-таблицы). - `source` ([String](../../sql-reference/data-types/string.md)) — Текст, описывающий [источник данных](../../sql-reference/dictionaries/external-dictionaries/external-dicts-dict-sources.md) для словаря. @@ -59,4 +59,3 @@ SELECT * FROM system.dictionaries │ dictdb │ dict │ LOADED │ dictdb.dict │ Flat │ UInt64 │ ['value_default','value_expression'] │ ['String','String'] │ 74032 │ 0 │ 1 │ 1 │ 0.0004887585532746823 │ ClickHouse: dictdb.dicttbl │ 0 │ 1 │ 2020-03-04 04:17:34 │ 2020-03-04 04:30:34 │ 0.002 │ │ └──────────┴──────┴────────┴─────────────┴──────┴────────┴──────────────────────────────────────┴─────────────────────┴─────────────────┴─────────────┴──────────┴───────────────┴───────────────────────┴────────────────────────────┴──────────────┴──────────────┴─────────────────────┴──────────────────────────────┘───────────────────────┴────────────────┘ ``` - From 98bb44785eef7f8936f08362d1bbc94acc6dd61f Mon Sep 17 00:00:00 2001 From: Evgenia Sudarikova <56156889+otrazhenia@users.noreply.github.com> Date: Fri, 14 May 2021 09:12:25 +0300 Subject: [PATCH 66/82] Update docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../external-dictionaries/external-dicts-dict-layout.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index 9b65db484cb..a0352d8bf13 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -94,7 +94,7 @@ LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000)) Словарь полностью хранится в оперативной памяти в виде хэш-таблиц. Словарь может содержать произвольное количество элементов с произвольными идентификаторами. На практике количество ключей может достигать десятков миллионов элементов. -Если `preallocate` имеет значение `true` (по умолчанию `false`), хэш-таблица будет предварительно определена (это ускорит загрузку словаря). Используйте этот метод только в случае, если: +Если `preallocate` имеет значение `true` (по умолчанию `false`), хеш-таблица будет предварительно определена (это ускорит загрузку словаря). Используйте этот метод только в случае, если: - Источник поддерживает произвольное количество элементов (пока поддерживается только источником `ClickHouse`). - В данных нет дубликатов (иначе это может увеличить объем используемой памяти хэш-таблицы). @@ -452,4 +452,3 @@ dictGetString('prefix', 'asn', tuple(IPv6StringToNum('2001:db8::1'))) Никакие другие типы не поддерживаются. Функция возвращает атрибут для префикса, соответствующего данному IP-адресу. Если есть перекрывающиеся префиксы, возвращается наиболее специфический. Данные должны полностью помещаться в оперативной памяти. - From c896268f673efc11b281ca6469e80c902980262e Mon Sep 17 00:00:00 2001 From: Evgenia Sudarikova <56156889+otrazhenia@users.noreply.github.com> Date: Fri, 14 May 2021 09:12:31 +0300 Subject: [PATCH 67/82] Update docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md Co-authored-by: olgarev <56617294+olgarev@users.noreply.github.com> --- .../external-dictionaries/external-dicts-dict-layout.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md index a0352d8bf13..e63b5574e30 100644 --- a/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md +++ b/docs/ru/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-layout.md @@ -97,7 +97,7 @@ LAYOUT(FLAT(INITIAL_ARRAY_SIZE 50000 MAX_ARRAY_SIZE 5000000)) Если `preallocate` имеет значение `true` (по умолчанию `false`), хеш-таблица будет предварительно определена (это ускорит загрузку словаря). Используйте этот метод только в случае, если: - Источник поддерживает произвольное количество элементов (пока поддерживается только источником `ClickHouse`). -- В данных нет дубликатов (иначе это может увеличить объем используемой памяти хэш-таблицы). +- В данных нет дубликатов (иначе это может увеличить объем используемой памяти хеш-таблицы). Поддерживаются все виды источников. При обновлении данные (из файла, из таблицы) читаются целиком. From 396f60f292685c24e07268749fa2318fbf2a1691 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Fri, 14 May 2021 14:43:51 +0800 Subject: [PATCH 68/82] Fix bug for explain pipeline --- src/Processors/Pipe.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Pipe.cpp b/src/Processors/Pipe.cpp index 044975448ad..abd4df1f51f 100644 --- a/src/Processors/Pipe.cpp +++ b/src/Processors/Pipe.cpp @@ -834,7 +834,7 @@ void Pipe::transform(const Transformer & transformer) if (collected_processors) { - for (const auto & processor : processors) + for (const auto & processor : new_processors) collected_processors->emplace_back(processor); } From 1f0990ff1d2eb095fadfe634d86690abb1a3bf48 Mon Sep 17 00:00:00 2001 From: hexiaoting Date: Fri, 14 May 2021 16:18:27 +0800 Subject: [PATCH 69/82] Add test case --- .../01861_explain_pipeline.reference | 31 +++++++++++++++++++ .../0_stateless/01861_explain_pipeline.sql | 10 ++++++ 2 files changed, 41 insertions(+) create mode 100644 tests/queries/0_stateless/01861_explain_pipeline.reference create mode 100644 tests/queries/0_stateless/01861_explain_pipeline.sql diff --git a/tests/queries/0_stateless/01861_explain_pipeline.reference b/tests/queries/0_stateless/01861_explain_pipeline.reference new file mode 100644 index 00000000000..8d755f807c0 --- /dev/null +++ b/tests/queries/0_stateless/01861_explain_pipeline.reference @@ -0,0 +1,31 @@ +(Expression) +ExpressionTransform + (SettingQuotaAndLimits) + (Expression) + ExpressionTransform + (MergingFinal) + ReplacingSorted 2 → 1 + (Expression) + ExpressionTransform × 2 + (ReadFromMergeTree) + MergeTree × 2 0 → 1 +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +(Expression) +ExpressionTransform × 2 + (SettingQuotaAndLimits) + (Expression) + ExpressionTransform × 2 + (MergingFinal) + ReplacingSorted × 2 2 → 1 + Copy × 2 1 → 2 + AddingSelector × 2 + (Expression) + ExpressionTransform × 2 + (ReadFromMergeTree) + MergeTree × 2 0 → 1 diff --git a/tests/queries/0_stateless/01861_explain_pipeline.sql b/tests/queries/0_stateless/01861_explain_pipeline.sql new file mode 100644 index 00000000000..aafecf57af1 --- /dev/null +++ b/tests/queries/0_stateless/01861_explain_pipeline.sql @@ -0,0 +1,10 @@ +DROP TABLE IF EXISTS test; +CREATE TABLE test(a Int, b Int) Engine=ReplacingMergeTree order by a; +INSERT INTO test select number, number from numbers(5); +INSERT INTO test select number, number from numbers(5,2); +set max_threads =1; +explain pipeline select * from test final; +select * from test final; +set max_threads =2; +explain pipeline select * from test final; +DROP TABLE test; From a1e9202b932796dd80e90ebe32039384c75e794d Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 May 2021 12:30:32 +0300 Subject: [PATCH 70/82] Review fixes. --- src/Interpreters/ActionsDAG.cpp | 43 ++++++++++++++++++--------------- 1 file changed, 24 insertions(+), 19 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index 917e9889972..dd99da23a36 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -501,7 +501,7 @@ static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * nod Block ActionsDAG::updateHeader(Block header) const { - std::unordered_map result_cache; + std::unordered_map node_to_column; std::vector pos_to_remove; { @@ -520,7 +520,7 @@ Block ActionsDAG::updateHeader(Block header) const { auto & list = it->second; pos_to_remove.push_back(pos); - result_cache[inputs[list.front()]] = std::move(col); + node_to_column[inputs[list.front()]] = std::move(col); list.pop_front(); } } @@ -529,30 +529,38 @@ Block ActionsDAG::updateHeader(Block header) const ColumnsWithTypeAndName result_columns; result_columns.reserve(index.size()); + struct Frame + { + const Node * node; + size_t next_child = 0; + }; + { for (const auto * output : index) { - if (result_cache.count(output) == 0) + if (node_to_column.count(output) == 0) { - std::stack stack; - stack.push(output); + std::stack stack; + stack.push({.node = output}); while (!stack.empty()) { - const Node * node = stack.top(); + auto & frame = stack.top(); + const auto * node = frame.node; - bool all_children_calculated = true; - for (const auto * child : node->children) + while (frame.next_child < node->children.size()) { - if (result_cache.count(child) == 0) + const auto * child = node->children[frame.next_child]; + if (node_to_column.count(child) == 0) { - stack.push(child); - all_children_calculated = false; + stack.push({.node = child}); break; } + + ++frame.next_child; } - if (!all_children_calculated) + if (frame.next_child < node->children.size()) continue; stack.pop(); @@ -560,22 +568,19 @@ Block ActionsDAG::updateHeader(Block header) const ColumnsWithTypeAndName arguments(node->children.size()); for (size_t i = 0; i < arguments.size(); ++i) { - arguments[i] = result_cache[node->children[i]]; + arguments[i] = node_to_column[node->children[i]]; if (!arguments[i].column) throw Exception(ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK, "Not found column {} in block", node->children[i]->result_name); } - if (node->type != ActionsDAG::ActionType::INPUT) - result_cache[node] = executeActionForHeader(node, std::move(arguments)); - else - result_cache[node] = {}; + node_to_column[node] = executeActionForHeader(node, std::move(arguments)); } } - auto & column = result_cache[output]; + auto & column = node_to_column[output]; if (column.column) - result_columns.push_back(result_cache[output]); + result_columns.push_back(node_to_column[output]); } } From 782f02b1dceb513df6f915f9479e729050e6ba58 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 14 May 2021 13:01:27 +0300 Subject: [PATCH 71/82] Review fixes. --- src/Interpreters/ActionsDAG.cpp | 14 +++----------- 1 file changed, 3 insertions(+), 11 deletions(-) diff --git a/src/Interpreters/ActionsDAG.cpp b/src/Interpreters/ActionsDAG.cpp index dd99da23a36..ede8e76a3e5 100644 --- a/src/Interpreters/ActionsDAG.cpp +++ b/src/Interpreters/ActionsDAG.cpp @@ -502,7 +502,7 @@ static ColumnWithTypeAndName executeActionForHeader(const ActionsDAG::Node * nod Block ActionsDAG::updateHeader(Block header) const { std::unordered_map node_to_column; - std::vector pos_to_remove; + std::set pos_to_remove; { std::unordered_map> input_positions; @@ -510,8 +510,6 @@ Block ActionsDAG::updateHeader(Block header) const for (size_t pos = 0; pos < inputs.size(); ++pos) input_positions[inputs[pos]->result_name].emplace_back(pos); - pos_to_remove.reserve(inputs.size()); - for (size_t pos = 0; pos < header.columns(); ++pos) { const auto & col = header.getByPosition(pos); @@ -519,7 +517,7 @@ Block ActionsDAG::updateHeader(Block header) const if (it != input_positions.end() && !it->second.empty()) { auto & list = it->second; - pos_to_remove.push_back(pos); + pos_to_remove.insert(pos); node_to_column[inputs[list.front()]] = std::move(col); list.pop_front(); } @@ -585,15 +583,9 @@ Block ActionsDAG::updateHeader(Block header) const } if (isInputProjected()) - { header.clear(); - } else - { - std::sort(pos_to_remove.rbegin(), pos_to_remove.rend()); - for (auto pos : pos_to_remove) - header.erase(pos); - } + header.erase(pos_to_remove); Block res; From 484cac619376ace4c8d9b7f48e5bbbac97fd5fe2 Mon Sep 17 00:00:00 2001 From: Pavel Kruglov Date: Fri, 14 May 2021 13:48:31 +0300 Subject: [PATCH 72/82] Return include_indices, fix arrays and add more tests --- .../Formats/Impl/ORCBlockInputFormat.cpp | 27 ++++++++++++++++++- .../Formats/Impl/ORCBlockInputFormat.h | 3 +++ .../01273_arrow_arrays_load.reference | 4 +++ .../0_stateless/01273_arrow_arrays_load.sh | 17 ++++++++++++ .../01273_arrow_nested_arrays_load.reference | 2 ++ .../01273_arrow_nested_arrays_load.sh | 17 ++++++++++++ ...01273_arrow_nullable_arrays_load.reference | 6 +++++ .../01273_arrow_nullable_arrays_load.sh | 15 +++++++++++ 8 files changed, 90 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/01273_arrow_arrays_load.reference create mode 100755 tests/queries/0_stateless/01273_arrow_arrays_load.sh create mode 100644 tests/queries/0_stateless/01273_arrow_nested_arrays_load.reference create mode 100755 tests/queries/0_stateless/01273_arrow_nested_arrays_load.sh create mode 100644 tests/queries/0_stateless/01273_arrow_nullable_arrays_load.reference create mode 100755 tests/queries/0_stateless/01273_arrow_nullable_arrays_load.sh diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp index f669f6695d1..24b524faeaf 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp @@ -42,7 +42,7 @@ Chunk ORCBlockInputFormat::generate() return res; std::shared_ptr batch_result; - arrow::Status batch_status = file_reader->ReadStripe(stripe_current, &batch_result); + arrow::Status batch_status = file_reader->ReadStripe(stripe_current, include_indices, &batch_result); if (!batch_status.ok()) throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", batch_status.ToString()); @@ -63,14 +63,39 @@ void ORCBlockInputFormat::resetParser() IInputFormat::resetParser(); file_reader.reset(); + include_indices.clear(); stripe_current = 0; } +size_t countIndicesForType(std::shared_ptr type) +{ + if (type->id() == arrow::Type::LIST) + return countIndicesForType(static_cast(type.get())->value_type()) + 1; + + return 1; +} + void ORCBlockInputFormat::prepareReader() { THROW_ARROW_NOT_OK(arrow::adapters::orc::ORCFileReader::Open(asArrowFile(in), arrow::default_memory_pool(), &file_reader)); stripe_total = file_reader->NumberOfStripes(); stripe_current = 0; + + std::shared_ptr schema; + THROW_ARROW_NOT_OK(file_reader->ReadSchema(&schema)); + + int index = 0; + for (int i = 0; i < schema->num_fields(); ++i) + { + if (getPort().getHeader().has(schema->field(i)->name())) + { + /// LIST type require 2 indices, so we should recursively + /// count the number of indices we need for this type. + int indexes_count = countIndicesForType(schema->field(i)->type()); + for (int j = 0; j != indexes_count; ++j) + include_indices.push_back(index++); + } + } } void registerInputFormatProcessorORC(FormatFactory &factory) diff --git a/src/Processors/Formats/Impl/ORCBlockInputFormat.h b/src/Processors/Formats/Impl/ORCBlockInputFormat.h index 5aec330a0c8..0c78290f3cc 100644 --- a/src/Processors/Formats/Impl/ORCBlockInputFormat.h +++ b/src/Processors/Formats/Impl/ORCBlockInputFormat.h @@ -30,6 +30,9 @@ private: int stripe_current = 0; + // indices of columns to read from ORC file + std::vector include_indices; + void prepareReader(); }; diff --git a/tests/queries/0_stateless/01273_arrow_arrays_load.reference b/tests/queries/0_stateless/01273_arrow_arrays_load.reference new file mode 100644 index 00000000000..edb6ba617a1 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_arrays_load.reference @@ -0,0 +1,4 @@ +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] +[] [] [] [] [] [] [] [] [] [] [] [] [] [] +[1,-2,3] [1,2,3] [100,-200,300] [100,200,300] [10000000,-20000000,30000000] [10000000,2000000,3000000] [100000000000000,-200000000000,3000000000000] [100000000000000,20000000000000,3000000000000] ['Some string','Some string','Some string'] ['0000','1111','2222'] [42.42,424.2,0.4242] [424242.424242,4242042420.242424,42] ['2000-01-01','2001-01-01','2002-01-01'] ['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00'] +[] [] [] [] [] [] [] [] [] [] [] [] [] [] diff --git a/tests/queries/0_stateless/01273_arrow_arrays_load.sh b/tests/queries/0_stateless/01273_arrow_arrays_load.sh new file mode 100755 index 00000000000..b8d1a85921d --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_arrays_load.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS orc_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_arrays (arr1 Array(Int8), arr2 Array(UInt8), arr3 Array(Int16), arr4 Array(UInt16), arr5 Array(Int32), arr6 Array(UInt32), arr7 Array(Int64), arr8 Array(UInt64), arr9 Array(String), arr10 Array(FixedString(4)), arr11 Array(Float32), arr12 Array(Float64), arr13 Array(Date), arr14 Array(Datetime)) ENGINE=Memory()" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_arrays VALUES ([1,-2,3],[1,2,3],[100,-200,300],[100,200,300],[10000000,-20000000,30000000],[10000000,2000000,3000000],[100000000000000,-200000000000,3000000000000],[100000000000000,20000000000000,3000000000000],['Some string','Some string','Some string'],['0000','1111','2222'],[42.42,424.2,0.4242],[424242.424242,4242042420.242424,42],['2000-01-01','2001-01-01','2002-01-01'],['2000-01-01 00:00:00','2001-01-01 00:00:00','2002-01-01 00:00:00']),([],[],[],[],[],[],[],[],[],[],[],[],[],[])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_arrays FORMAT Arrow" > "${CLICKHOUSE_TMP}"/arrays.arrow + +cat "${CLICKHOUSE_TMP}"/arrays.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_arrays FORMAT Arrow" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_arrays" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_arrays" diff --git a/tests/queries/0_stateless/01273_arrow_nested_arrays_load.reference b/tests/queries/0_stateless/01273_arrow_nested_arrays_load.reference new file mode 100644 index 00000000000..dd9c9900684 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_nested_arrays_load.reference @@ -0,0 +1,2 @@ +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] +[[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]] [[['Some string','Some string'],[]],[['Some string']],[[]]] [[NULL,1,2],[NULL],[1,2],[]] [['Some string',NULL,'Some string'],[NULL],[]] diff --git a/tests/queries/0_stateless/01273_arrow_nested_arrays_load.sh b/tests/queries/0_stateless/01273_arrow_nested_arrays_load.sh new file mode 100755 index 00000000000..cd2040260e9 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_nested_arrays_load.sh @@ -0,0 +1,17 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_nested_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_nested_arrays (arr1 Array(Array(Array(UInt32))), arr2 Array(Array(Array(String))), arr3 Array(Array(Nullable(UInt32))), arr4 Array(Array(Nullable(String)))) engine=Memory()" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_nested_arrays VALUES ([[[1,2,3],[1,2,3]],[[1,2,3]],[[],[1,2,3]]],[[['Some string','Some string'],[]],[['Some string']],[[]]],[[NULL,1,2],[NULL],[1,2],[]],[['Some string',NULL,'Some string'],[NULL],[]])" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_nested_arrays FORMAT Arrow" > "${CLICKHOUSE_TMP}"/nested_arrays.arrow + +cat "${CLICKHOUSE_TMP}"/nested_arrays.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_nested_arrays FORMAT Arrow" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_nested_arrays" +${CLICKHOUSE_CLIENT} --query="DROP table arrow_nested_arrays" diff --git a/tests/queries/0_stateless/01273_arrow_nullable_arrays_load.reference b/tests/queries/0_stateless/01273_arrow_nullable_arrays_load.reference new file mode 100644 index 00000000000..44b19f616d0 --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_nullable_arrays_load.reference @@ -0,0 +1,6 @@ +[1,NULL,2] [NULL,'Some string',NULL] [0,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] +[1,NULL,2] [NULL,'Some string',NULL] [0,NULL,42.42] +[NULL] [NULL] [NULL] +[] [] [] diff --git a/tests/queries/0_stateless/01273_arrow_nullable_arrays_load.sh b/tests/queries/0_stateless/01273_arrow_nullable_arrays_load.sh new file mode 100755 index 00000000000..d780cc6d06a --- /dev/null +++ b/tests/queries/0_stateless/01273_arrow_nullable_arrays_load.sh @@ -0,0 +1,15 @@ +#!/usr/bin/env bash + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS arrow_nullable_arrays" +${CLICKHOUSE_CLIENT} --query="CREATE TABLE arrow_nullable_arrays (arr1 Array(Nullable(UInt32)), arr2 Array(Nullable(String)), arr3 Array(Nullable(Float32))) ENGINE=Memory()" +${CLICKHOUSE_CLIENT} --query="INSERT INTO arrow_nullable_arrays VALUES ([1,NULL,2],[NULL,'Some string',NULL],[0.00,NULL,42.42]), ([NULL],[NULL],[NULL]), ([],[],[])" +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_nullable_arrays FORMAT Arrow" > "${CLICKHOUSE_TMP}"/nullable_arrays.arrow + +cat "${CLICKHOUSE_TMP}"/nullable_arrays.arrow | ${CLICKHOUSE_CLIENT} -q "INSERT INTO arrow_nullable_arrays FORMAT Arrow" + +${CLICKHOUSE_CLIENT} --query="SELECT * FROM arrow_nullable_arrays" +${CLICKHOUSE_CLIENT} --query="DROP TABLE arrow_nullable_arrays" From 6f16d92cfe843bc153dc6b6c26dd47af673df002 Mon Sep 17 00:00:00 2001 From: Alexander Kuzmenkov <36882414+akuzm@users.noreply.github.com> Date: Fri, 14 May 2021 15:50:30 +0300 Subject: [PATCH 73/82] Update Settings.h --- src/Core/Settings.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Core/Settings.h b/src/Core/Settings.h index c49e3884c31..f25393c9502 100644 --- a/src/Core/Settings.h +++ b/src/Core/Settings.h @@ -104,7 +104,7 @@ class IColumn; M(Bool, compile_expressions, false, "Compile some scalar functions and operators to native code.", 0) \ M(UInt64, min_count_to_compile_expression, 3, "The number of identical expressions before they are JIT-compiled", 0) \ M(UInt64, group_by_two_level_threshold, 100000, "From what number of keys, a two-level aggregation starts. 0 - the threshold is not set.", 0) \ - M(UInt64, group_by_two_level_threshold_bytes, 10000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ + M(UInt64, group_by_two_level_threshold_bytes, 50000000, "From what size of the aggregation state in bytes, a two-level aggregation begins to be used. 0 - the threshold is not set. Two-level aggregation is used when at least one of the thresholds is triggered.", 0) \ M(Bool, distributed_aggregation_memory_efficient, true, "Is the memory-saving mode of distributed aggregation enabled.", 0) \ M(UInt64, aggregation_memory_efficient_merge_threads, 0, "Number of threads to use for merge intermediate aggregation results in memory efficient mode. When bigger, then more memory is consumed. 0 means - same as 'max_threads'.", 0) \ \ From 4fa9649fa02d948dec10634adcd8377115c5400a Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 14 May 2021 19:22:24 +0300 Subject: [PATCH 74/82] Update clickhouse-format.md --- docs/en/operations/utilities/clickhouse-format.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/en/operations/utilities/clickhouse-format.md b/docs/en/operations/utilities/clickhouse-format.md index 33b528983b6..17948dce82d 100644 --- a/docs/en/operations/utilities/clickhouse-format.md +++ b/docs/en/operations/utilities/clickhouse-format.md @@ -41,12 +41,11 @@ $ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNIO Result: ```text -SELECT 1 -; - -SELECT 1 -UNION ALL +SELECT * +FROM ( + SELECT 1 AS x + UNION ALL SELECT 1 UNION DISTINCT SELECT 3 From 6062c7b5e59515fc3bc29d03a8475cd2d8c0fd52 Mon Sep 17 00:00:00 2001 From: alesapin Date: Fri, 14 May 2021 19:22:54 +0300 Subject: [PATCH 75/82] Update clickhouse-format.md --- docs/ru/operations/utilities/clickhouse-format.md | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/ru/operations/utilities/clickhouse-format.md b/docs/ru/operations/utilities/clickhouse-format.md index ca58b4a3b93..43043fcc1d5 100644 --- a/docs/ru/operations/utilities/clickhouse-format.md +++ b/docs/ru/operations/utilities/clickhouse-format.md @@ -41,12 +41,11 @@ $ clickhouse-format -n <<< "SELECT * FROM (SELECT 1 AS x UNION ALL SELECT 1 UNIO Результат: ```text -SELECT 1 -; - -SELECT 1 -UNION ALL +SELECT * +FROM ( + SELECT 1 AS x + UNION ALL SELECT 1 UNION DISTINCT SELECT 3 From 6028eb1c88e93c22d2b6c90cc174cb338ec05d01 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Thu, 13 May 2021 19:33:07 +0300 Subject: [PATCH 76/82] done --- CHANGELOG.md | 145 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 145 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index cc1ec835a7b..6a2c7cb66b8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,3 +1,148 @@ +## ClickHouse release 21.5 + +#### Backward Incompatible Change + +* Change comparison of integers and floating point numbers when integer is not exactly representable in the floating point data type. In new version comparison will return false as the rounding error will occur. Example: `9223372036854775808.0 != 9223372036854775808`, because the number `9223372036854775808` is not representable as floating point number exactly (and `9223372036854775808.0` is rounded to `9223372036854776000.0`). But in previous version the comparison will return as the numbers are equal, because if the floating point number `9223372036854776000.0` get converted back to UInt64, it will yield `9223372036854775808`. For the reference, the Python programming language also treats these numbers as equal. But this behaviour was dependend on CPU model (different results on AMD64 and AArch64 for some out-of-range numbers), so we make the comparison more precise. It will treat int and float numbers equal only if int is represented in floating point type exactly. [#22595](https://github.com/ClickHouse/ClickHouse/pull/22595) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Now replicas that are processing the `ALTER TABLE ATTACH PART[ITION]` command search in their `detached/` folders before fetching the data from other replicas. As an implementation detail, a new command `ATTACH_PART` is introduced in the replicated log. Parts are searched and compared by their checksums. [#18978](https://github.com/ClickHouse/ClickHouse/pull/18978) ([Mike Kot](https://github.com/myrrc)). + +#### New Feature + +* Added `Decimal256` type support in dictionaries. Closes [#20979](https://github.com/ClickHouse/ClickHouse/issues/20979). [#22960](https://github.com/ClickHouse/ClickHouse/pull/22960) ([Maksim Kita](https://github.com/kitaisreal)). +* Added `ALTER TABLE ... FETCH PART ...` query. It's similar to `FETCH PARTITION`, but fetches only one part. [#22706](https://github.com/ClickHouse/ClickHouse/pull/22706) ([turbo jason](https://github.com/songenjie)). +* FlatDictionary added `initial_array_size`, `max_array_size` options. [#22521](https://github.com/ClickHouse/ClickHouse/pull/22521) ([Maksim Kita](https://github.com/kitaisreal)). +* Add new setting `non_replicated_deduplication_window` for non-replicated MergeTree inserts deduplication. [#22514](https://github.com/ClickHouse/ClickHouse/pull/22514) ([alesapin](https://github.com/alesapin)). +* Update paths to the `CatBoost` model configs in config reloading. [#22434](https://github.com/ClickHouse/ClickHouse/pull/22434) ([Kruglov Pavel](https://github.com/Avogar)). +* Added function `dictGetOrNull`. It works like `dictGet`, but return `Null` in case key was not found in dictionary. Closes [#22375](https://github.com/ClickHouse/ClickHouse/issues/22375). [#22413](https://github.com/ClickHouse/ClickHouse/pull/22413) ([Maksim Kita](https://github.com/kitaisreal)). +* Added support for replicas and shards in MySQL/PostgreSQL table engine / table function. Replicas and shards, if used together, are added via new table engine ExternalDistributed. Closes [#20969](https://github.com/ClickHouse/ClickHouse/issues/20969). [#22217](https://github.com/ClickHouse/ClickHouse/pull/22217) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Improved performance of `dictGetHierarchy`, `dictIsIn` functions. Added functions `dictGetChildren(dictionary, key)`, `dictGetDescendants(dictionary, key, level)`. Function `dictGetChildren` return all children as an array if indexes. It is a inverse transformation for `dictGetHierarchy`. Function `dictGetDescendants` return all descendants as if `dictGetChildren` was applied `level` times recursively. Zero `level` value is equivalent to infinity. Closes [#14656](https://github.com/ClickHouse/ClickHouse/issues/14656). [#22096](https://github.com/ClickHouse/ClickHouse/pull/22096) ([Maksim Kita](https://github.com/kitaisreal)). +* Added a table function `s3Cluster`, which allows to process files from `s3` in parallel on every node of a specified cluster. [#22012](https://github.com/ClickHouse/ClickHouse/pull/22012) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Added a setting `max_distributed_depth` that limits the depth of recursive queries to `Distributed` tables. Closes [#20229](https://github.com/ClickHouse/ClickHouse/issues/20229). [#21942](https://github.com/ClickHouse/ClickHouse/pull/21942) ([flynn](https://github.com/ucasFL)). +* Added function `arrayFold(x1,...,xn,accum -> expression, array1,...,arrayn, init_accum)` that applies the expression to each element of the array (or set of parallel arrays) and collect result in accumulator. [#21589](https://github.com/ClickHouse/ClickHouse/pull/21589) ([Dmitry Krylov](https://github.com/dmalkr)). + +#### Performance Improvement + +* Improved performance of `intDiv` by dynamic dispatch for AVX2. This closes [#22314](https://github.com/ClickHouse/ClickHouse/issues/22314). [#23000](https://github.com/ClickHouse/ClickHouse/pull/23000) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Improved performance of reading from `ArrowStream` input format for sources other then local file (e.g. URL). [#22673](https://github.com/ClickHouse/ClickHouse/pull/22673) ([nvartolomei](https://github.com/nvartolomei)). +* Disabled compression by default when interacting with localhost (with clickhouse-client or server to server with distributed queries) via native protocol. It may improve performance of some import/export operations. This closes [#22234](https://github.com/ClickHouse/ClickHouse/issues/22234). [#22237](https://github.com/ClickHouse/ClickHouse/pull/22237) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Exclude values that does not belong to the shard from right part of IN section for distributed queries. [#21511](https://github.com/ClickHouse/ClickHouse/pull/21511) ([Azat Khuzhin](https://github.com/azat)). +* Improved performance of reading a subset of columns with File-like table engine and column-oriented format like Parquet, Arrow or ORC. This closes [#issue:20129](https://github.com/ClickHouse/ClickHouse/issues/20129). [#21302](https://github.com/ClickHouse/ClickHouse/pull/21302) ([keenwolf](https://github.com/keen-wolf)). + +#### Improvement + +* Enabled `async_socket_for_remote` by default. [#23683](https://github.com/ClickHouse/ClickHouse/pull/23683) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed `quantile(s)TDigest`. Added special handling of singleton centroids according to tdunning/t-digest 3.2+. Also a bug with over-compression of centroids in implementation of earlier version of the algorithm was fixed. [#23314](https://github.com/ClickHouse/ClickHouse/pull/23314) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Disable settings `use_hedged_requests` and `async_socket_for_remote` because there is an evidence that it may cause issues. [#23261](https://github.com/ClickHouse/ClickHouse/pull/23261) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make function `unhex` case insensitive for compatibility with MySQL. [#23229](https://github.com/ClickHouse/ClickHouse/pull/23229) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed memory tracking with min_bytes_to_use_mmap_io. [#23211](https://github.com/ClickHouse/ClickHouse/pull/23211) ([Azat Khuzhin](https://github.com/azat)). +* Implement functions `arrayHasAny`, `arrayHasAll`, `has`, `indexOf`, `countEqual` for generic case when types of array elements are different. In previous versions the functions `arrayHasAny`, `arrayHasAll` returned false and `has`, `indexOf`, `countEqual` thrown exception. Also add support for `Decimal` and big integer types in functions `has` and similar. This closes [#20272](https://github.com/ClickHouse/ClickHouse/issues/20272). [#23044](https://github.com/ClickHouse/ClickHouse/pull/23044) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Raised the threshold on max number of matches in result of the function `extractAllGroupsHorizontal`. [#23036](https://github.com/ClickHouse/ClickHouse/pull/23036) ([Vasily Nemkov](https://github.com/Enmk)). +* Do not perform `optimize_skip_unused_shards` for cluster with one node. [#22999](https://github.com/ClickHouse/ClickHouse/pull/22999) ([Azat Khuzhin](https://github.com/azat)). +* Increase `max_uri_size` (the maximum size of URL in HTTP interface) to 1 MiB by default. This closes [#21197](https://github.com/ClickHouse/ClickHouse/issues/21197). [#22997](https://github.com/ClickHouse/ClickHouse/pull/22997) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added ability to run clickhouse-keeper with SSL. Config settings `keeper_server.tcp_port_secure` can be used for secure interaction between client and keeper-server. `keeper_server.raft_configuration.secure` can be used to enable internal secure communication between nodes. [#22992](https://github.com/ClickHouse/ClickHouse/pull/22992) ([alesapin](https://github.com/alesapin)). +* Added ability to flush buffer only in background for StorageBuffer. [#22986](https://github.com/ClickHouse/ClickHouse/pull/22986) ([Azat Khuzhin](https://github.com/azat)). +* When selecting from MergeTree table with NULL in WHERE condition, in rare cases, exception was thrown. This closes [#20019](https://github.com/ClickHouse/ClickHouse/issues/20019). [#22978](https://github.com/ClickHouse/ClickHouse/pull/22978) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed an error handling in Poco HTTP Client for AWS. [#22973](https://github.com/ClickHouse/ClickHouse/pull/22973) ([kreuzerkrieg](https://github.com/kreuzerkrieg)). +* Respect `max_part_removal_threads` for ReplicatedMergeTree. [#22971](https://github.com/ClickHouse/ClickHouse/pull/22971) ([Azat Khuzhin](https://github.com/azat)). +* Fix inactive_parts_to_throw_insert=0 with inactive_parts_to_delay_insert>0. [#22947](https://github.com/ClickHouse/ClickHouse/pull/22947) ([Azat Khuzhin](https://github.com/azat)). +* Set `background_fetches_pool_size` to 8 that is better for production usage with frequent small insertions or slow ZooKeeper cluster. [#22945](https://github.com/ClickHouse/ClickHouse/pull/22945) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* `dateDiff` now works with `DateTime64` arguments (even for values outside of `DateTime` range) [#22931](https://github.com/ClickHouse/ClickHouse/pull/22931) ([Vasily Nemkov](https://github.com/Enmk)). +* Added an ability to replicate MySQL databases containing views without failing. This is accomplished by ignoring the views. [#22760](https://github.com/ClickHouse/ClickHouse/pull/22760) ([Christian](https://github.com/cfroystad)). +* Allow RBAC row policy via postgresql protocol. Closes [#22658](https://github.com/ClickHouse/ClickHouse/issues/22658). PostgreSQL protocol is enabled in configuration by default. [#22755](https://github.com/ClickHouse/ClickHouse/pull/22755) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Add metric to track how much time is spend during waiting for Buffer layer lock. [#22725](https://github.com/ClickHouse/ClickHouse/pull/22725) ([Azat Khuzhin](https://github.com/azat)). +* Allow to use CTE in VIEW definition. This closes [#22491](https://github.com/ClickHouse/ClickHouse/issues/22491). [#22657](https://github.com/ClickHouse/ClickHouse/pull/22657) ([Amos Bird](https://github.com/amosbird)). +* Clear the rest of the screen and show cursor in `clickhouse-client` if previous program has left garbage in terminal. This closes [#16518](https://github.com/ClickHouse/ClickHouse/issues/16518). [#22634](https://github.com/ClickHouse/ClickHouse/pull/22634) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Make `round` function to behave consistently on non-x86_64 platforms. Rounding half to nearest even (Banker's rounding) is used. [#22582](https://github.com/ClickHouse/ClickHouse/pull/22582) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Correctly check structure of async distributed blocks. [#22325](https://github.com/ClickHouse/ClickHouse/pull/22325) ([Azat Khuzhin](https://github.com/azat)). +* Added nanodbc instead of Poco::ODBC. Closes [#9678](https://github.com/ClickHouse/ClickHouse/issues/9678). Add support for DateTime64 and Decimal* for ODBC table engine. Closes [#21961](https://github.com/ClickHouse/ClickHouse/issues/21961). Fixed issue with cyrillic text being truncated. Closes [#16246](https://github.com/ClickHouse/ClickHouse/issues/16246). Added connection pools for odbc bridge. [#21972](https://github.com/ClickHouse/ClickHouse/pull/21972) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Allow publishing Kafka errors to a virtual column of Kafka engine, controlled by the `kafka_handle_error_mode` setting. [#21850](https://github.com/ClickHouse/ClickHouse/pull/21850) ([fastio](https://github.com/fastio)). +* Add aliases `simpleJSONExtract/simpleJSONHas` to `visitParam/visitParamExtract{UInt, Int, Bool, Float, Raw, String}`. Fixes [#21383](https://github.com/ClickHouse/ClickHouse/issues/21383). [#21519](https://github.com/ClickHouse/ClickHouse/pull/21519) ([fastio](https://github.com/fastio)). +* Add `clickhouse-library-bridge` for library dictionary source. Closes [#9502](https://github.com/ClickHouse/ClickHouse/issues/9502). [#21509](https://github.com/ClickHouse/ClickHouse/pull/21509) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Forbid to drop a column if it's referenced by materialized view. Closes [#21164](https://github.com/ClickHouse/ClickHouse/issues/21164). [#21303](https://github.com/ClickHouse/ClickHouse/pull/21303) ([flynn](https://github.com/ucasFL)). +* Support dynamic interserver credentials. [#14113](https://github.com/ClickHouse/ClickHouse/pull/14113) ([johnskopis](https://github.com/johnskopis)). + +#### Bug Fix + +* HashedDictionary complex key update field initial load fix. Closes [#23800](https://github.com/ClickHouse/ClickHouse/issues/23800). [#23824](https://github.com/ClickHouse/ClickHouse/pull/23824) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed an error `Can't initialize pipeline with empty pipe` for queries with `GLOBAL IN/JOIN` and `use_hedged_requests`. Fixes [#23431](https://github.com/ClickHouse/ClickHouse/issues/23431). [#23805](https://github.com/ClickHouse/ClickHouse/pull/23805) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed remote JDBC bridge timeout connection issue. Closes [#9609](https://github.com/ClickHouse/ClickHouse/issues/9609). [#23771](https://github.com/ClickHouse/ClickHouse/pull/23771) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed crash when `PREWHERE` and row policy filter are both in effect with empty result. [#23763](https://github.com/ClickHouse/ClickHouse/pull/23763) ([Amos Bird](https://github.com/amosbird)). +* Avoid possible "Cannot schedule a task" error (in case some exception had been occurred) on INSERT into Distributed. [#23744](https://github.com/ClickHouse/ClickHouse/pull/23744) ([Azat Khuzhin](https://github.com/azat)). +* Added an exception in case of completely the same values in both samples in aggregate function `mannWhitneyUTest`. This fixes [#23646](https://github.com/ClickHouse/ClickHouse/issues/23646). [#23654](https://github.com/ClickHouse/ClickHouse/pull/23654) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed server fault when inserting data through HTTP caused an exception. This fixes [#23512](https://github.com/ClickHouse/ClickHouse/issues/23512). [#23643](https://github.com/ClickHouse/ClickHouse/pull/23643) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed misinterpretation of some `LIKE` expressions with escape sequences. [#23610](https://github.com/ClickHouse/ClickHouse/pull/23610) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed restart / stop command hanging. Closes [#20214](https://github.com/ClickHouse/ClickHouse/issues/20214). [#23552](https://github.com/ClickHouse/ClickHouse/pull/23552) ([filimonov](https://github.com/filimonov)). +* Fixed `columns` function when multiple joins in select query. Closes [#22736](https://github.com/ClickHouse/ClickHouse/issues/22736). [#23501](https://github.com/ClickHouse/ClickHouse/pull/23501) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed a crash when modifying column's default value when a column itself is used as ReplacingMergeTree's parameter. [#23483](https://github.com/ClickHouse/ClickHouse/pull/23483) ([hexiaoting](https://github.com/hexiaoting)). +* Fixed corner cases in vertical merges with `ReplacingMergeTree`. In rare cases they could lead to fails of merges with exceptions like `Incomplete granules are not allowed while blocks are granules size`. [#23459](https://github.com/ClickHouse/ClickHouse/pull/23459) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed bug that does not allow cast from empty array literal, to array with dimensions greater than 1. Closes [#14476](https://github.com/ClickHouse/ClickHouse/issues/14476). [#23456](https://github.com/ClickHouse/ClickHouse/pull/23456) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed a bug when `deltaSum` aggregate function produced incorrect result after resetting the counter. [#23437](https://github.com/ClickHouse/ClickHouse/pull/23437) ([Russ Frank](https://github.com/rf)). +* Fixed `Cannot unlink file` error on unsuccessful creation of ReplicatedMergeTree table with multidisk configuration. This closes [#21755](https://github.com/ClickHouse/ClickHouse/issues/21755). [#23433](https://github.com/ClickHouse/ClickHouse/pull/23433) ([tavplubix](https://github.com/tavplubix)). +* Fixed a lack of support for Kafka storage with `arrow` and `arrowstream` format messages. [#23415](https://github.com/ClickHouse/ClickHouse/pull/23415) ([Chao Ma](https://github.com/godliness)). +* Allow to move more conditions to `PREWHERE` as it was before version 21.1. Insufficient number of moved condtions could lead to worse performance. [#23397](https://github.com/ClickHouse/ClickHouse/pull/23397) ([Anton Popov](https://github.com/CurtizJ)). +* Remove support for `argMin` and `argMax` for single `Tuple` argument. The code was not memory-safe. The feature was added by mistake and it is confusing for people. These functions can be reintroduced under different names later. This fixes [#22384](https://github.com/ClickHouse/ClickHouse/issues/22384) and reverts [#17359](https://github.com/ClickHouse/ClickHouse/issues/17359). [#23393](https://github.com/ClickHouse/ClickHouse/pull/23393) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed incompatible constant expression generation during partition pruning based on virtual columns. This fixes https://github.com/ClickHouse/ClickHouse/pull/21401#discussion_r611888913. [#23366](https://github.com/ClickHouse/ClickHouse/pull/23366) ([Amos Bird](https://github.com/amosbird)). +* Fixed a bug in dict join with join_algorithm = 'auto'. Close [#23002](https://github.com/ClickHouse/ClickHouse/issues/23002). [#23312](https://github.com/ClickHouse/ClickHouse/pull/23312) ([Vladimir](https://github.com/vdimir)). +* Don't relax NOT conditions during partition pruning. This fixes [#23305](https://github.com/ClickHouse/ClickHouse/issues/23305) and [#21539](https://github.com/ClickHouse/ClickHouse/issues/21539). [#23310](https://github.com/ClickHouse/ClickHouse/pull/23310) ([Amos Bird](https://github.com/amosbird)). +* Fix possible crash in case if `unknown packet` was received form remote query (with `async_socket_for_remote` enabled). Maybe fixes [#21167](https://github.com/ClickHouse/ClickHouse/issues/21167). [#23309](https://github.com/ClickHouse/ClickHouse/pull/23309) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed very rare race condition on background cleanup of old blocks. It might cause a block not to be deduplicated if it's too close to the end of deduplication window. [#23301](https://github.com/ClickHouse/ClickHouse/pull/23301) ([tavplubix](https://github.com/tavplubix)). +* Fixed very rare (distributed) race condition between creation and removal of ReplicatedMergeTree tables. It might cause exceptions like `node doesn't exist` on attempt to create replicated table. Fixes [#21419](https://github.com/ClickHouse/ClickHouse/issues/21419). [#23294](https://github.com/ClickHouse/ClickHouse/pull/23294) ([tavplubix](https://github.com/tavplubix)). +* Fixed simple key dictionary from DDL creation if primary key is not first attribute. Fixes [#23236](https://github.com/ClickHouse/ClickHouse/issues/23236). [#23262](https://github.com/ClickHouse/ClickHouse/pull/23262) ([Maksim Kita](https://github.com/kitaisreal)). +* Add type conversion for `optimize_skip_unused_shards_rewrite_in` (fixes `use-of-uninitialized-value` with `optimize_skip_unused_shards_rewrite_in`). [#23219](https://github.com/ClickHouse/ClickHouse/pull/23219) ([Azat Khuzhin](https://github.com/azat)). +* Fixed reading from ODBC when there are many long column names in a table. Closes [#8853](https://github.com/ClickHouse/ClickHouse/issues/8853). [#23215](https://github.com/ClickHouse/ClickHouse/pull/23215) ([Kseniia Sumarokova](https://github.com/kssenii)). +* Fixed missing semicolon in exception message. The user may find this exception message unpleasant to read. [#23208](https://github.com/ClickHouse/ClickHouse/pull/23208) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed missing whitespace in some exception messages about `LowCardinality` type. [#23207](https://github.com/ClickHouse/ClickHouse/pull/23207) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed the behavior when disabling `input_format_with_names_use_header ` setting discards all the input with CSVWithNames format. This fixes [#22406](https://github.com/ClickHouse/ClickHouse/issues/22406). [#23202](https://github.com/ClickHouse/ClickHouse/pull/23202) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed `Not found column` error when selecting from `MaterializeMySQL` with condition on key column. Fixes [#22432](https://github.com/ClickHouse/ClickHouse/issues/22432). [#23200](https://github.com/ClickHouse/ClickHouse/pull/23200) ([tavplubix](https://github.com/tavplubix)). +* QueryAliasVisitor to prefer alias for ASTWithAlias if subquery was optimized to constant. Fixes [#22924](https://github.com/ClickHouse/ClickHouse/issues/22924). Fixes [#10401](https://github.com/ClickHouse/ClickHouse/issues/10401). [#23191](https://github.com/ClickHouse/ClickHouse/pull/23191) ([Maksim Kita](https://github.com/kitaisreal)). +* Server might fail to start if `data_type_default_nullable` setting is enabled in default profile, it's fixed. Fixes [#22573](https://github.com/ClickHouse/ClickHouse/issues/22573). [#23185](https://github.com/ClickHouse/ClickHouse/pull/23185) ([tavplubix](https://github.com/tavplubix)). +* Fixed a crash on shutdown which happened because of wrong accounting of current connections. [#23154](https://github.com/ClickHouse/ClickHouse/pull/23154) ([Vitaly Baranov](https://github.com/vitlibar)). +* Some values were formatted with alignment in center in table cells in `Markdown` format. Not anymore. [#23096](https://github.com/ClickHouse/ClickHouse/pull/23096) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fixed `Table .inner_id... doesn't exist` error when selecting from Materialized View after detaching it from Atomic database and attaching back. [#23047](https://github.com/ClickHouse/ClickHouse/pull/23047) ([tavplubix](https://github.com/tavplubix)). +* Remove non-essential details from suggestions in clickhouse-client. This closes [#22158](https://github.com/ClickHouse/ClickHouse/issues/22158). [#23040](https://github.com/ClickHouse/ClickHouse/pull/23040) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix error `Cannot find column in ActionsDAG result` which may happen if subquery uses `untuple`. Fixes [#22290](https://github.com/ClickHouse/ClickHouse/issues/22290). [#22991](https://github.com/ClickHouse/ClickHouse/pull/22991) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fix usage of constant columns of type `Map` with nullable values. [#22939](https://github.com/ClickHouse/ClickHouse/pull/22939) ([Anton Popov](https://github.com/CurtizJ)). +* fixed `formatDateTime()` on `DateTime64` and "%C" format specifier fixed `toDateTime64()` for large values and non-zero scale. ... [#22937](https://github.com/ClickHouse/ClickHouse/pull/22937) ([Vasily Nemkov](https://github.com/Enmk)). +* Fixed a crash when using `mannWhitneyUTest` and `rankCorr` with window functions. This fixes [#22728](https://github.com/ClickHouse/ClickHouse/issues/22728). [#22876](https://github.com/ClickHouse/ClickHouse/pull/22876) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). +* Fixed `bytes_allocated` field in system.dictionaries for sparse_hashed dictionaries. [#22867](https://github.com/ClickHouse/ClickHouse/pull/22867) ([Azat Khuzhin](https://github.com/azat)). +* Fixed possible hanging in concurrent DROP/CREATE of TEMPORARY LIVE VIEW in `TemporaryLiveViewCleaner`, see https://gist.github.com/vzakaznikov/0c03195960fc86b56bfe2bc73a90019e. [#22858](https://github.com/ClickHouse/ClickHouse/pull/22858) ([Vitaly Baranov](https://github.com/vitlibar)). +* Fixed pushdown of `HAVING` in case, when filter column is used in aggregation. [#22763](https://github.com/ClickHouse/ClickHouse/pull/22763) ([Anton Popov](https://github.com/CurtizJ)). +* Fixed approximate total rows accounting for reverse reading from MergeTree. [#22726](https://github.com/ClickHouse/ClickHouse/pull/22726) ([Azat Khuzhin](https://github.com/azat)). +* Fixed possible hangs in Zookeeper requests in case of OOM exception. Fixes [#22438](https://github.com/ClickHouse/ClickHouse/issues/22438). [#22684](https://github.com/ClickHouse/ClickHouse/pull/22684) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). +* Fixed wait for mutations on several replicas for ReplicatedMergeTree table engines. Previously, mutation/alter query may finish before mutation actually executed on other replicas. [#22669](https://github.com/ClickHouse/ClickHouse/pull/22669) ([alesapin](https://github.com/alesapin)). +* Fixed LOGICAL_ERROR for Log with nested types w/o columns in the SELECT clause. [#22654](https://github.com/ClickHouse/ClickHouse/pull/22654) ([Azat Khuzhin](https://github.com/azat)). +* Fixed a bug with unlimited wait for auxiliary AWS requests. [#22594](https://github.com/ClickHouse/ClickHouse/pull/22594) ([Vladimir Chebotarev](https://github.com/excitoon)). +* Fixed a crash when client closes connection very early [#22579](https://github.com/ClickHouse/ClickHouse/issues/22579). [#22591](https://github.com/ClickHouse/ClickHouse/pull/22591) ([nvartolomei](https://github.com/nvartolomei)). +* Fixed an incorrect formatting of function `map` in distributed queries. [#22588](https://github.com/ClickHouse/ClickHouse/pull/22588) ([foolchi](https://github.com/foolchi)). +* Avoid UB in *Log engines for rwlock unlock due to unlock from another thread. [#22583](https://github.com/ClickHouse/ClickHouse/pull/22583) ([Azat Khuzhin](https://github.com/azat)). +* Fixed UB by unlocking the rwlock of the TinyLog from the same thread. [#22560](https://github.com/ClickHouse/ClickHouse/pull/22560) ([Azat Khuzhin](https://github.com/azat)). +* Fixed deserialization of empty string without newline at end of TSV format. This closes [#20244](https://github.com/ClickHouse/ClickHouse/issues/20244). Possible workaround without version update: set `input_format_null_as_default` to zero. It was zero in old versions. [#22527](https://github.com/ClickHouse/ClickHouse/pull/22527) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Fix bug in partial merge join with `LowCardinality`. Close [#22386](https://github.com/ClickHouse/ClickHouse/issues/22386), close [#22388](https://github.com/ClickHouse/ClickHouse/issues/22388). [#22510](https://github.com/ClickHouse/ClickHouse/pull/22510) ([Vladimir](https://github.com/vdimir)). +* Fixed `ClickHouseDictionarySource` configuration loop. Closes [#14314](https://github.com/ClickHouse/ClickHouse/issues/14314). [#22479](https://github.com/ClickHouse/ClickHouse/pull/22479) ([Maksim Kita](https://github.com/kitaisreal)). +* Fixed a race condition in HedgedConnections which leads to crash. This fixes [#22161](https://github.com/ClickHouse/ClickHouse/issues/22161). [#22443](https://github.com/ClickHouse/ClickHouse/pull/22443) ([Kruglov Pavel](https://github.com/Avogar)). +* Buffer overflow (on read) was possible in `tokenbf_v1` full text index. The excessive bytes are not used but the read operation may lead to crash in rare cases. This closes [#19233](https://github.com/ClickHouse/ClickHouse/issues/19233). [#22421](https://github.com/ClickHouse/ClickHouse/pull/22421) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Do not limit HTTP chunk size. Fixes [#21907](https://github.com/ClickHouse/ClickHouse/issues/21907). [#22322](https://github.com/ClickHouse/ClickHouse/pull/22322) ([Ivan](https://github.com/abyss7)). +* Fixed a bug, which leads to underaggregation of data in case of enabled `optimize_aggregation_in_order` and many parts in table. Slightly improve performance of aggregation with enabled `optimize_aggregation_in_order`. [#21889](https://github.com/ClickHouse/ClickHouse/pull/21889) ([Anton Popov](https://github.com/CurtizJ)). +* Check if table function view is used as a column. This complements https://github.com/ClickHouse/ClickHouse/pull/20350. [#21465](https://github.com/ClickHouse/ClickHouse/pull/21465) ([Amos Bird](https://github.com/amosbird)). +* * Fix "unknown column" error for tables with `Merge` engine in queris with `JOIN` and aggregation. Closes [#18368](https://github.com/ClickHouse/ClickHouse/issues/18368), close [#22226](https://github.com/ClickHouse/ClickHouse/issues/22226). [#21370](https://github.com/ClickHouse/ClickHouse/pull/21370) ([Vladimir](https://github.com/vdimir)). +* Fixed name clashes in `PredicateRewriteVisitor`. It caused incorrect `WHERE` filtration after full join. Close [#20497](https://github.com/ClickHouse/ClickHouse/issues/20497). [#20622](https://github.com/ClickHouse/ClickHouse/pull/20622) ([Vladimir](https://github.com/vdimir)). +* Fixed very rare bug when quorum insert with `quorum_parallel=1` is not really "quorum" because of deduplication. [#18215](https://github.com/ClickHouse/ClickHouse/pull/18215) ([filimonov](https://github.com/filimonov)). + +#### Build/Testing/Packaging Improvement + +* Simplify debian packages. This fixes [#21698](https://github.com/ClickHouse/ClickHouse/issues/21698). [#22976](https://github.com/ClickHouse/ClickHouse/pull/22976) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added support for ClickHouse build on Apple M1. [#21639](https://github.com/ClickHouse/ClickHouse/pull/21639) ([changvvb](https://github.com/changvvb)). +* Fixed ClickHouse Keeper build for MacOS. [#22860](https://github.com/ClickHouse/ClickHouse/pull/22860) ([alesapin](https://github.com/alesapin)). +* Fixed some tests on AArch64 platform. [#22596](https://github.com/ClickHouse/ClickHouse/pull/22596) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Added function alignment for possibly better performance. [#21431](https://github.com/ClickHouse/ClickHouse/pull/21431) ([Danila Kutenin](https://github.com/danlark1)). +* Adjust some tests to output identical results on amd64 and aarch64 (qemu). The result was depending on implementation specific CPU behaviour. [#22590](https://github.com/ClickHouse/ClickHouse/pull/22590) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow query profiling only on x86_64. See [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174)#issuecomment-812954965 and [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638)#issuecomment-703805337. This closes [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638). [#22580](https://github.com/ClickHouse/ClickHouse/pull/22580) ([alexey-milovidov](https://github.com/alexey-milovidov)). +* Allow building with unbundled xz (lzma) using USE_INTERNAL_XZ_LIBRARY=OFF ... [#22571](https://github.com/ClickHouse/ClickHouse/pull/22571) ([Kfir Itzhak](https://github.com/mastertheknife)). +* Enable the bundled openldap on ppc64le ... [#22487](https://github.com/ClickHouse/ClickHouse/pull/22487) ([Kfir Itzhak](https://github.com/mastertheknife)). +* Disable incompatible libraries (platform specific typically) on ppc64le ... [#22475](https://github.com/ClickHouse/ClickHouse/pull/22475) ([Kfir Itzhak](https://github.com/mastertheknife)). +* Add on-demand check for clickhouse Keeper. [#22373](https://github.com/ClickHouse/ClickHouse/pull/22373) ([alesapin](https://github.com/alesapin)). +* Run stateless tests in parallel in CI. Depends on [#22181](https://github.com/ClickHouse/ClickHouse/issues/22181). [#22300](https://github.com/ClickHouse/ClickHouse/pull/22300) ([alesapin](https://github.com/alesapin)). +* Build `jemalloc` with support for [heap profiling](https://github.com/jemalloc/jemalloc/wiki/Use-Case%3A-Heap-Profiling). [#22834](https://github.com/ClickHouse/ClickHouse/pull/22834) ([nvartolomei](https://github.com/nvartolomei)). + ## ClickHouse release 21.4 ### ClickHouse release 21.4.1 2021-04-12 From 703bd2aeb68580e932cf9bf3cb7801488c5e19de Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 15 May 2021 00:25:12 +0300 Subject: [PATCH 77/82] done --- tests/queries/0_stateless/arcadia_skip_list.txt | 1 + 1 file changed, 1 insertion(+) diff --git a/tests/queries/0_stateless/arcadia_skip_list.txt b/tests/queries/0_stateless/arcadia_skip_list.txt index e84934dbc34..4e8ac456455 100644 --- a/tests/queries/0_stateless/arcadia_skip_list.txt +++ b/tests/queries/0_stateless/arcadia_skip_list.txt @@ -224,6 +224,7 @@ 01305_polygons_union 01306_polygons_intersection 01702_system_query_log +01710_projection_fetch 01759_optimize_skip_unused_shards_zero_shards 01780_clickhouse_dictionary_source_loop 01790_dist_INSERT_block_structure_mismatch_types_and_names From d174afae5ad3734447f49dc2d2ec7cfd767c01f5 Mon Sep 17 00:00:00 2001 From: Nikita Mikhaylov Date: Sat, 15 May 2021 00:28:22 +0300 Subject: [PATCH 78/82] better --- CHANGELOG.md | 145 --------------------------------------------------- 1 file changed, 145 deletions(-) diff --git a/CHANGELOG.md b/CHANGELOG.md index 6a2c7cb66b8..cc1ec835a7b 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,148 +1,3 @@ -## ClickHouse release 21.5 - -#### Backward Incompatible Change - -* Change comparison of integers and floating point numbers when integer is not exactly representable in the floating point data type. In new version comparison will return false as the rounding error will occur. Example: `9223372036854775808.0 != 9223372036854775808`, because the number `9223372036854775808` is not representable as floating point number exactly (and `9223372036854775808.0` is rounded to `9223372036854776000.0`). But in previous version the comparison will return as the numbers are equal, because if the floating point number `9223372036854776000.0` get converted back to UInt64, it will yield `9223372036854775808`. For the reference, the Python programming language also treats these numbers as equal. But this behaviour was dependend on CPU model (different results on AMD64 and AArch64 for some out-of-range numbers), so we make the comparison more precise. It will treat int and float numbers equal only if int is represented in floating point type exactly. [#22595](https://github.com/ClickHouse/ClickHouse/pull/22595) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Now replicas that are processing the `ALTER TABLE ATTACH PART[ITION]` command search in their `detached/` folders before fetching the data from other replicas. As an implementation detail, a new command `ATTACH_PART` is introduced in the replicated log. Parts are searched and compared by their checksums. [#18978](https://github.com/ClickHouse/ClickHouse/pull/18978) ([Mike Kot](https://github.com/myrrc)). - -#### New Feature - -* Added `Decimal256` type support in dictionaries. Closes [#20979](https://github.com/ClickHouse/ClickHouse/issues/20979). [#22960](https://github.com/ClickHouse/ClickHouse/pull/22960) ([Maksim Kita](https://github.com/kitaisreal)). -* Added `ALTER TABLE ... FETCH PART ...` query. It's similar to `FETCH PARTITION`, but fetches only one part. [#22706](https://github.com/ClickHouse/ClickHouse/pull/22706) ([turbo jason](https://github.com/songenjie)). -* FlatDictionary added `initial_array_size`, `max_array_size` options. [#22521](https://github.com/ClickHouse/ClickHouse/pull/22521) ([Maksim Kita](https://github.com/kitaisreal)). -* Add new setting `non_replicated_deduplication_window` for non-replicated MergeTree inserts deduplication. [#22514](https://github.com/ClickHouse/ClickHouse/pull/22514) ([alesapin](https://github.com/alesapin)). -* Update paths to the `CatBoost` model configs in config reloading. [#22434](https://github.com/ClickHouse/ClickHouse/pull/22434) ([Kruglov Pavel](https://github.com/Avogar)). -* Added function `dictGetOrNull`. It works like `dictGet`, but return `Null` in case key was not found in dictionary. Closes [#22375](https://github.com/ClickHouse/ClickHouse/issues/22375). [#22413](https://github.com/ClickHouse/ClickHouse/pull/22413) ([Maksim Kita](https://github.com/kitaisreal)). -* Added support for replicas and shards in MySQL/PostgreSQL table engine / table function. Replicas and shards, if used together, are added via new table engine ExternalDistributed. Closes [#20969](https://github.com/ClickHouse/ClickHouse/issues/20969). [#22217](https://github.com/ClickHouse/ClickHouse/pull/22217) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Improved performance of `dictGetHierarchy`, `dictIsIn` functions. Added functions `dictGetChildren(dictionary, key)`, `dictGetDescendants(dictionary, key, level)`. Function `dictGetChildren` return all children as an array if indexes. It is a inverse transformation for `dictGetHierarchy`. Function `dictGetDescendants` return all descendants as if `dictGetChildren` was applied `level` times recursively. Zero `level` value is equivalent to infinity. Closes [#14656](https://github.com/ClickHouse/ClickHouse/issues/14656). [#22096](https://github.com/ClickHouse/ClickHouse/pull/22096) ([Maksim Kita](https://github.com/kitaisreal)). -* Added a table function `s3Cluster`, which allows to process files from `s3` in parallel on every node of a specified cluster. [#22012](https://github.com/ClickHouse/ClickHouse/pull/22012) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Added a setting `max_distributed_depth` that limits the depth of recursive queries to `Distributed` tables. Closes [#20229](https://github.com/ClickHouse/ClickHouse/issues/20229). [#21942](https://github.com/ClickHouse/ClickHouse/pull/21942) ([flynn](https://github.com/ucasFL)). -* Added function `arrayFold(x1,...,xn,accum -> expression, array1,...,arrayn, init_accum)` that applies the expression to each element of the array (or set of parallel arrays) and collect result in accumulator. [#21589](https://github.com/ClickHouse/ClickHouse/pull/21589) ([Dmitry Krylov](https://github.com/dmalkr)). - -#### Performance Improvement - -* Improved performance of `intDiv` by dynamic dispatch for AVX2. This closes [#22314](https://github.com/ClickHouse/ClickHouse/issues/22314). [#23000](https://github.com/ClickHouse/ClickHouse/pull/23000) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Improved performance of reading from `ArrowStream` input format for sources other then local file (e.g. URL). [#22673](https://github.com/ClickHouse/ClickHouse/pull/22673) ([nvartolomei](https://github.com/nvartolomei)). -* Disabled compression by default when interacting with localhost (with clickhouse-client or server to server with distributed queries) via native protocol. It may improve performance of some import/export operations. This closes [#22234](https://github.com/ClickHouse/ClickHouse/issues/22234). [#22237](https://github.com/ClickHouse/ClickHouse/pull/22237) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Exclude values that does not belong to the shard from right part of IN section for distributed queries. [#21511](https://github.com/ClickHouse/ClickHouse/pull/21511) ([Azat Khuzhin](https://github.com/azat)). -* Improved performance of reading a subset of columns with File-like table engine and column-oriented format like Parquet, Arrow or ORC. This closes [#issue:20129](https://github.com/ClickHouse/ClickHouse/issues/20129). [#21302](https://github.com/ClickHouse/ClickHouse/pull/21302) ([keenwolf](https://github.com/keen-wolf)). - -#### Improvement - -* Enabled `async_socket_for_remote` by default. [#23683](https://github.com/ClickHouse/ClickHouse/pull/23683) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed `quantile(s)TDigest`. Added special handling of singleton centroids according to tdunning/t-digest 3.2+. Also a bug with over-compression of centroids in implementation of earlier version of the algorithm was fixed. [#23314](https://github.com/ClickHouse/ClickHouse/pull/23314) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Disable settings `use_hedged_requests` and `async_socket_for_remote` because there is an evidence that it may cause issues. [#23261](https://github.com/ClickHouse/ClickHouse/pull/23261) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make function `unhex` case insensitive for compatibility with MySQL. [#23229](https://github.com/ClickHouse/ClickHouse/pull/23229) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed memory tracking with min_bytes_to_use_mmap_io. [#23211](https://github.com/ClickHouse/ClickHouse/pull/23211) ([Azat Khuzhin](https://github.com/azat)). -* Implement functions `arrayHasAny`, `arrayHasAll`, `has`, `indexOf`, `countEqual` for generic case when types of array elements are different. In previous versions the functions `arrayHasAny`, `arrayHasAll` returned false and `has`, `indexOf`, `countEqual` thrown exception. Also add support for `Decimal` and big integer types in functions `has` and similar. This closes [#20272](https://github.com/ClickHouse/ClickHouse/issues/20272). [#23044](https://github.com/ClickHouse/ClickHouse/pull/23044) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Raised the threshold on max number of matches in result of the function `extractAllGroupsHorizontal`. [#23036](https://github.com/ClickHouse/ClickHouse/pull/23036) ([Vasily Nemkov](https://github.com/Enmk)). -* Do not perform `optimize_skip_unused_shards` for cluster with one node. [#22999](https://github.com/ClickHouse/ClickHouse/pull/22999) ([Azat Khuzhin](https://github.com/azat)). -* Increase `max_uri_size` (the maximum size of URL in HTTP interface) to 1 MiB by default. This closes [#21197](https://github.com/ClickHouse/ClickHouse/issues/21197). [#22997](https://github.com/ClickHouse/ClickHouse/pull/22997) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added ability to run clickhouse-keeper with SSL. Config settings `keeper_server.tcp_port_secure` can be used for secure interaction between client and keeper-server. `keeper_server.raft_configuration.secure` can be used to enable internal secure communication between nodes. [#22992](https://github.com/ClickHouse/ClickHouse/pull/22992) ([alesapin](https://github.com/alesapin)). -* Added ability to flush buffer only in background for StorageBuffer. [#22986](https://github.com/ClickHouse/ClickHouse/pull/22986) ([Azat Khuzhin](https://github.com/azat)). -* When selecting from MergeTree table with NULL in WHERE condition, in rare cases, exception was thrown. This closes [#20019](https://github.com/ClickHouse/ClickHouse/issues/20019). [#22978](https://github.com/ClickHouse/ClickHouse/pull/22978) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed an error handling in Poco HTTP Client for AWS. [#22973](https://github.com/ClickHouse/ClickHouse/pull/22973) ([kreuzerkrieg](https://github.com/kreuzerkrieg)). -* Respect `max_part_removal_threads` for ReplicatedMergeTree. [#22971](https://github.com/ClickHouse/ClickHouse/pull/22971) ([Azat Khuzhin](https://github.com/azat)). -* Fix inactive_parts_to_throw_insert=0 with inactive_parts_to_delay_insert>0. [#22947](https://github.com/ClickHouse/ClickHouse/pull/22947) ([Azat Khuzhin](https://github.com/azat)). -* Set `background_fetches_pool_size` to 8 that is better for production usage with frequent small insertions or slow ZooKeeper cluster. [#22945](https://github.com/ClickHouse/ClickHouse/pull/22945) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* `dateDiff` now works with `DateTime64` arguments (even for values outside of `DateTime` range) [#22931](https://github.com/ClickHouse/ClickHouse/pull/22931) ([Vasily Nemkov](https://github.com/Enmk)). -* Added an ability to replicate MySQL databases containing views without failing. This is accomplished by ignoring the views. [#22760](https://github.com/ClickHouse/ClickHouse/pull/22760) ([Christian](https://github.com/cfroystad)). -* Allow RBAC row policy via postgresql protocol. Closes [#22658](https://github.com/ClickHouse/ClickHouse/issues/22658). PostgreSQL protocol is enabled in configuration by default. [#22755](https://github.com/ClickHouse/ClickHouse/pull/22755) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Add metric to track how much time is spend during waiting for Buffer layer lock. [#22725](https://github.com/ClickHouse/ClickHouse/pull/22725) ([Azat Khuzhin](https://github.com/azat)). -* Allow to use CTE in VIEW definition. This closes [#22491](https://github.com/ClickHouse/ClickHouse/issues/22491). [#22657](https://github.com/ClickHouse/ClickHouse/pull/22657) ([Amos Bird](https://github.com/amosbird)). -* Clear the rest of the screen and show cursor in `clickhouse-client` if previous program has left garbage in terminal. This closes [#16518](https://github.com/ClickHouse/ClickHouse/issues/16518). [#22634](https://github.com/ClickHouse/ClickHouse/pull/22634) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Make `round` function to behave consistently on non-x86_64 platforms. Rounding half to nearest even (Banker's rounding) is used. [#22582](https://github.com/ClickHouse/ClickHouse/pull/22582) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Correctly check structure of async distributed blocks. [#22325](https://github.com/ClickHouse/ClickHouse/pull/22325) ([Azat Khuzhin](https://github.com/azat)). -* Added nanodbc instead of Poco::ODBC. Closes [#9678](https://github.com/ClickHouse/ClickHouse/issues/9678). Add support for DateTime64 and Decimal* for ODBC table engine. Closes [#21961](https://github.com/ClickHouse/ClickHouse/issues/21961). Fixed issue with cyrillic text being truncated. Closes [#16246](https://github.com/ClickHouse/ClickHouse/issues/16246). Added connection pools for odbc bridge. [#21972](https://github.com/ClickHouse/ClickHouse/pull/21972) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Allow publishing Kafka errors to a virtual column of Kafka engine, controlled by the `kafka_handle_error_mode` setting. [#21850](https://github.com/ClickHouse/ClickHouse/pull/21850) ([fastio](https://github.com/fastio)). -* Add aliases `simpleJSONExtract/simpleJSONHas` to `visitParam/visitParamExtract{UInt, Int, Bool, Float, Raw, String}`. Fixes [#21383](https://github.com/ClickHouse/ClickHouse/issues/21383). [#21519](https://github.com/ClickHouse/ClickHouse/pull/21519) ([fastio](https://github.com/fastio)). -* Add `clickhouse-library-bridge` for library dictionary source. Closes [#9502](https://github.com/ClickHouse/ClickHouse/issues/9502). [#21509](https://github.com/ClickHouse/ClickHouse/pull/21509) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Forbid to drop a column if it's referenced by materialized view. Closes [#21164](https://github.com/ClickHouse/ClickHouse/issues/21164). [#21303](https://github.com/ClickHouse/ClickHouse/pull/21303) ([flynn](https://github.com/ucasFL)). -* Support dynamic interserver credentials. [#14113](https://github.com/ClickHouse/ClickHouse/pull/14113) ([johnskopis](https://github.com/johnskopis)). - -#### Bug Fix - -* HashedDictionary complex key update field initial load fix. Closes [#23800](https://github.com/ClickHouse/ClickHouse/issues/23800). [#23824](https://github.com/ClickHouse/ClickHouse/pull/23824) ([Maksim Kita](https://github.com/kitaisreal)). -* Fixed an error `Can't initialize pipeline with empty pipe` for queries with `GLOBAL IN/JOIN` and `use_hedged_requests`. Fixes [#23431](https://github.com/ClickHouse/ClickHouse/issues/23431). [#23805](https://github.com/ClickHouse/ClickHouse/pull/23805) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed remote JDBC bridge timeout connection issue. Closes [#9609](https://github.com/ClickHouse/ClickHouse/issues/9609). [#23771](https://github.com/ClickHouse/ClickHouse/pull/23771) ([Maksim Kita](https://github.com/kitaisreal)). -* Fixed crash when `PREWHERE` and row policy filter are both in effect with empty result. [#23763](https://github.com/ClickHouse/ClickHouse/pull/23763) ([Amos Bird](https://github.com/amosbird)). -* Avoid possible "Cannot schedule a task" error (in case some exception had been occurred) on INSERT into Distributed. [#23744](https://github.com/ClickHouse/ClickHouse/pull/23744) ([Azat Khuzhin](https://github.com/azat)). -* Added an exception in case of completely the same values in both samples in aggregate function `mannWhitneyUTest`. This fixes [#23646](https://github.com/ClickHouse/ClickHouse/issues/23646). [#23654](https://github.com/ClickHouse/ClickHouse/pull/23654) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed server fault when inserting data through HTTP caused an exception. This fixes [#23512](https://github.com/ClickHouse/ClickHouse/issues/23512). [#23643](https://github.com/ClickHouse/ClickHouse/pull/23643) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed misinterpretation of some `LIKE` expressions with escape sequences. [#23610](https://github.com/ClickHouse/ClickHouse/pull/23610) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed restart / stop command hanging. Closes [#20214](https://github.com/ClickHouse/ClickHouse/issues/20214). [#23552](https://github.com/ClickHouse/ClickHouse/pull/23552) ([filimonov](https://github.com/filimonov)). -* Fixed `columns` function when multiple joins in select query. Closes [#22736](https://github.com/ClickHouse/ClickHouse/issues/22736). [#23501](https://github.com/ClickHouse/ClickHouse/pull/23501) ([Maksim Kita](https://github.com/kitaisreal)). -* Fixed a crash when modifying column's default value when a column itself is used as ReplacingMergeTree's parameter. [#23483](https://github.com/ClickHouse/ClickHouse/pull/23483) ([hexiaoting](https://github.com/hexiaoting)). -* Fixed corner cases in vertical merges with `ReplacingMergeTree`. In rare cases they could lead to fails of merges with exceptions like `Incomplete granules are not allowed while blocks are granules size`. [#23459](https://github.com/ClickHouse/ClickHouse/pull/23459) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed bug that does not allow cast from empty array literal, to array with dimensions greater than 1. Closes [#14476](https://github.com/ClickHouse/ClickHouse/issues/14476). [#23456](https://github.com/ClickHouse/ClickHouse/pull/23456) ([Maksim Kita](https://github.com/kitaisreal)). -* Fixed a bug when `deltaSum` aggregate function produced incorrect result after resetting the counter. [#23437](https://github.com/ClickHouse/ClickHouse/pull/23437) ([Russ Frank](https://github.com/rf)). -* Fixed `Cannot unlink file` error on unsuccessful creation of ReplicatedMergeTree table with multidisk configuration. This closes [#21755](https://github.com/ClickHouse/ClickHouse/issues/21755). [#23433](https://github.com/ClickHouse/ClickHouse/pull/23433) ([tavplubix](https://github.com/tavplubix)). -* Fixed a lack of support for Kafka storage with `arrow` and `arrowstream` format messages. [#23415](https://github.com/ClickHouse/ClickHouse/pull/23415) ([Chao Ma](https://github.com/godliness)). -* Allow to move more conditions to `PREWHERE` as it was before version 21.1. Insufficient number of moved condtions could lead to worse performance. [#23397](https://github.com/ClickHouse/ClickHouse/pull/23397) ([Anton Popov](https://github.com/CurtizJ)). -* Remove support for `argMin` and `argMax` for single `Tuple` argument. The code was not memory-safe. The feature was added by mistake and it is confusing for people. These functions can be reintroduced under different names later. This fixes [#22384](https://github.com/ClickHouse/ClickHouse/issues/22384) and reverts [#17359](https://github.com/ClickHouse/ClickHouse/issues/17359). [#23393](https://github.com/ClickHouse/ClickHouse/pull/23393) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed incompatible constant expression generation during partition pruning based on virtual columns. This fixes https://github.com/ClickHouse/ClickHouse/pull/21401#discussion_r611888913. [#23366](https://github.com/ClickHouse/ClickHouse/pull/23366) ([Amos Bird](https://github.com/amosbird)). -* Fixed a bug in dict join with join_algorithm = 'auto'. Close [#23002](https://github.com/ClickHouse/ClickHouse/issues/23002). [#23312](https://github.com/ClickHouse/ClickHouse/pull/23312) ([Vladimir](https://github.com/vdimir)). -* Don't relax NOT conditions during partition pruning. This fixes [#23305](https://github.com/ClickHouse/ClickHouse/issues/23305) and [#21539](https://github.com/ClickHouse/ClickHouse/issues/21539). [#23310](https://github.com/ClickHouse/ClickHouse/pull/23310) ([Amos Bird](https://github.com/amosbird)). -* Fix possible crash in case if `unknown packet` was received form remote query (with `async_socket_for_remote` enabled). Maybe fixes [#21167](https://github.com/ClickHouse/ClickHouse/issues/21167). [#23309](https://github.com/ClickHouse/ClickHouse/pull/23309) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed very rare race condition on background cleanup of old blocks. It might cause a block not to be deduplicated if it's too close to the end of deduplication window. [#23301](https://github.com/ClickHouse/ClickHouse/pull/23301) ([tavplubix](https://github.com/tavplubix)). -* Fixed very rare (distributed) race condition between creation and removal of ReplicatedMergeTree tables. It might cause exceptions like `node doesn't exist` on attempt to create replicated table. Fixes [#21419](https://github.com/ClickHouse/ClickHouse/issues/21419). [#23294](https://github.com/ClickHouse/ClickHouse/pull/23294) ([tavplubix](https://github.com/tavplubix)). -* Fixed simple key dictionary from DDL creation if primary key is not first attribute. Fixes [#23236](https://github.com/ClickHouse/ClickHouse/issues/23236). [#23262](https://github.com/ClickHouse/ClickHouse/pull/23262) ([Maksim Kita](https://github.com/kitaisreal)). -* Add type conversion for `optimize_skip_unused_shards_rewrite_in` (fixes `use-of-uninitialized-value` with `optimize_skip_unused_shards_rewrite_in`). [#23219](https://github.com/ClickHouse/ClickHouse/pull/23219) ([Azat Khuzhin](https://github.com/azat)). -* Fixed reading from ODBC when there are many long column names in a table. Closes [#8853](https://github.com/ClickHouse/ClickHouse/issues/8853). [#23215](https://github.com/ClickHouse/ClickHouse/pull/23215) ([Kseniia Sumarokova](https://github.com/kssenii)). -* Fixed missing semicolon in exception message. The user may find this exception message unpleasant to read. [#23208](https://github.com/ClickHouse/ClickHouse/pull/23208) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed missing whitespace in some exception messages about `LowCardinality` type. [#23207](https://github.com/ClickHouse/ClickHouse/pull/23207) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed the behavior when disabling `input_format_with_names_use_header ` setting discards all the input with CSVWithNames format. This fixes [#22406](https://github.com/ClickHouse/ClickHouse/issues/22406). [#23202](https://github.com/ClickHouse/ClickHouse/pull/23202) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed `Not found column` error when selecting from `MaterializeMySQL` with condition on key column. Fixes [#22432](https://github.com/ClickHouse/ClickHouse/issues/22432). [#23200](https://github.com/ClickHouse/ClickHouse/pull/23200) ([tavplubix](https://github.com/tavplubix)). -* QueryAliasVisitor to prefer alias for ASTWithAlias if subquery was optimized to constant. Fixes [#22924](https://github.com/ClickHouse/ClickHouse/issues/22924). Fixes [#10401](https://github.com/ClickHouse/ClickHouse/issues/10401). [#23191](https://github.com/ClickHouse/ClickHouse/pull/23191) ([Maksim Kita](https://github.com/kitaisreal)). -* Server might fail to start if `data_type_default_nullable` setting is enabled in default profile, it's fixed. Fixes [#22573](https://github.com/ClickHouse/ClickHouse/issues/22573). [#23185](https://github.com/ClickHouse/ClickHouse/pull/23185) ([tavplubix](https://github.com/tavplubix)). -* Fixed a crash on shutdown which happened because of wrong accounting of current connections. [#23154](https://github.com/ClickHouse/ClickHouse/pull/23154) ([Vitaly Baranov](https://github.com/vitlibar)). -* Some values were formatted with alignment in center in table cells in `Markdown` format. Not anymore. [#23096](https://github.com/ClickHouse/ClickHouse/pull/23096) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fixed `Table .inner_id... doesn't exist` error when selecting from Materialized View after detaching it from Atomic database and attaching back. [#23047](https://github.com/ClickHouse/ClickHouse/pull/23047) ([tavplubix](https://github.com/tavplubix)). -* Remove non-essential details from suggestions in clickhouse-client. This closes [#22158](https://github.com/ClickHouse/ClickHouse/issues/22158). [#23040](https://github.com/ClickHouse/ClickHouse/pull/23040) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix error `Cannot find column in ActionsDAG result` which may happen if subquery uses `untuple`. Fixes [#22290](https://github.com/ClickHouse/ClickHouse/issues/22290). [#22991](https://github.com/ClickHouse/ClickHouse/pull/22991) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fix usage of constant columns of type `Map` with nullable values. [#22939](https://github.com/ClickHouse/ClickHouse/pull/22939) ([Anton Popov](https://github.com/CurtizJ)). -* fixed `formatDateTime()` on `DateTime64` and "%C" format specifier fixed `toDateTime64()` for large values and non-zero scale. ... [#22937](https://github.com/ClickHouse/ClickHouse/pull/22937) ([Vasily Nemkov](https://github.com/Enmk)). -* Fixed a crash when using `mannWhitneyUTest` and `rankCorr` with window functions. This fixes [#22728](https://github.com/ClickHouse/ClickHouse/issues/22728). [#22876](https://github.com/ClickHouse/ClickHouse/pull/22876) ([Nikita Mikhaylov](https://github.com/nikitamikhaylov)). -* Fixed `bytes_allocated` field in system.dictionaries for sparse_hashed dictionaries. [#22867](https://github.com/ClickHouse/ClickHouse/pull/22867) ([Azat Khuzhin](https://github.com/azat)). -* Fixed possible hanging in concurrent DROP/CREATE of TEMPORARY LIVE VIEW in `TemporaryLiveViewCleaner`, see https://gist.github.com/vzakaznikov/0c03195960fc86b56bfe2bc73a90019e. [#22858](https://github.com/ClickHouse/ClickHouse/pull/22858) ([Vitaly Baranov](https://github.com/vitlibar)). -* Fixed pushdown of `HAVING` in case, when filter column is used in aggregation. [#22763](https://github.com/ClickHouse/ClickHouse/pull/22763) ([Anton Popov](https://github.com/CurtizJ)). -* Fixed approximate total rows accounting for reverse reading from MergeTree. [#22726](https://github.com/ClickHouse/ClickHouse/pull/22726) ([Azat Khuzhin](https://github.com/azat)). -* Fixed possible hangs in Zookeeper requests in case of OOM exception. Fixes [#22438](https://github.com/ClickHouse/ClickHouse/issues/22438). [#22684](https://github.com/ClickHouse/ClickHouse/pull/22684) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). -* Fixed wait for mutations on several replicas for ReplicatedMergeTree table engines. Previously, mutation/alter query may finish before mutation actually executed on other replicas. [#22669](https://github.com/ClickHouse/ClickHouse/pull/22669) ([alesapin](https://github.com/alesapin)). -* Fixed LOGICAL_ERROR for Log with nested types w/o columns in the SELECT clause. [#22654](https://github.com/ClickHouse/ClickHouse/pull/22654) ([Azat Khuzhin](https://github.com/azat)). -* Fixed a bug with unlimited wait for auxiliary AWS requests. [#22594](https://github.com/ClickHouse/ClickHouse/pull/22594) ([Vladimir Chebotarev](https://github.com/excitoon)). -* Fixed a crash when client closes connection very early [#22579](https://github.com/ClickHouse/ClickHouse/issues/22579). [#22591](https://github.com/ClickHouse/ClickHouse/pull/22591) ([nvartolomei](https://github.com/nvartolomei)). -* Fixed an incorrect formatting of function `map` in distributed queries. [#22588](https://github.com/ClickHouse/ClickHouse/pull/22588) ([foolchi](https://github.com/foolchi)). -* Avoid UB in *Log engines for rwlock unlock due to unlock from another thread. [#22583](https://github.com/ClickHouse/ClickHouse/pull/22583) ([Azat Khuzhin](https://github.com/azat)). -* Fixed UB by unlocking the rwlock of the TinyLog from the same thread. [#22560](https://github.com/ClickHouse/ClickHouse/pull/22560) ([Azat Khuzhin](https://github.com/azat)). -* Fixed deserialization of empty string without newline at end of TSV format. This closes [#20244](https://github.com/ClickHouse/ClickHouse/issues/20244). Possible workaround without version update: set `input_format_null_as_default` to zero. It was zero in old versions. [#22527](https://github.com/ClickHouse/ClickHouse/pull/22527) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Fix bug in partial merge join with `LowCardinality`. Close [#22386](https://github.com/ClickHouse/ClickHouse/issues/22386), close [#22388](https://github.com/ClickHouse/ClickHouse/issues/22388). [#22510](https://github.com/ClickHouse/ClickHouse/pull/22510) ([Vladimir](https://github.com/vdimir)). -* Fixed `ClickHouseDictionarySource` configuration loop. Closes [#14314](https://github.com/ClickHouse/ClickHouse/issues/14314). [#22479](https://github.com/ClickHouse/ClickHouse/pull/22479) ([Maksim Kita](https://github.com/kitaisreal)). -* Fixed a race condition in HedgedConnections which leads to crash. This fixes [#22161](https://github.com/ClickHouse/ClickHouse/issues/22161). [#22443](https://github.com/ClickHouse/ClickHouse/pull/22443) ([Kruglov Pavel](https://github.com/Avogar)). -* Buffer overflow (on read) was possible in `tokenbf_v1` full text index. The excessive bytes are not used but the read operation may lead to crash in rare cases. This closes [#19233](https://github.com/ClickHouse/ClickHouse/issues/19233). [#22421](https://github.com/ClickHouse/ClickHouse/pull/22421) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Do not limit HTTP chunk size. Fixes [#21907](https://github.com/ClickHouse/ClickHouse/issues/21907). [#22322](https://github.com/ClickHouse/ClickHouse/pull/22322) ([Ivan](https://github.com/abyss7)). -* Fixed a bug, which leads to underaggregation of data in case of enabled `optimize_aggregation_in_order` and many parts in table. Slightly improve performance of aggregation with enabled `optimize_aggregation_in_order`. [#21889](https://github.com/ClickHouse/ClickHouse/pull/21889) ([Anton Popov](https://github.com/CurtizJ)). -* Check if table function view is used as a column. This complements https://github.com/ClickHouse/ClickHouse/pull/20350. [#21465](https://github.com/ClickHouse/ClickHouse/pull/21465) ([Amos Bird](https://github.com/amosbird)). -* * Fix "unknown column" error for tables with `Merge` engine in queris with `JOIN` and aggregation. Closes [#18368](https://github.com/ClickHouse/ClickHouse/issues/18368), close [#22226](https://github.com/ClickHouse/ClickHouse/issues/22226). [#21370](https://github.com/ClickHouse/ClickHouse/pull/21370) ([Vladimir](https://github.com/vdimir)). -* Fixed name clashes in `PredicateRewriteVisitor`. It caused incorrect `WHERE` filtration after full join. Close [#20497](https://github.com/ClickHouse/ClickHouse/issues/20497). [#20622](https://github.com/ClickHouse/ClickHouse/pull/20622) ([Vladimir](https://github.com/vdimir)). -* Fixed very rare bug when quorum insert with `quorum_parallel=1` is not really "quorum" because of deduplication. [#18215](https://github.com/ClickHouse/ClickHouse/pull/18215) ([filimonov](https://github.com/filimonov)). - -#### Build/Testing/Packaging Improvement - -* Simplify debian packages. This fixes [#21698](https://github.com/ClickHouse/ClickHouse/issues/21698). [#22976](https://github.com/ClickHouse/ClickHouse/pull/22976) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added support for ClickHouse build on Apple M1. [#21639](https://github.com/ClickHouse/ClickHouse/pull/21639) ([changvvb](https://github.com/changvvb)). -* Fixed ClickHouse Keeper build for MacOS. [#22860](https://github.com/ClickHouse/ClickHouse/pull/22860) ([alesapin](https://github.com/alesapin)). -* Fixed some tests on AArch64 platform. [#22596](https://github.com/ClickHouse/ClickHouse/pull/22596) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Added function alignment for possibly better performance. [#21431](https://github.com/ClickHouse/ClickHouse/pull/21431) ([Danila Kutenin](https://github.com/danlark1)). -* Adjust some tests to output identical results on amd64 and aarch64 (qemu). The result was depending on implementation specific CPU behaviour. [#22590](https://github.com/ClickHouse/ClickHouse/pull/22590) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow query profiling only on x86_64. See [#15174](https://github.com/ClickHouse/ClickHouse/issues/15174)#issuecomment-812954965 and [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638)#issuecomment-703805337. This closes [#15638](https://github.com/ClickHouse/ClickHouse/issues/15638). [#22580](https://github.com/ClickHouse/ClickHouse/pull/22580) ([alexey-milovidov](https://github.com/alexey-milovidov)). -* Allow building with unbundled xz (lzma) using USE_INTERNAL_XZ_LIBRARY=OFF ... [#22571](https://github.com/ClickHouse/ClickHouse/pull/22571) ([Kfir Itzhak](https://github.com/mastertheknife)). -* Enable the bundled openldap on ppc64le ... [#22487](https://github.com/ClickHouse/ClickHouse/pull/22487) ([Kfir Itzhak](https://github.com/mastertheknife)). -* Disable incompatible libraries (platform specific typically) on ppc64le ... [#22475](https://github.com/ClickHouse/ClickHouse/pull/22475) ([Kfir Itzhak](https://github.com/mastertheknife)). -* Add on-demand check for clickhouse Keeper. [#22373](https://github.com/ClickHouse/ClickHouse/pull/22373) ([alesapin](https://github.com/alesapin)). -* Run stateless tests in parallel in CI. Depends on [#22181](https://github.com/ClickHouse/ClickHouse/issues/22181). [#22300](https://github.com/ClickHouse/ClickHouse/pull/22300) ([alesapin](https://github.com/alesapin)). -* Build `jemalloc` with support for [heap profiling](https://github.com/jemalloc/jemalloc/wiki/Use-Case%3A-Heap-Profiling). [#22834](https://github.com/ClickHouse/ClickHouse/pull/22834) ([nvartolomei](https://github.com/nvartolomei)). - ## ClickHouse release 21.4 ### ClickHouse release 21.4.1 2021-04-12 From 5d2778dd2daf62bf3a265d222146a0a6ad3a2b13 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Sat, 15 May 2021 00:57:24 +0200 Subject: [PATCH 79/82] Update zstd to v1.5.0 --- contrib/zstd | 2 +- contrib/zstd-cmake/CMakeLists.txt | 9 +++++++-- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/contrib/zstd b/contrib/zstd index 10f0e6993f9..a488ba114ec 160000 --- a/contrib/zstd +++ b/contrib/zstd @@ -1 +1 @@ -Subproject commit 10f0e6993f9d2f682da6d04aa2385b7d53cbb4ee +Subproject commit a488ba114ec17ea1054b9057c26a046fc122b3b6 diff --git a/contrib/zstd-cmake/CMakeLists.txt b/contrib/zstd-cmake/CMakeLists.txt index d74dcdffd9c..226ee1a8067 100644 --- a/contrib/zstd-cmake/CMakeLists.txt +++ b/contrib/zstd-cmake/CMakeLists.txt @@ -66,6 +66,7 @@ SET(Sources "${LIBRARY_DIR}/compress/zstd_compress.c" "${LIBRARY_DIR}/compress/zstd_compress_literals.c" "${LIBRARY_DIR}/compress/zstd_compress_sequences.c" + "${LIBRARY_DIR}/compress/zstd_compress_superblock.c" "${LIBRARY_DIR}/compress/zstd_double_fast.c" "${LIBRARY_DIR}/compress/zstd_fast.c" "${LIBRARY_DIR}/compress/zstd_lazy.c" @@ -95,16 +96,19 @@ SET(Headers "${LIBRARY_DIR}/common/pool.h" "${LIBRARY_DIR}/common/threading.h" "${LIBRARY_DIR}/common/xxhash.h" - "${LIBRARY_DIR}/common/zstd_errors.h" + "${LIBRARY_DIR}/common/zstd_deps.h" "${LIBRARY_DIR}/common/zstd_internal.h" + "${LIBRARY_DIR}/common/zstd_trace.h" "${LIBRARY_DIR}/compress/hist.h" "${LIBRARY_DIR}/compress/zstd_compress_internal.h" "${LIBRARY_DIR}/compress/zstd_compress_literals.h" "${LIBRARY_DIR}/compress/zstd_compress_sequences.h" + "${LIBRARY_DIR}/compress/zstd_compress_superblock.h" "${LIBRARY_DIR}/compress/zstd_cwksp.h" "${LIBRARY_DIR}/compress/zstd_double_fast.h" "${LIBRARY_DIR}/compress/zstd_fast.h" "${LIBRARY_DIR}/compress/zstd_lazy.h" + "${LIBRARY_DIR}/compress/zstd_ldm_geartab.h" "${LIBRARY_DIR}/compress/zstd_ldm.h" "${LIBRARY_DIR}/compress/zstdmt_compress.h" "${LIBRARY_DIR}/compress/zstd_opt.h" @@ -113,7 +117,8 @@ SET(Headers "${LIBRARY_DIR}/decompress/zstd_decompress_internal.h" "${LIBRARY_DIR}/dictBuilder/cover.h" "${LIBRARY_DIR}/dictBuilder/divsufsort.h" - "${LIBRARY_DIR}/dictBuilder/zdict.h" + "${LIBRARY_DIR}/zdict.h" + "${LIBRARY_DIR}/zstd_errors.h" "${LIBRARY_DIR}/zstd.h") SET(ZSTD_LEGACY_SUPPORT true) From 0ddfc3df840414a35cdef095bf89f62d35797a91 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Sat, 15 May 2021 02:13:45 +0200 Subject: [PATCH 80/82] TestReadAfterAIO: Use the local path instead of /tmp for temporal files /tmp might be mounted as tmpfs (default in ArchLinux) which is incompatible with O_DIRECT (https://lore.kernel.org/lkml/459D290B.1040703@tmr.com/t/), making the test fail: ``` [ RUN ] ReadBufferAIOTest.TestReadAfterAIO unknown file: Failure C++ exception with description "Cannot open file /tmp/filei6ZsCa/foo, errno: 22, strerror: Invalid argument" thrown in the test body. [ FAILED ] ReadBufferAIOTest.TestReadAfterAIO (0 ms) ``` Instead create the tmp folder in the local path and delete it at the end --- src/IO/tests/gtest_aio_seek_back_after_eof.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/src/IO/tests/gtest_aio_seek_back_after_eof.cpp b/src/IO/tests/gtest_aio_seek_back_after_eof.cpp index 5f1c1c0b945..41cf3c5d146 100644 --- a/src/IO/tests/gtest_aio_seek_back_after_eof.cpp +++ b/src/IO/tests/gtest_aio_seek_back_after_eof.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include @@ -14,7 +15,7 @@ namespace { std::string createTmpFileForEOFtest() { - char pattern[] = "/tmp/fileXXXXXX"; + char pattern[] = "./EOFtestFolderXXXXXX"; if (char * dir = ::mkdtemp(pattern); dir) { return std::string(dir) + "/foo"; @@ -78,6 +79,11 @@ TEST(ReadBufferAIOTest, TestReadAfterAIO) size_t read_after_eof_big = testbuf.read(repeatdata.data(), repeatdata.size()); EXPECT_EQ(read_after_eof_big, data.length()); EXPECT_TRUE(testbuf.eof()); + + if (file_path[0] != '/') + { + std::filesystem::remove_all(file_path.substr(0, file_path.size() - 4)); + } } #endif From e358a0963c1c976eb4cc9ca766143b46fc6b705d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ra=C3=BAl=20Mar=C3=ADn?= Date: Sat, 15 May 2021 03:03:38 +0200 Subject: [PATCH 81/82] Clearer code based on review --- src/IO/tests/gtest_aio_seek_back_after_eof.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/IO/tests/gtest_aio_seek_back_after_eof.cpp b/src/IO/tests/gtest_aio_seek_back_after_eof.cpp index 41cf3c5d146..784f5479657 100644 --- a/src/IO/tests/gtest_aio_seek_back_after_eof.cpp +++ b/src/IO/tests/gtest_aio_seek_back_after_eof.cpp @@ -82,7 +82,9 @@ TEST(ReadBufferAIOTest, TestReadAfterAIO) if (file_path[0] != '/') { - std::filesystem::remove_all(file_path.substr(0, file_path.size() - 4)); + const size_t last_slash = file_path.rfind('/'); + const std::string temp_dir = file_path.substr(0, last_slash); + std::filesystem::remove_all(temp_dir); } } From 3252228852acfbf633ae5dd7748bdc95a2c00a3c Mon Sep 17 00:00:00 2001 From: Benjamin Naecker Date: Fri, 14 May 2021 18:56:16 -0700 Subject: [PATCH 82/82] Updates submodules bringing in several upstream changes to build on illumos --- .gitmodules | 1 + contrib/boringssl | 2 +- contrib/cppkafka | 2 +- contrib/grpc | 2 +- contrib/poco | 2 +- contrib/rocksdb | 2 +- contrib/zlib-ng | 2 +- 7 files changed, 7 insertions(+), 6 deletions(-) diff --git a/.gitmodules b/.gitmodules index f7dcf5f4ac1..66a2370f0da 100644 --- a/.gitmodules +++ b/.gitmodules @@ -17,6 +17,7 @@ [submodule "contrib/zlib-ng"] path = contrib/zlib-ng url = https://github.com/ClickHouse-Extras/zlib-ng.git + branch = clickhouse-new [submodule "contrib/googletest"] path = contrib/googletest url = https://github.com/google/googletest.git diff --git a/contrib/boringssl b/contrib/boringssl index 83c1cda8a02..a6a2e2ab3e4 160000 --- a/contrib/boringssl +++ b/contrib/boringssl @@ -1 +1 @@ -Subproject commit 83c1cda8a0224dc817cbad2966c7ed4acc35f02a +Subproject commit a6a2e2ab3e44d97ce98e51c558e989f211de7eb3 diff --git a/contrib/cppkafka b/contrib/cppkafka index b06e64ef5bf..57a599d99c5 160000 --- a/contrib/cppkafka +++ b/contrib/cppkafka @@ -1 +1 @@ -Subproject commit b06e64ef5bffd636d918a742c689f69130c1dbab +Subproject commit 57a599d99c540e647bcd0eb9ea77c523cca011b3 diff --git a/contrib/grpc b/contrib/grpc index 1085a941238..5b79aae85c5 160000 --- a/contrib/grpc +++ b/contrib/grpc @@ -1 +1 @@ -Subproject commit 1085a941238e66b13e3fb89c310533745380acbc +Subproject commit 5b79aae85c515e0df4abfb7b1e07975fdc7cecc1 diff --git a/contrib/poco b/contrib/poco index b7d9ec16ee3..59945069080 160000 --- a/contrib/poco +++ b/contrib/poco @@ -1 +1 @@ -Subproject commit b7d9ec16ee33ca76643d5fcd907ea9a33285640a +Subproject commit 5994506908028612869fee627d68d8212dfe7c1e diff --git a/contrib/rocksdb b/contrib/rocksdb index 54a0decabbc..07c77549a20 160000 --- a/contrib/rocksdb +++ b/contrib/rocksdb @@ -1 +1 @@ -Subproject commit 54a0decabbcf4c0bb5cf7befa9c597f28289bff5 +Subproject commit 07c77549a20b63ff6981b400085eba36bb5c80c4 diff --git a/contrib/zlib-ng b/contrib/zlib-ng index 5cc4d232020..db232d30b4c 160000 --- a/contrib/zlib-ng +++ b/contrib/zlib-ng @@ -1 +1 @@ -Subproject commit 5cc4d232020dc66d1d6c5438834457e2a2f6127b +Subproject commit db232d30b4c72fd58e6d7eae2d12cebf9c3d90db