From 566c08086afd037e9b7ce244ac6c222d286af48d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 6 May 2022 14:44:00 +0000 Subject: [PATCH 001/152] support Object type inside other types --- src/Columns/ColumnArray.h | 2 + src/Columns/ColumnObject.h | 5 +- src/Columns/ColumnTuple.cpp | 6 + src/Columns/ColumnTuple.h | 1 + src/Columns/IColumn.h | 2 + src/DataTypes/DataTypeArray.h | 1 + src/DataTypes/DataTypeObject.h | 1 + src/DataTypes/DataTypeTuple.cpp | 5 + src/DataTypes/DataTypeTuple.h | 1 + src/DataTypes/IDataType.h | 2 + src/DataTypes/ObjectUtils.cpp | 267 ++++++++++++++---- src/DataTypes/ObjectUtils.h | 39 +-- src/Functions/FunctionsConversion.h | 5 +- src/Interpreters/InterpreterCreateQuery.cpp | 4 +- src/Processors/Formats/IRowInputFormat.cpp | 5 +- .../Formats/Impl/ValuesBlockInputFormat.cpp | 4 +- src/Storages/MergeTree/MergeTask.cpp | 2 +- src/Storages/MergeTree/MergeTreeData.cpp | 16 +- src/Storages/MergeTree/MergeTreeData.h | 6 +- .../MergeTree/MergeTreeDataWriter.cpp | 12 +- src/Storages/MergeTree/MergeTreeDataWriter.h | 2 - src/Storages/MergeTree/MergeTreeSink.cpp | 4 +- .../MergeTree/ReplicatedMergeTreeSink.cpp | 5 +- src/Storages/StorageDistributed.cpp | 6 +- src/Storages/StorageInMemoryMetadata.cpp | 6 +- src/Storages/StorageMemory.cpp | 6 +- src/Storages/StorageSnapshot.cpp | 2 +- src/Storages/getStructureOfRemoteTable.cpp | 2 +- .../01825_type_json_in_array.reference | 17 ++ .../0_stateless/01825_type_json_in_array.sql | 27 ++ 30 files changed, 339 insertions(+), 124 deletions(-) create mode 100644 tests/queries/0_stateless/01825_type_json_in_array.reference create mode 100644 tests/queries/0_stateless/01825_type_json_in_array.sql diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 1d88b2e6a26..1657160fbae 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -168,6 +168,8 @@ public: void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; + void finalize() override { data->finalize(); } + bool isCollationSupported() const override { return getData().isCollationSupported(); } size_t getNumberOfDimensions() const; diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 9d61b165042..27a00e44698 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -177,8 +177,6 @@ public: Subcolumns & getSubcolumns() { return subcolumns; } PathsInData getKeys() const; - /// Finalizes all subcolumns. - void finalize(); bool isFinalized() const; /// Part of interface @@ -199,6 +197,9 @@ public: Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; + /// Finalizes all subcolumns. + void finalize() override; + /// All other methods throw exception. ColumnPtr decompress() const override { throwMustBeConcrete(); } diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 7c8d73edd16..8d6b2dbdbd9 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -572,4 +572,10 @@ SerializationInfoPtr ColumnTuple::getSerializationInfo() const return std::make_shared(std::move(infos), SerializationInfo::Settings{}); } +void ColumnTuple::finalize() +{ + for (auto & column : columns) + column->finalize(); +} + } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index e6797c2582c..7b554ed2fea 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -103,6 +103,7 @@ public: double getRatioOfDefaultRows(double sample_ratio) const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; SerializationInfoPtr getSerializationInfo() const override; + void finalize() override; size_t tupleSize() const { return columns.size(); } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index f4986799d47..4bf364d9b52 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -442,6 +442,8 @@ public: return getPtr(); } + /// Some columns may require finalization before using of other operations. + virtual void finalize() {} [[nodiscard]] static MutablePtr mutate(Ptr ptr) { diff --git a/src/DataTypes/DataTypeArray.h b/src/DataTypes/DataTypeArray.h index 122ac8e03a3..033a657c845 100644 --- a/src/DataTypes/DataTypeArray.h +++ b/src/DataTypes/DataTypeArray.h @@ -48,6 +48,7 @@ public: bool textCanContainOnlyValidUTF8() const override { return nested->textCanContainOnlyValidUTF8(); } bool isComparable() const override { return nested->isComparable(); } bool canBeComparedWithCollation() const override { return nested->canBeComparedWithCollation(); } + bool hasDynamicSubcolumns() const override { return nested->hasDynamicSubcolumns(); } bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { diff --git a/src/DataTypes/DataTypeObject.h b/src/DataTypes/DataTypeObject.h index 503947c3738..2f6ad961512 100644 --- a/src/DataTypes/DataTypeObject.h +++ b/src/DataTypes/DataTypeObject.h @@ -36,6 +36,7 @@ public: bool haveSubtypes() const override { return false; } bool equals(const IDataType & rhs) const override; bool isParametric() const override { return true; } + bool hasDynamicSubcolumns() const override { return true; } SerializationPtr doGetDefaultSerialization() const override; diff --git a/src/DataTypes/DataTypeTuple.cpp b/src/DataTypes/DataTypeTuple.cpp index 558b13927c1..6b49d0c969c 100644 --- a/src/DataTypes/DataTypeTuple.cpp +++ b/src/DataTypes/DataTypeTuple.cpp @@ -233,6 +233,11 @@ bool DataTypeTuple::haveMaximumSizeOfValue() const return std::all_of(elems.begin(), elems.end(), [](auto && elem) { return elem->haveMaximumSizeOfValue(); }); } +bool DataTypeTuple::hasDynamicSubcolumns() const +{ + return std::any_of(elems.begin(), elems.end(), [](auto && elem) { return elem->hasDynamicSubcolumns(); }); +} + bool DataTypeTuple::isComparable() const { return std::all_of(elems.begin(), elems.end(), [](auto && elem) { return elem->isComparable(); }); diff --git a/src/DataTypes/DataTypeTuple.h b/src/DataTypes/DataTypeTuple.h index 009a2284a0a..0c5c1e82fab 100644 --- a/src/DataTypes/DataTypeTuple.h +++ b/src/DataTypes/DataTypeTuple.h @@ -48,6 +48,7 @@ public: bool isComparable() const override; bool textCanContainOnlyValidUTF8() const override; bool haveMaximumSizeOfValue() const override; + bool hasDynamicSubcolumns() const override; size_t getMaximumSizeOfValueInMemory() const override; size_t getSizeOfValueInMemory() const override; diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index fc9e50dc55b..6d1d17c9e40 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -287,6 +287,8 @@ public: /// Strings, Numbers, Date, DateTime, Nullable virtual bool canBeInsideLowCardinality() const { return false; } + virtual bool hasDynamicSubcolumns() const { return false; } + /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint); diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index ab4ac847a1c..430bfe2af72 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -8,6 +8,7 @@ #include #include #include +#include #include #include #include @@ -128,44 +129,109 @@ static auto extractVector(const std::vector & vec) return res; } -void convertObjectsToTuples(Block & block, const NamesAndTypesList & extended_storage_columns) +static DataTypePtr recreateTupleWithElements(const DataTypeTuple & type_tuple, const DataTypes & elements) { - std::unordered_map storage_columns_map; - for (const auto & [name, type] : extended_storage_columns) - storage_columns_map[name] = type; + return type_tuple.haveExplicitNames() + ? std::make_shared(elements, type_tuple.getElementNames()) + : std::make_shared(elements); +} - for (auto & column : block) +static std::pair convertObjectColumnToTuple( + const ColumnObject & column_object, const DataTypeObject & type_object) +{ + const auto & subcolumns = column_object.getSubcolumns(); + + if (!column_object.isFinalized()) + throw Exception(ErrorCodes::LOGICAL_ERROR, + "Cannot convert to tuple column of type {}. Column should be finalized first", type_object.getName()); + + PathsInData tuple_paths; + DataTypes tuple_types; + Columns tuple_columns; + + for (const auto & entry : subcolumns) { - if (!isObject(column.type)) - continue; + tuple_paths.emplace_back(entry->path); + tuple_types.emplace_back(entry->data.getLeastCommonType()); + tuple_columns.emplace_back(entry->data.getFinalizedColumnPtr()); + } - const auto & column_object = assert_cast(*column.column); - const auto & subcolumns = column_object.getSubcolumns(); + return unflattenTuple(tuple_paths, tuple_types, tuple_columns); +} - if (!column_object.isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot convert to tuple column '{}' from type {}. Column should be finalized first", - column.name, column.type->getName()); +static std::pair recursivlyConvertDynamicColumnToTuple( + const ColumnPtr & column, const DataTypePtr & type) +{ + if (!type->hasDynamicSubcolumns()) + return {column, type}; - PathsInData tuple_paths; - DataTypes tuple_types; - Columns tuple_columns; + if (const auto * type_object = typeid_cast(type.get())) + { + const auto & column_object = assert_cast(*column); + return convertObjectColumnToTuple(column_object, *type_object); + } - for (const auto & entry : subcolumns) + if (const auto * type_array = typeid_cast(type.get())) + { + const auto & column_array = assert_cast(*column); + auto [new_column, new_type] = recursivlyConvertDynamicColumnToTuple( + column_array.getDataPtr(), type_array->getNestedType()); + + return { - tuple_paths.emplace_back(entry->path); - tuple_types.emplace_back(entry->data.getLeastCommonType()); - tuple_columns.emplace_back(entry->data.getFinalizedColumnPtr()); + ColumnArray::create(std::move(new_column), column_array.getOffsetsPtr()), + std::make_shared(std::move(new_type)), + }; + } + + if (const auto * type_tuple = typeid_cast(type.get())) + { + const auto & column_tuple = assert_cast(*column); + + const auto & tuple_columns = column_tuple.getColumns(); + const auto & tuple_types = type_tuple->getElements(); + assert(tuple_columns.size() == tuple_types.size()); + const size_t tuple_size = tuple_types.size(); + + Columns new_tuple_columns(tuple_size); + DataTypes new_tuple_types(tuple_size); + + for (size_t i = 0; i < tuple_size; ++i) + { + std::tie(new_tuple_columns[i], new_tuple_types[i]) + = recursivlyConvertDynamicColumnToTuple(tuple_columns[i], tuple_types[i]); } - auto it = storage_columns_map.find(column.name); - if (it == storage_columns_map.end()) + return + { + ColumnTuple::create(new_tuple_columns), + recreateTupleWithElements(*type_tuple, new_tuple_types) + }; + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type->getName()); +} + +void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & storage_snapshot) +{ + for (auto & column : block) + { + if (!column.type->hasDynamicSubcolumns()) + continue; + + std::tie(column.column, column.type) + = recursivlyConvertDynamicColumnToTuple(column.column, column.type); + + GetColumnsOptions options(GetColumnsOptions::AllPhysical); + auto storage_column = storage_snapshot->tryGetColumn(options, column.name); + if (!storage_column) throw Exception(ErrorCodes::LOGICAL_ERROR, "Column '{}' not found in storage", column.name); - std::tie(column.column, column.type) = unflattenTuple(tuple_paths, tuple_types, tuple_columns); + auto storage_column_concrete = storage_snapshot->getColumn(options.withExtendedObjects(), column.name); /// Check that constructed Tuple type and type in storage are compatible. - getLeastCommonTypeForObject({column.type, it->second}, true); + getLeastCommonTypeForDynamicColumns( + storage_column->type, {column.type, storage_column_concrete.type}, true); } } @@ -196,24 +262,8 @@ void checkObjectHasNoAmbiguosPaths(const PathsInData & paths) } } -DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool check_ambiguos_paths) +static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool check_ambiguos_paths) { - if (types.empty()) - return nullptr; - - bool all_equal = true; - for (size_t i = 1; i < types.size(); ++i) - { - if (!types[i]->equals(*types[0])) - { - all_equal = false; - break; - } - } - - if (all_equal) - return types[0]; - /// Types of subcolumns by path from all tuples. std::unordered_map subcolumns_types; @@ -266,19 +316,123 @@ DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool check_ambi return unflattenTuple(tuple_paths, tuple_types); } -NameSet getNamesOfObjectColumns(const NamesAndTypesList & columns_list) +DataTypePtr getLeastCommonTypeForDynamicColumnsImpl( + const DataTypePtr & type_in_storage, const DataTypes & concrete_types, bool check_ambiguos_paths) { - NameSet res; - for (const auto & [name, type] : columns_list) - if (isObject(type)) - res.insert(name); + if (!type_in_storage->hasDynamicSubcolumns()) + return type_in_storage; - return res; + if (isObject(type_in_storage)) + return getLeastCommonTypeForObject(concrete_types, check_ambiguos_paths); + + if (const auto * type_array = typeid_cast(type_in_storage.get())) + { + DataTypes nested_types; + nested_types.reserve(concrete_types.size()); + + for (const auto & type : concrete_types) + { + const auto * type_array_conctete = typeid_cast(type.get()); + if (!type_array_conctete) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected Array type, got {}", type->getName()); + + nested_types.push_back(type_array_conctete->getNestedType()); + } + + return std::make_shared( + getLeastCommonTypeForDynamicColumnsImpl( + type_array->getNestedType(), nested_types, check_ambiguos_paths)); + } + + if (const auto * type_tuple = typeid_cast(type_in_storage.get())) + { + const auto & element_types = type_tuple->getElements(); + DataTypes new_element_types(element_types.size()); + + for (size_t i = 0; i < element_types.size(); ++i) + { + DataTypes concrete_element_types; + concrete_element_types.reserve(concrete_types.size()); + + for (const auto & type : concrete_types) + { + const auto * type_tuple_conctete = typeid_cast(type.get()); + if (!type_tuple_conctete) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected Tuple type, got {}", type->getName()); + + concrete_element_types.push_back(type_tuple_conctete->getElement(i)); + } + + new_element_types[i] = getLeastCommonTypeForDynamicColumnsImpl( + element_types[i], concrete_element_types, check_ambiguos_paths); + } + + return recreateTupleWithElements(*type_tuple, new_element_types); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName()); } -bool hasObjectColumns(const ColumnsDescription & columns) +DataTypePtr getLeastCommonTypeForDynamicColumns( + const DataTypePtr & type_in_storage, const DataTypes & concrete_types, bool check_ambiguos_paths) { - return std::any_of(columns.begin(), columns.end(), [](const auto & column) { return isObject(column.type); }); + if (concrete_types.empty()) + return nullptr; + + bool all_equal = true; + for (size_t i = 1; i < concrete_types.size(); ++i) + { + if (!concrete_types[i]->equals(*concrete_types[0])) + { + all_equal = false; + break; + } + } + + if (all_equal) + return concrete_types[0]; + + return getLeastCommonTypeForDynamicColumnsImpl(type_in_storage, concrete_types, check_ambiguos_paths); +} + +DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage) +{ + if (!type_in_storage->hasDynamicSubcolumns()) + return type_in_storage; + + if (isObject(type_in_storage)) + { + return std::make_shared( + DataTypes{std::make_shared()}, + Names{ColumnObject::COLUMN_NAME_DUMMY}); + } + + if (const auto * type_array = typeid_cast(type_in_storage.get())) + return std::make_shared( + createConcreteEmptyDynamicColumn(type_array->getNestedType())); + + if (const auto * type_tuple = typeid_cast(type_in_storage.get())) + { + const auto & elements = type_tuple->getElements(); + DataTypes new_elements; + new_elements.reserve(elements.size()); + + for (const auto & element : elements) + new_elements.push_back(createConcreteEmptyDynamicColumn(element)); + + return recreateTupleWithElements(*type_tuple, new_elements); + } + + throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName()); +} + +bool hasDynamicSubcolumns(const ColumnsDescription & columns) +{ + return std::any_of(columns.begin(), columns.end(), + [](const auto & column) + { + return column.type->hasDynamicSubcolumns(); + }); } void extendObjectColumns(NamesAndTypesList & columns_list, const ColumnsDescription & object_columns, bool with_subcolumns) @@ -299,16 +453,20 @@ void extendObjectColumns(NamesAndTypesList & columns_list, const ColumnsDescript columns_list.splice(columns_list.end(), std::move(subcolumns_list)); } -void updateObjectColumns(ColumnsDescription & object_columns, const NamesAndTypesList & new_columns) +void updateObjectColumns( + ColumnsDescription & object_columns, + const ColumnsDescription & storage_columns, + const NamesAndTypesList & new_columns) { for (const auto & new_column : new_columns) { auto object_column = object_columns.tryGetColumn(GetColumnsOptions::All, new_column.name); if (object_column && !object_column->type->equals(*new_column.type)) { + auto storage_column = storage_columns.getColumn(GetColumnsOptions::All, new_column.name); object_columns.modify(new_column.name, [&](auto & column) { - column.type = getLeastCommonTypeForObject({object_column->type, new_column.type}); + column.type = getLeastCommonTypeForDynamicColumns(storage_column.type, {object_column->type, new_column.type}); }); } } @@ -686,13 +844,6 @@ void replaceMissedSubcolumnsByConstants( addConstantToWithClause(query, name, type); } -void finalizeObjectColumns(MutableColumns & columns) -{ - for (auto & column : columns) - if (auto * column_object = typeid_cast(column.get())) - column_object->finalize(); -} - Field FieldVisitorReplaceScalars::operator()(const Array & x) const { if (num_dimensions_to_keep == 0) diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 8dc46ceecf5..815e6654229 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -11,6 +11,9 @@ namespace DB { +struct StorageSnapshot; +using StorageSnapshotPtr = std::shared_ptr; + /// Returns number of dimensions in Array type. 0 if type is not array. size_t getNumberOfDimensions(const IDataType & type); @@ -37,25 +40,28 @@ DataTypePtr getDataTypeByColumn(const IColumn & column); /// Converts Object types and columns to Tuples in @columns_list and @block /// and checks that types are consistent with types in @extended_storage_columns. -void convertObjectsToTuples(Block & block, const NamesAndTypesList & extended_storage_columns); +void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & storage_snapshot); /// Checks that each path is not the prefix of any other path. void checkObjectHasNoAmbiguosPaths(const PathsInData & paths); /// Receives several Tuple types and deduces the least common type among them. -DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool check_ambiguos_paths = false); +DataTypePtr getLeastCommonTypeForDynamicColumns( + const DataTypePtr & type_in_storage, const DataTypes & types, bool check_ambiguos_paths = false); + +DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage); /// Converts types of object columns to tuples in @columns_list /// according to @object_columns and adds all tuple's subcolumns if needed. void extendObjectColumns(NamesAndTypesList & columns_list, const ColumnsDescription & object_columns, bool with_subcolumns); - -NameSet getNamesOfObjectColumns(const NamesAndTypesList & columns_list); -bool hasObjectColumns(const ColumnsDescription & columns); -void finalizeObjectColumns(MutableColumns & columns); +bool hasDynamicSubcolumns(const ColumnsDescription & columns); /// Updates types of objects in @object_columns inplace /// according to types in new_columns. -void updateObjectColumns(ColumnsDescription & object_columns, const NamesAndTypesList & new_columns); +void updateObjectColumns( + ColumnsDescription & object_columns, + const ColumnsDescription & storage_columns, + const NamesAndTypesList & new_columns); using DataTypeTuplePtr = std::shared_ptr; @@ -124,7 +130,7 @@ public: /// columns-like objects from entry to which Iterator points. /// columns-like object should have fields "name" and "type". template -ColumnsDescription getObjectColumns( +ColumnsDescription getConcreteObjectColumns( Iterator begin, Iterator end, const ColumnsDescription & storage_columns, EntryColumnsGetter && entry_columns_getter) @@ -135,14 +141,8 @@ ColumnsDescription getObjectColumns( { for (const auto & column : storage_columns) { - if (isObject(column.type)) - { - auto tuple_type = std::make_shared( - DataTypes{std::make_shared()}, - Names{ColumnObject::COLUMN_NAME_DUMMY}); - - res.add({column.name, std::move(tuple_type)}); - } + if (column.type->hasDynamicSubcolumns()) + res.add({column.name, createConcreteEmptyDynamicColumn(column.type)}); } return res; @@ -156,13 +156,16 @@ ColumnsDescription getObjectColumns( for (const auto & column : entry_columns) { auto storage_column = storage_columns.tryGetPhysical(column.name); - if (storage_column && isObject(storage_column->type)) + if (storage_column && storage_column->type->hasDynamicSubcolumns()) types_in_entries[column.name].push_back(column.type); } } for (const auto & [name, types] : types_in_entries) - res.add({name, getLeastCommonTypeForObject(types)}); + { + auto storage_column = storage_columns.getPhysical(name); + res.add({name, getLeastCommonTypeForDynamicColumns(storage_column.type, types)}); + } return res; } diff --git a/src/Functions/FunctionsConversion.h b/src/Functions/FunctionsConversion.h index 6926899eb63..54f302772de 100644 --- a/src/Functions/FunctionsConversion.h +++ b/src/Functions/FunctionsConversion.h @@ -3277,9 +3277,8 @@ private: { return [] (ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable * nullable_source, size_t input_rows_count) { - auto res = ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count); - auto & res_object = assert_cast(res->assumeMutableRef()); - res_object.finalize(); + auto res = ConvertImplGenericFromString::execute(arguments, result_type, nullable_source, input_rows_count)->assumeMutable(); + res->finalize(); return res; }; } diff --git a/src/Interpreters/InterpreterCreateQuery.cpp b/src/Interpreters/InterpreterCreateQuery.cpp index d65f4086e2d..bb7bc62f677 100644 --- a/src/Interpreters/InterpreterCreateQuery.cpp +++ b/src/Interpreters/InterpreterCreateQuery.cpp @@ -766,7 +766,7 @@ void InterpreterCreateQuery::validateTableStructure(const ASTCreateQuery & creat { for (const auto & [name, type] : properties.columns.getAllPhysical()) { - if (isObject(type)) + if (type->hasDynamicSubcolumns()) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column '{}' which type is '{}' " @@ -1299,7 +1299,7 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create, /// we can safely destroy the object without a call to "shutdown", because there is guarantee /// that no background threads/similar resources remain after exception from "startup". - if (!res->supportsDynamicSubcolumns() && hasObjectColumns(res->getInMemoryMetadataPtr()->getColumns())) + if (!res->supportsDynamicSubcolumns() && hasDynamicSubcolumns(res->getInMemoryMetadataPtr()->getColumns())) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot create table with column of type Object, " diff --git a/src/Processors/Formats/IRowInputFormat.cpp b/src/Processors/Formats/IRowInputFormat.cpp index 72864d83b86..6bed40b0166 100644 --- a/src/Processors/Formats/IRowInputFormat.cpp +++ b/src/Processors/Formats/IRowInputFormat.cpp @@ -211,9 +211,10 @@ Chunk IRowInputFormat::generate() return {}; } - finalizeObjectColumns(columns); + for (const auto & column : columns) + column->finalize(); + Chunk chunk(std::move(columns), num_rows); - //chunk.setChunkInfo(std::move(chunk_missing_values)); return chunk; } diff --git a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp index 956e640e202..6802262ca85 100644 --- a/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp +++ b/src/Processors/Formats/Impl/ValuesBlockInputFormat.cpp @@ -107,7 +107,9 @@ Chunk ValuesBlockInputFormat::generate() return {}; } - finalizeObjectColumns(columns); + for (const auto & column : columns) + column->finalize(); + size_t rows_in_block = columns[0]->size(); return Chunk{std::move(columns), rows_in_block}; } diff --git a/src/Storages/MergeTree/MergeTask.cpp b/src/Storages/MergeTree/MergeTask.cpp index cb01fe3fcfc..bd1c78e8e04 100644 --- a/src/Storages/MergeTree/MergeTask.cpp +++ b/src/Storages/MergeTree/MergeTask.cpp @@ -135,7 +135,7 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() global_ctx->all_column_names = global_ctx->metadata_snapshot->getColumns().getNamesOfPhysical(); global_ctx->storage_columns = global_ctx->metadata_snapshot->getColumns().getAllPhysical(); - auto object_columns = MergeTreeData::getObjectColumns(global_ctx->future_part->parts, global_ctx->metadata_snapshot->getColumns()); + auto object_columns = MergeTreeData::getConcreteObjectColumns(global_ctx->future_part->parts, global_ctx->metadata_snapshot->getColumns()); global_ctx->storage_snapshot = std::make_shared(*global_ctx->data, global_ctx->metadata_snapshot, object_columns); extendObjectColumns(global_ctx->storage_columns, object_columns, false); diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d84fb9d30d3..28f19819c19 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -6643,18 +6643,18 @@ ReservationPtr MergeTreeData::balancedReservation( return reserved_space; } -ColumnsDescription MergeTreeData::getObjectColumns( +ColumnsDescription MergeTreeData::getConcreteObjectColumns( const DataPartsVector & parts, const ColumnsDescription & storage_columns) { - return DB::getObjectColumns( + return DB::getConcreteObjectColumns( parts.begin(), parts.end(), storage_columns, [](const auto & part) -> const auto & { return part->getColumns(); }); } -ColumnsDescription MergeTreeData::getObjectColumns( +ColumnsDescription MergeTreeData::getConcreteObjectColumns( boost::iterator_range range, const ColumnsDescription & storage_columns) { - return DB::getObjectColumns( + return DB::getConcreteObjectColumns( range.begin(), range.end(), storage_columns, [](const auto & part) -> const auto & { return part->getColumns(); }); } @@ -6663,21 +6663,21 @@ void MergeTreeData::resetObjectColumnsFromActiveParts(const DataPartsLock & /*lo { auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & columns = metadata_snapshot->getColumns(); - if (!hasObjectColumns(columns)) + if (!hasDynamicSubcolumns(columns)) return; auto range = getDataPartsStateRange(DataPartState::Active); - object_columns = getObjectColumns(range, columns); + object_columns = getConcreteObjectColumns(range, columns); } void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPartsLock & /*lock*/) { auto metadata_snapshot = getInMemoryMetadataPtr(); const auto & columns = metadata_snapshot->getColumns(); - if (!hasObjectColumns(columns)) + if (!hasDynamicSubcolumns(columns)) return; - DB::updateObjectColumns(object_columns, part->getColumns()); + DB::updateObjectColumns(object_columns, columns, part->getColumns()); } StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 8515c8583b7..29f3ef3d40a 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -746,10 +746,10 @@ public: return column_sizes; } - const ColumnsDescription & getObjectColumns() const { return object_columns; } + const ColumnsDescription & getConcreteObjectColumns() const { return object_columns; } /// Creates desciprion of columns of data type Object from the range of data parts. - static ColumnsDescription getObjectColumns( + static ColumnsDescription getConcreteObjectColumns( const DataPartsVector & parts, const ColumnsDescription & storage_columns); IndexSizeByName getSecondaryIndexSizes() const override @@ -1101,7 +1101,7 @@ protected: } /// Creates desciprion of columns of data type Object from the range of data parts. - static ColumnsDescription getObjectColumns( + static ColumnsDescription getConcreteObjectColumns( boost::iterator_range range, const ColumnsDescription & storage_columns); std::optional totalRowsByPartitionPredicateImpl( diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 24ca7cc2f5a..2451af7594f 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -287,7 +287,7 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( auto columns = metadata_snapshot->getColumns().getAllPhysical().filter(block.getNames()); for (auto & column : columns) - if (isObject(column.type)) + if (column.type->hasDynamicSubcolumns()) column.type = block.getByName(column.name).type; static const String TMP_PREFIX = "tmp_insert_"; @@ -469,16 +469,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( return temp_part; } -void MergeTreeDataWriter::deduceTypesOfObjectColumns(const StorageSnapshotPtr & storage_snapshot, Block & block) -{ - if (!storage_snapshot->object_columns.empty()) - { - auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects(); - auto storage_columns = storage_snapshot->getColumns(options); - convertObjectsToTuples(block, storage_columns); - } -} - MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( const String & part_name, MergeTreeDataPartType part_type, diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 7b6bf8fb1db..86909ab2062 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -42,8 +42,6 @@ public: */ static BlocksWithPartition splitBlockIntoParts(const Block & block, size_t max_parts, const StorageMetadataPtr & metadata_snapshot, ContextPtr context); - static void deduceTypesOfObjectColumns(const StorageSnapshotPtr & storage_snapshot, Block & block); - /// This structure contains not completely written temporary part. /// Some writes may happen asynchronously, e.g. for blob storages. /// You should call finalize() to wait until all data is written. diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index 93b9f356595..4b0998edf69 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -2,6 +2,7 @@ #include #include #include +#include namespace DB @@ -51,8 +52,9 @@ void MergeTreeSink::consume(Chunk chunk) { auto block = getHeader().cloneWithColumns(chunk.detachColumns()); auto storage_snapshot = storage.getStorageSnapshot(metadata_snapshot, context); + if (!storage_snapshot->object_columns.empty()) + convertDynamicColumnsToTuples(block, storage_snapshot); - storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block); auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); using DelayedPartitions = std::vector; diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 126d34bcc1d..28eeb97bb55 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -4,6 +4,7 @@ #include #include #include +#include #include #include @@ -151,7 +152,9 @@ void ReplicatedMergeTreeSink::consume(Chunk chunk) checkQuorumPrecondition(zookeeper); auto storage_snapshot = storage.getStorageSnapshot(metadata_snapshot, context); - storage.writer.deduceTypesOfObjectColumns(storage_snapshot, block); + if (!storage_snapshot->object_columns.empty()) + convertDynamicColumnsToTuples(block, storage_snapshot); + auto part_blocks = storage.writer.splitBlockIntoParts(block, max_parts_per_block, metadata_snapshot, context); using DelayedPartitions = std::vector; diff --git a/src/Storages/StorageDistributed.cpp b/src/Storages/StorageDistributed.cpp index ae67a67af7f..66f8b7f8a7f 100644 --- a/src/Storages/StorageDistributed.cpp +++ b/src/Storages/StorageDistributed.cpp @@ -595,7 +595,7 @@ std::optional StorageDistributed::getOptimizedQueryP static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr query) { - if (!hasObjectColumns(all_columns)) + if (!hasDynamicSubcolumns(all_columns)) return false; if (!query) @@ -610,7 +610,7 @@ static bool requiresObjectColumns(const ColumnsDescription & all_columns, ASTPtr auto name_in_storage = Nested::splitName(required_column).first; auto column_in_storage = all_columns.tryGetPhysical(name_in_storage); - if (column_in_storage && isObject(column_in_storage->type)) + if (column_in_storage && column_in_storage->type->hasDynamicSubcolumns()) return true; } @@ -637,7 +637,7 @@ StorageSnapshotPtr StorageDistributed::getStorageSnapshotForQuery( metadata_snapshot->getColumns(), getContext()); - auto object_columns = DB::getObjectColumns( + auto object_columns = DB::getConcreteObjectColumns( snapshot_data->objects_by_shard.begin(), snapshot_data->objects_by_shard.end(), metadata_snapshot->getColumns(), diff --git a/src/Storages/StorageInMemoryMetadata.cpp b/src/Storages/StorageInMemoryMetadata.cpp index 66dcc938aef..a80f21834db 100644 --- a/src/Storages/StorageInMemoryMetadata.cpp +++ b/src/Storages/StorageInMemoryMetadata.cpp @@ -526,7 +526,7 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns) const auto * available_type = it->getMapped(); - if (!isObject(*available_type) + if (!available_type->hasDynamicSubcolumns() && !column.type->equals(*available_type) && !isCompatibleEnumTypes(available_type, column.type.get())) throw Exception( @@ -575,7 +575,7 @@ void StorageInMemoryMetadata::check(const NamesAndTypesList & provided_columns, const auto * provided_column_type = it->getMapped(); const auto * available_column_type = jt->getMapped(); - if (!isObject(*provided_column_type) + if (!provided_column_type->hasDynamicSubcolumns() && !provided_column_type->equals(*available_column_type) && !isCompatibleEnumTypes(available_column_type, provided_column_type)) throw Exception( @@ -619,7 +619,7 @@ void StorageInMemoryMetadata::check(const Block & block, bool need_all) const listOfColumns(available_columns)); const auto * available_type = it->getMapped(); - if (!isObject(*available_type) + if (!available_type->hasDynamicSubcolumns() && !column.type->equals(*available_type) && !isCompatibleEnumTypes(available_type, column.type.get())) throw Exception( diff --git a/src/Storages/StorageMemory.cpp b/src/Storages/StorageMemory.cpp index c660195c368..9392aec482d 100644 --- a/src/Storages/StorageMemory.cpp +++ b/src/Storages/StorageMemory.cpp @@ -141,7 +141,7 @@ public: auto extended_storage_columns = storage_snapshot->getColumns( GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects()); - convertObjectsToTuples(block, extended_storage_columns); + convertDynamicColumnsToTuples(block, storage_snapshot); } if (storage.compress) @@ -207,10 +207,10 @@ StorageSnapshotPtr StorageMemory::getStorageSnapshot(const StorageMetadataPtr & auto snapshot_data = std::make_unique(); snapshot_data->blocks = data.get(); - if (!hasObjectColumns(metadata_snapshot->getColumns())) + if (!hasDynamicSubcolumns(metadata_snapshot->getColumns())) return std::make_shared(*this, metadata_snapshot, ColumnsDescription{}, std::move(snapshot_data)); - auto object_columns = getObjectColumns( + auto object_columns = getConcreteObjectColumns( snapshot_data->blocks->begin(), snapshot_data->blocks->end(), metadata_snapshot->getColumns(), diff --git a/src/Storages/StorageSnapshot.cpp b/src/Storages/StorageSnapshot.cpp index a4b64c798f3..6745b8fb3f5 100644 --- a/src/Storages/StorageSnapshot.cpp +++ b/src/Storages/StorageSnapshot.cpp @@ -60,7 +60,7 @@ std::optional StorageSnapshot::tryGetColumn(const GetColumnsOpt { const auto & columns = getMetadataForQuery()->getColumns(); auto column = columns.tryGetColumn(options, column_name); - if (column && (!isObject(column->type) || !options.with_extended_objects)) + if (column && (!column->type->hasDynamicSubcolumns() || !options.with_extended_objects)) return column; if (options.with_extended_objects) diff --git a/src/Storages/getStructureOfRemoteTable.cpp b/src/Storages/getStructureOfRemoteTable.cpp index 8fa4d02e8e1..d022287a671 100644 --- a/src/Storages/getStructureOfRemoteTable.cpp +++ b/src/Storages/getStructureOfRemoteTable.cpp @@ -189,7 +189,7 @@ ColumnsDescriptionByShardNum getExtendedObjectsOfRemoteTables( auto type_name = get(type_col[i]); auto storage_column = storage_columns.tryGetPhysical(name); - if (storage_column && isObject(storage_column->type)) + if (storage_column && storage_column->type->hasDynamicSubcolumns()) res.add(ColumnDescription(std::move(name), DataTypeFactory::instance().get(type_name))); } } diff --git a/tests/queries/0_stateless/01825_type_json_in_array.reference b/tests/queries/0_stateless/01825_type_json_in_array.reference new file mode 100644 index 00000000000..1b6ca177b57 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_in_array.reference @@ -0,0 +1,17 @@ +{"id":1,"arr":[{"k1":1,"k2":{"k3":2,"k4":3,"k5":""}},{"k1":2,"k2":{"k3":0,"k4":0,"k5":"foo"}}]} +{"id":2,"arr":[{"k1":3,"k2":{"k3":4,"k4":5,"k5":""}}]} +1 [1,2] [2,0] [3,0] ['','foo'] +2 [3] [4] [5] [''] +{"arr":{"k1":1,"k2":{"k3":2,"k4":3,"k5":""}}} +{"arr":{"k1":2,"k2":{"k3":0,"k4":0,"k5":"foo"}}} +{"arr":{"k1":3,"k2":{"k3":4,"k4":5,"k5":""}}} +Array(Tuple(k1 Int8, k2 Tuple(k3 Int8, k4 Int8, k5 String))) +{"id":1,"arr":[{"k1":[{"k2":"aaa","k3":"bbb","k4":0},{"k2":"ccc","k3":"","k4":0}],"k5":{"k6":""}}]} +{"id":2,"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}]} +1 [['aaa','ccc']] [['bbb','']] [[]] [''] +2 [[]] [['ddd','']] [[10,20]] ['foo'] +{"k1":{"k2":"","k3":"","k4":20}} +{"k1":{"k2":"","k3":"ddd","k4":10}} +{"k1":{"k2":"aaa","k3":"bbb","k4":0}} +{"k1":{"k2":"ccc","k3":"","k4":0}} +Tuple(k2 String, k3 String, k4 Int8) diff --git a/tests/queries/0_stateless/01825_type_json_in_array.sql b/tests/queries/0_stateless/01825_type_json_in_array.sql new file mode 100644 index 00000000000..abb117fbfa4 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_in_array.sql @@ -0,0 +1,27 @@ +-- Tags: no-fasttest + +SET allow_experimental_object_type = 1; +DROP TABLE IF EXISTS t_json_array; + +CREATE TABLE t_json_array (id UInt32, arr Array(JSON)) ENGINE = MergeTree ORDER BY id; + +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": 1, "k2": {"k3": 2, "k4": 3}}, {"k1": 2, "k2": {"k5": "foo"}}]} +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": 3, "k2": {"k3": 4, "k4": 5}}]} + +SET output_format_json_named_tuples_as_objects = 1; + +SELECT * FROM t_json_array ORDER BY id FORMAT JSONEachRow; +SELECT id, arr.k1, arr.k2.k3, arr.k2.k4, arr.k2.k5 FROM t_json_array ORDER BY id; +SELECT arr FROM t_json_array ARRAY JOIN arr ORDER BY arr.k1 FORMAT JSONEachRow; +SELECT toTypeName(arr) FROM t_json_array LIMIT 1; + +TRUNCATE TABLE t_json_array; + +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 1, "arr": [{"k1": [{"k2": "aaa", "k3": "bbb"}, {"k2": "ccc"}]}]} +INSERT INTO t_json_array FORMAT JSONEachRow {"id": 2, "arr": [{"k1": [{"k3": "ddd", "k4": 10}, {"k4": 20}], "k5": {"k6": "foo"}}]} + +SELECT * FROM t_json_array ORDER BY id FORMAT JSONEachRow; +SELECT id, arr.k1.k2, arr.k1.k3, arr.k1.k4, arr.k5.k6 FROM t_json_array ORDER BY id; + +SELECT arrayJoin(arrayJoin(arr.k1)) AS k1 FROM t_json_array ORDER BY k1 FORMAT JSONEachRow; +SELECT toTypeName(arrayJoin(arrayJoin(arr.k1))) AS arr FROM t_json_array LIMIT 1; From 90c60741a172e641d70e2d3766d0b702c209d880 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 6 May 2022 17:13:26 +0000 Subject: [PATCH 002/152] more safe operations with ColumnObject --- src/Columns/ColumnObject.cpp | 102 ++++++++++-------- src/Columns/ColumnObject.h | 4 +- .../Serializations/SerializationObject.cpp | 9 +- src/Interpreters/convertFieldToType.cpp | 3 + .../01825_type_json_in_array.reference | 4 + .../0_stateless/01825_type_json_in_array.sql | 7 ++ 6 files changed, 83 insertions(+), 46 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 0913f6b8da9..8413a709e05 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -553,11 +553,9 @@ size_t ColumnObject::allocatedBytes() const void ColumnObject::forEachSubcolumn(ColumnCallback callback) { - if (!isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot iterate over non-finalized ColumnObject"); - for (auto & entry : subcolumns) - callback(entry->data.data.back()); + for (auto & part : entry->data.data) + callback(part); } void ColumnObject::insert(const Field & field) @@ -594,26 +592,43 @@ void ColumnObject::insertDefault() Field ColumnObject::operator[](size_t n) const { - if (!isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get Field from non-finalized ColumnObject"); - - Object object; - for (const auto & entry : subcolumns) - object[entry->path.getPath()] = (*entry->data.data.back())[n]; - + Field object; + get(n, object); return object; } void ColumnObject::get(size_t n, Field & res) const { - if (!isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot get Field from non-finalized ColumnObject"); - + assert(n < size()); + res = Object(); auto & object = res.get(); - for (const auto & entry : subcolumns) + + if (isFinalized()) { - auto it = object.try_emplace(entry->path.getPath()).first; - entry->data.data.back()->get(n, it->second); + for (const auto & entry : subcolumns) + { + auto it = object.try_emplace(entry->path.getPath()).first; + entry->data.data.back()->get(n, it->second); + } + } + else + { + for (const auto & entry : subcolumns) + { + size_t ind = n; + for (const auto & part : entry->data.data) + { + if (ind < part->size()) + { + auto it = object.try_emplace(entry->path.getPath()).first; + part->get(ind, it->second); + it->second = convertFieldToTypeOrThrow(it->second, *entry->data.getLeastCommonType()); + break; + } + + ind -= part->size(); + } + } } } @@ -625,19 +640,25 @@ void ColumnObject::insertFrom(const IColumn & src, size_t n) void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t length) { - const auto & src_object = assert_cast(src); - if (!src_object.isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot insertRangeFrom non-finalized ColumnObject"); + const auto * src_object = &assert_cast(src); + MutableColumnPtr column_holder; + + if (!src_object->isFinalized()) + { + column_holder = IColumn::mutate(src_object->getPtr()); + column_holder->finalize(); + src_object = &assert_cast(*column_holder); + } for (auto & entry : subcolumns) { - if (src_object.hasSubcolumn(entry->path)) - entry->data.insertRangeFrom(src_object.getSubcolumn(entry->path), start, length); + if (src_object->hasSubcolumn(entry->path)) + entry->data.insertRangeFrom(src_object->getSubcolumn(entry->path), start, length); else entry->data.insertManyDefaults(length); } - for (const auto & entry : src_object.subcolumns) + for (const auto & entry : src_object->subcolumns) { if (!hasSubcolumn(entry->path)) { @@ -668,21 +689,6 @@ void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t len finalize(); } -ColumnPtr ColumnObject::replicate(const Offsets & offsets) const -{ - if (!isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot replicate non-finalized ColumnObject"); - - auto res_column = ColumnObject::create(is_nullable); - for (const auto & entry : subcolumns) - { - auto replicated_data = entry->data.data.back()->replicate(offsets)->assumeMutable(); - res_column->addSubcolumn(entry->path, std::move(replicated_data)); - } - - return res_column; -} - void ColumnObject::popBack(size_t length) { for (auto & entry : subcolumns) @@ -692,10 +698,15 @@ void ColumnObject::popBack(size_t length) } template -ColumnPtr ColumnObject::applyForSubcolumns(Func && func, std::string_view func_name) const +ColumnPtr ColumnObject::applyForSubcolumns(Func && func) const { if (!isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot {} non-finalized ColumnObject", func_name); + { + auto finalized = IColumn::mutate(getPtr()); + auto & finalized_object = assert_cast(*finalized); + finalized_object.finalize(); + return finalized_object.applyForSubcolumns(std::move(func)); + } auto res = ColumnObject::create(is_nullable); for (const auto & subcolumn : subcolumns) @@ -708,17 +719,22 @@ ColumnPtr ColumnObject::applyForSubcolumns(Func && func, std::string_view func_n ColumnPtr ColumnObject::permute(const Permutation & perm, size_t limit) const { - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.permute(perm, limit); }, "permute"); + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.permute(perm, limit); }); } ColumnPtr ColumnObject::filter(const Filter & filter, ssize_t result_size_hint) const { - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.filter(filter, result_size_hint); }, "filter"); + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.filter(filter, result_size_hint); }); } ColumnPtr ColumnObject::index(const IColumn & indexes, size_t limit) const { - return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.index(indexes, limit); }, "index"); + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.index(indexes, limit); }); +} + +ColumnPtr ColumnObject::replicate(const Offsets & offsets) const +{ + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.replicate(offsets); }); } const ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key) const diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 6c69b0247b7..154122f6743 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -195,13 +195,13 @@ public: void insertDefault() override; void insertFrom(const IColumn & src, size_t n) override; void insertRangeFrom(const IColumn & src, size_t start, size_t length) override; - ColumnPtr replicate(const Offsets & offsets) const override; void popBack(size_t length) override; Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; + ColumnPtr replicate(const Offsets & offsets) const override; /// Finalizes all subcolumns. void finalize() override; @@ -237,7 +237,7 @@ private: } template - ColumnPtr applyForSubcolumns(Func && func, std::string_view func_name) const; + ColumnPtr applyForSubcolumns(Func && func) const; }; } diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 2d6555dcb43..0229754f677 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -241,7 +241,14 @@ void SerializationObject::serializeBinaryBulkWithMultipleStreams( const auto & column_object = assert_cast(column); if (!column_object.isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot write non-finalized ColumnObject"); + { + auto finalized = IColumn::mutate(column_object.getPtr()); + auto & finalized_object = assert_cast(*finalized); + finalized_object.finalize(); + serializeBinaryBulkWithMultipleStreams( + finalized_object, offset, limit, settings, state); + return; + } settings.path.push_back(Substream::ObjectStructure); if (auto * stream = settings.getter(settings.path)) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index dd23ad69ae2..36970895b65 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -370,6 +370,9 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } else if (isObject(type)) { + if (src.getType() == Field::Types::Object) + return src; /// Already in needed type. + const auto * from_type_tuple = typeid_cast(from_type_hint); if (src.getType() == Field::Types::Tuple && from_type_tuple && from_type_tuple->haveExplicitNames()) { diff --git a/tests/queries/0_stateless/01825_type_json_in_array.reference b/tests/queries/0_stateless/01825_type_json_in_array.reference index 1b6ca177b57..0bdb9917622 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.reference +++ b/tests/queries/0_stateless/01825_type_json_in_array.reference @@ -15,3 +15,7 @@ Array(Tuple(k1 Int8, k2 Tuple(k3 Int8, k4 Int8, k5 String))) {"k1":{"k2":"aaa","k3":"bbb","k4":0}} {"k1":{"k2":"ccc","k3":"","k4":0}} Tuple(k2 String, k3 String, k4 Int8) +{"arr":[{"x":1}]} +{"arr":{"x":{"y":1},"t":{"y":2}}} +{"arr":[1,{"y":1}]} +{"arr":[{"x":"aaa","y":[1,2,3]}]} diff --git a/tests/queries/0_stateless/01825_type_json_in_array.sql b/tests/queries/0_stateless/01825_type_json_in_array.sql index abb117fbfa4..b02202ec175 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.sql +++ b/tests/queries/0_stateless/01825_type_json_in_array.sql @@ -25,3 +25,10 @@ SELECT id, arr.k1.k2, arr.k1.k3, arr.k1.k4, arr.k5.k6 FROM t_json_array ORDER BY SELECT arrayJoin(arrayJoin(arr.k1)) AS k1 FROM t_json_array ORDER BY k1 FORMAT JSONEachRow; SELECT toTypeName(arrayJoin(arrayJoin(arr.k1))) AS arr FROM t_json_array LIMIT 1; + +DROP TABLE t_json_array; + +SELECT * FROM values('arr Array(JSON)', '[\'{"x" : 1}\']') FORMAT JSONEachRow; +SELECT * FROM values('arr Map(String, JSON)', '{\'x\' : \'{"y" : 1}\', \'t\' : \'{"y" : 2}\'}') FORMAT JSONEachRow; +SELECT * FROM values('arr Tuple(Int32, JSON)', '(1, \'{"y" : 1}\')') FORMAT JSONEachRow; +SELECT * FROM format(JSONEachRow, '{"arr" : [{"x" : "aaa", "y" : [1,2,3]}]}') FORMAT JSONEachRow; From b3e262f605e7f451471a0e93ddcf121ea578cd0c Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Fri, 6 May 2022 18:52:53 +0000 Subject: [PATCH 003/152] better cloneResized in ColumnObject --- src/Columns/ColumnObject.cpp | 20 +++++++++----------- src/Columns/ColumnObject.h | 5 +++-- 2 files changed, 12 insertions(+), 13 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 8413a709e05..64f92156175 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -525,16 +525,6 @@ size_t ColumnObject::size() const return num_rows; } -MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const -{ - /// cloneResized with new_size == 0 is used for cloneEmpty(). - if (new_size != 0) - throw Exception(ErrorCodes::NOT_IMPLEMENTED, - "ColumnObject doesn't support resize to non-zero length"); - - return ColumnObject::create(is_nullable); -} - size_t ColumnObject::byteSize() const { size_t res = 0; @@ -698,7 +688,7 @@ void ColumnObject::popBack(size_t length) } template -ColumnPtr ColumnObject::applyForSubcolumns(Func && func) const +MutableColumnPtr ColumnObject::applyForSubcolumns(Func && func) const { if (!isFinalized()) { @@ -737,6 +727,14 @@ ColumnPtr ColumnObject::replicate(const Offsets & offsets) const return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.replicate(offsets); }); } +MutableColumnPtr ColumnObject::cloneResized(size_t new_size) const +{ + if (new_size == 0) + return ColumnObject::create(is_nullable); + + return applyForSubcolumns([&](const auto & subcolumn) { return subcolumn.cloneResized(new_size); }); +} + const ColumnObject::Subcolumn & ColumnObject::getSubcolumn(const PathInData & key) const { if (const auto * node = subcolumns.findLeaf(key)) diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 154122f6743..8bed90bd0ce 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -187,7 +187,6 @@ public: TypeIndex getDataType() const override { return TypeIndex::Object; } size_t size() const override; - MutableColumnPtr cloneResized(size_t new_size) const override; size_t byteSize() const override; size_t allocatedBytes() const override; void forEachSubcolumn(ColumnCallback callback) override; @@ -198,10 +197,12 @@ public: void popBack(size_t length) override; Field operator[](size_t n) const override; void get(size_t n, Field & res) const override; + ColumnPtr permute(const Permutation & perm, size_t limit) const override; ColumnPtr filter(const Filter & filter, ssize_t result_size_hint) const override; ColumnPtr index(const IColumn & indexes, size_t limit) const override; ColumnPtr replicate(const Offsets & offsets) const override; + MutableColumnPtr cloneResized(size_t new_size) const override; /// Finalizes all subcolumns. void finalize() override; @@ -237,7 +238,7 @@ private: } template - ColumnPtr applyForSubcolumns(Func && func) const; + MutableColumnPtr applyForSubcolumns(Func && func) const; }; } From e1911a29b81bff19573549cbc5d7960ab89476df Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 9 May 2022 16:12:04 +0000 Subject: [PATCH 004/152] better text serialization of type Object --- src/Columns/ColumnArray.h | 1 + src/Columns/ColumnObject.cpp | 81 +++++++++++-------- src/Columns/ColumnObject.h | 7 +- src/Columns/ColumnTuple.cpp | 5 ++ src/Columns/ColumnTuple.h | 1 + src/Columns/IColumn.h | 8 ++ src/DataTypes/ObjectUtils.cpp | 5 +- .../Serializations/SerializationObject.cpp | 70 ++++++++++++---- .../Serializations/SerializationObject.h | 2 + 9 files changed, 127 insertions(+), 53 deletions(-) diff --git a/src/Columns/ColumnArray.h b/src/Columns/ColumnArray.h index 1657160fbae..3e3462c7cdf 100644 --- a/src/Columns/ColumnArray.h +++ b/src/Columns/ColumnArray.h @@ -169,6 +169,7 @@ public: void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; void finalize() override { data->finalize(); } + bool isFinalized() const override { return data->isFinalized(); } bool isCollationSupported() const override { return getData().isCollationSupported(); } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 64f92156175..d0c5997c071 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -24,6 +24,7 @@ namespace ErrorCodes extern const int NUMBER_OF_DIMENSIONS_MISMATHED; extern const int NOT_IMPLEMENTED; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int ARGUMENT_OUT_OF_BOUND; } namespace @@ -179,7 +180,7 @@ ColumnObject::Subcolumn::Subcolumn( { } -size_t ColumnObject::Subcolumn::Subcolumn::size() const +size_t ColumnObject::Subcolumn::size() const { size_t res = num_of_defaults_in_prefix; for (const auto & part : data) @@ -187,7 +188,7 @@ size_t ColumnObject::Subcolumn::Subcolumn::size() const return res; } -size_t ColumnObject::Subcolumn::Subcolumn::byteSize() const +size_t ColumnObject::Subcolumn::byteSize() const { size_t res = 0; for (const auto & part : data) @@ -195,7 +196,7 @@ size_t ColumnObject::Subcolumn::Subcolumn::byteSize() const return res; } -size_t ColumnObject::Subcolumn::Subcolumn::allocatedBytes() const +size_t ColumnObject::Subcolumn::allocatedBytes() const { size_t res = 0; for (const auto & part : data) @@ -203,6 +204,37 @@ size_t ColumnObject::Subcolumn::Subcolumn::allocatedBytes() const return res; } +void ColumnObject::Subcolumn::get(size_t n, Field & res) const +{ + if (isFinalized()) + { + getFinalizedColumn().get(n, res); + return; + } + + size_t ind = n; + if (ind < num_of_defaults_in_prefix) + { + res = least_common_type.get()->getDefault(); + return; + } + + ind -= num_of_defaults_in_prefix; + for (const auto & part : data) + { + if (ind < part->size()) + { + part->get(ind, res); + res = convertFieldToTypeOrThrow(res, *least_common_type.get()); + return; + } + + ind -= part->size(); + } + + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for getting field is out of range", n); +} + void ColumnObject::Subcolumn::checkTypes() const { DataTypes prefix_types; @@ -593,39 +625,16 @@ void ColumnObject::get(size_t n, Field & res) const res = Object(); auto & object = res.get(); - if (isFinalized()) + for (const auto & entry : subcolumns) { - for (const auto & entry : subcolumns) - { - auto it = object.try_emplace(entry->path.getPath()).first; - entry->data.data.back()->get(n, it->second); - } - } - else - { - for (const auto & entry : subcolumns) - { - size_t ind = n; - for (const auto & part : entry->data.data) - { - if (ind < part->size()) - { - auto it = object.try_emplace(entry->path.getPath()).first; - part->get(ind, it->second); - it->second = convertFieldToTypeOrThrow(it->second, *entry->data.getLeastCommonType()); - break; - } - - ind -= part->size(); - } - } + auto it = object.try_emplace(entry->path.getPath()).first; + entry->data.get(n, it->second); } } void ColumnObject::insertFrom(const IColumn & src, size_t n) { insert(src[n]); - finalize(); } void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t length) @@ -635,8 +644,7 @@ void ColumnObject::insertRangeFrom(const IColumn & src, size_t start, size_t len if (!src_object->isFinalized()) { - column_holder = IColumn::mutate(src_object->getPtr()); - column_holder->finalize(); + column_holder = src.cloneFinalized(); src_object = &assert_cast(*column_holder); } @@ -692,10 +700,9 @@ MutableColumnPtr ColumnObject::applyForSubcolumns(Func && func) const { if (!isFinalized()) { - auto finalized = IColumn::mutate(getPtr()); + auto finalized = cloneFinalized(); auto & finalized_object = assert_cast(*finalized); - finalized_object.finalize(); - return finalized_object.applyForSubcolumns(std::move(func)); + return finalized_object.applyForSubcolumns(std::forward(func)); } auto res = ColumnObject::create(is_nullable); @@ -704,6 +711,7 @@ MutableColumnPtr ColumnObject::applyForSubcolumns(Func && func) const auto new_subcolumn = func(subcolumn->data.getFinalizedColumn()); res->addSubcolumn(subcolumn->path, new_subcolumn->assumeMutable()); } + return res; } @@ -843,13 +851,16 @@ bool ColumnObject::isFinalized() const void ColumnObject::finalize() { + if (isFinalized()) + return; + size_t old_size = size(); Subcolumns new_subcolumns; for (auto && entry : subcolumns) { const auto & least_common_type = entry->data.getLeastCommonType(); - /// Do not add subcolumns, which consists only from NULLs. + /// Do not add subcolumns, which consist only from NULLs. if (isNothing(getBaseTypeOfArray(least_common_type))) continue; diff --git a/src/Columns/ColumnObject.h b/src/Columns/ColumnObject.h index 8bed90bd0ce..762049afd38 100644 --- a/src/Columns/ColumnObject.h +++ b/src/Columns/ColumnObject.h @@ -65,6 +65,7 @@ public: size_t size() const; size_t byteSize() const; size_t allocatedBytes() const; + void get(size_t n, Field & res) const; bool isFinalized() const; const DataTypePtr & getLeastCommonType() const { return least_common_type.get(); } @@ -101,6 +102,9 @@ public: const IColumn & getFinalizedColumn() const; const ColumnPtr & getFinalizedColumnPtr() const; + const std::vector & getData() const { return data; } + size_t getNumberOfDefaultsInPrefix() const { return num_of_defaults_in_prefix; } + friend class ColumnObject; private: @@ -179,8 +183,6 @@ public: Subcolumns & getSubcolumns() { return subcolumns; } PathsInData getKeys() const; - bool isFinalized() const; - /// Part of interface const char * getFamilyName() const override { return "Object"; } @@ -206,6 +208,7 @@ public: /// Finalizes all subcolumns. void finalize() override; + bool isFinalized() const override; /// All other methods throw exception. diff --git a/src/Columns/ColumnTuple.cpp b/src/Columns/ColumnTuple.cpp index 8d6b2dbdbd9..c260e48dde5 100644 --- a/src/Columns/ColumnTuple.cpp +++ b/src/Columns/ColumnTuple.cpp @@ -578,4 +578,9 @@ void ColumnTuple::finalize() column->finalize(); } +bool ColumnTuple::isFinalized() const +{ + return std::all_of(columns.begin(), columns.end(), [](const auto & column) { return column->isFinalized(); }); +} + } diff --git a/src/Columns/ColumnTuple.h b/src/Columns/ColumnTuple.h index 7b554ed2fea..93b937251c0 100644 --- a/src/Columns/ColumnTuple.h +++ b/src/Columns/ColumnTuple.h @@ -104,6 +104,7 @@ public: void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; SerializationInfoPtr getSerializationInfo() const override; void finalize() override; + bool isFinalized() const override; size_t tupleSize() const { return columns.size(); } diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 4bf364d9b52..1b67a3d65ad 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -444,6 +444,14 @@ public: /// Some columns may require finalization before using of other operations. virtual void finalize() {} + virtual bool isFinalized() const { return true; } + + MutablePtr cloneFinalized() const + { + auto finalized = IColumn::mutate(getPtr()); + finalized->finalize(); + return finalized; + } [[nodiscard]] static MutablePtr mutate(Ptr ptr) { diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 112998e176d..dea3dfe9e33 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -105,10 +105,11 @@ Array createEmptyArrayField(size_t num_dimensions) DataTypePtr getDataTypeByColumn(const IColumn & column) { auto idx = column.getDataType(); - if (WhichDataType(idx).isSimple()) + WhichDataType which(idx); + if (which.isSimple()) return DataTypeFactory::instance().get(String(magic_enum::enum_name(idx))); - if (WhichDataType(idx).isNothing()) + if (which.isNothing()) return std::make_shared(); if (const auto * column_array = checkAndGetColumn(&column)) diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 0229754f677..56d098b47b3 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -10,8 +10,6 @@ #include #include -#include - #include #include #include @@ -242,11 +240,8 @@ void SerializationObject::serializeBinaryBulkWithMultipleStreams( if (!column_object.isFinalized()) { - auto finalized = IColumn::mutate(column_object.getPtr()); - auto & finalized_object = assert_cast(*finalized); - finalized_object.finalize(); - serializeBinaryBulkWithMultipleStreams( - finalized_object, offset, limit, settings, state); + auto finalized = column_object.cloneFinalized(); + serializeBinaryBulkWithMultipleStreams(*finalized, offset, limit, settings, state); return; } @@ -374,21 +369,68 @@ void SerializationObject::serializeTextImpl(const IColumn & column, size const auto & column_object = assert_cast(column); const auto & subcolumns = column_object.getSubcolumns(); + bool first = true; writeChar('{', ostr); - for (auto it = subcolumns.begin(); it != subcolumns.end(); ++it) - { - if (it != subcolumns.begin()) - writeCString(",", ostr); - writeDoubleQuoted((*it)->path.getPath(), ostr); + for (const auto & entry : subcolumns) + { + WriteBufferFromOwnString value_buf; + bool have_value = serializeTextFromSubcolumn(entry->data, row_num, value_buf, settings); + if (!have_value) + continue; + + if (!first) + writeCString(",", ostr); + else + first = false; + + writeDoubleQuoted(entry->path.getPath(), ostr); writeChar(':', ostr); - auto serialization = (*it)->data.getLeastCommonType()->getDefaultSerialization(); - serialization->serializeTextJSON((*it)->data.getFinalizedColumn(), row_num, ostr, settings); + auto value = value_buf.stringRef(); + ostr.write(value.data, value.size); } writeChar('}', ostr); } +template +bool SerializationObject::serializeTextFromSubcolumn( + const ColumnObject::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const +{ + if (subcolumn.isFinalized()) + { + const auto & finalized_column = subcolumn.getFinalizedColumn(); + if (finalized_column.isDefaultAt(row_num)) + return false; + + auto serialization = subcolumn.getLeastCommonType()->getSerialization(*finalized_column.getSerializationInfo()); + serialization->serializeTextJSON(finalized_column, row_num, ostr, settings); + return true; + } + + size_t ind = row_num; + if (ind < subcolumn.getNumberOfDefaultsInPrefix()) + return false; + + ind -= subcolumn.getNumberOfDefaultsInPrefix(); + for (const auto & part : subcolumn.getData()) + { + if (ind < part->size()) + { + if (part->isDefaultAt(ind)) + return false; + + auto serialization = getDataTypeByColumn(*part)->getSerialization(*part->getSerializationInfo()); + serialization->serializeTextJSON(*part, ind, ostr, settings); + return true; + } + + ind -= part->size(); + } + + return false; +} + template void SerializationObject::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 549c8735aee..74ba2e28bdb 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -1,5 +1,6 @@ #pragma once +#include "Columns/ColumnObject.h" #include #include @@ -65,6 +66,7 @@ private: void deserializeTextImpl(IColumn & column, Reader && reader) const; void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; + bool serializeTextFromSubcolumn(const ColumnObject::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; /// Pool of parser objects to make SerializationObject thread safe. mutable SimpleObjectPool parsers_pool; From 9d2a4d970dcaf0adca714a88d715a21ee6753989 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 9 May 2022 18:05:55 +0000 Subject: [PATCH 005/152] better text serialization of type Object --- src/Columns/ColumnObject.cpp | 4 +- src/DataTypes/ObjectUtils.cpp | 11 +++-- .../Serializations/SerializationObject.cpp | 42 ++++++++----------- .../Serializations/SerializationObject.h | 2 +- .../01825_type_json_in_array.reference | 1 + .../0_stateless/01825_type_json_in_array.sql | 2 +- 6 files changed, 29 insertions(+), 33 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index d0c5997c071..85a80c29756 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -364,8 +364,8 @@ void ColumnObject::Subcolumn::insertRangeFrom(const Subcolumn & src, size_t star bool ColumnObject::Subcolumn::isFinalized() const { - return data.empty() || - (data.size() == 1 && !data[0]->isSparse() && num_of_defaults_in_prefix == 0); + return num_of_defaults_in_prefix == 0 && + (data.empty() || (data.size() == 1 && !data[0]->isSparse())); } void ColumnObject::Subcolumn::finalize() diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index dea3dfe9e33..4117bcc5c08 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -143,11 +143,14 @@ static DataTypePtr recreateTupleWithElements(const DataTypeTuple & type_tuple, c static std::pair convertObjectColumnToTuple( const ColumnObject & column_object, const DataTypeObject & type_object) { - const auto & subcolumns = column_object.getSubcolumns(); - if (!column_object.isFinalized()) - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Cannot convert to tuple column of type {}. Column should be finalized first", type_object.getName()); + { + auto finalized = column_object.cloneFinalized(); + const auto & finalized_object = assert_cast(*finalized); + return convertObjectColumnToTuple(finalized_object, type_object); + } + + const auto & subcolumns = column_object.getSubcolumns(); PathsInData tuple_paths; DataTypes tuple_types; diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 56d098b47b3..593a9a1bb02 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -23,6 +23,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; extern const int CANNOT_READ_ALL_DATA; extern const int LOGICAL_ERROR; + extern const int ARGUMENT_OUT_OF_BOUND; } namespace @@ -369,66 +370,57 @@ void SerializationObject::serializeTextImpl(const IColumn & column, size const auto & column_object = assert_cast(column); const auto & subcolumns = column_object.getSubcolumns(); - bool first = true; writeChar('{', ostr); - - for (const auto & entry : subcolumns) + for (auto it = subcolumns.begin(); it != subcolumns.end(); ++it) { - WriteBufferFromOwnString value_buf; - bool have_value = serializeTextFromSubcolumn(entry->data, row_num, value_buf, settings); - if (!have_value) - continue; - - if (!first) + const auto & entry = *it; + if (it != subcolumns.begin()) writeCString(",", ostr); - else - first = false; writeDoubleQuoted(entry->path.getPath(), ostr); writeChar(':', ostr); - - auto value = value_buf.stringRef(); - ostr.write(value.data, value.size); + serializeTextFromSubcolumn(entry->data, row_num, ostr, settings); } writeChar('}', ostr); } template -bool SerializationObject::serializeTextFromSubcolumn( +void SerializationObject::serializeTextFromSubcolumn( const ColumnObject::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const { if (subcolumn.isFinalized()) { const auto & finalized_column = subcolumn.getFinalizedColumn(); - if (finalized_column.isDefaultAt(row_num)) - return false; - auto serialization = subcolumn.getLeastCommonType()->getSerialization(*finalized_column.getSerializationInfo()); serialization->serializeTextJSON(finalized_column, row_num, ostr, settings); - return true; + return; } size_t ind = row_num; if (ind < subcolumn.getNumberOfDefaultsInPrefix()) - return false; + { + /// Suboptimal, but it should happen rarely. + auto tmp_column = subcolumn.getLeastCommonType()->createColumn(); + tmp_column->insertDefault(); + auto serialization = subcolumn.getLeastCommonType()->getSerialization(*tmp_column->getSerializationInfo()); + serialization->serializeTextJSON(*tmp_column, 0, ostr, settings); + return; + } ind -= subcolumn.getNumberOfDefaultsInPrefix(); for (const auto & part : subcolumn.getData()) { if (ind < part->size()) { - if (part->isDefaultAt(ind)) - return false; - auto serialization = getDataTypeByColumn(*part)->getSerialization(*part->getSerializationInfo()); serialization->serializeTextJSON(*part, ind, ostr, settings); - return true; + return; } ind -= part->size(); } - return false; + throw Exception(ErrorCodes::ARGUMENT_OUT_OF_BOUND, "Index ({}) for text serialization is out of range", row_num); } template diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 74ba2e28bdb..0abb6a98d63 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -66,7 +66,7 @@ private: void deserializeTextImpl(IColumn & column, Reader && reader) const; void serializeTextImpl(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; - bool serializeTextFromSubcolumn(const ColumnObject::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; + void serializeTextFromSubcolumn(const ColumnObject::Subcolumn & subcolumn, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const; /// Pool of parser objects to make SerializationObject thread safe. mutable SimpleObjectPool parsers_pool; diff --git a/tests/queries/0_stateless/01825_type_json_in_array.reference b/tests/queries/0_stateless/01825_type_json_in_array.reference index 0bdb9917622..3f6dc6ccd88 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.reference +++ b/tests/queries/0_stateless/01825_type_json_in_array.reference @@ -18,4 +18,5 @@ Tuple(k2 String, k3 String, k4 Int8) {"arr":[{"x":1}]} {"arr":{"x":{"y":1},"t":{"y":2}}} {"arr":[1,{"y":1}]} +{"arr":[2,{"y":2}]} {"arr":[{"x":"aaa","y":[1,2,3]}]} diff --git a/tests/queries/0_stateless/01825_type_json_in_array.sql b/tests/queries/0_stateless/01825_type_json_in_array.sql index b02202ec175..935969272fa 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.sql +++ b/tests/queries/0_stateless/01825_type_json_in_array.sql @@ -30,5 +30,5 @@ DROP TABLE t_json_array; SELECT * FROM values('arr Array(JSON)', '[\'{"x" : 1}\']') FORMAT JSONEachRow; SELECT * FROM values('arr Map(String, JSON)', '{\'x\' : \'{"y" : 1}\', \'t\' : \'{"y" : 2}\'}') FORMAT JSONEachRow; -SELECT * FROM values('arr Tuple(Int32, JSON)', '(1, \'{"y" : 1}\')') FORMAT JSONEachRow; +SELECT * FROM values('arr Tuple(Int32, JSON)', '(1, \'{"y" : 1}\')', '(2, \'{"y" : 2}\')') FORMAT JSONEachRow; SELECT * FROM format(JSONEachRow, '{"arr" : [{"x" : "aaa", "y" : [1,2,3]}]}') FORMAT JSONEachRow; From 1f91655cf95b257f8dd5ffec33250bf945cc2f6f Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Tue, 10 May 2022 15:32:17 +0000 Subject: [PATCH 006/152] support Object inside Map --- src/Columns/ColumnMap.h | 2 + src/Columns/ColumnObject.cpp | 13 +- src/DataTypes/DataTypeMap.cpp | 21 +++ src/DataTypes/DataTypeMap.h | 2 + src/DataTypes/IDataType.h | 1 + src/DataTypes/ObjectUtils.cpp | 121 +++++++++++------- .../Serializations/SerializationObject.h | 4 +- .../01825_type_json_in_other_types.reference | 17 +++ .../01825_type_json_in_other_types.sh | 91 +++++++++++++ 9 files changed, 219 insertions(+), 53 deletions(-) create mode 100644 tests/queries/0_stateless/01825_type_json_in_other_types.reference create mode 100755 tests/queries/0_stateless/01825_type_json_in_other_types.sh diff --git a/src/Columns/ColumnMap.h b/src/Columns/ColumnMap.h index 95838e70d10..3b9ff07c09e 100644 --- a/src/Columns/ColumnMap.h +++ b/src/Columns/ColumnMap.h @@ -92,6 +92,8 @@ public: bool structureEquals(const IColumn & rhs) const override; double getRatioOfDefaultRows(double sample_ratio) const override; void getIndicesOfNonDefaultRows(Offsets & indices, size_t from, size_t limit) const override; + void finalize() override { nested->finalize(); } + bool isFinalized() const override { return nested->isFinalized(); } const ColumnArray & getNestedColumn() const { return assert_cast(*nested); } ColumnArray & getNestedColumn() { return assert_cast(*nested); } diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 85a80c29756..34919ad2398 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -276,8 +276,8 @@ static bool isConversionRequiredBetweenIntegers(const IDataType & lhs, const IDa bool is_native_int = which_lhs.isNativeInt() && which_rhs.isNativeInt(); bool is_native_uint = which_lhs.isNativeUInt() && which_rhs.isNativeUInt(); - return (is_native_int || is_native_uint) - && lhs.getSizeOfValueInMemory() <= rhs.getSizeOfValueInMemory(); + return (!is_native_int && !is_native_uint) + || lhs.getSizeOfValueInMemory() > rhs.getSizeOfValueInMemory(); } void ColumnObject::Subcolumn::insert(Field field, FieldInfo info) @@ -320,7 +320,7 @@ void ColumnObject::Subcolumn::insert(Field field, FieldInfo info) } else if (!least_common_base_type->equals(*base_type) && !isNothing(base_type)) { - if (!isConversionRequiredBetweenIntegers(*base_type, *least_common_base_type)) + if (isConversionRequiredBetweenIntegers(*base_type, *least_common_base_type)) { base_type = getLeastSupertype(DataTypes{std::move(base_type), least_common_base_type}, true); type_changed = true; @@ -832,6 +832,10 @@ void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo & if (num_rows == 0) num_rows = new_size; + else if (new_size != num_rows) + throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, + "Required size of subcolumn {} ({}) is inconsistent with column size ({})", + key.getPath(), new_size, num_rows); } PathsInData ColumnObject::getKeys() const @@ -851,9 +855,6 @@ bool ColumnObject::isFinalized() const void ColumnObject::finalize() { - if (isFinalized()) - return; - size_t old_size = size(); Subcolumns new_subcolumns; for (auto && entry : subcolumns) diff --git a/src/DataTypes/DataTypeMap.cpp b/src/DataTypes/DataTypeMap.cpp index 42ec739c33b..d49c205fc59 100644 --- a/src/DataTypes/DataTypeMap.cpp +++ b/src/DataTypes/DataTypeMap.cpp @@ -22,6 +22,27 @@ namespace ErrorCodes extern const int BAD_ARGUMENTS; } +DataTypeMap::DataTypeMap(const DataTypePtr & nested_) + : nested(nested_) +{ + const auto * type_array = typeid_cast(nested.get()); + if (!type_array) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expected Array(Tuple(key, value)) type, got {}", nested->getName()); + + const auto * type_tuple = typeid_cast(type_array->getNestedType().get()); + if (!type_tuple) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expected Array(Tuple(key, value)) type, got {}", nested->getName()); + + if (type_tuple->getElements().size() != 2) + throw Exception(ErrorCodes::BAD_ARGUMENTS, + "Expected Array(Tuple(key, value)) type, got {}", nested->getName()); + + key_type = type_tuple->getElement(0); + value_type = type_tuple->getElement(1); + assertKeyType(); +} DataTypeMap::DataTypeMap(const DataTypes & elems_) { diff --git a/src/DataTypes/DataTypeMap.h b/src/DataTypes/DataTypeMap.h index 479008031fe..2ab5c602a25 100644 --- a/src/DataTypes/DataTypeMap.h +++ b/src/DataTypes/DataTypeMap.h @@ -23,6 +23,7 @@ private: public: static constexpr bool is_parametric = true; + explicit DataTypeMap(const DataTypePtr & nested_); explicit DataTypeMap(const DataTypes & elems); DataTypeMap(const DataTypePtr & key_type_, const DataTypePtr & value_type_); @@ -40,6 +41,7 @@ public: bool isComparable() const override { return key_type->isComparable() && value_type->isComparable(); } bool isParametric() const override { return true; } bool haveSubtypes() const override { return true; } + bool hasDynamicSubcolumns() const override { return nested->hasDynamicSubcolumns(); } const DataTypePtr & getKeyType() const { return key_type; } const DataTypePtr & getValueType() const { return value_type; } diff --git a/src/DataTypes/IDataType.h b/src/DataTypes/IDataType.h index 6d1d17c9e40..1fcac18b124 100644 --- a/src/DataTypes/IDataType.h +++ b/src/DataTypes/IDataType.h @@ -287,6 +287,7 @@ public: /// Strings, Numbers, Date, DateTime, Nullable virtual bool canBeInsideLowCardinality() const { return false; } + /// Object, Array(Object), Tuple(..., Object, ...) virtual bool hasDynamicSubcolumns() const { return false; } /// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column. diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 4117bcc5c08..4aea9a5f7e3 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -2,6 +2,7 @@ #include #include #include +#include #include #include #include @@ -12,6 +13,7 @@ #include #include #include +#include #include #include #include @@ -191,6 +193,19 @@ static std::pair recursivlyConvertDynamicColumnToTuple( }; } + if (const auto * type_map = typeid_cast(type.get())) + { + const auto & column_map = assert_cast(*column); + auto [new_column, new_type] = recursivlyConvertDynamicColumnToTuple( + column_map.getNestedColumnPtr(), type_map->getNestedType()); + + return + { + ColumnMap::create(std::move(new_column)), + std::make_shared(std::move(new_type)), + }; + } + if (const auto * type_tuple = typeid_cast(type.get())) { const auto & column_tuple = assert_cast(*column); @@ -343,7 +358,58 @@ static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool che return unflattenTuple(tuple_paths, tuple_types); } -DataTypePtr getLeastCommonTypeForDynamicColumnsImpl( +static DataTypePtr getLeastCommonTypeForDynamicColumnsImpl( + const DataTypePtr & type_in_storage, const DataTypes & concrete_types, bool check_ambiguos_paths); + +template +static DataTypePtr getLeastCommonTypeForColumnWithNestedType( + const Type & type, const DataTypes & concrete_types, bool check_ambiguos_paths) +{ + DataTypes nested_types; + nested_types.reserve(concrete_types.size()); + + for (const auto & concrete_type : concrete_types) + { + const auto * type_with_nested_conctete = typeid_cast(concrete_type.get()); + if (!type_with_nested_conctete) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected {} type, got {}", demangle(typeid(Type).name()), concrete_type->getName()); + + nested_types.push_back(type_with_nested_conctete->getNestedType()); + } + + return std::make_shared( + getLeastCommonTypeForDynamicColumnsImpl( + type.getNestedType(), nested_types, check_ambiguos_paths)); +} + +static DataTypePtr getLeastCommonTypeForTuple( + const DataTypeTuple & type, const DataTypes & concrete_types, bool check_ambiguos_paths) +{ + const auto & element_types = type.getElements(); + DataTypes new_element_types(element_types.size()); + + for (size_t i = 0; i < element_types.size(); ++i) + { + DataTypes concrete_element_types; + concrete_element_types.reserve(concrete_types.size()); + + for (const auto & type_concrete : concrete_types) + { + const auto * type_tuple_conctete = typeid_cast(type_concrete.get()); + if (!type_tuple_conctete) + throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected Tuple type, got {}", type_concrete->getName()); + + concrete_element_types.push_back(type_tuple_conctete->getElement(i)); + } + + new_element_types[i] = getLeastCommonTypeForDynamicColumnsImpl( + element_types[i], concrete_element_types, check_ambiguos_paths); + } + + return recreateTupleWithElements(type, new_element_types); +} + +static DataTypePtr getLeastCommonTypeForDynamicColumnsImpl( const DataTypePtr & type_in_storage, const DataTypes & concrete_types, bool check_ambiguos_paths) { if (!type_in_storage->hasDynamicSubcolumns()) @@ -353,49 +419,13 @@ DataTypePtr getLeastCommonTypeForDynamicColumnsImpl( return getLeastCommonTypeForObject(concrete_types, check_ambiguos_paths); if (const auto * type_array = typeid_cast(type_in_storage.get())) - { - DataTypes nested_types; - nested_types.reserve(concrete_types.size()); + return getLeastCommonTypeForColumnWithNestedType(*type_array, concrete_types, check_ambiguos_paths); - for (const auto & type : concrete_types) - { - const auto * type_array_conctete = typeid_cast(type.get()); - if (!type_array_conctete) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected Array type, got {}", type->getName()); - - nested_types.push_back(type_array_conctete->getNestedType()); - } - - return std::make_shared( - getLeastCommonTypeForDynamicColumnsImpl( - type_array->getNestedType(), nested_types, check_ambiguos_paths)); - } + if (const auto * type_map = typeid_cast(type_in_storage.get())) + return getLeastCommonTypeForColumnWithNestedType(*type_map, concrete_types, check_ambiguos_paths); if (const auto * type_tuple = typeid_cast(type_in_storage.get())) - { - const auto & element_types = type_tuple->getElements(); - DataTypes new_element_types(element_types.size()); - - for (size_t i = 0; i < element_types.size(); ++i) - { - DataTypes concrete_element_types; - concrete_element_types.reserve(concrete_types.size()); - - for (const auto & type : concrete_types) - { - const auto * type_tuple_conctete = typeid_cast(type.get()); - if (!type_tuple_conctete) - throw Exception(ErrorCodes::TYPE_MISMATCH, "Expected Tuple type, got {}", type->getName()); - - concrete_element_types.push_back(type_tuple_conctete->getElement(i)); - } - - new_element_types[i] = getLeastCommonTypeForDynamicColumnsImpl( - element_types[i], concrete_element_types, check_ambiguos_paths); - } - - return recreateTupleWithElements(*type_tuple, new_element_types); - } + return getLeastCommonTypeForTuple(*type_tuple, concrete_types, check_ambiguos_paths); throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName()); } @@ -428,16 +458,17 @@ DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage return type_in_storage; if (isObject(type_in_storage)) - { return std::make_shared( - DataTypes{std::make_shared()}, - Names{ColumnObject::COLUMN_NAME_DUMMY}); - } + DataTypes{std::make_shared()}, Names{ColumnObject::COLUMN_NAME_DUMMY}); if (const auto * type_array = typeid_cast(type_in_storage.get())) return std::make_shared( createConcreteEmptyDynamicColumn(type_array->getNestedType())); + if (const auto * type_map = typeid_cast(type_in_storage.get())) + return std::make_shared( + createConcreteEmptyDynamicColumn(type_map->getNestedType())); + if (const auto * type_tuple = typeid_cast(type_in_storage.get())) { const auto & elements = type_tuple->getElements(); diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index 0abb6a98d63..fc34508f1bb 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -1,6 +1,6 @@ #pragma once -#include "Columns/ColumnObject.h" +#include #include #include @@ -8,7 +8,7 @@ namespace DB { /// Serialization for data type Object. -/// Supported only test serialization/deserialization. +/// Supported only text serialization/deserialization. /// and binary bulk serialization/deserialization without position independent /// encoding, i.e. serialization/deserialization into Native format. template diff --git a/tests/queries/0_stateless/01825_type_json_in_other_types.reference b/tests/queries/0_stateless/01825_type_json_in_other_types.reference new file mode 100644 index 00000000000..b94885a65ab --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_in_other_types.reference @@ -0,0 +1,17 @@ +Tuple(String, Map(String, Array(Tuple(k1 Nested(k2 Int8, k3 Int8, k5 String), k4 String))), Tuple(k1 String, k2 Tuple(k3 String, k4 String))) +============= +{"id":1,"data":["foo",{"aa":[{"k1":[{"k2":1,"k3":2,"k5":""},{"k2":0,"k3":3,"k5":""}],"k4":""},{"k1":[{"k2":4,"k3":0,"k5":""},{"k2":0,"k3":5,"k5":""},{"k2":6,"k3":0,"k5":""}],"k4":"qqq"}],"bb":[{"k1":[],"k4":"www"},{"k1":[{"k2":7,"k3":8,"k5":""},{"k2":9,"k3":10,"k5":""},{"k2":11,"k3":12,"k5":""}],"k4":""}]},{"k1":"aa","k2":{"k3":"bb","k4":"c"}}]} +{"id":2,"data":["bar",{"aa":[{"k1":[{"k2":13,"k3":14,"k5":""},{"k2":15,"k3":16,"k5":""}],"k4":"www"}]},{"k1":"","k2":{"k3":"","k4":""}}]} +{"id":3,"data":["some",{"aa":[{"k1":[{"k2":0,"k3":20,"k5":"some"}],"k4":""}]},{"k1":"eee","k2":{"k3":"","k4":""}}]} +============= +{"aa":[{"k1":[{"k2":1,"k3":2,"k5":""},{"k2":0,"k3":3,"k5":""}],"k4":""},{"k1":[{"k2":4,"k3":0,"k5":""},{"k2":0,"k3":5,"k5":""},{"k2":6,"k3":0,"k5":""}],"k4":"qqq"}],"bb":[{"k1":[],"k4":"www"},{"k1":[{"k2":7,"k3":8,"k5":""},{"k2":9,"k3":10,"k5":""},{"k2":11,"k3":12,"k5":""}],"k4":""}]} +{"aa":[{"k1":[{"k2":13,"k3":14,"k5":""},{"k2":15,"k3":16,"k5":""}],"k4":"www"}],"bb":[]} +{"aa":[{"k1":[{"k2":0,"k3":20,"k5":"some"}],"k4":""}],"bb":[]} +============= +{"k1":[[{"k2":1,"k3":2,"k5":""},{"k2":0,"k3":3,"k5":""}],[{"k2":4,"k3":0,"k5":""},{"k2":0,"k3":5,"k5":""},{"k2":6,"k3":0,"k5":""}]],"k4":["","qqq"]} +{"k1":[[{"k2":13,"k3":14,"k5":""},{"k2":15,"k3":16,"k5":""}]],"k4":["www"]} +{"k1":[[{"k2":0,"k3":20,"k5":"some"}]],"k4":[""]} +============= +{"obj":{"k1":"aa","k2":{"k3":"bb","k4":"c"}}} +{"obj":{"k1":"","k2":{"k3":"","k4":""}}} +{"obj":{"k1":"eee","k2":{"k3":"","k4":""}}} diff --git a/tests/queries/0_stateless/01825_type_json_in_other_types.sh b/tests/queries/0_stateless/01825_type_json_in_other_types.sh new file mode 100755 index 00000000000..e9cf0bcaca1 --- /dev/null +++ b/tests/queries/0_stateless/01825_type_json_in_other_types.sh @@ -0,0 +1,91 @@ +#!/usr/bin/env bash +# Tags: no-fasttest + +CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +# shellcheck source=../shell_config.sh +. "$CUR_DIR"/../shell_config.sh + +${CLICKHOUSE_CLIENT} -q "SET allow_experimental_object_type = 1" +${CLICKHOUSE_CLIENT} -q "DROP TABLE IF EXISTS t_json_nested" + +${CLICKHOUSE_CLIENT} -q " + CREATE TABLE t_json_nested + ( + id UInt32, + data Tuple(String, Map(String, Array(JSON)), JSON) + ) + ENGINE = MergeTree ORDER BY id" --allow_experimental_object_type 1 + +cat < Date: Wed, 11 May 2022 13:37:02 +0000 Subject: [PATCH 007/152] fix checks --- src/Columns/ColumnObject.cpp | 1 - src/DataTypes/ObjectUtils.cpp | 4 ++-- src/DataTypes/Serializations/SerializationObject.cpp | 1 - 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/src/Columns/ColumnObject.cpp b/src/Columns/ColumnObject.cpp index 34919ad2398..c6a63cc4172 100644 --- a/src/Columns/ColumnObject.cpp +++ b/src/Columns/ColumnObject.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes extern const int ILLEGAL_COLUMN; extern const int DUPLICATE_COLUMN; extern const int NUMBER_OF_DIMENSIONS_MISMATHED; - extern const int NOT_IMPLEMENTED; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; extern const int ARGUMENT_OUT_OF_BOUND; } diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 4aea9a5f7e3..e75e2b7bc62 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -188,7 +188,7 @@ static std::pair recursivlyConvertDynamicColumnToTuple( return { - ColumnArray::create(std::move(new_column), column_array.getOffsetsPtr()), + ColumnArray::create(new_column, column_array.getOffsetsPtr()), std::make_shared(std::move(new_type)), }; } @@ -201,7 +201,7 @@ static std::pair recursivlyConvertDynamicColumnToTuple( return { - ColumnMap::create(std::move(new_column)), + ColumnMap::create(new_column), std::make_shared(std::move(new_type)), }; } diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 593a9a1bb02..af816568790 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -22,7 +22,6 @@ namespace ErrorCodes extern const int NOT_IMPLEMENTED; extern const int INCORRECT_DATA; extern const int CANNOT_READ_ALL_DATA; - extern const int LOGICAL_ERROR; extern const int ARGUMENT_OUT_OF_BOUND; } From 86b29b7f1a088fd0da3e2a72ac69b7aee9bf4357 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Sep 2022 15:16:39 +0000 Subject: [PATCH 008/152] fix serilization of Object inside other types --- src/DataTypes/Serializations/ISerialization.h | 2 + .../Serializations/SerializationArray.cpp | 4 +- .../Serializations/SerializationArray.h | 1 + .../SerializationLowCardinality.cpp | 1 + .../SerializationLowCardinality.h | 1 + .../Serializations/SerializationMap.cpp | 3 +- .../Serializations/SerializationMap.h | 1 + .../Serializations/SerializationNamed.cpp | 3 +- .../Serializations/SerializationNamed.h | 1 + .../Serializations/SerializationNullable.cpp | 4 +- .../Serializations/SerializationNullable.h | 1 + .../Serializations/SerializationObject.cpp | 43 ++++++++----------- .../Serializations/SerializationObject.h | 1 + .../Serializations/SerializationSparse.cpp | 4 +- .../Serializations/SerializationSparse.h | 1 + .../Serializations/SerializationTuple.cpp | 3 +- .../Serializations/SerializationTuple.h | 1 + .../Serializations/SerializationWrapper.cpp | 3 +- .../Serializations/SerializationWrapper.h | 1 + .../tests/gtest_object_serialization.cpp | 2 +- src/Formats/NativeWriter.cpp | 2 +- src/Functions/blockSerializedSize.cpp | 2 +- .../MergeTreeDataPartWriterCompact.cpp | 2 +- .../MergeTree/MergeTreeDataPartWriterWide.cpp | 2 +- src/Storages/MergeTree/MergeTreeIndexSet.cpp | 5 ++- src/Storages/StorageLog.cpp | 2 +- 26 files changed, 57 insertions(+), 39 deletions(-) diff --git a/src/DataTypes/Serializations/ISerialization.h b/src/DataTypes/Serializations/ISerialization.h index b5d2082631e..f23b1095749 100644 --- a/src/DataTypes/Serializations/ISerialization.h +++ b/src/DataTypes/Serializations/ISerialization.h @@ -219,7 +219,9 @@ public: }; /// Call before serializeBinaryBulkWithMultipleStreams chain to write something before first mark. + /// Column may be used only to retrieve the structure. virtual void serializeBinaryBulkStatePrefix( + const IColumn & /*column*/, SerializeBinaryBulkSettings & /*settings*/, SerializeBinaryBulkStatePtr & /*state*/) const {} diff --git a/src/DataTypes/Serializations/SerializationArray.cpp b/src/DataTypes/Serializations/SerializationArray.cpp index 625f2dce0b0..13577633327 100644 --- a/src/DataTypes/Serializations/SerializationArray.cpp +++ b/src/DataTypes/Serializations/SerializationArray.cpp @@ -235,11 +235,13 @@ void SerializationArray::enumerateStreams( } void SerializationArray::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { settings.path.push_back(Substream::ArrayElements); - nested->serializeBinaryBulkStatePrefix(settings, state); + const auto & column_array = assert_cast(column); + nested->serializeBinaryBulkStatePrefix(column_array.getData(), settings, state); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationArray.h b/src/DataTypes/Serializations/SerializationArray.h index 3769f8a4513..89bf50697fc 100644 --- a/src/DataTypes/Serializations/SerializationArray.h +++ b/src/DataTypes/Serializations/SerializationArray.h @@ -41,6 +41,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.cpp b/src/DataTypes/Serializations/SerializationLowCardinality.cpp index 8e19c5a740b..942ebbbe595 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.cpp +++ b/src/DataTypes/Serializations/SerializationLowCardinality.cpp @@ -225,6 +225,7 @@ struct DeserializeStateLowCardinality : public ISerialization::DeserializeBinary }; void SerializationLowCardinality::serializeBinaryBulkStatePrefix( + const IColumn & /*column*/, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { diff --git a/src/DataTypes/Serializations/SerializationLowCardinality.h b/src/DataTypes/Serializations/SerializationLowCardinality.h index 96e3a297d6a..c42f30f223a 100644 --- a/src/DataTypes/Serializations/SerializationLowCardinality.h +++ b/src/DataTypes/Serializations/SerializationLowCardinality.h @@ -23,6 +23,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationMap.cpp b/src/DataTypes/Serializations/SerializationMap.cpp index ea22070b5b1..824953da322 100644 --- a/src/DataTypes/Serializations/SerializationMap.cpp +++ b/src/DataTypes/Serializations/SerializationMap.cpp @@ -273,10 +273,11 @@ void SerializationMap::enumerateStreams( } void SerializationMap::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - nested->serializeBinaryBulkStatePrefix(settings, state); + nested->serializeBinaryBulkStatePrefix(extractNestedColumn(column), settings, state); } void SerializationMap::serializeBinaryBulkStateSuffix( diff --git a/src/DataTypes/Serializations/SerializationMap.h b/src/DataTypes/Serializations/SerializationMap.h index 93b3e179499..746598c2cc4 100644 --- a/src/DataTypes/Serializations/SerializationMap.h +++ b/src/DataTypes/Serializations/SerializationMap.h @@ -37,6 +37,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationNamed.cpp b/src/DataTypes/Serializations/SerializationNamed.cpp index 097e9cedfbe..5217bb6c068 100644 --- a/src/DataTypes/Serializations/SerializationNamed.cpp +++ b/src/DataTypes/Serializations/SerializationNamed.cpp @@ -17,11 +17,12 @@ void SerializationNamed::enumerateStreams( } void SerializationNamed::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { addToPath(settings.path); - nested_serialization->serializeBinaryBulkStatePrefix(settings, state); + nested_serialization->serializeBinaryBulkStatePrefix(column, settings, state); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationNamed.h b/src/DataTypes/Serializations/SerializationNamed.h index 343b96c16e3..12e5ce63e93 100644 --- a/src/DataTypes/Serializations/SerializationNamed.h +++ b/src/DataTypes/Serializations/SerializationNamed.h @@ -31,6 +31,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationNullable.cpp b/src/DataTypes/Serializations/SerializationNullable.cpp index a6273deaa30..0ee0443f251 100644 --- a/src/DataTypes/Serializations/SerializationNullable.cpp +++ b/src/DataTypes/Serializations/SerializationNullable.cpp @@ -73,11 +73,13 @@ void SerializationNullable::enumerateStreams( } void SerializationNullable::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { settings.path.push_back(Substream::NullableElements); - nested->serializeBinaryBulkStatePrefix(settings, state); + const auto & column_nullable = assert_cast(column); + nested->serializeBinaryBulkStatePrefix(column_nullable.getNestedColumn(), settings, state); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationNullable.h b/src/DataTypes/Serializations/SerializationNullable.h index e6e0e4f33c2..bbf9a1e0c48 100644 --- a/src/DataTypes/Serializations/SerializationNullable.h +++ b/src/DataTypes/Serializations/SerializationNullable.h @@ -19,6 +19,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index e7a403106a8..5577cab5982 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -139,7 +139,6 @@ void SerializationObject::checkSerializationIsSupported(const TSettings template struct SerializationObject::SerializeStateObject : public ISerialization::SerializeBinaryBulkState { - bool is_first = true; DataTypePtr nested_type; SerializationPtr nested_serialization; SerializeBinaryBulkStatePtr nested_state; @@ -156,6 +155,7 @@ struct SerializationObject::DeserializeStateObject : public ISerializati template void SerializationObject::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -166,13 +166,25 @@ void SerializationObject::serializeBinaryBulkStatePrefix( settings.path.push_back(Substream::ObjectStructure); auto * stream = settings.getter(settings.path); - settings.path.pop_back(); if (!stream) throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for kind of binary serialization"); writeIntBinary(static_cast(BinarySerializationKind::TUPLE), *stream); - state = std::make_shared(); + + const auto & column_object = assert_cast(column); + auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); + writeStringBinary(tuple_type->getName(), *stream); + + auto state_object = std::make_shared(); + state_object->nested_type = tuple_type; + state_object->nested_serialization = tuple_type->getDefaultSerialization(); + + settings.path.back() = Substream::ObjectData; + state_object->nested_serialization->serializeBinaryBulkStatePrefix(*tuple_column, settings, state_object->nested_state); + + state = std::move(state_object); + settings.path.pop_back(); } template @@ -266,25 +278,7 @@ void SerializationObject::serializeBinaryBulkWithMultipleStreams( auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); - if (state_object->is_first) - { - /// Actually it's a part of serializeBinaryBulkStatePrefix, - /// but it cannot be done there, because we have to know the - /// structure of column. - - settings.path.push_back(Substream::ObjectStructure); - if (auto * stream = settings.getter(settings.path)) - writeStringBinary(tuple_type->getName(), *stream); - - state_object->nested_type = tuple_type; - state_object->nested_serialization = tuple_type->getDefaultSerialization(); - state_object->is_first = false; - - settings.path.back() = Substream::ObjectData; - state_object->nested_serialization->serializeBinaryBulkStatePrefix(settings, state_object->nested_state); - settings.path.pop_back(); - } - else if (!state_object->nested_type->equals(*tuple_type)) + if (!state_object->nested_type->equals(*tuple_type)) { throw Exception(ErrorCodes::LOGICAL_ERROR, "Types of internal column of Object mismatched. Expected: {}, Got: {}", @@ -452,8 +446,9 @@ void SerializationObject::serializeTextFromSubcolumn( { if (ind < part->size()) { - auto info = least_common_type->getSerializationInfo(*part); - auto serialization = least_common_type->getSerialization(*info); + auto part_type = getDataTypeByColumn(*part); + auto info = part_type->getSerializationInfo(*part); + auto serialization = part_type->getSerialization(*info); serialization->serializeTextJSON(*part, ind, ostr, settings); return; } diff --git a/src/DataTypes/Serializations/SerializationObject.h b/src/DataTypes/Serializations/SerializationObject.h index ef8019ead77..47a7127cd1c 100644 --- a/src/DataTypes/Serializations/SerializationObject.h +++ b/src/DataTypes/Serializations/SerializationObject.h @@ -31,6 +31,7 @@ public: */ void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 6fa40e460c5..a3d7e071215 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -184,11 +184,13 @@ void SerializationSparse::enumerateStreams( } void SerializationSparse::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { settings.path.push_back(Substream::SparseElements); - nested->serializeBinaryBulkStatePrefix(settings, state); + const auto & column_sparse = assert_cast(column); + nested->serializeBinaryBulkStatePrefix(column_sparse.getValuesColumn(), settings, state); settings.path.pop_back(); } diff --git a/src/DataTypes/Serializations/SerializationSparse.h b/src/DataTypes/Serializations/SerializationSparse.h index 54ab4853360..f6082f9a949 100644 --- a/src/DataTypes/Serializations/SerializationSparse.h +++ b/src/DataTypes/Serializations/SerializationSparse.h @@ -33,6 +33,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationTuple.cpp b/src/DataTypes/Serializations/SerializationTuple.cpp index 8138b15c9af..21f6d47f1f9 100644 --- a/src/DataTypes/Serializations/SerializationTuple.cpp +++ b/src/DataTypes/Serializations/SerializationTuple.cpp @@ -317,6 +317,7 @@ struct DeserializeBinaryBulkStateTuple : public ISerialization::DeserializeBinar void SerializationTuple::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { @@ -324,7 +325,7 @@ void SerializationTuple::serializeBinaryBulkStatePrefix( tuple_state->states.resize(elems.size()); for (size_t i = 0; i < elems.size(); ++i) - elems[i]->serializeBinaryBulkStatePrefix(settings, tuple_state->states[i]); + elems[i]->serializeBinaryBulkStatePrefix(extractElementColumn(column, i), settings, tuple_state->states[i]); state = std::move(tuple_state); } diff --git a/src/DataTypes/Serializations/SerializationTuple.h b/src/DataTypes/Serializations/SerializationTuple.h index e82d8473645..0eab14431d3 100644 --- a/src/DataTypes/Serializations/SerializationTuple.h +++ b/src/DataTypes/Serializations/SerializationTuple.h @@ -39,6 +39,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/SerializationWrapper.cpp b/src/DataTypes/Serializations/SerializationWrapper.cpp index 271c53dfcf1..8d41746055d 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.cpp +++ b/src/DataTypes/Serializations/SerializationWrapper.cpp @@ -13,10 +13,11 @@ void SerializationWrapper::enumerateStreams( } void SerializationWrapper::serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const { - nested_serialization->serializeBinaryBulkStatePrefix(settings, state); + nested_serialization->serializeBinaryBulkStatePrefix(column, settings, state); } void SerializationWrapper::serializeBinaryBulkStateSuffix( diff --git a/src/DataTypes/Serializations/SerializationWrapper.h b/src/DataTypes/Serializations/SerializationWrapper.h index 43fc7e9914a..a3a30b8f6b7 100644 --- a/src/DataTypes/Serializations/SerializationWrapper.h +++ b/src/DataTypes/Serializations/SerializationWrapper.h @@ -26,6 +26,7 @@ public: const SubstreamData & data) const override; void serializeBinaryBulkStatePrefix( + const IColumn & column, SerializeBinaryBulkSettings & settings, SerializeBinaryBulkStatePtr & state) const override; diff --git a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp index f1fbbe115e2..fc7432d5bf6 100644 --- a/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp +++ b/src/DataTypes/Serializations/tests/gtest_object_serialization.cpp @@ -31,7 +31,7 @@ TEST(SerializationObject, FromString) settings.getter = [&out](const auto &) { return &out; }; writeIntBinary(static_cast(1), out); - serialization->serializeBinaryBulkStatePrefix(settings, state); + serialization->serializeBinaryBulkStatePrefix(*column_string, settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*column_string, 0, column_string->size(), settings, state); serialization->serializeBinaryBulkStateSuffix(settings, state); } diff --git a/src/Formats/NativeWriter.cpp b/src/Formats/NativeWriter.cpp index 9d4cfb68d56..499f3215d63 100644 --- a/src/Formats/NativeWriter.cpp +++ b/src/Formats/NativeWriter.cpp @@ -59,7 +59,7 @@ static void writeData(const ISerialization & serialization, const ColumnPtr & co settings.low_cardinality_max_dictionary_size = 0; //-V1048 ISerialization::SerializeBinaryBulkStatePtr state; - serialization.serializeBinaryBulkStatePrefix(settings, state); + serialization.serializeBinaryBulkStatePrefix(*full_column, settings, state); serialization.serializeBinaryBulkWithMultipleStreams(*full_column, offset, limit, settings, state); serialization.serializeBinaryBulkStateSuffix(settings, state); } diff --git a/src/Functions/blockSerializedSize.cpp b/src/Functions/blockSerializedSize.cpp index d406984c51c..35be65f3fed 100644 --- a/src/Functions/blockSerializedSize.cpp +++ b/src/Functions/blockSerializedSize.cpp @@ -54,7 +54,7 @@ public: auto serialization = elem.type->getDefaultSerialization(); - serialization->serializeBinaryBulkStatePrefix(settings, state); + serialization->serializeBinaryBulkStatePrefix(*full_column, settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*full_column, 0 /** offset */, 0 /** limit */, settings, state); diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp index 771248b99c6..d0311f0250c 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterCompact.cpp @@ -121,7 +121,7 @@ void writeColumnSingleGranule( serialize_settings.position_independent_encoding = true; //-V1048 serialize_settings.low_cardinality_max_dictionary_size = 0; //-V1048 - serialization->serializeBinaryBulkStatePrefix(serialize_settings, state); + serialization->serializeBinaryBulkStatePrefix(*column.column, serialize_settings, state); serialization->serializeBinaryBulkWithMultipleStreams(*column.column, from_row, number_of_rows, serialize_settings, state); serialization->serializeBinaryBulkStateSuffix(serialize_settings, state); } diff --git a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp index 3d4aa0a7707..2f2a4fef9cb 100644 --- a/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp +++ b/src/Storages/MergeTree/MergeTreeDataPartWriterWide.cpp @@ -349,7 +349,7 @@ void MergeTreeDataPartWriterWide::writeColumn( { ISerialization::SerializeBinaryBulkSettings serialize_settings; serialize_settings.getter = createStreamGetter(name_and_type, offset_columns); - serialization->serializeBinaryBulkStatePrefix(serialize_settings, it->second); + serialization->serializeBinaryBulkStatePrefix(column, serialize_settings, it->second); } const auto & global_settings = storage.getContext()->getSettingsRef(); diff --git a/src/Storages/MergeTree/MergeTreeIndexSet.cpp b/src/Storages/MergeTree/MergeTreeIndexSet.cpp index 3c31deda823..0e15f2c4cb6 100644 --- a/src/Storages/MergeTree/MergeTreeIndexSet.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexSet.cpp @@ -74,8 +74,9 @@ void MergeTreeIndexGranuleSet::serializeBinary(WriteBuffer & ostr) const auto serialization = type->getDefaultSerialization(); ISerialization::SerializeBinaryBulkStatePtr state; - serialization->serializeBinaryBulkStatePrefix(settings, state); - serialization->serializeBinaryBulkWithMultipleStreams(*block.getByPosition(i).column, 0, size(), settings, state); + const auto & column = *block.getByPosition(i).column; + serialization->serializeBinaryBulkStatePrefix(column, settings, state); + serialization->serializeBinaryBulkWithMultipleStreams(column, 0, size(), settings, state); serialization->serializeBinaryBulkStateSuffix(settings, state); } } diff --git a/src/Storages/StorageLog.cpp b/src/Storages/StorageLog.cpp index c6bc55fd620..1f623503f97 100644 --- a/src/Storages/StorageLog.cpp +++ b/src/Storages/StorageLog.cpp @@ -462,7 +462,7 @@ void LogSink::writeData(const NameAndTypePair & name_and_type, const IColumn & c settings.getter = createStreamGetter(name_and_type); if (!serialize_states.contains(name)) - serialization->serializeBinaryBulkStatePrefix(settings, serialize_states[name]); + serialization->serializeBinaryBulkStatePrefix(column, settings, serialize_states[name]); if (storage.use_marks_file) { From bbb7974dfbf1123dc648946c36749b72c8e3cebd Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Thu, 8 Sep 2022 15:47:43 +0000 Subject: [PATCH 009/152] fix style check --- src/DataTypes/ObjectUtils.cpp | 1 + src/DataTypes/ObjectUtils.h | 2 +- src/DataTypes/Serializations/SerializationObject.cpp | 1 + tests/queries/0_stateless/01825_type_json_in_array.reference | 4 ++-- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 580583df5a4..b7f83ac6f63 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -939,6 +939,7 @@ size_t FieldVisitorToNumberOfDimensions::operator()(const Array & x) { const size_t size = x.size(); size_t dimensions = 0; + for (size_t i = 0; i < size; ++i) { size_t element_dimensions = applyVisitor(*this, x[i]); diff --git a/src/DataTypes/ObjectUtils.h b/src/DataTypes/ObjectUtils.h index 3c2812b6297..4160064c09b 100644 --- a/src/DataTypes/ObjectUtils.h +++ b/src/DataTypes/ObjectUtils.h @@ -39,7 +39,7 @@ Array createEmptyArrayField(size_t num_dimensions); DataTypePtr getDataTypeByColumn(const IColumn & column); /// Converts Object types and columns to Tuples in @columns_list and @block -/// and checks that types are consistent with types in @extended_storage_columns. +/// and checks that types are consistent with types in @storage_snapshot. void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & storage_snapshot); /// Checks that each path is not the prefix of any other path. diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 5577cab5982..25c458aad64 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -29,6 +29,7 @@ namespace ErrorCodes extern const int INCORRECT_DATA; extern const int CANNOT_READ_ALL_DATA; extern const int ARGUMENT_OUT_OF_BOUND; + extern const int LOGICAL_ERROR; } template diff --git a/tests/queries/0_stateless/01825_type_json_in_array.reference b/tests/queries/0_stateless/01825_type_json_in_array.reference index 3f6dc6ccd88..d06320ca291 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.reference +++ b/tests/queries/0_stateless/01825_type_json_in_array.reference @@ -8,8 +8,8 @@ Array(Tuple(k1 Int8, k2 Tuple(k3 Int8, k4 Int8, k5 String))) {"id":1,"arr":[{"k1":[{"k2":"aaa","k3":"bbb","k4":0},{"k2":"ccc","k3":"","k4":0}],"k5":{"k6":""}}]} {"id":2,"arr":[{"k1":[{"k2":"","k3":"ddd","k4":10},{"k2":"","k3":"","k4":20}],"k5":{"k6":"foo"}}]} -1 [['aaa','ccc']] [['bbb','']] [[]] [''] -2 [[]] [['ddd','']] [[10,20]] ['foo'] +1 [['aaa','ccc']] [['bbb','']] [[0,0]] [''] +2 [['','']] [['ddd','']] [[10,20]] ['foo'] {"k1":{"k2":"","k3":"","k4":20}} {"k1":{"k2":"","k3":"ddd","k4":10}} {"k1":{"k2":"aaa","k3":"bbb","k4":0}} From ad664cb6ddde7b7a7ebc8e2864df106b8cf37388 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 12 Sep 2022 18:14:02 +0000 Subject: [PATCH 010/152] fix serializePrefix for sparse columns --- src/DataTypes/Serializations/SerializationSparse.cpp | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationSparse.cpp b/src/DataTypes/Serializations/SerializationSparse.cpp index 1580706b09f..cd09cd7be5a 100644 --- a/src/DataTypes/Serializations/SerializationSparse.cpp +++ b/src/DataTypes/Serializations/SerializationSparse.cpp @@ -183,8 +183,11 @@ void SerializationSparse::serializeBinaryBulkStatePrefix( SerializeBinaryBulkStatePtr & state) const { settings.path.push_back(Substream::SparseElements); - const auto & column_sparse = assert_cast(column); - nested->serializeBinaryBulkStatePrefix(column_sparse.getValuesColumn(), settings, state); + if (const auto * column_sparse = typeid_cast(&column)) + nested->serializeBinaryBulkStatePrefix(column_sparse->getValuesColumn(), settings, state); + else + nested->serializeBinaryBulkStatePrefix(column, settings, state); + settings.path.pop_back(); } From c74b5c8126a7b32e2df22f2c27a5627a448e31e1 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Sep 2022 16:59:17 +0000 Subject: [PATCH 011/152] Fix schema inference in s3Cluster and improve in hdfsCluster --- src/Interpreters/getTableExpressions.cpp | 17 ++++++ src/Interpreters/getTableExpressions.h | 3 ++ src/Storages/HDFS/StorageHDFSCluster.cpp | 37 ++++++++++++- src/Storages/HDFS/StorageHDFSCluster.h | 1 + src/Storages/StorageS3.cpp | 54 ++----------------- src/Storages/StorageS3.h | 16 ------ src/Storages/StorageS3Cluster.cpp | 47 ++++++++++++---- src/Storages/StorageS3Cluster.h | 1 + .../TableFunctionHDFSCluster.cpp | 2 +- src/TableFunctions/TableFunctionS3.cpp | 4 +- src/TableFunctions/TableFunctionS3Cluster.cpp | 1 + 11 files changed, 104 insertions(+), 79 deletions(-) diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 830f0ea4411..43c9438ecec 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -155,4 +156,20 @@ TablesWithColumns getDatabaseAndTablesWithColumns( return tables_with_columns; } +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) +{ + auto * select_query = query->as(); + if (!select_query || !select_query->tables()) + return nullptr; + + auto * tables = select_query->tables()->as(); + auto * table_expression = tables->children[0]->as()->table_expression->as(); + if (!table_expression->table_function) + return nullptr; + + auto * table_function = table_expression->table_function->as(); + return table_function->arguments->as(); +} + + } diff --git a/src/Interpreters/getTableExpressions.h b/src/Interpreters/getTableExpressions.h index c4ca01ee3c3..4497b376cc7 100644 --- a/src/Interpreters/getTableExpressions.h +++ b/src/Interpreters/getTableExpressions.h @@ -3,6 +3,7 @@ #include #include #include +#include namespace DB { @@ -23,4 +24,6 @@ ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number TablesWithColumns getDatabaseAndTablesWithColumns( const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, bool include_materialized_cols); +ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query); + } diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 47a6fbf5eaa..7851532a6a4 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -25,6 +25,7 @@ #include #include #include +#include #include @@ -32,6 +33,34 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +static ASTPtr addColumnsStructureToQuery(const ASTPtr & query, const String & structure) +{ + /// Add argument with table structure to hdfsCluster table function in select query. + auto result_query = query->clone(); + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(result_query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function hdfsCluster, got '{}'", queryToString(query)); + + auto structure_literal = std::make_shared(structure); + + if (expression_list->children.size() != 2 && expression_list->children.size() != 3) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 or 3 arguments in hdfsCluster table functions, got {}", expression_list->children.size()); + + if (expression_list->children.size() == 2) + { + auto format_literal = std::make_shared("auto"); + expression_list->children.push_back(format_literal); + } + + expression_list->children.push_back(structure_literal); + return result_query; +} + StorageHDFSCluster::StorageHDFSCluster( ContextPtr context_, String cluster_name_, @@ -56,6 +85,7 @@ StorageHDFSCluster::StorageHDFSCluster( { auto columns = StorageHDFS::getTableStructureFromData(format_name, uri_, compression_method, context_); storage_metadata.setColumns(columns); + need_to_add_structure_to_query = true; } else storage_metadata.setColumns(columns_); @@ -92,6 +122,11 @@ Pipe StorageHDFSCluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; + auto query_to_send = query_info.original_query; + if (need_to_add_structure_to_query) + query_to_send = addColumnsStructureToQuery( + query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll())); + for (const auto & replicas : cluster->getShardsAddresses()) { /// There will be only one replica, because we consider each replica as a shard @@ -110,7 +145,7 @@ Pipe StorageHDFSCluster::read( /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( connection, - queryToString(query_info.original_query), + queryToString(query_to_send), header, context, /*throttler=*/nullptr, diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 21ae73c11ea..d5af257f803 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -44,6 +44,7 @@ private: String uri; String format_name; String compression_method; + bool need_to_add_structure_to_query = false; }; diff --git a/src/Storages/StorageS3.cpp b/src/Storages/StorageS3.cpp index e15956f78be..8a6349007ec 100644 --- a/src/Storages/StorageS3.cpp +++ b/src/Storages/StorageS3.cpp @@ -361,39 +361,6 @@ String StorageS3Source::KeysIterator::next() return pimpl->next(); } -class StorageS3Source::ReadTasksIterator::Impl -{ -public: - explicit Impl(const std::vector & read_tasks_, const ReadTaskCallback & new_read_tasks_callback_) - : read_tasks(read_tasks_), new_read_tasks_callback(new_read_tasks_callback_) - { - } - - String next() - { - size_t current_index = index.fetch_add(1, std::memory_order_relaxed); - if (current_index >= read_tasks.size()) - return new_read_tasks_callback(); - return read_tasks[current_index]; - } - -private: - std::atomic_size_t index = 0; - std::vector read_tasks; - ReadTaskCallback new_read_tasks_callback; -}; - -StorageS3Source::ReadTasksIterator::ReadTasksIterator( - const std::vector & read_tasks_, const ReadTaskCallback & new_read_tasks_callback_) - : pimpl(std::make_shared(read_tasks_, new_read_tasks_callback_)) -{ -} - -String StorageS3Source::ReadTasksIterator::next() -{ - return pimpl->next(); -} - Block StorageS3Source::getHeader(Block sample_block, const std::vector & requested_virtual_columns) { for (const auto & virtual_column : requested_virtual_columns) @@ -802,8 +769,7 @@ StorageS3::StorageS3( distributed_processing_, is_key_with_globs, format_settings, - context_, - &read_tasks_used_in_schema_inference); + context_); storage_metadata.setColumns(columns); } else @@ -831,19 +797,14 @@ std::shared_ptr StorageS3::createFileIterator( ContextPtr local_context, ASTPtr query, const Block & virtual_block, - const std::vector & read_tasks, std::unordered_map * object_infos, Strings * read_keys) { if (distributed_processing) { return std::make_shared( - [read_tasks_iterator = std::make_shared(read_tasks, local_context->getReadTaskCallback()), read_keys]() -> String - { - auto key = read_tasks_iterator->next(); - if (read_keys) - read_keys->push_back(key); - return key; + [callback = local_context->getReadTaskCallback()]() -> String { + return callback(); }); } else if (is_key_with_globs) @@ -903,7 +864,6 @@ Pipe StorageS3::read( local_context, query_info.query, virtual_block, - read_tasks_used_in_schema_inference, &object_infos); ColumnsDescription columns_description; @@ -1201,7 +1161,7 @@ ColumnsDescription StorageS3::getTableStructureFromData( return getTableStructureFromDataImpl( configuration.format, s3_configuration, configuration.compression_method, distributed_processing, - s3_configuration.uri.key.find_first_of("*?{") != std::string::npos, format_settings, ctx, nullptr, object_infos); + s3_configuration.uri.key.find_first_of("*?{") != std::string::npos, format_settings, ctx, object_infos); } ColumnsDescription StorageS3::getTableStructureFromDataImpl( @@ -1212,13 +1172,12 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( bool is_key_with_globs, const std::optional & format_settings, ContextPtr ctx, - std::vector * read_keys_in_distributed_processing, std::unordered_map * object_infos) { std::vector read_keys; auto file_iterator - = createFileIterator(s3_configuration, {s3_configuration.uri.key}, is_key_with_globs, distributed_processing, ctx, nullptr, {}, {}, object_infos, &read_keys); + = createFileIterator(s3_configuration, {s3_configuration.uri.key}, is_key_with_globs, distributed_processing, ctx, nullptr, {}, object_infos, &read_keys); std::optional columns_from_cache; size_t prev_read_keys_size = read_keys.size(); @@ -1271,9 +1230,6 @@ ColumnsDescription StorageS3::getTableStructureFromDataImpl( if (ctx->getSettingsRef().schema_inference_use_cache_for_s3) addColumnsToCache(read_keys, s3_configuration, columns, format, format_settings, ctx); - if (distributed_processing && read_keys_in_distributed_processing) - *read_keys_in_distributed_processing = std::move(read_keys); - return columns; } diff --git a/src/Storages/StorageS3.h b/src/Storages/StorageS3.h index c63508c8e6a..7c6970bab0e 100644 --- a/src/Storages/StorageS3.h +++ b/src/Storages/StorageS3.h @@ -66,18 +66,6 @@ public: std::shared_ptr pimpl; }; - class ReadTasksIterator - { - public: - ReadTasksIterator(const std::vector & read_tasks_, const ReadTaskCallback & new_read_tasks_callback_); - String next(); - - private: - class Impl; - /// shared_ptr to have copy constructor - std::shared_ptr pimpl; - }; - using IteratorWrapper = std::function; static Block getHeader(Block sample_block, const std::vector & requested_virtual_columns); @@ -238,8 +226,6 @@ private: ASTPtr partition_by; bool is_key_with_globs = false; - std::vector read_tasks_used_in_schema_inference; - std::unordered_map object_infos; static void updateS3Configuration(ContextPtr, S3Configuration &); @@ -252,7 +238,6 @@ private: ContextPtr local_context, ASTPtr query, const Block & virtual_block, - const std::vector & read_tasks = {}, std::unordered_map * object_infos = nullptr, Strings * read_keys = nullptr); @@ -264,7 +249,6 @@ private: bool is_key_with_globs, const std::optional & format_settings, ContextPtr ctx, - std::vector * read_keys_in_distributed_processing = nullptr, std::unordered_map * object_infos = nullptr); bool supportsSubsetOfColumns() const override; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 800bce0afde..245888f5ce3 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -5,14 +5,9 @@ #if USE_AWS_S3 #include "Common/Exception.h" -#include #include "Client/Connection.h" #include "Core/QueryProcessingStage.h" -#include -#include -#include #include -#include #include #include #include @@ -23,28 +18,54 @@ #include #include #include -#include "Processors/ISource.h" #include #include #include #include +#include #include #include #include +#include #include #include #include #include -#include #include #include -#include -#include namespace DB { + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +static ASTPtr addColumnsStructureToQuery(const ASTPtr & query, const String & structure) +{ + /// Add argument with table structure to s3Cluster table function in select query. + auto result_query = query->clone(); + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(result_query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function from s3Cluster, got '{}'", queryToString(query)); + auto structure_literal = std::make_shared(structure); + + if (expression_list->children.size() < 2 || expression_list->children.size() > 5) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to 5 arguments in s3Cluster table functions, got {}", expression_list->children.size()); + + if (expression_list->children.size() == 2 || expression_list->children.size() == 4) + { + auto format_literal = std::make_shared("auto"); + expression_list->children.push_back(format_literal); + } + + expression_list->children.push_back(structure_literal); + return result_query; +} + StorageS3Cluster::StorageS3Cluster( const StorageS3ClusterConfiguration & configuration_, const StorageID & table_id_, @@ -72,6 +93,7 @@ StorageS3Cluster::StorageS3Cluster( auto columns = StorageS3::getTableStructureFromDataImpl(format_name, s3_configuration, compression_method, /*distributed_processing_*/false, is_key_with_globs, /*format_settings=*/std::nullopt, context_); storage_metadata.setColumns(columns); + need_to_add_structure_to_query = true; } else storage_metadata.setColumns(columns_); @@ -117,6 +139,11 @@ Pipe StorageS3Cluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; + ASTPtr query_to_send = query_info.original_query; + if (need_to_add_structure_to_query) + query_to_send = addColumnsStructureToQuery( + query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll())); + for (const auto & replicas : cluster->getShardsAddresses()) { /// There will be only one replica, because we consider each replica as a shard @@ -135,7 +162,7 @@ Pipe StorageS3Cluster::read( /// So, task_identifier is passed as constructor argument. It is more obvious. auto remote_query_executor = std::make_shared( connection, - queryToString(query_info.original_query), + queryToString(query_to_send), header, context, /*throttler=*/nullptr, diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index e5ca3b58123..7e4d4801090 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -46,6 +46,7 @@ private: String compression_method; NamesAndTypesList virtual_columns; Block virtual_block; + bool need_to_add_structure_to_query = false; }; diff --git a/src/TableFunctions/TableFunctionHDFSCluster.cpp b/src/TableFunctions/TableFunctionHDFSCluster.cpp index 385d280a100..fd2ef7bb052 100644 --- a/src/TableFunctions/TableFunctionHDFSCluster.cpp +++ b/src/TableFunctions/TableFunctionHDFSCluster.cpp @@ -48,7 +48,7 @@ void TableFunctionHDFSCluster::parseArguments(const ASTPtr & ast_function, Conte const auto message = fmt::format( "The signature of table function {} shall be the following:\n" \ " - cluster, uri\n",\ - " - cluster, format\n",\ + " - cluster, uri, format\n",\ " - cluster, uri, format, structure\n",\ " - cluster, uri, format, structure, compression_method", getName()); diff --git a/src/TableFunctions/TableFunctionS3.cpp b/src/TableFunctions/TableFunctionS3.cpp index 0bf33007760..0ceb7cd5e5b 100644 --- a/src/TableFunctions/TableFunctionS3.cpp +++ b/src/TableFunctions/TableFunctionS3.cpp @@ -64,7 +64,7 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar if (args.size() == 4) { auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); - if (FormatFactory::instance().getAllFormats().contains(second_arg)) + if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) args_to_idx = {{"format", 1}, {"structure", 2}, {"compression_method", 3}}; else @@ -77,7 +77,7 @@ void TableFunctionS3::parseArgumentsImpl(const String & error_message, ASTs & ar { auto second_arg = checkAndGetLiteralArgument(args[1], "format/access_key_id"); - if (FormatFactory::instance().getAllFormats().contains(second_arg)) + if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg)) args_to_idx = {{"format", 1}, {"structure", 2}}; else args_to_idx = {{"access_key_id", 1}, {"secret_access_key", 2}}; diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 99c3ff85009..668cd67dd08 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -77,6 +77,7 @@ void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, Context /// StorageS3ClusterConfiguration inherints from StorageS3Configuration, so it is safe to upcast it. TableFunctionS3::parseArgumentsImpl(message, clipped_args, context, static_cast(configuration)); + LOG_DEBUG(&Poco::Logger::get("TableFunctionS3Cluster"), "Structure: {}", configuration.structure); } From addd36f4369ac0e45b08f048450eab73172f1301 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Sep 2022 17:00:25 +0000 Subject: [PATCH 012/152] Remove logging --- src/TableFunctions/TableFunctionS3Cluster.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/TableFunctions/TableFunctionS3Cluster.cpp b/src/TableFunctions/TableFunctionS3Cluster.cpp index 668cd67dd08..99c3ff85009 100644 --- a/src/TableFunctions/TableFunctionS3Cluster.cpp +++ b/src/TableFunctions/TableFunctionS3Cluster.cpp @@ -77,7 +77,6 @@ void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, Context /// StorageS3ClusterConfiguration inherints from StorageS3Configuration, so it is safe to upcast it. TableFunctionS3::parseArgumentsImpl(message, clipped_args, context, static_cast(configuration)); - LOG_DEBUG(&Poco::Logger::get("TableFunctionS3Cluster"), "Structure: {}", configuration.structure); } From e005b83d790e604bf5140819fc360963a5b76ca6 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 30 Sep 2022 17:56:43 +0000 Subject: [PATCH 013/152] Add tests --- ...2457_s3_cluster_schema_inference.reference | 44 +++++++++++++++++++ .../02457_s3_cluster_schema_inference.sql | 13 ++++++ ...58_hdfs_cluster_schema_inference.reference | 10 +++++ .../02458_hdfs_cluster_schema_inference.sql | 12 +++++ 4 files changed, 79 insertions(+) create mode 100644 tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference create mode 100644 tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql create mode 100644 tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.reference create mode 100644 tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql diff --git a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference new file mode 100644 index 00000000000..b918bf2b155 --- /dev/null +++ b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.reference @@ -0,0 +1,44 @@ +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 +0 0 0 +0 0 0 +1 2 3 +4 5 6 +7 8 9 +10 11 12 +13 14 15 +16 17 18 diff --git a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql new file mode 100644 index 00000000000..039c7a9fd7f --- /dev/null +++ b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql @@ -0,0 +1,13 @@ +-- Tags: no-fasttest +-- Tag no-fasttest: Depends on AWS + +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest'); +desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV'); + +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c2; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV') order by c1, c2, c2; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest') order by c1, c2, c2; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV') order by c1, c2, c2; + diff --git a/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.reference b/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.reference new file mode 100644 index 00000000000..a812e64a642 --- /dev/null +++ b/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.reference @@ -0,0 +1,10 @@ +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +c1 Nullable(Int64) +c2 Nullable(Int64) +c3 Nullable(Int64) +1 2 3 +4 5 6 +1 2 3 +4 5 6 diff --git a/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql b/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql new file mode 100644 index 00000000000..42e88fc44b2 --- /dev/null +++ b/tests/queries/0_stateless/02458_hdfs_cluster_schema_inference.sql @@ -0,0 +1,12 @@ +-- Tags: no-fasttest, no-parallel, no-cpu-aarch64 +-- Tag no-fasttest: Depends on Java + +insert into table function hdfs('hdfs://localhost:12222/test_02458_1.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 1, 2, 3 settings hdfs_truncate_on_insert=1; +insert into table function hdfs('hdfs://localhost:12222/test_02458_2.tsv', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') select 4, 5, 6 settings hdfs_truncate_on_insert=1; + +desc hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv'); +desc hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv', 'TSV'); + +select * from hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv') order by c1, c2, c3; +select * from hdfsCluster('test_cluster_one_shard_three_replicas_localhost', 'hdfs://localhost:12222/test_02458_{1,2}.tsv', 'TSV') order by c1, c2, c3; + From 8db899e550608a1c5b1854e88520d318ff5bb331 Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 3 Oct 2022 14:44:33 +0000 Subject: [PATCH 014/152] fix assertion in fuzzer --- .../Serializations/SerializationObject.cpp | 13 ++++++++++--- .../0_stateless/01825_type_json_in_array.reference | 1 + .../0_stateless/01825_type_json_in_array.sql | 1 + 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/src/DataTypes/Serializations/SerializationObject.cpp b/src/DataTypes/Serializations/SerializationObject.cpp index 25c458aad64..98a94886f67 100644 --- a/src/DataTypes/Serializations/SerializationObject.cpp +++ b/src/DataTypes/Serializations/SerializationObject.cpp @@ -165,16 +165,23 @@ void SerializationObject::serializeBinaryBulkStatePrefix( throw Exception(ErrorCodes::NOT_IMPLEMENTED, "DataTypeObject doesn't support serialization with non-trivial state"); + const auto & column_object = assert_cast(column); + if (!column_object.isFinalized()) + { + auto finalized = column_object.cloneFinalized(); + serializeBinaryBulkStatePrefix(*finalized, settings, state); + return; + } + settings.path.push_back(Substream::ObjectStructure); auto * stream = settings.getter(settings.path); if (!stream) throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for kind of binary serialization"); - writeIntBinary(static_cast(BinarySerializationKind::TUPLE), *stream); - - const auto & column_object = assert_cast(column); auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object); + + writeIntBinary(static_cast(BinarySerializationKind::TUPLE), *stream); writeStringBinary(tuple_type->getName(), *stream); auto state_object = std::make_shared(); diff --git a/tests/queries/0_stateless/01825_type_json_in_array.reference b/tests/queries/0_stateless/01825_type_json_in_array.reference index d06320ca291..c36a22e6951 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.reference +++ b/tests/queries/0_stateless/01825_type_json_in_array.reference @@ -20,3 +20,4 @@ Tuple(k2 String, k3 String, k4 Int8) {"arr":[1,{"y":1}]} {"arr":[2,{"y":2}]} {"arr":[{"x":"aaa","y":[1,2,3]}]} +{"arr":[{"x":1}]} diff --git a/tests/queries/0_stateless/01825_type_json_in_array.sql b/tests/queries/0_stateless/01825_type_json_in_array.sql index 935969272fa..e5c20d7ba6b 100644 --- a/tests/queries/0_stateless/01825_type_json_in_array.sql +++ b/tests/queries/0_stateless/01825_type_json_in_array.sql @@ -32,3 +32,4 @@ SELECT * FROM values('arr Array(JSON)', '[\'{"x" : 1}\']') FORMAT JSONEachRow; SELECT * FROM values('arr Map(String, JSON)', '{\'x\' : \'{"y" : 1}\', \'t\' : \'{"y" : 2}\'}') FORMAT JSONEachRow; SELECT * FROM values('arr Tuple(Int32, JSON)', '(1, \'{"y" : 1}\')', '(2, \'{"y" : 2}\')') FORMAT JSONEachRow; SELECT * FROM format(JSONEachRow, '{"arr" : [{"x" : "aaa", "y" : [1,2,3]}]}') FORMAT JSONEachRow; +SELECT * FROM values('arr Array(JSON)', '[\'{"x" : 1}\']') FORMAT JSONEachRow; From 5cb63575493092a26bec969f6415ef63efd76f3d Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Mon, 3 Oct 2022 19:37:27 +0000 Subject: [PATCH 015/152] fix style check --- src/DataTypes/ObjectUtils.cpp | 1 - src/Storages/MergeTree/MergeTreeSink.cpp | 1 - src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp | 1 - 3 files changed, 3 deletions(-) diff --git a/src/DataTypes/ObjectUtils.cpp b/src/DataTypes/ObjectUtils.cpp index 34ece825a16..e711b34ffa9 100644 --- a/src/DataTypes/ObjectUtils.cpp +++ b/src/DataTypes/ObjectUtils.cpp @@ -1,4 +1,3 @@ -#include #include #include #include diff --git a/src/Storages/MergeTree/MergeTreeSink.cpp b/src/Storages/MergeTree/MergeTreeSink.cpp index dcbd721267b..a3b0e4a0a78 100644 --- a/src/Storages/MergeTree/MergeTreeSink.cpp +++ b/src/Storages/MergeTree/MergeTreeSink.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include diff --git a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp index 2577144b3fb..8069e9b3f6a 100644 --- a/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp +++ b/src/Storages/MergeTree/ReplicatedMergeTreeSink.cpp @@ -1,7 +1,6 @@ #include #include #include -#include #include #include #include From c8642514d0a9c6a5f583902f3aacee87b4b50452 Mon Sep 17 00:00:00 2001 From: Suzy Wang Date: Mon, 17 Oct 2022 12:02:28 -0700 Subject: [PATCH 016/152] format ipv6 in s390x, endianess support --- src/Common/formatIPv6.h | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Common/formatIPv6.h b/src/Common/formatIPv6.h index 83b9d6e9fb1..31d5e83760a 100644 --- a/src/Common/formatIPv6.h +++ b/src/Common/formatIPv6.h @@ -5,6 +5,7 @@ #include #include #include +#include #include #include @@ -55,8 +56,11 @@ inline bool parseIPv4(const char * src, unsigned char * dst) } if (*(src - 1) != '\0') return false; - +#if __BYTE_ORDER__ == __ORDER_BIG_ENDIAN__ + reverseMemcpy(dst, &result, sizeof(result)); +#else memcpy(dst, &result, sizeof(result)); +#endif return true; } From 2c2f977096570207529166bbec2cf942bb0100c7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Oct 2022 13:03:55 +0000 Subject: [PATCH 017/152] Make better --- src/Interpreters/getTableExpressions.cpp | 16 ------- src/Interpreters/getTableExpressions.h | 2 - src/Storages/HDFS/StorageHDFSCluster.cpp | 35 +++----------- src/Storages/HDFS/StorageHDFSCluster.h | 2 +- src/Storages/StorageS3Cluster.cpp | 30 ++---------- ...lumnsStructureToQueryWithClusterEngine.cpp | 47 +++++++++++++++++++ ...ColumnsStructureToQueryWithClusterEngine.h | 11 +++++ 7 files changed, 69 insertions(+), 74 deletions(-) create mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp create mode 100644 src/Storages/addColumnsStructureToQueryWithClusterEngine.h diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 43c9438ecec..26fdab5b836 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -156,20 +156,4 @@ TablesWithColumns getDatabaseAndTablesWithColumns( return tables_with_columns; } -ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) -{ - auto * select_query = query->as(); - if (!select_query || !select_query->tables()) - return nullptr; - - auto * tables = select_query->tables()->as(); - auto * table_expression = tables->children[0]->as()->table_expression->as(); - if (!table_expression->table_function) - return nullptr; - - auto * table_function = table_expression->table_function->as(); - return table_function->arguments->as(); -} - - } diff --git a/src/Interpreters/getTableExpressions.h b/src/Interpreters/getTableExpressions.h index 4497b376cc7..22fb30fbad3 100644 --- a/src/Interpreters/getTableExpressions.h +++ b/src/Interpreters/getTableExpressions.h @@ -24,6 +24,4 @@ ASTPtr extractTableExpression(const ASTSelectQuery & select, size_t table_number TablesWithColumns getDatabaseAndTablesWithColumns( const ASTTableExprConstPtrs & table_expressions, ContextPtr context, bool include_alias_cols, bool include_materialized_cols); -ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query); - } diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 53170ee2439..7b110bb09fc 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -11,7 +11,6 @@ #include #include #include -#include #include #include #include @@ -26,6 +25,7 @@ #include #include #include +#include #include @@ -38,29 +38,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static ASTPtr addColumnsStructureToQuery(const ASTPtr & query, const String & structure) -{ - /// Add argument with table structure to hdfsCluster table function in select query. - auto result_query = query->clone(); - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(result_query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function hdfsCluster, got '{}'", queryToString(query)); - - auto structure_literal = std::make_shared(structure); - - if (expression_list->children.size() != 2 && expression_list->children.size() != 3) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 or 3 arguments in hdfsCluster table functions, got {}", expression_list->children.size()); - - if (expression_list->children.size() == 2) - { - auto format_literal = std::make_shared("auto"); - expression_list->children.push_back(format_literal); - } - - expression_list->children.push_back(structure_literal); - return result_query; -} - StorageHDFSCluster::StorageHDFSCluster( ContextPtr context_, String cluster_name_, @@ -85,7 +62,7 @@ StorageHDFSCluster::StorageHDFSCluster( { auto columns = StorageHDFS::getTableStructureFromData(format_name, uri_, compression_method, context_); storage_metadata.setColumns(columns); - need_to_add_structure_to_query = true; + add_columns_structure_to_query = true; } else storage_metadata.setColumns(columns_); @@ -122,10 +99,10 @@ Pipe StorageHDFSCluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - auto query_to_send = query_info.original_query; - if (need_to_add_structure_to_query) - query_to_send = addColumnsStructureToQuery( - query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll())); + auto query_to_send = query_info.original_query->clone(); + if (add_columns_structure_to_query) + addColumnsStructureToQueryWithClusterEngine( + query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 3, getName()); for (const auto & replicas : cluster->getShardsAddresses()) { diff --git a/src/Storages/HDFS/StorageHDFSCluster.h b/src/Storages/HDFS/StorageHDFSCluster.h index 88ad2c8871c..1eb823582d5 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.h +++ b/src/Storages/HDFS/StorageHDFSCluster.h @@ -44,7 +44,7 @@ private: String uri; String format_name; String compression_method; - bool need_to_add_structure_to_query = false; + bool add_columns_structure_to_query = false; }; diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index d4bd551aa56..eaa2c9a46d2 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -14,7 +14,6 @@ #include #include #include -#include #include #include #include @@ -27,6 +26,7 @@ #include #include #include +#include #include #include @@ -44,28 +44,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -static ASTPtr addColumnsStructureToQuery(const ASTPtr & query, const String & structure) -{ - /// Add argument with table structure to s3Cluster table function in select query. - auto result_query = query->clone(); - ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(result_query); - if (!expression_list) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function from s3Cluster, got '{}'", queryToString(query)); - auto structure_literal = std::make_shared(structure); - - if (expression_list->children.size() < 2 || expression_list->children.size() > 5) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to 5 arguments in s3Cluster table functions, got {}", expression_list->children.size()); - - if (expression_list->children.size() == 2 || expression_list->children.size() == 4) - { - auto format_literal = std::make_shared("auto"); - expression_list->children.push_back(format_literal); - } - - expression_list->children.push_back(structure_literal); - return result_query; -} - StorageS3Cluster::StorageS3Cluster( const StorageS3ClusterConfiguration & configuration_, const StorageID & table_id_, @@ -139,10 +117,10 @@ Pipe StorageS3Cluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; - ASTPtr query_to_send = query_info.original_query; + ASTPtr query_to_send = query_info.original_query->clone(); if (need_to_add_structure_to_query) - query_to_send = addColumnsStructureToQuery( - query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll())); + addColumnsStructureToQueryWithClusterEngine( + query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName()); for (const auto & replicas : cluster->getShardsAddresses()) { diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp new file mode 100644 index 00000000000..34b05445b0e --- /dev/null +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp @@ -0,0 +1,47 @@ +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +static ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) +{ + auto * select_query = query->as(); + if (!select_query || !select_query->tables()) + return nullptr; + + auto * tables = select_query->tables()->as(); + auto * table_expression = tables->children[0]->as()->table_expression->as(); + if (!table_expression->table_function) + return nullptr; + + auto * table_function = table_expression->table_function->as(); + return table_function->arguments->as(); +} + +void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name) +{ + /// Add argument with table structure to s3Cluster table function in select query. + ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); + if (!expression_list) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function {}, got '{}'", function_name, queryToString(query)); + auto structure_literal = std::make_shared(structure); + + if (expression_list->children.size() < 2 || expression_list->children.size() > max_arguments) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 2 to {} arguments in {} table functions, got {}", function_name, max_arguments, expression_list->children.size()); + + if (expression_list->children.size() == 2 || expression_list->children.size() == max_arguments - 1) + { + auto format_literal = std::make_shared("auto"); + expression_list->children.push_back(format_literal); + } + + expression_list->children.push_back(structure_literal); +} + +} diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.h b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h new file mode 100644 index 00000000000..f39f3a31630 --- /dev/null +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.h @@ -0,0 +1,11 @@ +#pragma once + +#include + +namespace DB +{ + +/// Add structure argument for queries with s3Cluster/hdfsCluster table function. +void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name); + +} From 9a7fbcecb00c7419f14f656ad2481e1717eac3ae Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Oct 2022 13:04:42 +0000 Subject: [PATCH 018/152] Remove unneded include --- src/Interpreters/getTableExpressions.cpp | 1 - src/Interpreters/getTableExpressions.h | 1 - 2 files changed, 2 deletions(-) diff --git a/src/Interpreters/getTableExpressions.cpp b/src/Interpreters/getTableExpressions.cpp index 26fdab5b836..830f0ea4411 100644 --- a/src/Interpreters/getTableExpressions.cpp +++ b/src/Interpreters/getTableExpressions.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include namespace DB diff --git a/src/Interpreters/getTableExpressions.h b/src/Interpreters/getTableExpressions.h index 22fb30fbad3..c4ca01ee3c3 100644 --- a/src/Interpreters/getTableExpressions.h +++ b/src/Interpreters/getTableExpressions.h @@ -3,7 +3,6 @@ #include #include #include -#include namespace DB { From 3bb11618597facbfbdedd4522b6292c7829e4293 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Oct 2022 13:05:40 +0000 Subject: [PATCH 019/152] Better naming --- src/Storages/StorageS3Cluster.cpp | 4 ++-- src/Storages/StorageS3Cluster.h | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index eaa2c9a46d2..6376fac4cfc 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -71,7 +71,7 @@ StorageS3Cluster::StorageS3Cluster( auto columns = StorageS3::getTableStructureFromDataImpl(format_name, s3_configuration, compression_method, /*distributed_processing_*/false, is_key_with_globs, /*format_settings=*/std::nullopt, context_); storage_metadata.setColumns(columns); - need_to_add_structure_to_query = true; + add_columns_structure_to_query = true; } else storage_metadata.setColumns(columns_); @@ -118,7 +118,7 @@ Pipe StorageS3Cluster::read( const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState; ASTPtr query_to_send = query_info.original_query->clone(); - if (need_to_add_structure_to_query) + if (add_columns_structure_to_query) addColumnsStructureToQueryWithClusterEngine( query_to_send, StorageDictionary::generateNamesAndTypesDescription(storage_snapshot->metadata->getColumns().getAll()), 5, getName()); diff --git a/src/Storages/StorageS3Cluster.h b/src/Storages/StorageS3Cluster.h index 3f77152833d..c3ff4bdb94b 100644 --- a/src/Storages/StorageS3Cluster.h +++ b/src/Storages/StorageS3Cluster.h @@ -46,7 +46,7 @@ private: String compression_method; NamesAndTypesList virtual_columns; Block virtual_block; - bool need_to_add_structure_to_query = false; + bool add_columns_structure_to_query = false; }; From be9c6b2c47e35f4bd382be652a9c79cef0ff22e7 Mon Sep 17 00:00:00 2001 From: avogar Date: Tue, 18 Oct 2022 13:08:45 +0000 Subject: [PATCH 020/152] Fix typo in test --- .../0_stateless/02457_s3_cluster_schema_inference.sql | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql index 039c7a9fd7f..03e8785b24b 100644 --- a/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql +++ b/tests/queries/0_stateless/02457_s3_cluster_schema_inference.sql @@ -6,8 +6,8 @@ desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localh desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest'); desc s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV'); -select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c2; -select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV') order by c1, c2, c2; -select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest') order by c1, c2, c2; -select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV') order by c1, c2, c2; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'TSV') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest') order by c1, c2, c3; +select * from s3Cluster('test_cluster_one_shard_three_replicas_localhost', 'http://localhost:11111/test/{a,b}.tsv', 'test', 'testtest', 'TSV') order by c1, c2, c3; From ba3e213a8392f6205374522270d83182dc87d864 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Tue, 18 Oct 2022 17:03:04 +0200 Subject: [PATCH 021/152] Remove outdated comment --- src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp index 34b05445b0e..9f09c61ec2f 100644 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp @@ -26,7 +26,6 @@ static ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & void addColumnsStructureToQueryWithClusterEngine(ASTPtr & query, const String & structure, size_t max_arguments, const String & function_name) { - /// Add argument with table structure to s3Cluster table function in select query. ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query); if (!expression_list) throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function {}, got '{}'", function_name, queryToString(query)); From 549597fe804353800b6505c2b03c029a073f9782 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Wed, 19 Oct 2022 12:15:27 +0800 Subject: [PATCH 022/152] Fix misbehavior of key analysis. When key types cannot be inside Nullable, it can break key analysis. See https://github.com/ClickHouse/ClickHouse/issues/42456. --- src/Interpreters/castColumn.cpp | 10 ++++++++++ src/Interpreters/castColumn.h | 1 + src/Storages/MergeTree/KeyCondition.cpp | 6 ++++-- ...dition_with_types_that_cannot_be_nullable.reference | 1 + ...ey_condition_with_types_that_cannot_be_nullable.sql | 9 +++++++++ 5 files changed, 25 insertions(+), 2 deletions(-) create mode 100644 tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.reference create mode 100644 tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.sql diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index dc9882b84b0..744cfff2527 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -51,4 +51,14 @@ ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const Data return castColumn(arg, type); } +ColumnPtr tryCastColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type) +try +{ + return castColumn(arg, type); +} +catch (...) +{ + return nullptr; +} + } diff --git a/src/Interpreters/castColumn.h b/src/Interpreters/castColumn.h index fcbea0f4646..c0a2cfbefbc 100644 --- a/src/Interpreters/castColumn.h +++ b/src/Interpreters/castColumn.h @@ -8,5 +8,6 @@ namespace DB ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type); ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type); ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type); +ColumnPtr tryCastColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 9f5f2873b98..3ecaff2c19a 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -1182,11 +1182,13 @@ bool KeyCondition::transformConstantWithValidFunctions( { auto const_type = cur_node->result_type; auto const_column = out_type->createColumnConst(1, out_value); - auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0]; + auto maybe_const_value_column = tryCastColumnAccurate({const_column, out_type, ""}, const_type); - if (const_value.isNull()) + if (maybe_const_value_column == nullptr) return false; + auto const_value = (*maybe_const_value_column)[0]; + while (!chain.empty()) { const auto * func = chain.top(); diff --git a/tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.reference b/tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.reference new file mode 100644 index 00000000000..13b65c29f05 --- /dev/null +++ b/tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.reference @@ -0,0 +1 @@ +printer1 diff --git a/tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.sql b/tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.sql new file mode 100644 index 00000000000..690ec6c70e0 --- /dev/null +++ b/tests/queries/0_stateless/02457_key_condition_with_types_that_cannot_be_nullable.sql @@ -0,0 +1,9 @@ +drop table if exists test; + +create table test (Printer LowCardinality(String), IntervalStart DateTime) engine MergeTree partition by (hiveHash(Printer), toYear(IntervalStart)) order by (Printer, IntervalStart); + +insert into test values ('printer1', '2006-02-07 06:28:15'); + +select Printer from test where Printer='printer1'; + +drop table test; From 830ab3a5a959134ff3fad278b36b160d5e185f1e Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Oct 2022 21:48:01 +0200 Subject: [PATCH 023/152] Fix style --- src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp index 9f09c61ec2f..31f49fa5490 100644 --- a/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp +++ b/src/Storages/addColumnsStructureToQueryWithClusterEngine.cpp @@ -9,6 +9,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + static ASTExpressionList * extractTableFunctionArgumentsFromSelectQuery(ASTPtr & query) { auto * select_query = query->as(); From ae3fa705946745a4340bd0fe0a29acea7cb0a955 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Oct 2022 21:48:20 +0200 Subject: [PATCH 024/152] Fix style --- src/Storages/StorageS3Cluster.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/StorageS3Cluster.cpp b/src/Storages/StorageS3Cluster.cpp index 6376fac4cfc..bdf08327a97 100644 --- a/src/Storages/StorageS3Cluster.cpp +++ b/src/Storages/StorageS3Cluster.cpp @@ -39,11 +39,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - StorageS3Cluster::StorageS3Cluster( const StorageS3ClusterConfiguration & configuration_, const StorageID & table_id_, From df96acc80f087a876387798e6107c5aeb13161ff Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Wed, 19 Oct 2022 21:48:45 +0200 Subject: [PATCH 025/152] Fix style --- src/Storages/HDFS/StorageHDFSCluster.cpp | 5 ----- 1 file changed, 5 deletions(-) diff --git a/src/Storages/HDFS/StorageHDFSCluster.cpp b/src/Storages/HDFS/StorageHDFSCluster.cpp index 7b110bb09fc..2b0222b7349 100644 --- a/src/Storages/HDFS/StorageHDFSCluster.cpp +++ b/src/Storages/HDFS/StorageHDFSCluster.cpp @@ -33,11 +33,6 @@ namespace DB { -namespace ErrorCodes -{ - extern const int LOGICAL_ERROR; -} - StorageHDFSCluster::StorageHDFSCluster( ContextPtr context_, String cluster_name_, From 2499ab3b3a9bb9fcf9ff09172fb0b869f7db4e19 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Thu, 20 Oct 2022 17:53:52 +0800 Subject: [PATCH 026/152] Use convertFieldToType --- src/Storages/MergeTree/KeyCondition.cpp | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 3ecaff2c19a..194cfbdabfc 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -6,7 +6,6 @@ #include #include #include -#include #include #include #include @@ -1181,14 +1180,10 @@ bool KeyCondition::transformConstantWithValidFunctions( if (is_valid_chain) { auto const_type = cur_node->result_type; - auto const_column = out_type->createColumnConst(1, out_value); - auto maybe_const_value_column = tryCastColumnAccurate({const_column, out_type, ""}, const_type); - - if (maybe_const_value_column == nullptr) + auto const_value = convertFieldToType(out_value, *const_type); + if (const_value.isNull()) return false; - auto const_value = (*maybe_const_value_column)[0]; - while (!chain.empty()) { const auto * func = chain.top(); From 867bcdbb1c5f433db91ea5c7141b168993c75425 Mon Sep 17 00:00:00 2001 From: Kruglov Pavel <48961922+Avogar@users.noreply.github.com> Date: Thu, 20 Oct 2022 16:46:25 +0200 Subject: [PATCH 027/152] Fix typo in setting name that led to bad usage of schema inference cache --- src/Formats/EscapingRuleUtils.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Formats/EscapingRuleUtils.cpp b/src/Formats/EscapingRuleUtils.cpp index e47525d089a..e80ab50968d 100644 --- a/src/Formats/EscapingRuleUtils.cpp +++ b/src/Formats/EscapingRuleUtils.cpp @@ -859,7 +859,7 @@ String getAdditionalFormatInfoByEscapingRule(const FormatSettings & settings, Fo result += fmt::format( ", use_best_effort_in_schema_inference={}, bool_true_representation={}, bool_false_representation={}," " null_representation={}, delimiter={}, tuple_delimiter={}", - settings.tsv.use_best_effort_in_schema_inference, + settings.csv.use_best_effort_in_schema_inference, settings.bool_true_representation, settings.bool_false_representation, settings.csv.null_representation, From 1a462fddea3c518e51c68f74a2f7c6193644042a Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Fri, 21 Oct 2022 13:28:24 +0800 Subject: [PATCH 028/152] Fix test --- src/Interpreters/convertFieldToType.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Interpreters/convertFieldToType.cpp b/src/Interpreters/convertFieldToType.cpp index 4e7562ef451..fdbae838ab4 100644 --- a/src/Interpreters/convertFieldToType.cpp +++ b/src/Interpreters/convertFieldToType.cpp @@ -218,10 +218,11 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const ID } if (which_type.isDateTime64() - && (which_from_type.isNativeInt() || which_from_type.isNativeUInt() || which_from_type.isDate() || which_from_type.isDate32() || which_from_type.isDateTime() || which_from_type.isDateTime64())) + && (src.getType() == Field::Types::UInt64 || src.getType() == Field::Types::Int64 || src.getType() == Field::Types::Decimal64)) { const auto scale = static_cast(type).getScale(); - const auto decimal_value = DecimalUtils::decimalFromComponents(applyVisitor(FieldVisitorConvertToNumber(), src), 0, scale); + const auto decimal_value + = DecimalUtils::decimalFromComponents(applyVisitor(FieldVisitorConvertToNumber(), src), 0, scale); return Field(DecimalField(decimal_value, scale)); } } From f104dd08cdf820382a375a32fb5b53ab2507fd63 Mon Sep 17 00:00:00 2001 From: Amos Bird Date: Sat, 22 Oct 2022 15:47:13 +0800 Subject: [PATCH 029/152] Only fix LowCardinality for now --- src/Interpreters/castColumn.cpp | 10 ---------- src/Interpreters/castColumn.h | 1 - src/Storages/MergeTree/KeyCondition.cpp | 8 ++++++-- 3 files changed, 6 insertions(+), 13 deletions(-) diff --git a/src/Interpreters/castColumn.cpp b/src/Interpreters/castColumn.cpp index 744cfff2527..dc9882b84b0 100644 --- a/src/Interpreters/castColumn.cpp +++ b/src/Interpreters/castColumn.cpp @@ -51,14 +51,4 @@ ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const Data return castColumn(arg, type); } -ColumnPtr tryCastColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type) -try -{ - return castColumn(arg, type); -} -catch (...) -{ - return nullptr; -} - } diff --git a/src/Interpreters/castColumn.h b/src/Interpreters/castColumn.h index c0a2cfbefbc..fcbea0f4646 100644 --- a/src/Interpreters/castColumn.h +++ b/src/Interpreters/castColumn.h @@ -8,6 +8,5 @@ namespace DB ColumnPtr castColumn(const ColumnWithTypeAndName & arg, const DataTypePtr & type); ColumnPtr castColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type); ColumnPtr castColumnAccurateOrNull(const ColumnWithTypeAndName & arg, const DataTypePtr & type); -ColumnPtr tryCastColumnAccurate(const ColumnWithTypeAndName & arg, const DataTypePtr & type); } diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 194cfbdabfc..cc5f6998955 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -6,6 +6,7 @@ #include #include #include +#include #include #include #include @@ -1179,8 +1180,11 @@ bool KeyCondition::transformConstantWithValidFunctions( if (is_valid_chain) { - auto const_type = cur_node->result_type; - auto const_value = convertFieldToType(out_value, *const_type); + out_type = removeLowCardinality(out_type); + auto const_type = removeLowCardinality(cur_node->result_type); + auto const_column = out_type->createColumnConst(1, out_value); + auto const_value = (*castColumnAccurateOrNull({const_column, out_type, ""}, const_type))[0]; + if (const_value.isNull()) return false; From 0c5011419009f39be9ca7aeec6adca30be8511a6 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Mon, 24 Oct 2022 14:41:38 -0400 Subject: [PATCH 030/152] Update Fedora build info I am using Fedora 36. yum install of `clang-c++` failed for me, `clang` worked. And I had an error message telling me to install `ccache` also. --- docs/en/development/build.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index f397dc0d037..8982a3bc0a4 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -105,7 +105,7 @@ ninja Example for Fedora Rawhide: ``` bash sudo yum update -yum --nogpg install git cmake make clang-c++ python3 +sudo yum --nogpg install git cmake make clang python3 ccache git clone --recursive https://github.com/ClickHouse/ClickHouse.git mkdir build && cd build cmake ../ClickHouse From dd4121a2906883d23c65e0eeb96bbc0c2c3c338e Mon Sep 17 00:00:00 2001 From: peter279k Date: Tue, 25 Oct 2022 15:27:52 +0800 Subject: [PATCH 031/152] Migration method for installing the deb-packages --- docs/en/getting-started/install.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/docs/en/getting-started/install.md b/docs/en/getting-started/install.md index 61303eddab9..e88e9e06a68 100644 --- a/docs/en/getting-started/install.md +++ b/docs/en/getting-started/install.md @@ -128,6 +128,24 @@ clickhouse-client # or "clickhouse-client --password" if you set up a password. +
+Migration Method for installing the deb-packages + +```bash +sudo apt-key del E0C56BD4 +sudo apt-key adv --keyserver hkp://keyserver.ubuntu.com:80 --recv 8919F6BD2B48D754 +echo "deb https://packages.clickhouse.com/deb stable main" | sudo tee \ + /etc/apt/sources.list.d/clickhouse.list +sudo apt-get update + +sudo apt-get install -y clickhouse-server clickhouse-client + +sudo service clickhouse-server start +clickhouse-client # or "clickhouse-client --password" if you set up a password. +``` + +
+ You can replace `stable` with `lts` to use different [release kinds](/docs/en/faq/operations/production.md) based on your needs. You can also download and install packages manually from [here](https://packages.clickhouse.com/deb/pool/main/c/). From fc223aa6731e15928e94a17f96b87644dc4017ee Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 25 Oct 2022 23:52:31 +0000 Subject: [PATCH 032/152] replace throttler algorithm by token bucket --- src/Common/Throttler.cpp | 80 ++++++++++++++++++---------------------- src/Common/Throttler.h | 37 ++++++++++--------- 2 files changed, 55 insertions(+), 62 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 2c9279e21e1..5a0f7e227a6 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -3,7 +3,6 @@ #include #include #include -#include namespace ProfileEvents { @@ -21,63 +20,56 @@ namespace ErrorCodes /// Just 10^9. static constexpr auto NS = 1000000000UL; -/// Tracking window. Actually the size is not really important. We just want to avoid -/// throttles when there are no actions for a long period time. -static const double window_ns = 1ULL * NS; +static const size_t default_burst_seconds = 10; + +Throttler::Throttler(size_t max_speed_, const std::shared_ptr & parent_) + : max_speed(max_speed_) + , max_burst(max_speed_ * default_burst_seconds) + , limit_exceeded_exception_message("") + , tokens(max_burst) + , parent(parent_) +{} + +Throttler::Throttler(size_t max_speed_, size_t limit_, const char * limit_exceeded_exception_message_, + const std::shared_ptr & parent_) + : max_speed(max_speed_) + , max_burst(max_speed_ * default_burst_seconds) + , limit(limit_) + , limit_exceeded_exception_message(limit_exceeded_exception_message_) + , tokens(max_burst) + , parent(parent_) +{} void Throttler::add(size_t amount) { - size_t new_count; - /// This outer variable is always equal to smoothed_speed. - /// We use to avoid race condition. - double current_speed = 0; - + // Values obtained under lock to be checked after release + size_t count_value; + double tokens_value; { std::lock_guard lock(mutex); - auto now = clock_gettime_ns_adjusted(prev_ns); - /// If prev_ns is equal to zero (first `add` call) we known nothing about speed - /// and don't track anything. if (max_speed && prev_ns != 0) { - /// Time spent to process the amount of bytes - double time_spent = now - prev_ns; - - /// The speed in bytes per second is equal to amount / time_spent in seconds - auto new_speed = amount / (time_spent / NS); - - /// We want to make old values of speed less important for our smoothed value - /// so we decay it's value with coef. - auto decay_coeff = std::pow(0.5, time_spent / window_ns); - - /// Weighted average between previous and new speed - smoothed_speed = smoothed_speed * decay_coeff + (1 - decay_coeff) * new_speed; - current_speed = smoothed_speed; + double delta_seconds = static_cast(now - prev_ns) / NS; + tokens = std::max(tokens + max_speed * delta_seconds - amount, max_burst); } - count += amount; - new_count = count; + count_value = count; + tokens_value = tokens; prev_ns = now; } - if (limit && new_count > limit) + if (limit && count_value > limit) throw Exception(limit_exceeded_exception_message + std::string(" Maximum: ") + toString(limit), ErrorCodes::LIMIT_EXCEEDED); - if (max_speed && current_speed > max_speed) + /// Wait unless there is positive amount of tokens - throttling + if (max_speed && tokens_value < 0) { - /// If we was too fast then we have to sleep until our smoothed speed became <= max_speed - int64_t sleep_time = static_cast(-window_ns * std::log2(max_speed / current_speed)); - - if (sleep_time > 0) - { - accumulated_sleep += sleep_time; - - sleepForNanoseconds(sleep_time); - - accumulated_sleep -= sleep_time; - - ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time / 1000UL); - } + int64_t sleep_time = static_cast(-tokens_value * max_speed * NS); + accumulated_sleep += sleep_time; + sleepForNanoseconds(sleep_time); + accumulated_sleep -= sleep_time; + ProfileEvents::increment(ProfileEvents::ThrottlerSleepMicroseconds, sleep_time / 1000UL); } if (parent) @@ -89,9 +81,9 @@ void Throttler::reset() std::lock_guard lock(mutex); count = 0; - accumulated_sleep = 0; - smoothed_speed = 0; + tokens = max_burst; prev_ns = 0; + // NOTE: do not zero `accumulated_sleep` to avoid races } bool Throttler::isThrottling() const diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index 6d44ad6ca5f..a33637783e7 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -10,24 +10,26 @@ namespace DB { -/** Allows you to limit the speed of something (in entities per second) using sleep. - * Specifics of work: - * Tracks exponentially (pow of 1/2) smoothed speed with hardcoded window. - * See more comments in .cpp file. - * - * Also allows you to set a limit on the maximum number of entities. If exceeded, an exception will be thrown. +/** Allows you to limit the speed of something (in tokens per second) using sleep. + * Implemented using Token Bucket Throttling algorithm. + * Also allows you to set a limit on the maximum number of tokens. If exceeded, an exception will be thrown. */ class Throttler { public: - explicit Throttler(size_t max_speed_, const std::shared_ptr & parent_ = nullptr) - : max_speed(max_speed_), limit_exceeded_exception_message(""), parent(parent_) {} + Throttler(size_t max_speed_, size_t max_burst_, const std::shared_ptr & parent_ = nullptr) + : max_speed(max_speed_), max_burst(max_burst_), limit_exceeded_exception_message(""), tokens(max_burst), parent(parent_) {} + + explicit Throttler(size_t max_speed_, const std::shared_ptr & parent_ = nullptr); + + Throttler(size_t max_speed_, size_t max_burst_, size_t limit_, const char * limit_exceeded_exception_message_, + const std::shared_ptr & parent_ = nullptr) + : max_speed(max_speed_), max_burst(max_burst_), limit(limit_), limit_exceeded_exception_message(limit_exceeded_exception_message_), tokens(max_burst), parent(parent_) {} Throttler(size_t max_speed_, size_t limit_, const char * limit_exceeded_exception_message_, - const std::shared_ptr & parent_ = nullptr) - : max_speed(max_speed_), limit(limit_), limit_exceeded_exception_message(limit_exceeded_exception_message_), parent(parent_) {} + const std::shared_ptr & parent_ = nullptr); - /// Calculates the smoothed speed, sleeps if required and throws exception on + /// Use `amount` ent, sleeps if required and throws exception on /// limit overflow. void add(size_t amount); @@ -45,15 +47,14 @@ public: private: size_t count{0}; - const size_t max_speed{0}; - const uint64_t limit{0}; /// 0 - not limited. + const size_t max_speed{0}; /// in tokens per second. + const size_t max_burst{0}; /// in tokens. + const uint64_t limit{0}; /// 0 - not limited. const char * limit_exceeded_exception_message = nullptr; std::mutex mutex; - std::atomic accumulated_sleep{0}; - /// Smoothed value of current speed. Updated in `add` method. - double smoothed_speed{0}; - /// previous `add` call time (in nanoseconds) - uint64_t prev_ns{0}; + std::atomic accumulated_sleep{0}; // Accumulated sleep time over all waiting threads + double tokens{0}; /// Amount of tokens available in token bucket. Updated in `add` method. + uint64_t prev_ns{0}; /// Previous `add` call time (in nanoseconds). /// Used to implement a hierarchy of throttlers std::shared_ptr parent; From 27599ab70cf9c3b5b19efe497a554b1266b6210e Mon Sep 17 00:00:00 2001 From: serxa Date: Tue, 25 Oct 2022 23:58:25 +0000 Subject: [PATCH 033/152] fix comment --- src/Common/Throttler.h | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Common/Throttler.h b/src/Common/Throttler.h index a33637783e7..9b6eff13506 100644 --- a/src/Common/Throttler.h +++ b/src/Common/Throttler.h @@ -29,8 +29,7 @@ public: Throttler(size_t max_speed_, size_t limit_, const char * limit_exceeded_exception_message_, const std::shared_ptr & parent_ = nullptr); - /// Use `amount` ent, sleeps if required and throws exception on - /// limit overflow. + /// Use `amount` tokens, sleeps if required or throws exception on limit overflow. void add(size_t amount); /// Not thread safe From 471b391ab2b92ebd18ca019ea00dca2aa6f99c1c Mon Sep 17 00:00:00 2001 From: serxa Date: Wed, 26 Oct 2022 00:00:40 +0000 Subject: [PATCH 034/152] fix --- src/Common/Throttler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 5a0f7e227a6..a3ae966c49b 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -51,7 +51,7 @@ void Throttler::add(size_t amount) if (max_speed && prev_ns != 0) { double delta_seconds = static_cast(now - prev_ns) / NS; - tokens = std::max(tokens + max_speed * delta_seconds - amount, max_burst); + tokens = std::min(tokens + max_speed * delta_seconds - amount, max_burst); } count += amount; count_value = count; From 3eca9ada3b25ba789c049753138b799280c71fb8 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 26 Oct 2022 08:00:12 +0000 Subject: [PATCH 035/152] Add Date32 to formatDateTIme --- src/Functions/formatDateTime.cpp | 15 +++++++++++++-- 1 file changed, 13 insertions(+), 2 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index a10c059b342..4f28dae7a66 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -45,6 +46,7 @@ template <> struct ActionValueTypeMap { using ActionValueTyp template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; template <> struct ActionValueTypeMap { using ActionValueType = UInt16; }; +template <> struct ActionValueTypeMap { using ActionValueType = Int32; }; template <> struct ActionValueTypeMap { using ActionValueType = UInt32; }; // TODO(vnemkov): to add sub-second format instruction, make that DateTime64 and do some math in Action. template <> struct ActionValueTypeMap { using ActionValueType = Int64; }; @@ -324,7 +326,7 @@ public: "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + " when arguments size is 1. Should be integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); - if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDateTime64(arguments[0].type))) + if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDate32(arguments[0].type) || isDateTime64(arguments[0].type))) throw Exception( "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + " when arguments size is 2 or 3. Should be a integer or a date with time", @@ -337,7 +339,7 @@ public: "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) + ", should be 2 or 3", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); - if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDateTime64(arguments[0].type)) + if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception( "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() + ". Should be a date or a date with time", @@ -393,6 +395,7 @@ public: })) { if (!((res = executeType(arguments, result_type)) + || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)))) throw Exception( @@ -405,6 +408,7 @@ public: else { if (!((res = executeType(arguments, result_type)) + || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)))) throw Exception( @@ -496,6 +500,13 @@ public: instruction.perform(pos, static_cast(c.whole), time_zone); } } + else if constexpr (std::is_same_v) + { + for (auto & instruction : instructions) + { + instruction.perform(pos, static_cast(vec[i]), time_zone); + } + } else { for (auto & instruction : instructions) From 1b6293f6db33b474f518d40784a6cfb758c6f306 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 26 Oct 2022 12:43:29 +0200 Subject: [PATCH 036/152] MergeTree indexes use ActionsDAG --- .../QueryPlan/ReadFromMergeTree.cpp | 7 +- src/Storages/MergeTree/KeyCondition.cpp | 581 ++++-------------- src/Storages/MergeTree/KeyCondition.h | 82 +-- .../MergeTreeIndexConditionBloomFilter.cpp | 325 +++++----- .../MergeTreeIndexConditionBloomFilter.h | 24 +- .../MergeTree/MergeTreeIndexFullText.cpp | 151 ++--- .../MergeTree/MergeTreeIndexFullText.h | 8 +- .../MergeTree/MergeTreeIndexUtils.cpp | 47 ++ src/Storages/MergeTree/MergeTreeIndexUtils.h | 13 + src/Storages/MergeTree/RPNBuilder.cpp | 380 ++++++++++++ src/Storages/MergeTree/RPNBuilder.h | 276 +++++++-- 11 files changed, 1063 insertions(+), 831 deletions(-) create mode 100644 src/Storages/MergeTree/MergeTreeIndexUtils.cpp create mode 100644 src/Storages/MergeTree/MergeTreeIndexUtils.h create mode 100644 src/Storages/MergeTree/RPNBuilder.cpp diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 62b6eddf6ce..71f8378ae41 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -906,8 +906,11 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( for (const auto & node : added_filter_nodes.nodes) nodes.nodes.push_back(node); - key_condition.emplace( - std::move(nodes), query_info.syntax_analyzer_result, query_info.prepared_sets, context, primary_key_columns, primary_key.expression); + key_condition.emplace(std::move(nodes), + context, + primary_key_columns, + primary_key.expression, + query_info.syntax_analyzer_result->getArrayJoinSourceNameSet()); } else { diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index b282fb79514..596345cade7 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -27,6 +27,7 @@ #include #include #include +#include #include #include @@ -112,289 +113,6 @@ static String firstStringThatIsGreaterThanAllStringsWithPrefix(const String & pr return res; } -static void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool legacy = false) -{ - switch (node.type) - { - case (ActionsDAG::ActionType::INPUT): - writeString(node.result_name, out); - break; - case (ActionsDAG::ActionType::COLUMN): - { - /// If it was created from ASTLiteral, then result_name can be an alias. - /// We need to convert value back to string here. - if (const auto * column_const = typeid_cast(node.column.get())) - writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out); - /// It may be possible that column is ColumnSet - else - writeString(node.result_name, out); - break; - } - case (ActionsDAG::ActionType::ALIAS): - appendColumnNameWithoutAlias(*node.children.front(), out, legacy); - break; - case (ActionsDAG::ActionType::ARRAY_JOIN): - writeCString("arrayJoin(", out); - appendColumnNameWithoutAlias(*node.children.front(), out, legacy); - writeChar(')', out); - break; - case (ActionsDAG::ActionType::FUNCTION): - { - auto name = node.function_base->getName(); - if (legacy && name == "modulo") - writeCString("moduleLegacy", out); - else - writeString(name, out); - - writeChar('(', out); - bool first = true; - for (const auto * arg : node.children) - { - if (!first) - writeCString(", ", out); - first = false; - - appendColumnNameWithoutAlias(*arg, out, legacy); - } - writeChar(')', out); - } - } -} - -static std::string getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool legacy = false) -{ - WriteBufferFromOwnString out; - appendColumnNameWithoutAlias(node, out, legacy); - return std::move(out.str()); -} - -class KeyCondition::Tree -{ -public: - explicit Tree(const IAST * ast_) : ast(ast_) { assert(ast); } - explicit Tree(const ActionsDAG::Node * dag_) : dag(dag_) { assert(dag); } - - std::string getColumnName() const - { - if (ast) - return ast->getColumnNameWithoutAlias(); - else - return getColumnNameWithoutAlias(*dag); - } - - std::string getColumnNameLegacy() const - { - if (ast) - { - auto adjusted_ast = ast->clone(); - KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast); - return adjusted_ast->getColumnNameWithoutAlias(); - } - else - return getColumnNameWithoutAlias(*dag, true); - } - - bool isFunction() const - { - if (ast) - return typeid_cast(ast); - else - return dag->type == ActionsDAG::ActionType::FUNCTION; - } - - bool isConstant() const - { - if (ast) - return typeid_cast(ast); - else - return dag->column && isColumnConst(*dag->column); - } - - ColumnWithTypeAndName getConstant() const - { - if (!isConstant()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a constant"); - - ColumnWithTypeAndName res; - - if (ast) - { - const auto * literal = assert_cast(ast); - res.type = applyVisitor(FieldToDataType(), literal->value); - res.column = res.type->createColumnConst(0, literal->value); - - } - else - { - res.type = dag->result_type; - res.column = dag->column; - } - - return res; - } - - bool tryGetConstant(const Block & block_with_constants, Field & out_value, DataTypePtr & out_type) const - { - if (ast) - { - // Constant expr should use alias names if any - String column_name = ast->getColumnName(); - - if (const auto * lit = ast->as()) - { - /// By default block_with_constants has only one column named "_dummy". - /// If block contains only constants it's may not be preprocessed by - // ExpressionAnalyzer, so try to look up in the default column. - if (!block_with_constants.has(column_name)) - column_name = "_dummy"; - - /// Simple literal - out_value = lit->value; - out_type = block_with_constants.getByName(column_name).type; - - /// If constant is not Null, we can assume it's type is not Nullable as well. - if (!out_value.isNull()) - out_type = removeNullable(out_type); - - return true; - } - else if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column)) - { - /// An expression which is dependent on constants only - const auto & expr_info = block_with_constants.getByName(column_name); - out_value = (*expr_info.column)[0]; - out_type = expr_info.type; - - if (!out_value.isNull()) - out_type = removeNullable(out_type); - - return true; - } - } - else - { - if (dag->column && isColumnConst(*dag->column)) - { - out_value = (*dag->column)[0]; - out_type = dag->result_type; - - if (!out_value.isNull()) - out_type = removeNullable(out_type); - - return true; - } - } - - return false; - } - - ConstSetPtr tryGetPreparedSet( - const PreparedSetsPtr & sets, - const std::vector & indexes_mapping, - const DataTypes & data_types) const - { - if (sets && ast) - { - if (ast->as() || ast->as()) - return sets->get(PreparedSetKey::forSubquery(*ast)); - - /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information - /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets - /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check - /// that the types it was prepared with are compatible with the types of the primary key. - auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set) - { - assert(indexes_mapping.size() == data_types.size()); - - for (size_t i = 0; i < indexes_mapping.size(); ++i) - { - if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) - return false; - } - - return true; - }; - - for (const auto & set : sets->getByTreeHash(ast->getTreeHash())) - { - if (types_match(set)) - return set; - } - } - else if (dag->column) - { - const IColumn * col = dag->column.get(); - if (const auto * col_const = typeid_cast(col)) - col = &col_const->getDataColumn(); - - if (const auto * col_set = typeid_cast(col)) - { - auto set = col_set->getData(); - if (set->isCreated()) - return set; - } - } - - return nullptr; - } - - FunctionTree asFunction() const; - -protected: - const IAST * ast = nullptr; - const ActionsDAG::Node * dag = nullptr; -}; - -class KeyCondition::FunctionTree : public KeyCondition::Tree -{ -public: - std::string getFunctionName() const - { - if (ast) - return assert_cast(ast)->name; - else - return dag->function_base->getName(); - } - - size_t numArguments() const - { - if (ast) - { - const auto * func = assert_cast(ast); - return func->arguments ? func->arguments->children.size() : 0; - } - else - return dag->children.size(); - } - - Tree getArgumentAt(size_t idx) const - { - if (ast) - return Tree(assert_cast(ast)->arguments->children[idx].get()); - else - return Tree(dag->children[idx]); - } - -private: - using Tree::Tree; - - friend class Tree; -}; - - -KeyCondition::FunctionTree KeyCondition::Tree::asFunction() const -{ - if (!isFunction()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "KeyCondition::Tree node is not a function"); - - if (ast) - return KeyCondition::FunctionTree(ast); - else - return KeyCondition::FunctionTree(dag); -} - - -/// A dictionary containing actions to the corresponding functions to turn them into `RPNElement` const KeyCondition::AtomMap KeyCondition::atom_map { { @@ -869,16 +587,17 @@ static NameSet getAllSubexpressionNames(const ExpressionActions & key_expr) KeyCondition::KeyCondition( const ASTPtr & query, const ASTs & additional_filter_asts, - TreeRewriterResultPtr syntax_analyzer_result, - PreparedSetsPtr prepared_sets_, + Block block_with_constants, + PreparedSetsPtr prepared_sets, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr_, + NameSet array_joined_column_names_, bool single_point_, bool strict_) : key_expr(key_expr_) , key_subexpr_names(getAllSubexpressionNames(*key_expr)) - , prepared_sets(prepared_sets_) + , array_joined_column_names(std::move(array_joined_column_names_)) , single_point(single_point_) , strict(strict_) { @@ -889,82 +608,64 @@ KeyCondition::KeyCondition( key_columns[name] = i; } - if (!syntax_analyzer_result) + auto filter_node = buildFilterNode(query, additional_filter_asts); + + if (!filter_node) { rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); return; } - /** Evaluation of expressions that depend only on constants. - * For the index to be used, if it is written, for example `WHERE Date = toDate(now())`. + /** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)), + * the use of NOT operator in predicate will result in the indexing algorithm leave out some data. + * This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict + * when parsing the AST into internal RPN representation. + * To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's + * are pushed down and applied (when possible) to leaf nodes. */ - Block block_with_constants = getBlockWithConstants(query, syntax_analyzer_result, context); + auto inverted_filter_node = cloneASTWithInversionPushDown(filter_node); - if (syntax_analyzer_result) - { - for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) - array_joined_columns.insert(name); - } + RPNBuilder builder( + inverted_filter_node, + std::move(context), + std::move(block_with_constants), + std::move(prepared_sets), + [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); + rpn = std::move(builder).extractRPN(); +} - const ASTSelectQuery & select = query->as(); - - ASTs filters; - if (select.where()) - filters.push_back(select.where()); - - if (select.prewhere()) - filters.push_back(select.prewhere()); - - for (const auto & filter_ast : additional_filter_asts) - filters.push_back(filter_ast); - - if (!filters.empty()) - { - ASTPtr filter_query; - if (filters.size() == 1) - { - filter_query = filters.front(); - } - else - { - auto function = std::make_shared(); - - function->name = "and"; - function->arguments = std::make_shared(); - function->children.push_back(function->arguments); - function->arguments->children = std::move(filters); - - filter_query = function; - } - - /** When non-strictly monotonic functions are employed in functional index (e.g. ORDER BY toStartOfHour(dateTime)), - * the use of NOT operator in predicate will result in the indexing algorithm leave out some data. - * This is caused by rewriting in KeyCondition::tryParseAtomFromAST of relational operators to less strict - * when parsing the AST into internal RPN representation. - * To overcome the problem, before parsing the AST we transform it to its semantically equivalent form where all NOT's - * are pushed down and applied (when possible) to leaf nodes. - */ - auto ast = cloneASTWithInversionPushDown(filter_query); - traverseAST(Tree(ast.get()), context, block_with_constants); - } - else - { - rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); - } +KeyCondition::KeyCondition( + const SelectQueryInfo & query_info, + ContextPtr context, + const Names & key_column_names, + const ExpressionActionsPtr & key_expr_, + bool single_point_, + bool strict_) + : KeyCondition( + query_info.query, + query_info.filter_asts, + KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context), + query_info.prepared_sets, + context, + key_column_names, + key_expr_, + query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(), + single_point_, + strict_) +{ } KeyCondition::KeyCondition( ActionDAGNodes dag_nodes, - TreeRewriterResultPtr syntax_analyzer_result, - PreparedSetsPtr prepared_sets_, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr_, + NameSet array_joined_column_names_, bool single_point_, bool strict_) : key_expr(key_expr_) , key_subexpr_names(getAllSubexpressionNames(*key_expr)) - , prepared_sets(prepared_sets_) + , array_joined_column_names(std::move(array_joined_column_names_)) , single_point(single_point_) , strict(strict_) { @@ -975,29 +676,23 @@ KeyCondition::KeyCondition( key_columns[name] = i; } - if (!syntax_analyzer_result) + if (dag_nodes.nodes.empty()) { rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); return; } - for (const auto & [name, _] : syntax_analyzer_result->array_join_result_to_source) - array_joined_columns.insert(name); + auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context); + assert(inverted_dag->getOutputs().size() == 1); - if (!dag_nodes.nodes.empty()) + const auto * inverted_dag_filter_node = inverted_dag->getOutputs()[0]; + + RPNBuilder builder(inverted_dag_filter_node, context, [&](const RPNBuilderTreeNode & node, RPNElement & out) { - auto inverted_dag = cloneASTWithInversionPushDown(std::move(dag_nodes.nodes), context); + return extractAtomFromTree(node, out); + }); - // std::cerr << "========== inverted dag: " << inverted_dag->dumpDAG() << std::endl; - - Block empty; - for (const auto * node : inverted_dag->getOutputs()) - traverseAST(Tree(node), context, empty); - } - else - { - rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); - } + rpn = std::move(builder).extractRPN(); } bool KeyCondition::addCondition(const String & column, const Range & range) @@ -1009,12 +704,12 @@ bool KeyCondition::addCondition(const String & column, const Range & range) return true; } -/** Computes value of constant expression and its data type. - * Returns false, if expression isn't constant. - */ bool KeyCondition::getConstant(const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type) { - return Tree(expr.get()).tryGetConstant(block_with_constants, out_value, out_type); + RPNBuilderTreeContext tree_context(nullptr, block_with_constants, nullptr); + RPNBuilderTreeNode node(expr.get(), tree_context); + + return node.tryGetConstant(out_value, out_type); } @@ -1098,39 +793,6 @@ static FieldRef applyFunction(const FunctionBasePtr & func, const DataTypePtr & return {field.columns, field.row_idx, result_idx}; } -void KeyCondition::traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants) -{ - RPNElement element; - - if (node.isFunction()) - { - auto func = node.asFunction(); - if (tryParseLogicalOperatorFromAST(func, element)) - { - size_t num_args = func.numArguments(); - for (size_t i = 0; i < num_args; ++i) - { - traverseAST(func.getArgumentAt(i), context, block_with_constants); - - /** The first part of the condition is for the correct support of `and` and `or` functions of arbitrary arity - * - in this case `n - 1` elements are added (where `n` is the number of arguments). - */ - if (i != 0 || element.function == RPNElement::FUNCTION_NOT) - rpn.emplace_back(element); - } - - return; - } - } - - if (!tryParseAtomFromAST(node, context, block_with_constants, element)) - { - element.function = RPNElement::FUNCTION_UNKNOWN; - } - - rpn.emplace_back(std::move(element)); -} - /** The key functional expression constraint may be inferred from a plain column in the expression. * For example, if the key contains `toStartOfHour(Timestamp)` and query contains `WHERE Timestamp >= now()`, * it can be assumed that if `toStartOfHour()` is monotonic on [now(), inf), the `toStartOfHour(Timestamp) >= toStartOfHour(now())` @@ -1251,7 +913,7 @@ bool KeyCondition::transformConstantWithValidFunctions( } bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( - const Tree & node, + const RPNBuilderTreeNode & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, @@ -1259,7 +921,7 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( { String expr_name = node.getColumnName(); - if (array_joined_columns.contains(expr_name)) + if (array_joined_column_names.contains(expr_name)) return false; if (!key_subexpr_names.contains(expr_name)) @@ -1286,11 +948,15 @@ bool KeyCondition::canConstantBeWrappedByMonotonicFunctions( /// Looking for possible transformation of `column = constant` into `partition_expr = function(constant)` bool KeyCondition::canConstantBeWrappedByFunctions( - const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type) + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_column_type, + Field & out_value, + DataTypePtr & out_type) { String expr_name = node.getColumnName(); - if (array_joined_columns.contains(expr_name)) + if (array_joined_column_names.contains(expr_name)) return false; if (!key_subexpr_names.contains(expr_name)) @@ -1304,7 +970,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( /// The case `f(modulo(...))` for totally monotonic `f ` is considered to be rare. /// /// Note: for negative values, we can filter more partitions then needed. - expr_name = node.getColumnNameLegacy(); + expr_name = node.getColumnNameWithModuloLegacy(); if (!key_subexpr_names.contains(expr_name)) return false; @@ -1321,8 +987,7 @@ bool KeyCondition::canConstantBeWrappedByFunctions( } bool KeyCondition::tryPrepareSetIndex( - const FunctionTree & func, - ContextPtr context, + const RPNBuilderFunctionTreeNode & func, RPNElement & out, size_t & out_key_column_num) { @@ -1332,13 +997,12 @@ bool KeyCondition::tryPrepareSetIndex( std::vector indexes_mapping; DataTypes data_types; - auto get_key_tuple_position_mapping = [&](const Tree & node, size_t tuple_index) + auto get_key_tuple_position_mapping = [&](const RPNBuilderTreeNode & node, size_t tuple_index) { MergeTreeSetIndex::KeyTuplePositionMapping index_mapping; index_mapping.tuple_index = tuple_index; DataTypePtr data_type; - if (isKeyPossiblyWrappedByMonotonicFunctions( - node, context, index_mapping.key_index, data_type, index_mapping.functions)) + if (isKeyPossiblyWrappedByMonotonicFunctions(node, index_mapping.key_index, data_type, index_mapping.functions)) { indexes_mapping.push_back(index_mapping); data_types.push_back(data_type); @@ -1352,25 +1016,29 @@ bool KeyCondition::tryPrepareSetIndex( { /// Note: in case of ActionsDAG, tuple may be a constant. /// In this case, there is no keys in tuple. So, we don't have to check it. - auto left_arg_tuple = left_arg.asFunction(); + auto left_arg_tuple = left_arg.toFunctionNode(); if (left_arg_tuple.getFunctionName() == "tuple") { - left_args_count = left_arg_tuple.numArguments(); + left_args_count = left_arg_tuple.getArgumentsSize(); for (size_t i = 0; i < left_args_count; ++i) get_key_tuple_position_mapping(left_arg_tuple.getArgumentAt(i), i); } else + { get_key_tuple_position_mapping(left_arg, 0); + } } else + { get_key_tuple_position_mapping(left_arg, 0); + } if (indexes_mapping.empty()) return false; const auto right_arg = func.getArgumentAt(1); - auto prepared_set = right_arg.tryGetPreparedSet(prepared_sets, indexes_mapping, data_types); + auto prepared_set = right_arg.tryGetPreparedSet(indexes_mapping, data_types); if (!prepared_set) return false; @@ -1462,13 +1130,12 @@ private: bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( - const Tree & node, - ContextPtr context, + const RPNBuilderTreeNode & node, size_t & out_key_column_num, DataTypePtr & out_key_res_column_type, MonotonicFunctionsChain & out_functions_chain) { - std::vector chain_not_tested_for_monotonicity; + std::vector chain_not_tested_for_monotonicity; DataTypePtr key_column_type; if (!isKeyPossiblyWrappedByMonotonicFunctionsImpl(node, out_key_column_num, key_column_type, chain_not_tested_for_monotonicity)) @@ -1477,17 +1144,17 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( for (auto it = chain_not_tested_for_monotonicity.rbegin(); it != chain_not_tested_for_monotonicity.rend(); ++it) { auto function = *it; - auto func_builder = FunctionFactory::instance().tryGet(function.getFunctionName(), context); + auto func_builder = FunctionFactory::instance().tryGet(function.getFunctionName(), node.getTreeContext().getQueryContext()); if (!func_builder) return false; ColumnsWithTypeAndName arguments; ColumnWithTypeAndName const_arg; FunctionWithOptionalConstArg::Kind kind = FunctionWithOptionalConstArg::Kind::NO_CONST; - if (function.numArguments() == 2) + if (function.getArgumentsSize() == 2) { if (function.getArgumentAt(0).isConstant()) { - const_arg = function.getArgumentAt(0).getConstant(); + const_arg = function.getArgumentAt(0).getConstantColumn(); arguments.push_back(const_arg); arguments.push_back({ nullptr, key_column_type, "" }); kind = FunctionWithOptionalConstArg::Kind::LEFT_CONST; @@ -1495,7 +1162,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( else if (function.getArgumentAt(1).isConstant()) { arguments.push_back({ nullptr, key_column_type, "" }); - const_arg = function.getArgumentAt(1).getConstant(); + const_arg = function.getArgumentAt(1).getConstantColumn(); arguments.push_back(const_arg); kind = FunctionWithOptionalConstArg::Kind::RIGHT_CONST; } @@ -1521,10 +1188,10 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctions( } bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( - const Tree & node, + const RPNBuilderTreeNode & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, - std::vector & out_functions_chain) + std::vector & out_functions_chain) { /** By itself, the key column can be a functional expression. for example, `intHash32(UserID)`. * Therefore, use the full name of the expression for search. @@ -1534,7 +1201,7 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( // Key columns should use canonical names for index analysis String name = node.getColumnName(); - if (array_joined_columns.contains(name)) + if (array_joined_column_names.contains(name)) return false; auto it = key_columns.find(name); @@ -1547,37 +1214,39 @@ bool KeyCondition::isKeyPossiblyWrappedByMonotonicFunctionsImpl( if (node.isFunction()) { - auto func = node.asFunction(); + auto function_node = node.toFunctionNode(); - size_t num_args = func.numArguments(); - if (num_args > 2 || num_args == 0) + size_t arguments_size = function_node.getArgumentsSize(); + if (arguments_size > 2 || arguments_size == 0) return false; - out_functions_chain.push_back(func); - bool ret = false; - if (num_args == 2) + out_functions_chain.push_back(function_node); + + bool result = false; + if (arguments_size == 2) { - if (func.getArgumentAt(0).isConstant()) + if (function_node.getArgumentAt(0).isConstant()) { - ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain); + result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(1), out_key_column_num, out_key_column_type, out_functions_chain); } - else if (func.getArgumentAt(1).isConstant()) + else if (function_node.getArgumentAt(1).isConstant()) { - ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); + result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); } } else { - ret = isKeyPossiblyWrappedByMonotonicFunctionsImpl(func.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); + result = isKeyPossiblyWrappedByMonotonicFunctionsImpl(function_node.getArgumentAt(0), out_key_column_num, out_key_column_type, out_functions_chain); } - return ret; + + return result; } return false; } -static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const KeyCondition::Tree & node) +static void castValueToType(const DataTypePtr & desired_type, Field & src_value, const DataTypePtr & src_type, const String & node_column_name) { try { @@ -1587,13 +1256,13 @@ static void castValueToType(const DataTypePtr & desired_type, Field & src_value, { throw Exception("Key expression contains comparison between inconvertible types: " + desired_type->getName() + " and " + src_type->getName() + - " inside " + node.getColumnName(), + " inside " + node_column_name, ErrorCodes::BAD_TYPE_OF_FIELD); } } -bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out) +bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) { /** Functions < > = != <= >= in `notIn` isNull isNotNull, where one argument is a constant, and the other is one of columns of key, * or itself, wrapped in a chain of possibly-monotonic functions, @@ -1603,8 +1272,8 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl DataTypePtr const_type; if (node.isFunction()) { - auto func = node.asFunction(); - size_t num_args = func.numArguments(); + auto func = node.toFunctionNode(); + size_t num_args = func.getArgumentsSize(); DataTypePtr key_expr_type; /// Type of expression containing key column size_t key_column_num = -1; /// Number of a key column (inside key_column_names array) @@ -1616,7 +1285,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl if (num_args == 1) { - if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain))) + if (!(isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, chain))) return false; if (key_column_num == static_cast(-1)) @@ -1647,7 +1316,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl if (functionIsInOrGlobalInOperator(func_name)) { - if (tryPrepareSetIndex(func, context, out, key_column_num)) + if (tryPrepareSetIndex(func, out, key_column_num)) { key_arg_pos = 0; is_set_const = true; @@ -1655,7 +1324,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl else return false; } - else if (func.getArgumentAt(1).tryGetConstant(block_with_constants, const_value, const_type)) + else if (func.getArgumentAt(1).tryGetConstant(const_value, const_type)) { /// If the const operand is null, the atom will be always false if (const_value.isNull()) @@ -1664,7 +1333,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl return true; } - if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), context, key_column_num, key_expr_type, chain)) + if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(0), key_column_num, key_expr_type, chain)) { key_arg_pos = 0; } @@ -1685,7 +1354,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl else return false; } - else if (func.getArgumentAt(0).tryGetConstant(block_with_constants, const_value, const_type)) + else if (func.getArgumentAt(0).tryGetConstant(const_value, const_type)) { /// If the const operand is null, the atom will be always false if (const_value.isNull()) @@ -1694,7 +1363,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl return true; } - if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), context, key_column_num, key_expr_type, chain)) + if (isKeyPossiblyWrappedByMonotonicFunctions(func.getArgumentAt(1), key_column_num, key_expr_type, chain)) { key_arg_pos = 1; } @@ -1774,7 +1443,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl if (!const_type->equals(*common_type)) { - castValueToType(common_type, const_value, const_type, node); + castValueToType(common_type, const_value, const_type, node.getColumnName()); // Need to set is_constant_transformed unless we're doing exact conversion if (!key_expr_type_not_null->equals(*common_type)) @@ -1819,7 +1488,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl return atom_it->second(out, const_value); } - else if (node.tryGetConstant(block_with_constants, const_value, const_type)) + else if (node.tryGetConstant(const_value, const_type)) { /// For cases where it says, for example, `WHERE 0 AND something` @@ -1842,32 +1511,6 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl return false; } -bool KeyCondition::tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out) -{ - /// Functions AND, OR, NOT. - /// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses - /// (or, the same thing - calling the function `and` from one argument). - - if (func.getFunctionName() == "not") - { - if (func.numArguments() != 1) - return false; - - out.function = RPNElement::FUNCTION_NOT; - } - else - { - if (func.getFunctionName() == "and" || func.getFunctionName() == "indexHint") - out.function = RPNElement::FUNCTION_AND; - else if (func.getFunctionName() == "or") - out.function = RPNElement::FUNCTION_OR; - else - return false; - } - - return true; -} - String KeyCondition::toString() const { String res; diff --git a/src/Storages/MergeTree/KeyCondition.h b/src/Storages/MergeTree/KeyCondition.h index d00a25a1077..fe1bffa9305 100644 --- a/src/Storages/MergeTree/KeyCondition.h +++ b/src/Storages/MergeTree/KeyCondition.h @@ -2,11 +2,16 @@ #include -#include #include -#include -#include +#include + +#include +#include +#include + +#include +#include namespace DB { @@ -205,45 +210,37 @@ public: class KeyCondition { public: - /// Does not take into account the SAMPLE section. all_columns - the set of all columns of the table. + /// Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters KeyCondition( const ASTPtr & query, const ASTs & additional_filter_asts, - TreeRewriterResultPtr syntax_analyzer_result, + Block block_with_constants, PreparedSetsPtr prepared_sets_, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr, + NameSet array_joined_column_names, bool single_point_ = false, bool strict_ = false); + /** Construct key condition from AST SELECT query WHERE, PREWHERE and additional filters. + * Select query, additional filters, prepared sets are initialized using query info. + */ KeyCondition( const SelectQueryInfo & query_info, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr_, bool single_point_ = false, - bool strict_ = false) - : KeyCondition( - query_info.query, - query_info.filter_asts, - query_info.syntax_analyzer_result, - query_info.prepared_sets, - context, - key_column_names, - key_expr_, - single_point_, - strict_) - { - } + bool strict_ = false); + /// Construct key condition from ActionsDAG nodes KeyCondition( ActionDAGNodes dag_nodes, - TreeRewriterResultPtr syntax_analyzer_result, - PreparedSetsPtr prepared_sets_, ContextPtr context, const Names & key_column_names, const ExpressionActionsPtr & key_expr, + NameSet array_joined_column_names, bool single_point_ = false, bool strict_ = false); @@ -275,6 +272,7 @@ public: /// Checks that the index can not be used /// FUNCTION_UNKNOWN will be AND'ed (if any). bool alwaysUnknownOrTrue() const; + /// Checks that the index can not be used /// Does not allow any FUNCTION_UNKNOWN (will instantly return true). bool anyUnknownOrAlwaysTrue() const; @@ -313,10 +311,18 @@ public: * Returns false, if expression isn't constant. */ static bool getConstant( - const ASTPtr & expr, Block & block_with_constants, Field & out_value, DataTypePtr & out_type); + const ASTPtr & expr, + Block & block_with_constants, + Field & out_value, + DataTypePtr & out_type); + /** Calculate expressions, that depend only on constants. + * For index to work when something like "WHERE Date = toDate(now())" is written. + */ static Block getBlockWithConstants( - const ASTPtr & query, const TreeRewriterResultPtr & syntax_analyzer_result, ContextPtr context); + const ASTPtr & query, + const TreeRewriterResultPtr & syntax_analyzer_result, + ContextPtr context); static std::optional applyMonotonicFunctionsChainToRange( Range key_range, @@ -373,14 +379,11 @@ private: using RPN = std::vector; using ColumnIndices = std::map; - using AtomMap = std::unordered_map; public: + using AtomMap = std::unordered_map; static const AtomMap atom_map; - class Tree; - class FunctionTree; - private: BoolMask checkInRange( size_t used_key_size, @@ -390,9 +393,7 @@ private: bool right_bounded, BoolMask initial_mask) const; - void traverseAST(const Tree & node, ContextPtr context, Block & block_with_constants); - bool tryParseAtomFromAST(const Tree & node, ContextPtr context, Block & block_with_constants, RPNElement & out); - static bool tryParseLogicalOperatorFromAST(const FunctionTree & func, RPNElement & out); + bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out); /** Is node the key column * or expression in which column of key is wrapped by chain of functions, @@ -401,17 +402,16 @@ private: * and fills chain of possibly-monotonic functions. */ bool isKeyPossiblyWrappedByMonotonicFunctions( - const Tree & node, - ContextPtr context, + const RPNBuilderTreeNode & node, size_t & out_key_column_num, DataTypePtr & out_key_res_column_type, MonotonicFunctionsChain & out_functions_chain); bool isKeyPossiblyWrappedByMonotonicFunctionsImpl( - const Tree & node, + const RPNBuilderTreeNode & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, - std::vector & out_functions_chain); + std::vector & out_functions_chain); bool transformConstantWithValidFunctions( const String & expr_name, @@ -422,21 +422,24 @@ private: std::function always_monotonic) const; bool canConstantBeWrappedByMonotonicFunctions( - const Tree & node, + const RPNBuilderTreeNode & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); bool canConstantBeWrappedByFunctions( - const Tree & node, size_t & out_key_column_num, DataTypePtr & out_key_column_type, Field & out_value, DataTypePtr & out_type); + const RPNBuilderTreeNode & node, + size_t & out_key_column_num, + DataTypePtr & out_key_column_type, + Field & out_value, + DataTypePtr & out_type); /// If it's possible to make an RPNElement /// that will filter values (possibly tuples) by the content of 'prepared_set', /// do it and return true. bool tryPrepareSetIndex( - const FunctionTree & func, - ContextPtr context, + const RPNBuilderFunctionTreeNode & func, RPNElement & out, size_t & out_key_column_num); @@ -472,11 +475,12 @@ private: /// All intermediate columns are used to calculate key_expr. const NameSet key_subexpr_names; - NameSet array_joined_columns; - PreparedSetsPtr prepared_sets; + /// Array joined column names + NameSet array_joined_column_names; // If true, always allow key_expr to be wrapped by function bool single_point; + // If true, do not use always_monotonic information to transform constants bool strict; }; diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 3dd0568107e..db1b9762668 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -6,11 +6,13 @@ #include #include #include +#include #include #include #include #include #include +#include #include #include #include @@ -28,19 +30,7 @@ namespace ErrorCodes namespace { -PreparedSetKey getPreparedSetKey(const ASTPtr & node, const DataTypePtr & data_type) -{ - /// If the data type is tuple, let's try unbox once - if (node->as() || node->as()) - return PreparedSetKey::forSubquery(*node); - - if (const auto * date_type_tuple = typeid_cast(&*data_type)) - return PreparedSetKey::forLiteral(*node, date_type_tuple->getElements()); - - return PreparedSetKey::forLiteral(*node, DataTypes(1, data_type)); -} - -ColumnWithTypeAndName getPreparedSetInfo(const SetPtr & prepared_set) +ColumnWithTypeAndName getPreparedSetInfo(const ConstSetPtr & prepared_set) { if (prepared_set->getDataTypes().size() == 1) return {prepared_set->getSetElements()[0], prepared_set->getElementsTypes()[0], "dummy"}; @@ -110,8 +100,22 @@ MergeTreeIndexConditionBloomFilter::MergeTreeIndexConditionBloomFilter( const SelectQueryInfo & info_, ContextPtr context_, const Block & header_, size_t hash_functions_) : WithContext(context_), header(header_), query_info(info_), hash_functions(hash_functions_) { - auto atom_from_ast = [this](auto & node, auto, auto & constants, auto & out) { return traverseAtomAST(node, constants, out); }; - rpn = std::move(RPNBuilder(info_, getContext(), atom_from_ast).extractRPN()); + ASTPtr filter_node = buildFilterNode(query_info.query); + + if (!filter_node) + { + rpn.push_back(RPNElement::FUNCTION_UNKNOWN); + return; + } + + auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context_); + RPNBuilder builder( + filter_node, + context_, + std::move(block_with_constants), + query_info.prepared_sets, + [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); + rpn = std::move(builder).extractRPN(); } bool MergeTreeIndexConditionBloomFilter::alwaysUnknownOrTrue() const @@ -235,12 +239,13 @@ bool MergeTreeIndexConditionBloomFilter::mayBeTrueOnGranule(const MergeTreeIndex return rpn_stack[0].can_be_true; } -bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out) +bool MergeTreeIndexConditionBloomFilter::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) { { Field const_value; DataTypePtr const_type; - if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type)) + + if (node.tryGetConstant(const_value, const_type)) { if (const_value.getType() == Field::Types::UInt64) { @@ -262,56 +267,57 @@ bool MergeTreeIndexConditionBloomFilter::traverseAtomAST(const ASTPtr & node, Bl } } - return traverseFunction(node, block_with_constants, out, nullptr); + return traverseFunction(node, out, nullptr /*parent*/); } -bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent) +bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent) { bool maybe_useful = false; - if (const auto * function = node->as()) + if (node.isFunction()) { - if (!function->arguments) + const auto function = node.toFunctionNode(); + auto arguments_size = function.getArgumentsSize(); + auto function_name = function.getFunctionName(); + + if (arguments_size != 2) return false; - const ASTs & arguments = function->arguments->children; - for (const auto & arg : arguments) + auto lhs_argument = function.getArgumentAt(0); + auto rhs_argument = function.getArgumentAt(1); + + auto lhs_argument_column_name = lhs_argument.getColumnName(); + + if (functionIsInOrGlobalInOperator(function_name)) { - if (traverseFunction(arg, block_with_constants, out, node)) - maybe_useful = true; - } + ConstSetPtr prepared_set = rhs_argument.tryGetPreparedSet(); - if (arguments.size() != 2) - return false; - - if (functionIsInOrGlobalInOperator(function->name)) - { - auto prepared_set = getPreparedSet(arguments[1]); - - if (prepared_set) + if (prepared_set && prepared_set->hasExplicitSetElements()) { - if (traverseASTIn(function->name, arguments[0], prepared_set, out)) + const auto prepared_info = getPreparedSetInfo(prepared_set); + if (traverseTreeIn(function_name, lhs_argument, prepared_set, prepared_info.type, prepared_info.column, out)) maybe_useful = true; } } - else if (function->name == "equals" || - function->name == "notEquals" || - function->name == "has" || - function->name == "mapContains" || - function->name == "indexOf" || - function->name == "hasAny" || - function->name == "hasAll") + else if (function_name == "equals" || + function_name == "notEquals" || + function_name == "has" || + function_name == "mapContains" || + function_name == "indexOf" || + function_name == "hasAny" || + function_name == "hasAll") { Field const_value; DataTypePtr const_type; - if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type)) + + if (rhs_argument.tryGetConstant(const_value, const_type)) { - if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out, parent)) + if (traverseTreeEquals(function_name, lhs_argument, const_type, const_value, out, parent)) maybe_useful = true; } - else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type)) + else if (lhs_argument.tryGetConstant(const_value, const_type)) { - if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out, parent)) + if (traverseTreeEquals(function_name, rhs_argument, const_type, const_value, out, parent)) maybe_useful = true; } } @@ -320,28 +326,20 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const ASTPtr & node, B return maybe_useful; } -bool MergeTreeIndexConditionBloomFilter::traverseASTIn( +bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( const String & function_name, - const ASTPtr & key_ast, - const SetPtr & prepared_set, - RPNElement & out) -{ - const auto prepared_info = getPreparedSetInfo(prepared_set); - return traverseASTIn(function_name, key_ast, prepared_set, prepared_info.type, prepared_info.column, out); -} - -bool MergeTreeIndexConditionBloomFilter::traverseASTIn( - const String & function_name, - const ASTPtr & key_ast, - const SetPtr & prepared_set, + const RPNBuilderTreeNode & key_node, + const ConstSetPtr & prepared_set, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out) { - if (header.has(key_ast->getColumnName())) + auto key_node_column_name = key_node.getColumnName(); + + if (header.has(key_node_column_name)) { size_t row_size = column->size(); - size_t position = header.getPositionByName(key_ast->getColumnName()); + size_t position = header.getPositionByName(key_node_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; const auto & converted_column = castColumn(ColumnWithTypeAndName{column, type, ""}, index_type); out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithColumn(index_type, converted_column, 0, row_size))); @@ -355,30 +353,33 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn( return true; } - if (const auto * function = key_ast->as()) + if (key_node.isFunction()) { + auto key_node_function = key_node.toFunctionNode(); + auto key_node_function_name = key_node_function.getFunctionName(); + size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); + WhichDataType which(type); - if (which.isTuple() && function->name == "tuple") + if (which.isTuple() && function_name == "tuple") { const auto & tuple_column = typeid_cast(column.get()); const auto & tuple_data_type = typeid_cast(type.get()); - const ASTs & arguments = typeid_cast(*function->arguments).children; - if (tuple_data_type->getElements().size() != arguments.size() || tuple_column->getColumns().size() != arguments.size()) + if (tuple_data_type->getElements().size() != key_node_function_arguments_size || tuple_column->getColumns().size() != key_node_function_arguments_size) throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); bool match_with_subtype = false; const auto & sub_columns = tuple_column->getColumns(); const auto & sub_data_types = tuple_data_type->getElements(); - for (size_t index = 0; index < arguments.size(); ++index) - match_with_subtype |= traverseASTIn(function_name, arguments[index], nullptr, sub_data_types[index], sub_columns[index], out); + for (size_t index = 0; index < key_node_function_arguments_size; ++index) + match_with_subtype |= traverseTreeIn(function_name, key_node_function.getArgumentAt(index), nullptr, sub_data_types[index], sub_columns[index], out); return match_with_subtype; } - if (function->name == "arrayElement") + if (key_node_function_name == "arrayElement") { /** Try to parse arrayElement for mapKeys index. * It is important to ignore keys like column_map['Key'] IN ('') because if key does not exists in map @@ -388,9 +389,6 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn( * that way we skip necessary granules where map key does not exists. */ - if (!prepared_set) - return false; - auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" }; ColumnsWithTypeAndName default_columns_with_type_to_check = {default_column_with_type_to_check}; @@ -400,11 +398,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn( if (set_contain_default_value) return false; - const auto * column_ast_identifier = function->arguments.get()->children[0].get()->as(); - if (!column_ast_identifier) - return false; - - const auto & col_name = column_ast_identifier->name(); + const auto & col_name = key_node_function.getArgumentAt(0).getColumnName(); auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); auto map_values_index_column_name = fmt::format("mapValues({})", col_name); @@ -412,16 +406,17 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn( { /// For mapKeys we serialize key argument with bloom filter - auto & argument = function->arguments.get()->children[1]; + auto first_argument = key_node_function.getArgumentAt(1); - if (const auto * literal = argument->as()) + Field constant_value; + DataTypePtr constant_type; + + if (first_argument.tryGetConstant(constant_value, constant_type)) { size_t position = header.getPositionByName(map_keys_index_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; - - auto element_key = literal->value; const DataTypePtr actual_type = BloomFilter::getPrimitiveType(index_type); - out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), element_key))); + out.predicate.emplace_back(std::make_pair(position, BloomFilterHash::hashWithField(actual_type.get(), constant_value))); } else { @@ -459,74 +454,97 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTIn( } -static bool indexOfCanUseBloomFilter(const ASTPtr & parent) +static bool indexOfCanUseBloomFilter(const RPNBuilderTreeNode * parent) { if (!parent) return true; + if (!parent->isFunction()) + return false; + + auto function = parent->toFunctionNode(); + auto function_name = function.getFunctionName(); + /// `parent` is a function where `indexOf` is located. /// Example: `indexOf(arr, x) = 1`, parent is a function named `equals`. - if (const auto * function = parent->as()) + if (function_name == "and") { - if (function->name == "and") + return true; + } + else if (function_name == "equals" /// notEquals is not applicable + || function_name == "greater" || function_name == "greaterOrEquals" + || function_name == "less" || function_name == "lessOrEquals") + { + size_t function_arguments_size = function.getArgumentsSize(); + if (function_arguments_size != 2) + return false; + + /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. + + /// We should return true when the corresponding expression implies that the array contains the element. + /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element + /// (at least at 11th position but it does not matter). + + bool reversed = false; + Field constant_value; + DataTypePtr constant_type; + + if (function.getArgumentAt(0).tryGetConstant(constant_value, constant_type)) { + reversed = true; + } + else if (function.getArgumentAt(1).tryGetConstant(constant_value, constant_type)) + { + } + else + { + return false; + } + + Field zero(0); + bool constant_equal_zero = applyVisitor(FieldVisitorAccurateEquals(), constant_value, zero); + + if (function_name == "equals" && !constant_equal_zero) + { + /// indexOf(...) = c, c != 0 return true; } - else if (function->name == "equals" /// notEquals is not applicable - || function->name == "greater" || function->name == "greaterOrEquals" - || function->name == "less" || function->name == "lessOrEquals") + else if (function_name == "notEquals" && constant_equal_zero) { - if (function->arguments->children.size() != 2) - return false; - - /// We don't allow constant expressions like `indexOf(arr, x) = 1 + 0` but it's negligible. - - /// We should return true when the corresponding expression implies that the array contains the element. - /// Example: when `indexOf(arr, x)` > 10 is written, it means that arr definitely should contain the element - /// (at least at 11th position but it does not matter). - - bool reversed = false; - const ASTLiteral * constant = nullptr; - - if (const ASTLiteral * left = function->arguments->children[0]->as()) - { - constant = left; - reversed = true; - } - else if (const ASTLiteral * right = function->arguments->children[1]->as()) - { - constant = right; - } - else - return false; - - Field zero(0); - return (function->name == "equals" /// indexOf(...) = c, c != 0 - && !applyVisitor(FieldVisitorAccurateEquals(), constant->value, zero)) - || (function->name == "notEquals" /// indexOf(...) != c, c = 0 - && applyVisitor(FieldVisitorAccurateEquals(), constant->value, zero)) - || (function->name == (reversed ? "less" : "greater") /// indexOf(...) > c, c >= 0 - && !applyVisitor(FieldVisitorAccurateLess(), constant->value, zero)) - || (function->name == (reversed ? "lessOrEquals" : "greaterOrEquals") /// indexOf(...) >= c, c > 0 - && applyVisitor(FieldVisitorAccurateLess(), zero, constant->value)); + /// indexOf(...) != c, c = 0 + return true; } + else if (function_name == (reversed ? "less" : "greater") && !applyVisitor(FieldVisitorAccurateLess(), constant_value, zero)) + { + /// indexOf(...) > c, c >= 0 + return true; + } + else if (function_name == (reversed ? "lessOrEquals" : "greaterOrEquals") && applyVisitor(FieldVisitorAccurateLess(), zero, constant_value)) + { + /// indexOf(...) >= c, c > 0 + return true; + } + + return false; } return false; } -bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( +bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( const String & function_name, - const ASTPtr & key_ast, + const RPNBuilderTreeNode & key_node, const DataTypePtr & value_type, const Field & value_field, RPNElement & out, - const ASTPtr & parent) + const RPNBuilderTreeNode * parent) { - if (header.has(key_ast->getColumnName())) + auto key_column_name = key_node.getColumnName(); + + if (header.has(key_column_name)) { - size_t position = header.getPositionByName(key_ast->getColumnName()); + size_t position = header.getPositionByName(key_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; const auto * array_type = typeid_cast(index_type.get()); @@ -602,13 +620,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( if (function_name == "mapContains" || function_name == "has") { - const auto * key_ast_identifier = key_ast.get()->as(); - if (!key_ast_identifier) - return false; - - const auto & col_name = key_ast_identifier->name(); - auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); - + auto map_keys_index_column_name = fmt::format("mapKeys({})", key_column_name); if (!header.has(map_keys_index_column_name)) return false; @@ -629,29 +641,32 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( return true; } - if (const auto * function = key_ast->as()) + if (key_node.isFunction()) { WhichDataType which(value_type); - if (which.isTuple() && function->name == "tuple") + auto key_node_function = key_node.toFunctionNode(); + auto key_node_function_name = key_node_function.getFunctionName(); + size_t key_node_function_arguments_size = key_node_function.getArgumentsSize(); + + if (which.isTuple() && key_node_function_name == "tuple") { const Tuple & tuple = value_field.get(); const auto * value_tuple_data_type = typeid_cast(value_type.get()); - const ASTs & arguments = typeid_cast(*function->arguments).children; - if (tuple.size() != arguments.size()) + if (tuple.size() != key_node_function_arguments_size) throw Exception("Illegal types of arguments of function " + function_name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); bool match_with_subtype = false; const DataTypes & subtypes = value_tuple_data_type->getElements(); for (size_t index = 0; index < tuple.size(); ++index) - match_with_subtype |= traverseASTEquals(function_name, arguments[index], subtypes[index], tuple[index], out, key_ast); + match_with_subtype |= traverseTreeEquals(function_name, key_node_function.getArgumentAt(index), subtypes[index], tuple[index], out, &key_node); return match_with_subtype; } - if (function->name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) + if (key_node_function_name == "arrayElement" && (function_name == "equals" || function_name == "notEquals")) { /** Try to parse arrayElement for mapKeys index. * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map @@ -663,27 +678,22 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( if (value_field == value_type->getDefault()) return false; - const auto * column_ast_identifier = function->arguments.get()->children[0].get()->as(); - if (!column_ast_identifier) - return false; - - const auto & col_name = column_ast_identifier->name(); + auto first_argument = key_node_function.getArgumentAt(0); + const auto col_name = first_argument.getColumnName(); auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); auto map_values_index_column_name = fmt::format("mapValues({})", col_name); size_t position = 0; Field const_value = value_field; + DataTypePtr const_type; if (header.has(map_keys_index_column_name)) { position = header.getPositionByName(map_keys_index_column_name); + auto second_argument = key_node_function.getArgumentAt(1); - auto & argument = function->arguments.get()->children[1]; - - if (const auto * literal = argument->as()) - const_value = literal->value; - else + if (!second_argument.tryGetConstant(const_value, const_type)) return false; } else if (header.has(map_values_index_column_name)) @@ -708,23 +718,4 @@ bool MergeTreeIndexConditionBloomFilter::traverseASTEquals( return false; } -SetPtr MergeTreeIndexConditionBloomFilter::getPreparedSet(const ASTPtr & node) -{ - if (header.has(node->getColumnName())) - { - const auto & column_and_type = header.getByName(node->getColumnName()); - auto set_key = getPreparedSetKey(node, column_and_type.type); - if (auto prepared_set = query_info.prepared_sets->get(set_key)) - return prepared_set; - } - else - { - for (const auto & set : query_info.prepared_sets->getByTreeHash(node->getTreeHash())) - if (set->hasExplicitSetElements()) - return set; - } - - return DB::SetPtr(); -} - } diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h index 27fd701c67b..5d7ea371a83 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.h @@ -62,35 +62,27 @@ private: const size_t hash_functions; std::vector rpn; - SetPtr getPreparedSet(const ASTPtr & node); - bool mayBeTrueOnGranule(const MergeTreeIndexGranuleBloomFilter * granule) const; - bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); + bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out); - bool traverseFunction(const ASTPtr & node, Block & block_with_constants, RPNElement & out, const ASTPtr & parent); + bool traverseFunction(const RPNBuilderTreeNode & node, RPNElement & out, const RPNBuilderTreeNode * parent); - bool traverseASTIn( + bool traverseTreeIn( const String & function_name, - const ASTPtr & key_ast, - const SetPtr & prepared_set, - RPNElement & out); - - bool traverseASTIn( - const String & function_name, - const ASTPtr & key_ast, - const SetPtr & prepared_set, + const RPNBuilderTreeNode & key_node, + const ConstSetPtr & prepared_set, const DataTypePtr & type, const ColumnPtr & column, RPNElement & out); - bool traverseASTEquals( + bool traverseTreeEquals( const String & function_name, - const ASTPtr & key_ast, + const RPNBuilderTreeNode & key_node, const DataTypePtr & value_type, const Field & value_field, RPNElement & out, - const ASTPtr & parent); + const RPNBuilderTreeNode * parent); }; } diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index ff924290783..9074485a020 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -11,9 +11,11 @@ #include #include #include +#include #include #include #include +#include #include #include @@ -148,13 +150,22 @@ MergeTreeConditionFullText::MergeTreeConditionFullText( , token_extractor(token_extactor_) , prepared_sets(query_info.prepared_sets) { - rpn = std::move( - RPNBuilder( - query_info, context, - [this] (const ASTPtr & node, ContextPtr /* context */, Block & block_with_constants, RPNElement & out) -> bool - { - return this->traverseAtomAST(node, block_with_constants, out); - }).extractRPN()); + ASTPtr filter_node = buildFilterNode(query_info.query); + + if (!filter_node) + { + rpn.push_back(RPNElement::FUNCTION_UNKNOWN); + return; + } + + auto block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, context); + RPNBuilder builder( + filter_node, + context, + std::move(block_with_constants), + query_info.prepared_sets, + [&](const RPNBuilderTreeNode & node, RPNElement & out) { return extractAtomFromTree(node, out); }); + rpn = std::move(builder).extractRPN(); } bool MergeTreeConditionFullText::alwaysUnknownOrTrue() const @@ -306,13 +317,13 @@ bool MergeTreeConditionFullText::getKey(const std::string & key_column_name, siz return true; } -bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out) +bool MergeTreeConditionFullText::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out) { { Field const_value; DataTypePtr const_type; - if (KeyCondition::getConstant(node, block_with_constants, const_value, const_type)) + if (node.tryGetConstant(const_value, const_type)) { /// Check constant like in KeyCondition if (const_value.getType() == Field::Types::UInt64 @@ -329,53 +340,56 @@ bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & bl } } - if (const auto * function = node->as()) + if (node.isFunction()) { - if (!function->arguments) + auto function_node = node.toFunctionNode(); + auto function_name = function_node.getFunctionName(); + + size_t arguments_size = function_node.getArgumentsSize(); + if (arguments_size != 2) return false; - const ASTs & arguments = function->arguments->children; + auto left_argument = function_node.getArgumentAt(0); + auto right_argument = function_node.getArgumentAt(1); - if (arguments.size() != 2) - return false; - - if (functionIsInOrGlobalInOperator(function->name)) + if (functionIsInOrGlobalInOperator(function_name)) { - if (tryPrepareSetBloomFilter(arguments, out)) + if (tryPrepareSetBloomFilter(left_argument, right_argument, out)) { - if (function->name == "notIn") + if (function_name == "notIn") { out.function = RPNElement::FUNCTION_NOT_IN; return true; } - else if (function->name == "in") + else if (function_name == "in") { out.function = RPNElement::FUNCTION_IN; return true; } } } - else if (function->name == "equals" || - function->name == "notEquals" || - function->name == "has" || - function->name == "mapContains" || - function->name == "like" || - function->name == "notLike" || - function->name == "hasToken" || - function->name == "startsWith" || - function->name == "endsWith" || - function->name == "multiSearchAny") + else if (function_name == "equals" || + function_name == "notEquals" || + function_name == "has" || + function_name == "mapContains" || + function_name == "like" || + function_name == "notLike" || + function_name == "hasToken" || + function_name == "startsWith" || + function_name == "endsWith" || + function_name == "multiSearchAny") { Field const_value; DataTypePtr const_type; - if (KeyCondition::getConstant(arguments[1], block_with_constants, const_value, const_type)) + + if (right_argument.tryGetConstant(const_value, const_type)) { - if (traverseASTEquals(function->name, arguments[0], const_type, const_value, out)) + if (traverseTreeEquals(function_name, left_argument, const_type, const_value, out)) return true; } - else if (KeyCondition::getConstant(arguments[0], block_with_constants, const_value, const_type) && (function->name == "equals" || function->name == "notEquals")) + else if (left_argument.tryGetConstant(const_value, const_type) && (function_name == "equals" || function_name == "notEquals")) { - if (traverseASTEquals(function->name, arguments[1], const_type, const_value, out)) + if (traverseTreeEquals(function_name, right_argument, const_type, const_value, out)) return true; } } @@ -384,9 +398,9 @@ bool MergeTreeConditionFullText::traverseAtomAST(const ASTPtr & node, Block & bl return false; } -bool MergeTreeConditionFullText::traverseASTEquals( +bool MergeTreeConditionFullText::traverseTreeEquals( const String & function_name, - const ASTPtr & key_ast, + const RPNBuilderTreeNode & key_node, const DataTypePtr & value_type, const Field & value_field, RPNElement & out) @@ -397,13 +411,17 @@ bool MergeTreeConditionFullText::traverseASTEquals( Field const_value = value_field; + auto column_name = key_node.getColumnName(); size_t key_column_num = 0; - bool key_exists = getKey(key_ast->getColumnName(), key_column_num); - bool map_key_exists = getKey(fmt::format("mapKeys({})", key_ast->getColumnName()), key_column_num); + bool key_exists = getKey(column_name, key_column_num); + bool map_key_exists = getKey(fmt::format("mapKeys({})", column_name), key_column_num); - if (const auto * function = key_ast->as()) + if (key_node.isFunction()) { - if (function->name == "arrayElement") + auto key_function_node = key_node.toFunctionNode(); + auto key_function_node_function_name = key_function_node.getFunctionName(); + + if (key_function_node_function_name == "arrayElement") { /** Try to parse arrayElement for mapKeys index. * It is important to ignore keys like column_map['Key'] = '' because if key does not exists in map @@ -415,11 +433,8 @@ bool MergeTreeConditionFullText::traverseASTEquals( if (value_field == value_type->getDefault()) return false; - const auto * column_ast_identifier = function->arguments.get()->children[0].get()->as(); - if (!column_ast_identifier) - return false; - - const auto & map_column_name = column_ast_identifier->name(); + auto first_argument = key_function_node.getArgumentAt(0); + const auto & map_column_name = first_argument.getColumnName(); size_t map_keys_key_column_num = 0; auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name); @@ -431,12 +446,11 @@ bool MergeTreeConditionFullText::traverseASTEquals( if (map_keys_exists) { - auto & argument = function->arguments.get()->children[1]; + auto second_argument = key_function_node.getArgumentAt(1); + DataTypePtr const_type; - if (const auto * literal = argument->as()) + if (second_argument.tryGetConstant(const_value, const_type)) { - auto element_key = literal->value; - const_value = element_key; key_column_num = map_keys_key_column_num; key_exists = true; } @@ -567,33 +581,37 @@ bool MergeTreeConditionFullText::traverseASTEquals( } bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( - const ASTs & args, + const RPNBuilderTreeNode & left_argument, + const RPNBuilderTreeNode & right_argument, RPNElement & out) { - const ASTPtr & left_arg = args[0]; - const ASTPtr & right_arg = args[1]; - std::vector key_tuple_mapping; DataTypes data_types; - const auto * left_arg_tuple = typeid_cast(left_arg.get()); - if (left_arg_tuple && left_arg_tuple->name == "tuple") + if (left_argument.isFunction()) { - const auto & tuple_elements = left_arg_tuple->arguments->children; - for (size_t i = 0; i < tuple_elements.size(); ++i) + auto left_argument_function_node = left_argument.toFunctionNode(); + auto left_argument_function_node_name = left_argument_function_node.getFunctionName(); + + if (left_argument_function_node_name == "tuple") { - size_t key = 0; - if (getKey(tuple_elements[i]->getColumnName(), key)) + size_t left_argument_function_node_arguments_size = left_argument_function_node.getArgumentsSize(); + + for (size_t i = 0; i < left_argument_function_node_arguments_size; ++i) { - key_tuple_mapping.emplace_back(i, key); - data_types.push_back(index_data_types[key]); + size_t key = 0; + if (getKey(left_argument_function_node.getArgumentAt(i).getColumnName(), key)) + { + key_tuple_mapping.emplace_back(i, key); + data_types.push_back(index_data_types[key]); + } } } } else { size_t key = 0; - if (getKey(left_arg->getColumnName(), key)) + if (getKey(left_argument.getColumnName(), key)) { key_tuple_mapping.emplace_back(0, key); data_types.push_back(index_data_types[key]); @@ -603,19 +621,10 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( if (key_tuple_mapping.empty()) return false; - PreparedSetKey set_key; - if (typeid_cast(right_arg.get()) || typeid_cast(right_arg.get())) - set_key = PreparedSetKey::forSubquery(*right_arg); - else - set_key = PreparedSetKey::forLiteral(*right_arg, data_types); - - auto prepared_set = prepared_sets->get(set_key); + auto prepared_set = right_argument.tryGetPreparedSet(data_types); if (!prepared_set) return false; - if (!prepared_set->hasExplicitSetElements()) - return false; - for (const auto & data_type : prepared_set->getDataTypes()) if (data_type->getTypeId() != TypeIndex::String && data_type->getTypeId() != TypeIndex::FixedString) return false; diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.h b/src/Storages/MergeTree/MergeTreeIndexFullText.h index bb4f52a463e..ad487816aef 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.h +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.h @@ -122,17 +122,17 @@ private: using RPN = std::vector; - bool traverseAtomAST(const ASTPtr & node, Block & block_with_constants, RPNElement & out); + bool extractAtomFromTree(const RPNBuilderTreeNode & node, RPNElement & out); - bool traverseASTEquals( + bool traverseTreeEquals( const String & function_name, - const ASTPtr & key_ast, + const RPNBuilderTreeNode & key_node, const DataTypePtr & value_type, const Field & value_field, RPNElement & out); bool getKey(const std::string & key_column_name, size_t & key_column_num); - bool tryPrepareSetBloomFilter(const ASTs & args, RPNElement & out); + bool tryPrepareSetBloomFilter(const RPNBuilderTreeNode & left_argument, const RPNBuilderTreeNode & right_argument, RPNElement & out); static bool createFunctionEqualsCondition( RPNElement & out, const Field & value, const BloomFilterParameters & params, TokenExtractorPtr token_extractor); diff --git a/src/Storages/MergeTree/MergeTreeIndexUtils.cpp b/src/Storages/MergeTree/MergeTreeIndexUtils.cpp new file mode 100644 index 00000000000..652f0c853d4 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexUtils.cpp @@ -0,0 +1,47 @@ +#include + +#include +#include +#include + +namespace DB +{ + +ASTPtr buildFilterNode(const ASTPtr & select_query, ASTs additional_filters) +{ + auto & select_query_typed = select_query->as(); + + ASTs filters; + if (select_query_typed.where()) + filters.push_back(select_query_typed.where()); + + if (select_query_typed.prewhere()) + filters.push_back(select_query_typed.prewhere()); + + filters.insert(filters.end(), additional_filters.begin(), additional_filters.end()); + + if (filters.empty()) + return nullptr; + + ASTPtr filter_node; + + if (filters.size() == 1) + { + filter_node = filters.front(); + } + else + { + auto function = std::make_shared(); + + function->name = "and"; + function->arguments = std::make_shared(); + function->children.push_back(function->arguments); + function->arguments->children = std::move(filters); + + filter_node = std::move(function); + } + + return filter_node; +} + +} diff --git a/src/Storages/MergeTree/MergeTreeIndexUtils.h b/src/Storages/MergeTree/MergeTreeIndexUtils.h new file mode 100644 index 00000000000..23cd92f3d99 --- /dev/null +++ b/src/Storages/MergeTree/MergeTreeIndexUtils.h @@ -0,0 +1,13 @@ +#pragma once + +#include + +namespace DB +{ + +/** Build AST filter node for index analysis from WHERE and PREWHERE sections of select query and additional filters. + * If select query does not have WHERE or PREWHERE and additional filters are empty null is returned. + */ +ASTPtr buildFilterNode(const ASTPtr & select_query, ASTs additional_filters = {}); + +} diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp new file mode 100644 index 00000000000..362e02e8498 --- /dev/null +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -0,0 +1,380 @@ +#include + +#include + +#include +#include +#include +#include + +#include +#include + +#include +#include + +#include + +#include + + +namespace DB +{ + +namespace +{ + +void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & out, bool legacy = false) +{ + switch (node.type) + { + case (ActionsDAG::ActionType::INPUT): + writeString(node.result_name, out); + break; + case (ActionsDAG::ActionType::COLUMN): + { + /// If it was created from ASTLiteral, then result_name can be an alias. + /// We need to convert value back to string here. + if (const auto * column_const = typeid_cast(node.column.get())) + writeString(applyVisitor(FieldVisitorToString(), column_const->getField()), out); + /// It may be possible that column is ColumnSet + else + writeString(node.result_name, out); + break; + } + case (ActionsDAG::ActionType::ALIAS): + appendColumnNameWithoutAlias(*node.children.front(), out, legacy); + break; + case (ActionsDAG::ActionType::ARRAY_JOIN): + writeCString("arrayJoin(", out); + appendColumnNameWithoutAlias(*node.children.front(), out, legacy); + writeChar(')', out); + break; + case (ActionsDAG::ActionType::FUNCTION): + { + auto name = node.function_base->getName(); + if (legacy && name == "modulo") + writeCString("moduleLegacy", out); + else + writeString(name, out); + + writeChar('(', out); + bool first = true; + for (const auto * arg : node.children) + { + if (!first) + writeCString(", ", out); + first = false; + + appendColumnNameWithoutAlias(*arg, out, legacy); + } + writeChar(')', out); + } + } +} + +String getColumnNameWithoutAlias(const ActionsDAG::Node & node, bool legacy = false) +{ + WriteBufferFromOwnString out; + appendColumnNameWithoutAlias(node, out, legacy); + return std::move(out.str()); +} + +} + +RPNBuilderTreeContext::RPNBuilderTreeContext(ContextPtr query_context_) + : query_context(std::move(query_context_)) +{} + +RPNBuilderTreeContext::RPNBuilderTreeContext(ContextPtr query_context_, Block block_with_constants_, PreparedSetsPtr prepared_sets_) + : query_context(std::move(query_context_)) + , block_with_constants(std::move(block_with_constants_)) + , prepared_sets(std::move(prepared_sets_)) +{} + +RPNBuilderTreeNode::RPNBuilderTreeNode(const ActionsDAG::Node * dag_node_, RPNBuilderTreeContext & tree_context_) + : dag_node(dag_node_) + , tree_context(tree_context_) +{ + assert(dag_node); +} + +RPNBuilderTreeNode::RPNBuilderTreeNode(const IAST * ast_node_, RPNBuilderTreeContext & tree_context_) + : ast_node(ast_node_) + , tree_context(tree_context_) +{ + assert(ast_node); +} + +std::string RPNBuilderTreeNode::getColumnName() const +{ + if (ast_node) + return ast_node->getColumnNameWithoutAlias(); + else + return getColumnNameWithoutAlias(*dag_node); +} + +std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const +{ + if (ast_node) + { + auto adjusted_ast = ast_node->clone(); + KeyDescription::moduloToModuloLegacyRecursive(adjusted_ast); + return adjusted_ast->getColumnNameWithoutAlias(); + } + else + { + return getColumnNameWithoutAlias(*dag_node, true); + } +} + +bool RPNBuilderTreeNode::isFunction() const +{ + if (ast_node) + return typeid_cast(ast_node); + else + return dag_node->type == ActionsDAG::ActionType::FUNCTION; +} + +bool RPNBuilderTreeNode::isConstant() const +{ + if (ast_node) + return typeid_cast(ast_node); + else + return dag_node->column && isColumnConst(*dag_node->column); +} + +ColumnWithTypeAndName RPNBuilderTreeNode::getConstantColumn() const +{ + if (!isConstant()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a constant"); + + ColumnWithTypeAndName res; + + if (ast_node) + { + const auto * literal = assert_cast(ast_node); + res.type = applyVisitor(FieldToDataType(), literal->value); + res.column = res.type->createColumnConst(0, literal->value); + } + else + { + res.type = dag_node->result_type; + res.column = dag_node->column; + } + + return res; +} + +bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & output_type) const +{ + if (!isConstant()) + return false; + + if (ast_node) + { + // Constant expr should use alias names if any + String column_name = ast_node->getColumnName(); + const auto & block_with_constants = tree_context.getBlockWithConstants(); + + if (const auto * lit = ast_node->as()) + { + /// By default block_with_constants has only one column named "_dummy". + /// If block contains only constants it's may not be preprocessed by + // ExpressionAnalyzer, so try to look up in the default column. + if (!block_with_constants.has(column_name)) + column_name = "_dummy"; + + /// Simple literal + output_value = lit->value; + output_type = block_with_constants.getByName(column_name).type; + + /// If constant is not Null, we can assume it's type is not Nullable as well. + if (!output_value.isNull()) + output_type = removeNullable(output_type); + + return true; + } + else if (block_with_constants.has(column_name) && + isColumnConst(*block_with_constants.getByName(column_name).column)) + { + /// An expression which is dependent on constants only + const auto & expr_info = block_with_constants.getByName(column_name); + output_value = (*expr_info.column)[0]; + output_type = expr_info.type; + + if (!output_value.isNull()) + output_type = removeNullable(output_type); + + return true; + } + } + else + { + if (dag_node->column && isColumnConst(*dag_node->column)) + { + output_value = (*dag_node->column)[0]; + output_type = dag_node->result_type; + + if (!output_value.isNull()) + output_type = removeNullable(output_type); + + return true; + } + } + + return false; +} + +namespace +{ + +ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) +{ + if (!dag_node->column) + return {}; + + const IColumn * col = dag_node->column.get(); + if (const auto * col_const = typeid_cast(col)) + col = &col_const->getDataColumn(); + + if (const auto * col_set = typeid_cast(col)) + { + auto set = col_set->getData(); + + if (set->isCreated()) + return set; + } + + return {}; +} + +} + +ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet() const +{ + const auto & prepared_sets = getTreeContext().getPreparedSets(); + + if (ast_node && prepared_sets) + { + auto prepared_sets_with_same_hash = prepared_sets->getByTreeHash(ast_node->getTreeHash()); + for (auto & set : prepared_sets_with_same_hash) + if (set->isCreated()) + return set; + } + else if (dag_node) + { + return tryGetSetFromDAGNode(dag_node); + } + + return {}; +} + +ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet(const DataTypes & data_types) const +{ + const auto & prepared_sets = getTreeContext().getPreparedSets(); + + if (prepared_sets && ast_node) + { + if (ast_node->as() || ast_node->as()) + return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + + return prepared_sets->get(PreparedSetKey::forLiteral(*ast_node, data_types)); + } + else if (dag_node) + { + return tryGetSetFromDAGNode(dag_node); + } + + return nullptr; +} + +ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( + const std::vector & indexes_mapping, + const DataTypes & data_types) const +{ + const auto & prepared_sets = getTreeContext().getPreparedSets(); + + if (prepared_sets && ast_node) + { + if (ast_node->as() || ast_node->as()) + return prepared_sets->get(PreparedSetKey::forSubquery(*ast_node)); + + /// We have `PreparedSetKey::forLiteral` but it is useless here as we don't have enough information + /// about types in left argument of the IN operator. Instead, we manually iterate through all the sets + /// and find the one for the right arg based on the AST structure (getTreeHash), after that we check + /// that the types it was prepared with are compatible with the types of the primary key. + auto types_match = [&indexes_mapping, &data_types](const SetPtr & candidate_set) + { + assert(indexes_mapping.size() == data_types.size()); + + for (size_t i = 0; i < indexes_mapping.size(); ++i) + { + if (!candidate_set->areTypesEqual(indexes_mapping[i].tuple_index, data_types[i])) + return false; + } + + return true; + }; + + auto tree_hash = ast_node->getTreeHash(); + for (const auto & set : prepared_sets->getByTreeHash(tree_hash)) + { + if (types_match(set)) + return set; + } + } + else if (dag_node->column) + { + return tryGetSetFromDAGNode(dag_node); + } + + return nullptr; +} + +RPNBuilderFunctionTreeNode RPNBuilderTreeNode::toFunctionNode() const +{ + if (!isFunction()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a constant"); + + if (this->ast_node) + return RPNBuilderFunctionTreeNode(this->ast_node, tree_context); + else + return RPNBuilderFunctionTreeNode(this->dag_node, tree_context); +} + +std::string RPNBuilderFunctionTreeNode::getFunctionName() const +{ + if (ast_node) + return assert_cast(ast_node)->name; + else + return dag_node->function_base->getName(); +} + +size_t RPNBuilderFunctionTreeNode::getArgumentsSize() const +{ + if (ast_node) + { + const auto * ast_function = assert_cast(ast_node); + return ast_function->arguments ? ast_function->arguments->children.size() : 0; + } + else + { + return dag_node->children.size(); + } +} + +RPNBuilderTreeNode RPNBuilderFunctionTreeNode::getArgumentAt(size_t index) const +{ + if (ast_node) + { + const auto * ast_function = assert_cast(ast_node); + return RPNBuilderTreeNode(ast_function->arguments->children[index].get(), tree_context); + } + else + { + return RPNBuilderTreeNode(dag_node->children[index], tree_context); + } +} + +} diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index 27b616dc301..d4fc09dcdab 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -1,111 +1,262 @@ #pragma once #include -#include -#include -#include -#include -#include -#include -#include +#include +#include +#include +#include namespace DB { -/// Builds reverse polish notation -template -class RPNBuilder : WithContext +/** Context of RPNBuilderTree. + * + * For AST tree context, precalculated block with constansts and prepared sets are required for index analysis. + * For DAG tree precalculated block with constants and prepared sets are not required, because constants and sets already + * calculated inside COLUMN actions dag node. + */ +class RPNBuilderTreeContext { public: - using RPN = std::vector; - using AtomFromASTFunc = std::function< - bool(const ASTPtr & node, ContextPtr context, Block & block_with_constants, RPNElement & out)>; + /// Construct RPNBuilderTreeContext for ActionsDAG tree + explicit RPNBuilderTreeContext(ContextPtr query_context_); - RPNBuilder(const SelectQueryInfo & query_info, ContextPtr context_, const AtomFromASTFunc & atom_from_ast_) - : WithContext(context_), atom_from_ast(atom_from_ast_) + /// Construct RPNBuilderTreeContext for AST tree + explicit RPNBuilderTreeContext(ContextPtr query_context_, Block block_with_constants_, PreparedSetsPtr prepared_sets_); + + /// Get query context + const ContextPtr & getQueryContext() const { - /** Evaluation of expressions that depend only on constants. - * For the index to be used, if it is written, for example `WHERE Date = toDate(now())`. - */ - block_with_constants = KeyCondition::getBlockWithConstants(query_info.query, query_info.syntax_analyzer_result, getContext()); - - /// Transform WHERE section to Reverse Polish notation - const ASTSelectQuery & select = typeid_cast(*query_info.query); - if (select.where()) - { - traverseAST(select.where()); - - if (select.prewhere()) - { - traverseAST(select.prewhere()); - rpn.emplace_back(RPNElement::FUNCTION_AND); - } - } - else if (select.prewhere()) - { - traverseAST(select.prewhere()); - } - else - { - rpn.emplace_back(RPNElement::FUNCTION_UNKNOWN); - } + return query_context; } - RPN && extractRPN() { return std::move(rpn); } + /** Get block with constants. + * Valid only for AST tree. + */ + const Block & getBlockWithConstants() const + { + return block_with_constants; + } + + /** Get prepared sets. + * Valid only for AST tree. + */ + const PreparedSetsPtr & getPreparedSets() const + { + return prepared_sets; + } private: - void traverseAST(const ASTPtr & node) + /// Valid for both AST and ActionDAG tree + ContextPtr query_context; + + /// Valid only for AST tree + Block block_with_constants; + + /// Valid only for AST tree + PreparedSetsPtr prepared_sets; +}; + +class RPNBuilderFunctionTreeNode; + +/** RPNBuilderTreeNode is wrapper around DAG or AST node. + * It defines unified interface for index analysis. + */ +class RPNBuilderTreeNode +{ +public: + /// Construct RPNBuilderTreeNode with non null dag node and tree context + explicit RPNBuilderTreeNode(const ActionsDAG::Node * dag_node_, RPNBuilderTreeContext & tree_context_); + + /// Construct RPNBuilderTreeNode with non null ast node and tree context + explicit RPNBuilderTreeNode(const IAST * ast_node_, RPNBuilderTreeContext & tree_context_); + + /// Get column name + std::string getColumnName() const; + + /** Get column name. + * Function `modulo` is replaced with `moduloLegacy`. + */ + std::string getColumnNameWithModuloLegacy() const; + + /// Is node function + bool isFunction() const; + + /// Is node constant + bool isConstant() const; + + /** Get constant as constant column. + * Node must be constant before calling these method, otherwise logical exception is thrown. + */ + ColumnWithTypeAndName getConstantColumn() const; + + /** Try get constant from node. If node is constant returns true, and constant value and constant type output parameters are set. + * Otherwise false is returned. + */ + bool tryGetConstant(Field & output_value, DataTypePtr & output_type) const; + + /// Try get prepared set from node + ConstSetPtr tryGetPreparedSet() const; + + /// Try get prepared set from node that match data types + ConstSetPtr tryGetPreparedSet(const DataTypes & data_types) const; + + /// Try get prepared set from node that match indexes mapping and data types + ConstSetPtr tryGetPreparedSet( + const std::vector & indexes_mapping, + const DataTypes & data_types) const; + + /** Convert node to function node. + * Node must be function before calling these method, otherwise exception is thrown. + */ + RPNBuilderFunctionTreeNode toFunctionNode() const; + + /// Get tree context + const RPNBuilderTreeContext & getTreeContext() const + { + return tree_context; + } + + /// Get tree context + RPNBuilderTreeContext & getTreeContext() + { + return tree_context; + } + +protected: + const IAST * ast_node = nullptr; + const ActionsDAG::Node * dag_node = nullptr; + RPNBuilderTreeContext & tree_context; +}; + +/** RPNBuilderFunctionTreeNode is wrapper around RPNBuilderTreeNode with function type. + * It provide additional functionality that is specific for function. + */ +class RPNBuilderFunctionTreeNode : public RPNBuilderTreeNode +{ +public: + /// Get function name + std::string getFunctionName() const; + + /// Get function arguments size + size_t getArgumentsSize() const; + + /// Get argument at index + RPNBuilderTreeNode getArgumentAt(size_t index) const; + + using RPNBuilderTreeNode::RPNBuilderTreeNode; +}; + +/** RPN Builder build stack of reverse polish notation elements (RPNElements) required for index analysis. + * + * RPNBuilder client must provide RPNElement type that has following interface: + * + * struct RPNElementInterface + * { + * enum Function + * { + * FUNCTION_UNKNOWN, /// Can take any value. + * /// Operators of the logical expression. + * FUNCTION_NOT, + * FUNCTION_AND, + * FUNCTION_OR, + * ... + * }; + * + * RPNElementInterface(); + * + * Function function = FUNCTION_UNKNOWN; + * + * } + * + * RPNBuilder take care of building stack of RPNElements with `NOT`, `AND`, `OR` types. + * In addition client must provide ExtractAtomFromTreeFunction that returns true and RPNElement as output parameter, + * if it can convert RPNBuilderTree node to RPNElement, false otherwise. + */ +template +class RPNBuilder +{ +public: + using RPNElements = std::vector; + using ExtractAtomFromTreeFunction = std::function; + + explicit RPNBuilder(const ActionsDAG::Node * & filter_actions_dag_node, + ContextPtr query_context_, + const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_) + : tree_context(std::move(query_context_)) + , extract_atom_from_tree_function(extract_atom_from_tree_function_) + { + traverseTree(RPNBuilderTreeNode(filter_actions_dag_node, tree_context)); + } + + RPNBuilder(const ASTPtr & filter_node, + ContextPtr query_context_, + Block block_with_constants_, + PreparedSetsPtr prepared_sets_, + const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_) + : tree_context(std::move(query_context_), std::move(block_with_constants_), std::move(prepared_sets_)) + , extract_atom_from_tree_function(extract_atom_from_tree_function_) + { + traverseTree(RPNBuilderTreeNode(filter_node.get(), tree_context)); + } + + RPNElements && extractRPN() && { return std::move(rpn_elements); } + +private: + void traverseTree(const RPNBuilderTreeNode & node) { RPNElement element; - if (ASTFunction * func = typeid_cast(&*node)) + if (node.isFunction()) { - if (operatorFromAST(func, element)) + auto function_node = node.toFunctionNode(); + + if (extractLogicalOperatorFromTree(function_node, element)) { - auto & args = typeid_cast(*func->arguments).children; - for (size_t i = 0, size = args.size(); i < size; ++i) + size_t arguments_size = function_node.getArgumentsSize(); + + for (size_t argument_index = 0; argument_index < arguments_size; ++argument_index) { - traverseAST(args[i]); + auto function_node_argument = function_node.getArgumentAt(argument_index); + traverseTree(function_node_argument); /** The first part of the condition is for the correct support of `and` and `or` functions of arbitrary arity * - in this case `n - 1` elements are added (where `n` is the number of arguments). */ - if (i != 0 || element.function == RPNElement::FUNCTION_NOT) - rpn.emplace_back(std::move(element)); + if (argument_index != 0 || element.function == RPNElement::FUNCTION_NOT) + rpn_elements.emplace_back(std::move(element)); } return; } } - if (!atom_from_ast(node, getContext(), block_with_constants, element)) - { + if (!extract_atom_from_tree_function(node, element)) element.function = RPNElement::FUNCTION_UNKNOWN; - } - rpn.emplace_back(std::move(element)); + rpn_elements.emplace_back(std::move(element)); } - bool operatorFromAST(const ASTFunction * func, RPNElement & out) + bool extractLogicalOperatorFromTree(const RPNBuilderFunctionTreeNode & function_node, RPNElement & out) { /// Functions AND, OR, NOT. /// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses /// (or, the same thing - calling the function `and` from one argument). - const ASTs & args = typeid_cast(*func->arguments).children; - if (func->name == "not") + auto function_name = function_node.getFunctionName(); + if (function_name == "not") { - if (args.size() != 1) + if (function_node.getArgumentsSize() != 1) return false; out.function = RPNElement::FUNCTION_NOT; } else { - if (func->name == "and" || func->name == "indexHint") + if (function_name == "and" || function_name == "indexHint") out.function = RPNElement::FUNCTION_AND; - else if (func->name == "or") + else if (function_name == "or") out.function = RPNElement::FUNCTION_OR; else return false; @@ -114,10 +265,9 @@ private: return true; } - const AtomFromASTFunc & atom_from_ast; - Block block_with_constants; - RPN rpn; + RPNBuilderTreeContext tree_context; + const ExtractAtomFromTreeFunction & extract_atom_from_tree_function; + RPNElements rpn_elements; }; - } From c47ca522f40178e3e5b4cd1a675499e2d42327b8 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 26 Oct 2022 13:27:14 +0200 Subject: [PATCH 037/152] Fixed style check --- src/Storages/MergeTree/RPNBuilder.cpp | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 362e02e8498..05cc6492624 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -21,6 +21,11 @@ namespace DB { +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + namespace { From 1c17e9d45446f7d6e3dd4d6dfbc4aa64f203a074 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 26 Oct 2022 15:54:37 +0200 Subject: [PATCH 038/152] Fixed tests --- .../optimizePrimaryKeyCondition.cpp | 16 +++--- .../QueryPlan/ReadFromMergeTree.cpp | 6 ++- src/Storages/MergeTree/RPNBuilder.cpp | 51 +++++++++++++------ src/Storages/MergeTree/RPNBuilder.h | 2 +- 4 files changed, 50 insertions(+), 25 deletions(-) diff --git a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp index 7d682c408e5..984c76701ba 100644 --- a/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp +++ b/src/Processors/QueryPlan/Optimizations/optimizePrimaryKeyCondition.cpp @@ -17,7 +17,7 @@ void optimizePrimaryKeyCondition(QueryPlan::Node & root) size_t next_child = 0; }; - std::deque stack; + std::vector stack; stack.push_back({.node = &root}); while (!stack.empty()) @@ -27,29 +27,29 @@ void optimizePrimaryKeyCondition(QueryPlan::Node & root) /// Traverse all children first. if (frame.next_child < frame.node->children.size()) { - stack.push_back({.node = frame.node->children[frame.next_child]}); - + auto next_frame = Frame{.node = frame.node->children[frame.next_child]}; ++frame.next_child; + stack.push_back(next_frame); continue; } - auto add_filter = [&](auto & storage) + auto add_read_from_storage_filter = [&](auto & storage) { - for (auto iter=stack.rbegin() + 1; iter!=stack.rend(); ++iter) + for (auto iter = stack.rbegin() + 1; iter != stack.rend(); ++iter) { if (auto * filter_step = typeid_cast(iter->node->step.get())) storage.addFilter(filter_step->getExpression(), filter_step->getFilterColumnName()); else if (typeid_cast(iter->node->step.get())) - ; + continue; else break; } }; if (auto * read_from_merge_tree = typeid_cast(frame.node->step.get())) - add_filter(*read_from_merge_tree); + add_read_from_storage_filter(*read_from_merge_tree); else if (auto * read_from_merge = typeid_cast(frame.node->step.get())) - add_filter(*read_from_merge); + add_read_from_storage_filter(*read_from_merge); stack.pop_back(); } diff --git a/src/Processors/QueryPlan/ReadFromMergeTree.cpp b/src/Processors/QueryPlan/ReadFromMergeTree.cpp index 71f8378ae41..53568b6a79e 100644 --- a/src/Processors/QueryPlan/ReadFromMergeTree.cpp +++ b/src/Processors/QueryPlan/ReadFromMergeTree.cpp @@ -906,11 +906,15 @@ MergeTreeDataSelectAnalysisResultPtr ReadFromMergeTree::selectRangesToRead( for (const auto & node : added_filter_nodes.nodes) nodes.nodes.push_back(node); + NameSet array_join_name_set; + if (query_info.syntax_analyzer_result) + array_join_name_set = query_info.syntax_analyzer_result->getArrayJoinSourceNameSet(); + key_condition.emplace(std::move(nodes), context, primary_key_columns, primary_key.expression, - query_info.syntax_analyzer_result->getArrayJoinSourceNameSet()); + array_join_name_set); } else { diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 05cc6492624..4cc311b8b9e 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -33,10 +33,10 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o { switch (node.type) { - case (ActionsDAG::ActionType::INPUT): + case ActionsDAG::ActionType::INPUT: writeString(node.result_name, out); break; - case (ActionsDAG::ActionType::COLUMN): + case ActionsDAG::ActionType::COLUMN: { /// If it was created from ASTLiteral, then result_name can be an alias. /// We need to convert value back to string here. @@ -47,15 +47,15 @@ void appendColumnNameWithoutAlias(const ActionsDAG::Node & node, WriteBuffer & o writeString(node.result_name, out); break; } - case (ActionsDAG::ActionType::ALIAS): + case ActionsDAG::ActionType::ALIAS: appendColumnNameWithoutAlias(*node.children.front(), out, legacy); break; - case (ActionsDAG::ActionType::ARRAY_JOIN): + case ActionsDAG::ActionType::ARRAY_JOIN: writeCString("arrayJoin(", out); appendColumnNameWithoutAlias(*node.children.front(), out, legacy); writeChar(')', out); break; - case (ActionsDAG::ActionType::FUNCTION): + case ActionsDAG::ActionType::FUNCTION: { auto name = node.function_base->getName(); if (legacy && name == "modulo") @@ -144,9 +144,23 @@ bool RPNBuilderTreeNode::isFunction() const bool RPNBuilderTreeNode::isConstant() const { if (ast_node) - return typeid_cast(ast_node); + { + bool is_literal = typeid_cast(ast_node); + if (is_literal) + return true; + + String column_name = ast_node->getColumnName(); + const auto & block_with_constants = tree_context.getBlockWithConstants(); + + if (block_with_constants.has(column_name) && isColumnConst(*block_with_constants.getByName(column_name).column)) + return true; + + return false; + } else + { return dag_node->column && isColumnConst(*dag_node->column); + } } ColumnWithTypeAndName RPNBuilderTreeNode::getConstantColumn() const @@ -154,28 +168,35 @@ ColumnWithTypeAndName RPNBuilderTreeNode::getConstantColumn() const if (!isConstant()) throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a constant"); - ColumnWithTypeAndName res; + ColumnWithTypeAndName result; if (ast_node) { const auto * literal = assert_cast(ast_node); - res.type = applyVisitor(FieldToDataType(), literal->value); - res.column = res.type->createColumnConst(0, literal->value); + if (literal) + { + result.type = applyVisitor(FieldToDataType(), literal->value); + result.column = result.type->createColumnConst(0, literal->value); + + return result; + } + + String column_name = ast_node->getColumnName(); + const auto & block_with_constants = tree_context.getBlockWithConstants(); + + return block_with_constants.getByName(column_name); } else { - res.type = dag_node->result_type; - res.column = dag_node->column; + result.type = dag_node->result_type; + result.column = dag_node->column; } - return res; + return result; } bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & output_type) const { - if (!isConstant()) - return false; - if (ast_node) { // Constant expr should use alias names if any diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index d4fc09dcdab..f6fc4a70102 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -181,7 +181,7 @@ public: using RPNElements = std::vector; using ExtractAtomFromTreeFunction = std::function; - explicit RPNBuilder(const ActionsDAG::Node * & filter_actions_dag_node, + explicit RPNBuilder(const ActionsDAG::Node * filter_actions_dag_node, ContextPtr query_context_, const ExtractAtomFromTreeFunction & extract_atom_from_tree_function_) : tree_context(std::move(query_context_)) From 2b44a00fbc39db06f0c2e11db35b5608b9341aa1 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 26 Oct 2022 16:47:40 +0200 Subject: [PATCH 039/152] UNION node remove unnecessary union modes --- src/Analyzer/QueryTreeBuilder.cpp | 23 ++++++++--------- src/Analyzer/UnionNode.cpp | 42 ++++++++----------------------- src/Analyzer/UnionNode.h | 25 +++--------------- 3 files changed, 23 insertions(+), 67 deletions(-) diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index 890aa2b01a2..b68d7d3d387 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -145,12 +145,10 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectWithUnionExpression(const ASTPtr & if (select_lists.children.size() == 1) return buildSelectOrUnionExpression(select_lists.children[0], is_subquery, cte_name); - auto union_node = std::make_shared(); + auto union_node = std::make_shared(select_with_union_query_typed.union_mode); union_node->setIsSubquery(is_subquery); union_node->setIsCTE(!cte_name.empty()); union_node->setCTEName(cte_name); - union_node->setUnionMode(select_with_union_query_typed.union_mode); - union_node->setUnionModes(select_with_union_query_typed.list_of_modes); union_node->setOriginalAST(select_with_union_query); size_t select_lists_children_size = select_lists.children.size(); @@ -173,23 +171,22 @@ QueryTreeNodePtr QueryTreeBuilder::buildSelectIntersectExceptQuery(const ASTPtr if (select_lists.size() == 1) return buildSelectExpression(select_lists[0], is_subquery, cte_name); - auto union_node = std::make_shared(); - union_node->setIsSubquery(is_subquery); - union_node->setIsCTE(!cte_name.empty()); - union_node->setCTEName(cte_name); - + SelectUnionMode union_mode; if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_ALL) - union_node->setUnionMode(SelectUnionMode::INTERSECT_ALL); + union_mode = SelectUnionMode::INTERSECT_ALL; else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::INTERSECT_DISTINCT) - union_node->setUnionMode(SelectUnionMode::INTERSECT_DISTINCT); + union_mode = SelectUnionMode::INTERSECT_DISTINCT; else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_ALL) - union_node->setUnionMode(SelectUnionMode::EXCEPT_ALL); + union_mode = SelectUnionMode::EXCEPT_ALL; else if (select_intersect_except_query_typed.final_operator == ASTSelectIntersectExceptQuery::Operator::EXCEPT_DISTINCT) - union_node->setUnionMode(SelectUnionMode::EXCEPT_DISTINCT); + union_mode = SelectUnionMode::EXCEPT_DISTINCT; else throw Exception(ErrorCodes::LOGICAL_ERROR, "UNION type is not initialized"); - union_node->setUnionModes(SelectUnionModes(select_lists.size() - 1, union_node->getUnionMode())); + auto union_node = std::make_shared(union_mode); + union_node->setIsSubquery(is_subquery); + union_node->setIsCTE(!cte_name.empty()); + union_node->setCTEName(cte_name); union_node->setOriginalAST(select_intersect_except_query); size_t select_lists_size = select_lists.size(); diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index b8ed46c645e..74992652a1c 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -32,9 +32,15 @@ namespace ErrorCodes extern const int TYPE_MISMATCH; } -UnionNode::UnionNode() +UnionNode::UnionNode(SelectUnionMode union_mode_) : IQueryTreeNode(children_size) + , union_mode(union_mode_) { + if (union_mode == SelectUnionMode::UNION_DEFAULT || + union_mode == SelectUnionMode::EXCEPT_DEFAULT || + union_mode == SelectUnionMode::INTERSECT_DEFAULT) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION mode must be normalized"); + children[queries_child_index] = std::make_shared(); } @@ -109,20 +115,6 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", union_mode: " << toString(union_mode); - size_t union_modes_size = union_modes.size(); - buffer << '\n' << std::string(indent + 2, ' ') << "UNION MODES " << union_modes_size << '\n'; - - for (size_t i = 0; i < union_modes_size; ++i) - { - buffer << std::string(indent + 4, ' '); - - auto query_union_mode = union_modes[i]; - buffer << toString(query_union_mode); - - if (i + 1 != union_modes_size) - buffer << '\n'; - } - buffer << '\n' << std::string(indent + 2, ' ') << "QUERIES\n"; getQueriesNode()->dumpTreeImpl(buffer, format_state, indent + 4); } @@ -145,7 +137,7 @@ bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs) const return false; return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && - union_mode == rhs_typed.union_mode && union_modes == rhs_typed.union_modes; + union_mode == rhs_typed.union_mode; } void UnionNode::updateTreeHashImpl(HashState & state) const @@ -158,10 +150,6 @@ void UnionNode::updateTreeHashImpl(HashState & state) const state.update(static_cast(union_mode)); - state.update(union_modes.size()); - for (const auto & query_union_mode : union_modes) - state.update(static_cast(query_union_mode)); - if (constant_value) { auto constant_dump = applyVisitor(FieldVisitorToString(), constant_value->getValue()); @@ -179,14 +167,11 @@ void UnionNode::updateTreeHashImpl(HashState & state) const QueryTreeNodePtr UnionNode::cloneImpl() const { - auto result_union_node = std::make_shared(); + auto result_union_node = std::make_shared(union_mode); result_union_node->is_subquery = is_subquery; result_union_node->is_cte = is_cte; result_union_node->cte_name = cte_name; - result_union_node->union_mode = union_mode; - result_union_node->union_modes = union_modes; - result_union_node->union_modes_set = union_modes_set; result_union_node->constant_value = constant_value; result_union_node->table_expression_modifiers = table_expression_modifiers; @@ -197,14 +182,7 @@ ASTPtr UnionNode::toASTImpl() const { auto select_with_union_query = std::make_shared(); select_with_union_query->union_mode = union_mode; - - if (union_mode != SelectUnionMode::UNION_DEFAULT && - union_mode != SelectUnionMode::EXCEPT_DEFAULT && - union_mode != SelectUnionMode::INTERSECT_DEFAULT) - select_with_union_query->is_normalized = true; - - select_with_union_query->list_of_modes = union_modes; - select_with_union_query->set_of_modes = union_modes_set; + select_with_union_query->is_normalized = true; select_with_union_query->children.push_back(getQueriesNode()->toAST()); select_with_union_query->list_of_selects = select_with_union_query->children.back(); diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index 05e70b87a27..f192741faa0 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -19,6 +19,7 @@ namespace ErrorCodes } /** Union node represents union of queries in query tree. + * Union node must be initialized with normalized union mode. * * Example: (SELECT id FROM test_table) UNION ALL (SELECT id FROM test_table_2); * Example: (SELECT id FROM test_table) UNION DISTINCT (SELECT id FROM test_table_2); @@ -41,7 +42,8 @@ using UnionNodePtr = std::shared_ptr; class UnionNode final : public IQueryTreeNode { public: - explicit UnionNode(); + /// Construct union node with normalized union mode + explicit UnionNode(SelectUnionMode union_mode_); /// Returns true if union node is subquery, false otherwise bool isSubquery() const @@ -85,25 +87,6 @@ public: return union_mode; } - /// Set union mode value - void setUnionMode(SelectUnionMode union_mode_value) - { - union_mode = union_mode_value; - } - - /// Get union modes - const SelectUnionModes & getUnionModes() const - { - return union_modes; - } - - /// Set union modes value - void setUnionModes(const SelectUnionModes & union_modes_value) - { - union_modes = union_modes_value; - union_modes_set = SelectUnionModesSet(union_modes.begin(), union_modes.end()); - } - /// Get union node queries const ListNode & getQueries() const { @@ -189,8 +172,6 @@ private: bool is_cte = false; std::string cte_name; SelectUnionMode union_mode; - SelectUnionModes union_modes; - SelectUnionModesSet union_modes_set; ConstantValuePtr constant_value; std::optional table_expression_modifiers; From 6a0e3fc31da1c07e50798b77a1bd27b569b3ede7 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 26 Oct 2022 17:19:29 +0200 Subject: [PATCH 040/152] Subqueries remove unnecessary table expression modifiers --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 67 +++++++++---------- src/Analyzer/QueryNode.cpp | 17 ----- src/Analyzer/QueryNode.h | 19 ------ src/Analyzer/QueryTreeBuilder.cpp | 26 ++++--- src/Analyzer/UnionNode.cpp | 17 ----- src/Analyzer/UnionNode.h | 19 ------ src/Analyzer/Utils.cpp | 5 -- ...eries_table_expression_modifiers.reference | 0 ..._subqueries_table_expression_modifiers.sql | 17 +++++ 9 files changed, 65 insertions(+), 122 deletions(-) create mode 100644 tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.reference create mode 100644 tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 6b91d0f8053..019b002c527 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -1621,34 +1621,7 @@ void QueryAnalyzer::validateTableExpressionModifiers(const QueryTreeNodePtr & ta table_expression_node->formatASTForErrorMessage(), scope.scope_node->formatASTForErrorMessage()); - if (query_node || union_node) - { - auto table_expression_modifiers = query_node ? query_node->getTableExpressionModifiers() : union_node->getTableExpressionModifiers(); - - if (table_expression_modifiers.has_value()) - { - String table_expression_modifiers_error_message; - - if (table_expression_modifiers->hasFinal()) - { - table_expression_modifiers_error_message += "FINAL"; - - if (table_expression_modifiers->hasSampleSizeRatio()) - table_expression_modifiers_error_message += ", SAMPLE"; - } - else if (table_expression_modifiers->hasSampleSizeRatio()) - { - table_expression_modifiers_error_message += "SAMPLE"; - } - - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, - "Table expression modifiers {} are not supported for subquery {}. In scope {}", - table_expression_modifiers_error_message, - table_expression_node->formatASTForErrorMessage(), - scope.scope_node->formatASTForErrorMessage()); - } - } - else if (table_node || table_function_node) + if (table_node || table_function_node) { auto table_expression_modifiers = table_node ? table_node->getTableExpressionModifiers() : table_function_node->getTableExpressionModifiers(); @@ -4661,17 +4634,37 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod auto table_expression_modifiers = from_table_identifier.getTableExpressionModifiers(); - if (auto * resolved_identifier_query_node = resolved_identifier->as()) + auto * resolved_identifier_query_node = resolved_identifier->as(); + auto * resolved_identifier_union_node = resolved_identifier->as(); + + if (resolved_identifier_query_node || resolved_identifier_union_node) { - resolved_identifier_query_node->setIsCTE(false); + if (resolved_identifier_query_node) + resolved_identifier_query_node->setIsCTE(false); + else + resolved_identifier_union_node->setIsCTE(false); + if (table_expression_modifiers.has_value()) - resolved_identifier_query_node->setTableExpressionModifiers(*table_expression_modifiers); - } - else if (auto * resolved_identifier_union_node = resolved_identifier->as()) - { - resolved_identifier_union_node->setIsCTE(false); - if (table_expression_modifiers.has_value()) - resolved_identifier_union_node->setTableExpressionModifiers(*table_expression_modifiers); + { + String table_expression_modifiers_error_message; + + if (table_expression_modifiers->hasFinal()) + { + table_expression_modifiers_error_message += "FINAL"; + + if (table_expression_modifiers->hasSampleSizeRatio()) + table_expression_modifiers_error_message += ", SAMPLE"; + } + else if (table_expression_modifiers->hasSampleSizeRatio()) + { + table_expression_modifiers_error_message += "SAMPLE"; + } + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Table expression modifiers {} are not supported for subquery {}", + table_expression_modifiers_error_message, + resolved_identifier->formatASTForErrorMessage()); + } } else if (auto * resolved_identifier_table_node = resolved_identifier->as()) { diff --git a/src/Analyzer/QueryNode.cpp b/src/Analyzer/QueryNode.cpp index 557baad2654..c5bbc193544 100644 --- a/src/Analyzer/QueryNode.cpp +++ b/src/Analyzer/QueryNode.cpp @@ -74,12 +74,6 @@ void QueryNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", constant_value_type: " << constant_value->getType()->getName(); } - if (table_expression_modifiers) - { - buffer << ", "; - table_expression_modifiers->dump(buffer); - } - if (hasWith()) { buffer << '\n' << std::string(indent + 2, ' ') << "WITH\n"; @@ -195,13 +189,6 @@ bool QueryNode::isEqualImpl(const IQueryTreeNode & rhs) const else if (!constant_value && rhs_typed.constant_value) return false; - if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) - return false; - else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) - return false; - else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) - return false; - return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && @@ -250,9 +237,6 @@ void QueryNode::updateTreeHashImpl(HashState & state) const state.update(constant_value_type_name.size()); state.update(constant_value_type_name); } - - if (table_expression_modifiers) - table_expression_modifiers->updateTreeHash(state); } QueryTreeNodePtr QueryNode::cloneImpl() const @@ -270,7 +254,6 @@ QueryTreeNodePtr QueryNode::cloneImpl() const result_query_node->cte_name = cte_name; result_query_node->projection_columns = projection_columns; result_query_node->constant_value = constant_value; - result_query_node->table_expression_modifiers = table_expression_modifiers; return result_query_node; } diff --git a/src/Analyzer/QueryNode.h b/src/Analyzer/QueryNode.h index 6bb6613fc2b..1bb381c95c9 100644 --- a/src/Analyzer/QueryNode.h +++ b/src/Analyzer/QueryNode.h @@ -176,24 +176,6 @@ public: is_group_by_with_grouping_sets = is_group_by_with_grouping_sets_value; } - /// Return true if query node has table expression modifiers, false otherwise - bool hasTableExpressionModifiers() const - { - return table_expression_modifiers.has_value(); - } - - /// Get table expression modifiers - const std::optional & getTableExpressionModifiers() const - { - return table_expression_modifiers; - } - - /// Set table expression modifiers - void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) - { - table_expression_modifiers = std::move(table_expression_modifiers_value); - } - /// Returns true if query node WITH section is not empty, false otherwise bool hasWith() const { @@ -602,7 +584,6 @@ private: std::string cte_name; NamesAndTypes projection_columns; ConstantValuePtr constant_value; - std::optional table_expression_modifiers; SettingsChanges settings_changes; static constexpr size_t with_child_index = 0; diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index b68d7d3d387..c5bb3ae7074 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -673,14 +673,24 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select if (table_expression_modifiers) { - if (auto * query_node = node->as()) - query_node->setTableExpressionModifiers(*table_expression_modifiers); - else if (auto * union_node = node->as()) - union_node->setTableExpressionModifiers(*table_expression_modifiers); - else - throw Exception(ErrorCodes::LOGICAL_ERROR, - "Unexpected table expression subquery node. Expected union or query. Actual {}", - node->formatASTForErrorMessage()); + String table_expression_modifiers_error_message; + + if (table_expression_modifiers->hasFinal()) + { + table_expression_modifiers_error_message += "FINAL"; + + if (table_expression_modifiers->hasSampleSizeRatio()) + table_expression_modifiers_error_message += ", SAMPLE"; + } + else if (table_expression_modifiers->hasSampleSizeRatio()) + { + table_expression_modifiers_error_message += "SAMPLE"; + } + + throw Exception(ErrorCodes::UNSUPPORTED_METHOD, + "Table expression modifiers {} are not supported for subquery {}", + table_expression_modifiers_error_message, + node->formatASTForErrorMessage()); } table_expressions.push_back(std::move(node)); diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 74992652a1c..039077bd08a 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -107,12 +107,6 @@ void UnionNode::dumpTreeImpl(WriteBuffer & buffer, FormatState & format_state, s buffer << ", constant_value_type: " << constant_value->getType()->getName(); } - if (table_expression_modifiers) - { - buffer << ", "; - table_expression_modifiers->dump(buffer); - } - buffer << ", union_mode: " << toString(union_mode); buffer << '\n' << std::string(indent + 2, ' ') << "QUERIES\n"; @@ -129,13 +123,6 @@ bool UnionNode::isEqualImpl(const IQueryTreeNode & rhs) const else if (!constant_value && rhs_typed.constant_value) return false; - if (table_expression_modifiers && rhs_typed.table_expression_modifiers && table_expression_modifiers != rhs_typed.table_expression_modifiers) - return false; - else if (table_expression_modifiers && !rhs_typed.table_expression_modifiers) - return false; - else if (!table_expression_modifiers && rhs_typed.table_expression_modifiers) - return false; - return is_subquery == rhs_typed.is_subquery && is_cte == rhs_typed.is_cte && cte_name == rhs_typed.cte_name && union_mode == rhs_typed.union_mode; } @@ -160,9 +147,6 @@ void UnionNode::updateTreeHashImpl(HashState & state) const state.update(constant_value_type_name.size()); state.update(constant_value_type_name); } - - if (table_expression_modifiers) - table_expression_modifiers->updateTreeHash(state); } QueryTreeNodePtr UnionNode::cloneImpl() const @@ -173,7 +157,6 @@ QueryTreeNodePtr UnionNode::cloneImpl() const result_union_node->is_cte = is_cte; result_union_node->cte_name = cte_name; result_union_node->constant_value = constant_value; - result_union_node->table_expression_modifiers = table_expression_modifiers; return result_union_node; } diff --git a/src/Analyzer/UnionNode.h b/src/Analyzer/UnionNode.h index f192741faa0..9ef76591597 100644 --- a/src/Analyzer/UnionNode.h +++ b/src/Analyzer/UnionNode.h @@ -111,24 +111,6 @@ public: return children[queries_child_index]; } - /// Return true if union node has table expression modifiers, false otherwise - bool hasTableExpressionModifiers() const - { - return table_expression_modifiers.has_value(); - } - - /// Get table expression modifiers - const std::optional & getTableExpressionModifiers() const - { - return table_expression_modifiers; - } - - /// Set table expression modifiers - void setTableExpressionModifiers(TableExpressionModifiers table_expression_modifiers_value) - { - table_expression_modifiers = std::move(table_expression_modifiers_value); - } - /// Compute union node projection columns NamesAndTypes computeProjectionColumns() const; @@ -173,7 +155,6 @@ private: std::string cte_name; SelectUnionMode union_mode; ConstantValuePtr constant_value; - std::optional table_expression_modifiers; static constexpr size_t queries_child_index = 0; static constexpr size_t children_size = queries_child_index + 1; diff --git a/src/Analyzer/Utils.cpp b/src/Analyzer/Utils.cpp index 5f0d682865f..b504a5b5787 100644 --- a/src/Analyzer/Utils.cpp +++ b/src/Analyzer/Utils.cpp @@ -98,11 +98,6 @@ static ASTPtr convertIntoTableExpressionAST(const QueryTreeNodePtr & table_expre if (node_type == QueryTreeNodeType::QUERY || node_type == QueryTreeNodeType::UNION) { - if (auto * query_node = table_expression_node->as()) - table_expression_modifiers = query_node->getTableExpressionModifiers(); - else if (auto * union_node = table_expression_node->as()) - table_expression_modifiers = union_node->getTableExpressionModifiers(); - result_table_expression->subquery = result_table_expression->children.back(); } else if (node_type == QueryTreeNodeType::TABLE || node_type == QueryTreeNodeType::IDENTIFIER) diff --git a/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.reference b/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql b/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql new file mode 100644 index 00000000000..456783cad26 --- /dev/null +++ b/tests/queries/0_stateless/02474_analyzer_subqueries_table_expression_modifiers.sql @@ -0,0 +1,17 @@ +SET allow_experimental_analyzer = 1; + +SELECT * FROM (SELECT 1) FINAL; -- { serverError 1 } +SELECT * FROM (SELECT 1) SAMPLE 1/2; -- { serverError 1 } +SELECT * FROM (SELECT 1) FINAL SAMPLE 1/2; -- { serverError 1 } + +WITH cte_subquery AS (SELECT 1) SELECT * FROM cte_subquery FINAL; -- { serverError 1 } +WITH cte_subquery AS (SELECT 1) SELECT * FROM cte_subquery SAMPLE 1/2; -- { serverError 1 } +WITH cte_subquery AS (SELECT 1) SELECT * FROM cte_subquery FINAL SAMPLE 1/2; -- { serverError 1 } + +SELECT * FROM (SELECT 1 UNION ALL SELECT 1) FINAL; -- { serverError 1 } +SELECT * FROM (SELECT 1 UNION ALL SELECT 1) SAMPLE 1/2; -- { serverError 1 } +SELECT * FROM (SELECT 1 UNION ALL SELECT 1) FINAL SAMPLE 1/2; -- { serverError 1 } + +WITH cte_subquery AS (SELECT 1 UNION ALL SELECT 1) SELECT * FROM cte_subquery FINAL; -- { serverError 1 } +WITH cte_subquery AS (SELECT 1 UNION ALL SELECT 1) SELECT * FROM cte_subquery SAMPLE 1/2; -- { serverError 1 } +WITH cte_subquery AS (SELECT 1 UNION ALL SELECT 1) SELECT * FROM cte_subquery FINAL SAMPLE 1/2; -- { serverError 1 } From f5f6f1b5933b4d19702f9fabf63fbb4dc02f21a6 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Oct 2022 13:39:41 +0200 Subject: [PATCH 041/152] Add profile events for jemalloc purge Signed-off-by: Azat Khuzhin --- src/Common/MemoryTracker.cpp | 6 ++++++ src/Common/ProfileEvents.cpp | 2 ++ 2 files changed, 8 insertions(+) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 8bd31681706..41634e8f561 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -10,6 +10,7 @@ #include #include #include +#include #include #include "config.h" @@ -86,6 +87,8 @@ inline std::string_view toDescription(OvercommitResult result) namespace ProfileEvents { extern const Event QueryMemoryLimitExceeded; + extern const Event MemoryAllocatorPurge; + extern const Event MemoryAllocatorPurgeTimeMicroseconds; } using namespace std::chrono_literals; @@ -229,7 +232,10 @@ void MemoryTracker::allocImpl(Int64 size, bool throw_if_memory_exceeded, MemoryT { if (free_memory_in_allocator_arenas.exchange(-current_free_memory_in_allocator_arenas) > 0) { + Stopwatch watch; mallctl("arena." STRINGIFY(MALLCTL_ARENAS_ALL) ".purge", nullptr, nullptr, nullptr, 0); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurge); + ProfileEvents::increment(ProfileEvents::MemoryAllocatorPurgeTimeMicroseconds, watch.elapsedMicroseconds()); } } diff --git a/src/Common/ProfileEvents.cpp b/src/Common/ProfileEvents.cpp index 46bec669626..2f801e496fa 100644 --- a/src/Common/ProfileEvents.cpp +++ b/src/Common/ProfileEvents.cpp @@ -229,6 +229,8 @@ The server successfully detected this situation and will download merged part fr M(UserTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in user space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \ M(SystemTimeMicroseconds, "Total time spent in processing (queries and other tasks) threads executing CPU instructions in OS kernel space. This include time CPU pipeline was stalled due to cache misses, branch mispredictions, hyper-threading, etc.") \ M(MemoryOvercommitWaitTimeMicroseconds, "Total time spent in waiting for memory to be freed in OvercommitTracker.") \ + M(MemoryAllocatorPurge, "Total number of times memory allocator purge was requested") \ + M(MemoryAllocatorPurgeTimeMicroseconds, "Total number of times memory allocator purge was requested") \ M(SoftPageFaults, "The number of soft page faults in query execution threads. Soft page fault usually means a miss in the memory allocator cache which required a new memory mapping from the OS and subsequent allocation of a page of physical memory.") \ M(HardPageFaults, "The number of hard page faults in query execution threads. High values indicate either that you forgot to turn off swap on your server, or eviction of memory pages of the ClickHouse binary during very high memory pressure, or successful usage of the 'mmap' read method for the tables data.") \ \ From 2faefc0c0973b666397b6ac4a96c6732706e250b Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Oct 2022 13:40:37 +0200 Subject: [PATCH 042/152] Add amount of memory used by allocator into logs Signed-off-by: Azat Khuzhin --- src/Interpreters/AsynchronousMetrics.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 338ae1bbbfd..2e43a594960 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -713,9 +713,10 @@ void AsynchronousMetrics::update(TimePoint update_time) /// Log only if difference is high. This is for convenience. The threshold is arbitrary. if (difference >= 1048576 || difference <= -1048576) LOG_TRACE(log, - "MemoryTracking: was {}, peak {}, will set to {} (RSS), difference: {}", + "MemoryTracking: was {}, peak {}, free memory in arenas {}, will set to {} (RSS), difference: {}", ReadableSize(amount), ReadableSize(peak), + ReadableSize(free_memory_in_allocator_arenas), ReadableSize(rss), ReadableSize(difference)); From 7b69a70e828a6fab644086aa589d4352c5d86883 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 22 Oct 2022 18:39:36 +0200 Subject: [PATCH 043/152] Fix frequent memory drift message and clarify things in comments Somethine like: 2022.09.28 06:33:34.001433 [ 3133669 ] {} AsynchronousMetrics: MemoryTracking: was 562.20 MiB, peak 562.21 MiB, will set to 562.20 MiB (RSS), difference: -70.46 MiB 2022.09.28 06:33:35.001639 [ 3133669 ] {} AsynchronousMetrics: MemoryTracking: was 562.20 MiB, peak 562.21 MiB, will set to 562.20 MiB (RSS), difference: -70.45 MiB Signed-off-by: Azat Khuzhin --- src/Common/MemoryTracker.cpp | 2 +- src/Interpreters/AsynchronousMetrics.cpp | 12 +++++++++--- 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/src/Common/MemoryTracker.cpp b/src/Common/MemoryTracker.cpp index 41634e8f561..b530410ec63 100644 --- a/src/Common/MemoryTracker.cpp +++ b/src/Common/MemoryTracker.cpp @@ -438,7 +438,7 @@ void MemoryTracker::reset() void MemoryTracker::setRSS(Int64 rss_, Int64 free_memory_in_allocator_arenas_) { - Int64 new_amount = rss_; // - free_memory_in_allocator_arenas_; + Int64 new_amount = rss_; total_memory_tracker.amount.store(new_amount, std::memory_order_relaxed); free_memory_in_allocator_arenas.store(free_memory_in_allocator_arenas_, std::memory_order_relaxed); diff --git a/src/Interpreters/AsynchronousMetrics.cpp b/src/Interpreters/AsynchronousMetrics.cpp index 2e43a594960..488ac77e956 100644 --- a/src/Interpreters/AsynchronousMetrics.cpp +++ b/src/Interpreters/AsynchronousMetrics.cpp @@ -703,12 +703,18 @@ void AsynchronousMetrics::update(TimePoint update_time) Int64 free_memory_in_allocator_arenas = 0; #if USE_JEMALLOC - /// This is a memory which is kept by allocator. - /// Will subsract it from RSS to decrease memory drift. + /// According to jemalloc man, pdirty is: + /// + /// Number of pages within unused extents that are potentially + /// dirty, and for which madvise() or similar has not been called. + /// + /// So they will be subtracted from RSS to make accounting more + /// accurate, since those pages are not really RSS but a memory + /// that can be used at anytime via jemalloc. free_memory_in_allocator_arenas = je_malloc_pdirty * getPageSize(); #endif - Int64 difference = rss - free_memory_in_allocator_arenas - amount; + Int64 difference = rss - amount; /// Log only if difference is high. This is for convenience. The threshold is arbitrary. if (difference >= 1048576 || difference <= -1048576) From 09fe9c3ed130cbf2f70bcf5018063d0593a17a6b Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Wed, 26 Oct 2022 16:07:56 +0000 Subject: [PATCH 044/152] Use {} in exceptions --- src/Functions/formatDateTime.cpp | 99 ++++++++++++++------------------ 1 file changed, 42 insertions(+), 57 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 4f28dae7a66..c5240abf7a1 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -317,44 +317,39 @@ public: if constexpr (support_integer) { if (arguments.size() != 1 && arguments.size() != 2 && arguments.size() != 3) - throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) - + ", should be 1, 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 1, 2 or 3", + getName(), arguments.size()); if (arguments.size() == 1 && !isInteger(arguments[0].type)) - throw Exception( - "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() - + " when arguments size is 1. Should be integer", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1 argument of function {} when arguments size is 1. Should be integer", + arguments[0].type->getName(), getName()); if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDate32(arguments[0].type) || isDateTime64(arguments[0].type))) - throw Exception( - "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() - + " when arguments size is 2 or 3. Should be a integer or a date with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1 argument of function {} when arguments size is 2 or 3. Should be a integer or a date with time", + arguments[0].type->getName(), getName()); } else { if (arguments.size() != 2 && arguments.size() != 3) - throw Exception( - "Number of arguments for function " + getName() + " doesn't match: passed " + toString(arguments.size()) - + ", should be 2 or 3", - ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, + "Number of arguments for function {} doesn't match: passed {}, should be 2 or 3", + getName(), arguments.size()); if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime64(arguments[0].type)) - throw Exception( - "Illegal type " + arguments[0].type->getName() + " of 1 argument of function " + getName() - + ". Should be a date or a date with time", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 1 argument of function {}. Should be a date or a date with time", + arguments[0].type->getName(), getName()); } if (arguments.size() == 2 && !WhichDataType(arguments[1].type).isString()) - throw Exception( - "Illegal type " + arguments[1].type->getName() + " of 2 argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 2 argument of function {}. Must be String.", + arguments[1].type->getName(), getName()); if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString()) - throw Exception( - "Illegal type " + arguments[2].type->getName() + " of 3 argument of function " + getName() + ". Must be String.", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT); + throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, + "Illegal type {} of 3 argument of function {}. Must be String.", + arguments[2].type->getName(), getName()); if (arguments.size() == 1) return std::make_shared(); @@ -375,10 +370,9 @@ public: return true; })) { - throw Exception( - "Illegal column " + arguments[0].column->getName() + " of function " + getName() - + ", must be Integer or DateTime when arguments size is 1.", - ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of function {}, must be Integer or DateTime when arguments size is 1.", + arguments[0].column->getName(), getName()); } } else @@ -387,10 +381,9 @@ public: { using FromDataType = std::decay_t; if (!(res = executeType(arguments, result_type))) - throw Exception( - "Illegal column " + arguments[0].column->getName() + " of function " + getName() - + ", must be Integer or DateTime.", - ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of function {}, must be Integer or DateTime.", + arguments[0].column->getName(), getName()); return true; })) { @@ -398,10 +391,9 @@ public: || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)))) - throw Exception( - "Illegal column " + arguments[0].column->getName() + " of function " + getName() - + ", must be Integer or DateTime.", - ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of function {}, must be Integer or DateTime.", + arguments[0].column->getName(), getName()); } } } @@ -411,10 +403,9 @@ public: || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)) || (res = executeType(arguments, result_type)))) - throw Exception( - "Illegal column " + arguments[0].column->getName() + " of function " + getName() - + ", must be Date or DateTime.", - ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of function {}, must be Date or DateTime.", + arguments[0].column->getName(), getName()); } return res; @@ -429,10 +420,9 @@ public: const ColumnConst * pattern_column = checkAndGetColumnConst(arguments[1].column.get()); if (!pattern_column) - throw Exception("Illegal column " + arguments[1].column->getName() - + " of second ('format') argument of function " + getName() - + ". Must be constant string.", - ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Illegal column {} of second ('format') argument of function {}. Must be constant string.", + arguments[1].column->getName(), getName()); String pattern = pattern_column->getValue(); @@ -500,13 +490,6 @@ public: instruction.perform(pos, static_cast(c.whole), time_zone); } } - else if constexpr (std::is_same_v) - { - for (auto & instruction : instructions) - { - instruction.perform(pos, static_cast(vec[i]), time_zone); - } - } else { for (auto & instruction : instructions) @@ -723,12 +706,14 @@ public: // Unimplemented case 'U': [[fallthrough]]; case 'W': - throw Exception("Wrong pattern '" + pattern + "', symbol '" + *pos + " is not implemented ' for function " + getName(), - ErrorCodes::NOT_IMPLEMENTED); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, + "Wrong pattern '{}', symbol '{}' is not implemented for function {}", + pattern, *pos, getName()); default: - throw Exception( - "Wrong pattern '" + pattern + "', unexpected symbol '" + *pos + "' for function " + getName(), ErrorCodes::ILLEGAL_COLUMN); + throw Exception(ErrorCodes::ILLEGAL_COLUMN, + "Wrong pattern '{}', unexpected symbol '{}' for function {}", + pattern, *pos, getName()); } ++pos; From f04f6638189bd4a0f9c9defdb43c66b3e8c5ef37 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Wed, 26 Oct 2022 18:55:14 +0200 Subject: [PATCH 045/152] Fixed tests --- src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index db1b9762668..1413f084536 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -361,7 +361,7 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( WhichDataType which(type); - if (which.isTuple() && function_name == "tuple") + if (which.isTuple() && key_node_function_name == "tuple") { const auto & tuple_column = typeid_cast(column.get()); const auto & tuple_data_type = typeid_cast(type.get()); @@ -388,6 +388,8 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( * We cannot skip keys that does not exist in map if comparison is with default type value because * that way we skip necessary granules where map key does not exists. */ + if (!prepared_set) + return false; auto default_column_to_check = type->createColumnConstWithDefaultValue(1)->convertToFullColumnIfConst(); ColumnWithTypeAndName default_column_with_type_to_check { default_column_to_check, type, "" }; From c8444f751f968ebd3b048d6edcc9d90c1c6ac02d Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Wed, 26 Oct 2022 14:07:10 -0400 Subject: [PATCH 046/152] Move SonarCloud Job to nightly Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 53 +++++++++++++++++++++++++++++ .github/workflows/sonar.yml | 64 ----------------------------------- 2 files changed, 53 insertions(+), 64 deletions(-) delete mode 100644 .github/workflows/sonar.yml diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 801f7eda94a..834f74822a1 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -122,3 +122,56 @@ jobs: docker ps --quiet | xargs --no-run-if-empty docker kill ||: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" + SonarCloud: + name: Sonar Cloud + runs-on: [self-hosted, builder] + env: + SONAR_SCANNER_VERSION: 4.7.0.2747 + SONAR_SERVER_URL: "https://sonarcloud.io" + BUILD_WRAPPER_OUT_DIR: build_wrapper_output_directory # Directory where build-wrapper output will be placed + steps: + - uses: actions/checkout@v2 + with: + fetch-depth: 0 # Shallow clones should be disabled for a better relevancy of analysis + submodules: true + - name: Set up JDK 11 + uses: actions/setup-java@v1 + with: + java-version: 11 + - name: Download and set up sonar-scanner + env: + SONAR_SCANNER_DOWNLOAD_URL: https://binaries.sonarsource.com/Distribution/sonar-scanner-cli/sonar-scanner-cli-${{ env.SONAR_SCANNER_VERSION }}-linux.zip + run: | + mkdir -p $HOME/.sonar + curl -sSLo $HOME/.sonar/sonar-scanner.zip ${{ env.SONAR_SCANNER_DOWNLOAD_URL }} + unzip -o $HOME/.sonar/sonar-scanner.zip -d $HOME/.sonar/ + echo "$HOME/.sonar/sonar-scanner-${{ env.SONAR_SCANNER_VERSION }}-linux/bin" >> $GITHUB_PATH + - name: Download and set up build-wrapper + env: + BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip + run: | + curl -sSLo $HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }} + unzip -o $HOME/.sonar/build-wrapper-linux-x86.zip -d $HOME/.sonar/ + echo "$HOME/.sonar/build-wrapper-linux-x86" >> $GITHUB_PATH + - name: Set Up Build Tools + run: | + sudo apt-get update + sudo apt-get install -yq git cmake ccache python3 ninja-build + sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" + - name: Run build-wrapper + run: | + mkdir build + cd build + cmake .. + cd .. + build-wrapper-linux-x86-64 --out-dir ${{ env.BUILD_WRAPPER_OUT_DIR }} cmake --build build/ + - name: Run sonar-scanner + env: + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + SONAR_TOKEN: ${{ secrets.SONAR_TOKEN }} + run: | + sonar-scanner \ + --define sonar.host.url="${{ env.SONAR_SERVER_URL }}" \ + --define sonar.cfamily.build-wrapper-output="${{ env.BUILD_WRAPPER_OUT_DIR }}" \ + --define sonar.projectKey="ClickHouse_ClickHouse" \ + --define sonar.organization="clickhouse-java" \ No newline at end of file diff --git a/.github/workflows/sonar.yml b/.github/workflows/sonar.yml deleted file mode 100644 index 316f1e90d40..00000000000 --- a/.github/workflows/sonar.yml +++ /dev/null @@ -1,64 +0,0 @@ -name: Sonar Cloud -on: - push: - branches: - - master - pull_request: - types: [opened, synchronize, reopened] -env: - CC: clang-15 - CXX: clang++-15 -jobs: - sonar_cloud: - name: Sonar Cloud - runs-on: [self-hosted, builder] - env: - SONAR_SCANNER_VERSION: 4.7.0.2747 - SONAR_SERVER_URL: "https://sonarcloud.io" - BUILD_WRAPPER_OUT_DIR: build_wrapper_output_directory # Directory where build-wrapper output will be placed - steps: - - uses: actions/checkout@v2 - with: - fetch-depth: 0 # Shallow clones should be disabled for a better relevancy of analysis - submodules: true - - name: Set up JDK 11 - uses: actions/setup-java@v1 - with: - java-version: 11 - - name: Download and set up sonar-scanner - env: - SONAR_SCANNER_DOWNLOAD_URL: https://binaries.sonarsource.com/Distribution/sonar-scanner-cli/sonar-scanner-cli-${{ env.SONAR_SCANNER_VERSION }}-linux.zip - run: | - mkdir -p $HOME/.sonar - curl -sSLo $HOME/.sonar/sonar-scanner.zip ${{ env.SONAR_SCANNER_DOWNLOAD_URL }} - unzip -o $HOME/.sonar/sonar-scanner.zip -d $HOME/.sonar/ - echo "$HOME/.sonar/sonar-scanner-${{ env.SONAR_SCANNER_VERSION }}-linux/bin" >> $GITHUB_PATH - - name: Download and set up build-wrapper - env: - BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip - run: | - curl -sSLo $HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }} - unzip -o $HOME/.sonar/build-wrapper-linux-x86.zip -d $HOME/.sonar/ - echo "$HOME/.sonar/build-wrapper-linux-x86" >> $GITHUB_PATH - - name: Set Up Build Tools - run: | - sudo apt-get update - sudo apt-get install -yq git cmake ccache python3 ninja-build - sudo bash -c "$(wget -O - https://apt.llvm.org/llvm.sh)" - - name: Run build-wrapper - run: | - mkdir build - cd build - cmake .. - cd .. - build-wrapper-linux-x86-64 --out-dir ${{ env.BUILD_WRAPPER_OUT_DIR }} cmake --build build/ - - name: Run sonar-scanner - env: - GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} - SONAR_TOKEN: ${{ secrets.SONAR_TOKEN }} - run: | - sonar-scanner \ - --define sonar.host.url="${{ env.SONAR_SERVER_URL }}" \ - --define sonar.cfamily.build-wrapper-output="${{ env.BUILD_WRAPPER_OUT_DIR }}" \ - --define sonar.projectKey="clickhouse-java" \ - --define sonar.organization="ClickHouse" From 488c2200466571ebe97dec9ac98925d6eb4a8f48 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Oct 2022 01:45:38 +0000 Subject: [PATCH 047/152] Fix bug in ParserFunction --- src/Parsers/ExpressionListParsers.cpp | 10 ++++++++++ .../02474_fix_function_parser_bug.reference | 0 .../0_stateless/02474_fix_function_parser_bug.sql | 1 + 3 files changed, 11 insertions(+) create mode 100644 tests/queries/0_stateless/02474_fix_function_parser_bug.reference create mode 100644 tests/queries/0_stateless/02474_fix_function_parser_bug.sql diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 4af4dabb12e..2f39162e104 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -830,6 +830,16 @@ public: explicit FunctionLayer(String function_name_, bool allow_function_parameters_ = true) : function_name(function_name_), allow_function_parameters(allow_function_parameters_){} + bool getResult(ASTPtr & node) override + { + // FunctionLayer can be the only layer in our Layers stack, + // so we need to check that we exited the main cycle properly + if (!finished) + return false; + + return Layer::getResult(node); + } + bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// | 0 | 1 | 2 | diff --git a/tests/queries/0_stateless/02474_fix_function_parser_bug.reference b/tests/queries/0_stateless/02474_fix_function_parser_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02474_fix_function_parser_bug.sql b/tests/queries/0_stateless/02474_fix_function_parser_bug.sql new file mode 100644 index 00000000000..12e9e03f151 --- /dev/null +++ b/tests/queries/0_stateless/02474_fix_function_parser_bug.sql @@ -0,0 +1 @@ +CREATE DATABASE conv_mian ENGINE QALL(COLUMNS('|T.D'),¸mp} -- { clientError 62 } From 0fc0e81c0c3428c865fa719c6a421dc3bcfeed37 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Thu, 8 Sep 2022 20:36:08 +0200 Subject: [PATCH 048/152] Add Element::type() to JSONParser --- src/Common/JSONParsers/DummyJSONParser.h | 2 ++ src/Common/JSONParsers/ElementTypes.h | 16 ++++++++++++++++ src/Common/JSONParsers/RapidJSONParser.h | 16 +++++++++++++++- src/Common/JSONParsers/SimdJSONParser.h | 17 ++++++++++++++++- 4 files changed, 49 insertions(+), 2 deletions(-) create mode 100644 src/Common/JSONParsers/ElementTypes.h diff --git a/src/Common/JSONParsers/DummyJSONParser.h b/src/Common/JSONParsers/DummyJSONParser.h index 3cedd59decd..50c112affe2 100644 --- a/src/Common/JSONParsers/DummyJSONParser.h +++ b/src/Common/JSONParsers/DummyJSONParser.h @@ -3,6 +3,7 @@ #include #include #include +#include "ElementTypes.h" namespace DB @@ -25,6 +26,7 @@ struct DummyJSONParser { public: Element() = default; + static ElementType type() { return ElementType::NULL_VALUE; } static bool isInt64() { return false; } static bool isUInt64() { return false; } static bool isDouble() { return false; } diff --git a/src/Common/JSONParsers/ElementTypes.h b/src/Common/JSONParsers/ElementTypes.h new file mode 100644 index 00000000000..0dcfa7115aa --- /dev/null +++ b/src/Common/JSONParsers/ElementTypes.h @@ -0,0 +1,16 @@ +#pragma once + +namespace DB +{ +// Enum values match simdjson's for fast conversion +enum class ElementType { + ARRAY = '[', + OBJECT = '{', + INT64 = 'l', + UINT64 = 'u', + DOUBLE = 'd', + STRING = '"', + BOOL = 't', + NULL_VALUE = 'n' +}; +} diff --git a/src/Common/JSONParsers/RapidJSONParser.h b/src/Common/JSONParsers/RapidJSONParser.h index 01730bc0692..16a87a0e0bf 100644 --- a/src/Common/JSONParsers/RapidJSONParser.h +++ b/src/Common/JSONParsers/RapidJSONParser.h @@ -6,7 +6,7 @@ # include # include # include - +# include "ElementTypes.h" namespace DB { @@ -26,6 +26,20 @@ struct RapidJSONParser ALWAYS_INLINE Element() = default; ALWAYS_INLINE Element(const rapidjson::Value & value_) : ptr(&value_) {} /// NOLINT + ALWAYS_INLINE ElementType type() const { + switch (ptr->GetType()) { + case rapidjson::kNumberType: return ptr->IsDouble() ? ElementType::DOUBLE : (ptr->IsUint64() ? ElementType::UINT64 : ElementType::INT64); + case rapidjson::kStringType: return ElementType::STRING; + case rapidjson::kArrayType: return ElementType::ARRAY; + case rapidjson::kObjectType: return ElementType::OBJECT; + case rapidjson::kTrueType: return ElementType::BOOL; + case rapidjson::kFalseType: return ElementType::BOOL; + case rapidjson::kNullType: + default: + return ElementType::NULL_VALUE; + } + } + ALWAYS_INLINE bool isInt64() const { return ptr->IsInt64(); } ALWAYS_INLINE bool isUInt64() const { return ptr->IsUint64(); } ALWAYS_INLINE bool isDouble() const { return ptr->IsDouble(); } diff --git a/src/Common/JSONParsers/SimdJSONParser.h b/src/Common/JSONParsers/SimdJSONParser.h index 14eb3cd6d78..dd2077ba768 100644 --- a/src/Common/JSONParsers/SimdJSONParser.h +++ b/src/Common/JSONParsers/SimdJSONParser.h @@ -7,7 +7,7 @@ # include # include # include - +# include "ElementTypes.h" namespace DB { @@ -31,6 +31,21 @@ struct SimdJSONParser ALWAYS_INLINE Element() {} /// NOLINT ALWAYS_INLINE Element(const simdjson::dom::element & element_) : element(element_) {} /// NOLINT + ALWAYS_INLINE ElementType type() const { + switch (element.type()) { + case simdjson::dom::element_type::INT64: return ElementType::INT64; + case simdjson::dom::element_type::UINT64: return ElementType::UINT64; + case simdjson::dom::element_type::DOUBLE: return ElementType::DOUBLE; + case simdjson::dom::element_type::STRING: return ElementType::STRING; + case simdjson::dom::element_type::ARRAY: return ElementType::ARRAY; + case simdjson::dom::element_type::OBJECT: return ElementType::OBJECT; + case simdjson::dom::element_type::BOOL: return ElementType::BOOL; + case simdjson::dom::element_type::NULL_VALUE: + default: + return ElementType::NULL_VALUE; + } + } + ALWAYS_INLINE bool isInt64() const { return element.type() == simdjson::dom::element_type::INT64; } ALWAYS_INLINE bool isUInt64() const { return element.type() == simdjson::dom::element_type::UINT64; } ALWAYS_INLINE bool isDouble() const { return element.type() == simdjson::dom::element_type::DOUBLE; } From c3bcb1899df9dee8d6a98bfddec9414c3feed247 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Thu, 8 Sep 2022 20:36:21 +0200 Subject: [PATCH 049/152] Add convenience ctor to ReadBufferFromMemory --- src/IO/ReadBufferFromMemory.h | 2 ++ 1 file changed, 2 insertions(+) diff --git a/src/IO/ReadBufferFromMemory.h b/src/IO/ReadBufferFromMemory.h index dc5c464604b..ad96e4bfa28 100644 --- a/src/IO/ReadBufferFromMemory.h +++ b/src/IO/ReadBufferFromMemory.h @@ -16,6 +16,8 @@ public: requires (sizeof(CharT) == 1) ReadBufferFromMemory(const CharT * buf, size_t size) : SeekableReadBuffer(const_cast(reinterpret_cast(buf)), size, 0) {} + explicit ReadBufferFromMemory(const std::string_view&& str) + : SeekableReadBuffer(const_cast(str.data()), str.size(), 0) {} off_t seek(off_t off, int whence) override; From 40b572c23b317a0c9a40db0733b85bacc6b0a128 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Fri, 9 Sep 2022 16:52:39 +0200 Subject: [PATCH 050/152] Remove redundant default case in switch statement --- src/Common/JSONParsers/RapidJSONParser.h | 4 +--- src/Common/JSONParsers/SimdJSONParser.h | 4 +--- 2 files changed, 2 insertions(+), 6 deletions(-) diff --git a/src/Common/JSONParsers/RapidJSONParser.h b/src/Common/JSONParsers/RapidJSONParser.h index 16a87a0e0bf..4a0035c8360 100644 --- a/src/Common/JSONParsers/RapidJSONParser.h +++ b/src/Common/JSONParsers/RapidJSONParser.h @@ -34,9 +34,7 @@ struct RapidJSONParser case rapidjson::kObjectType: return ElementType::OBJECT; case rapidjson::kTrueType: return ElementType::BOOL; case rapidjson::kFalseType: return ElementType::BOOL; - case rapidjson::kNullType: - default: - return ElementType::NULL_VALUE; + case rapidjson::kNullType: return ElementType::NULL_VALUE; } } diff --git a/src/Common/JSONParsers/SimdJSONParser.h b/src/Common/JSONParsers/SimdJSONParser.h index dd2077ba768..2af0e1d6c01 100644 --- a/src/Common/JSONParsers/SimdJSONParser.h +++ b/src/Common/JSONParsers/SimdJSONParser.h @@ -40,9 +40,7 @@ struct SimdJSONParser case simdjson::dom::element_type::ARRAY: return ElementType::ARRAY; case simdjson::dom::element_type::OBJECT: return ElementType::OBJECT; case simdjson::dom::element_type::BOOL: return ElementType::BOOL; - case simdjson::dom::element_type::NULL_VALUE: - default: - return ElementType::NULL_VALUE; + case simdjson::dom::element_type::NULL_VALUE: return ElementType::NULL_VALUE; } } From 1ca230ff6d8a578aa60a8298c92fd129cfa57f4c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Mon, 19 Sep 2022 14:00:48 +0200 Subject: [PATCH 051/152] Allow coercing in JSONExtract* functions --- src/Common/JSONParsers/ElementTypes.h | 3 +- src/Common/JSONParsers/RapidJSONParser.h | 6 +- src/Common/JSONParsers/SimdJSONParser.h | 6 +- src/Functions/FunctionsJSON.cpp | 179 +++++++++++++----- src/IO/readDecimalText.h | 41 ++-- .../00700_to_decimal_or_something.reference | 4 +- .../00918_json_functions.reference | 36 ++++ .../0_stateless/00918_json_functions.sql | 36 ++++ .../01186_conversion_to_nullable.reference | 2 +- ...685_json_extract_double_as_float.reference | 4 +- .../02013_json_function_null_column.reference | 3 +- .../02013_json_function_null_column.sql | 1 + 12 files changed, 239 insertions(+), 82 deletions(-) diff --git a/src/Common/JSONParsers/ElementTypes.h b/src/Common/JSONParsers/ElementTypes.h index 0dcfa7115aa..44e4c850a2f 100644 --- a/src/Common/JSONParsers/ElementTypes.h +++ b/src/Common/JSONParsers/ElementTypes.h @@ -3,7 +3,8 @@ namespace DB { // Enum values match simdjson's for fast conversion -enum class ElementType { +enum class ElementType +{ ARRAY = '[', OBJECT = '{', INT64 = 'l', diff --git a/src/Common/JSONParsers/RapidJSONParser.h b/src/Common/JSONParsers/RapidJSONParser.h index 4a0035c8360..6c5ea938bfe 100644 --- a/src/Common/JSONParsers/RapidJSONParser.h +++ b/src/Common/JSONParsers/RapidJSONParser.h @@ -26,8 +26,10 @@ struct RapidJSONParser ALWAYS_INLINE Element() = default; ALWAYS_INLINE Element(const rapidjson::Value & value_) : ptr(&value_) {} /// NOLINT - ALWAYS_INLINE ElementType type() const { - switch (ptr->GetType()) { + ALWAYS_INLINE ElementType type() const + { + switch (ptr->GetType()) + { case rapidjson::kNumberType: return ptr->IsDouble() ? ElementType::DOUBLE : (ptr->IsUint64() ? ElementType::UINT64 : ElementType::INT64); case rapidjson::kStringType: return ElementType::STRING; case rapidjson::kArrayType: return ElementType::ARRAY; diff --git a/src/Common/JSONParsers/SimdJSONParser.h b/src/Common/JSONParsers/SimdJSONParser.h index 2af0e1d6c01..f0f8f91109f 100644 --- a/src/Common/JSONParsers/SimdJSONParser.h +++ b/src/Common/JSONParsers/SimdJSONParser.h @@ -31,8 +31,10 @@ struct SimdJSONParser ALWAYS_INLINE Element() {} /// NOLINT ALWAYS_INLINE Element(const simdjson::dom::element & element_) : element(element_) {} /// NOLINT - ALWAYS_INLINE ElementType type() const { - switch (element.type()) { + ALWAYS_INLINE ElementType type() const + { + switch (element.type()) + { case simdjson::dom::element_type::INT64: return ElementType::INT64; case simdjson::dom::element_type::UINT64: return ElementType::UINT64; case simdjson::dom::element_type::DOUBLE: return ElementType::DOUBLE; diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 493fc36ca3c..7c221209071 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -25,7 +25,6 @@ #include #include #include -#include #include #include #include @@ -40,6 +39,7 @@ #include #include +#include #include @@ -623,24 +623,32 @@ public: static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { UInt8 type; - if (element.isInt64()) - type = 'i'; - else if (element.isUInt64()) - type = 'u'; - else if (element.isDouble()) - type = 'd'; - else if (element.isBool()) - type = 'b'; - else if (element.isString()) - type = '"'; - else if (element.isArray()) - type = '['; - else if (element.isObject()) - type = '{'; - else if (element.isNull()) - type = 0; - else - return false; + switch (element.type()) + { + case ElementType::INT64: + type = 'i'; + break; + case ElementType::UINT64: + type = 'u'; + break; + case ElementType::DOUBLE: + type = 'd'; + break; + case ElementType::STRING: + type = '"'; + break; + case ElementType::ARRAY: + type = '['; + break; + case ElementType::OBJECT: + type = '{'; + break; + case ElementType::NULL_VALUE: + type = 0; + break; + default: + return false; + } ColumnVector & col_vec = assert_cast &>(dest); col_vec.insertValue(type); @@ -666,34 +674,51 @@ public: { NumberType value; - if (element.isInt64()) + switch (element.type()) { - if (!accurate::convertNumeric(element.getInt64(), value)) + case ElementType::DOUBLE: + if constexpr (std::is_floating_point_v) + { + /// We permit inaccurate conversion of double to float. + /// Example: double 0.1 from JSON is not representable in float. + /// But it will be more convenient for user to perform conversion. + value = static_cast(element.getDouble()); + } + else if (!accurate::convertNumeric(element.getDouble(), value)) + return false; + break; + case ElementType::UINT64: + if (!accurate::convertNumeric(element.getUInt64(), value)) + return false; + break; + case ElementType::INT64: + if (!accurate::convertNumeric(element.getInt64(), value)) + return false; + break; + case ElementType::BOOL: + if constexpr (is_integer && convert_bool_to_integer) + { + value = static_cast(element.getBool()); + break; + } return false; - } - else if (element.isUInt64()) - { - if (!accurate::convertNumeric(element.getUInt64(), value)) - return false; - } - else if (element.isDouble()) - { - if constexpr (std::is_floating_point_v) - { - /// We permit inaccurate conversion of double to float. - /// Example: double 0.1 from JSON is not representable in float. - /// But it will be more convenient for user to perform conversion. - value = static_cast(element.getDouble()); + case ElementType::STRING: { + auto rb = ReadBufferFromMemory{element.getString()}; + if constexpr (std::is_floating_point_v) + { + if (!tryReadFloatText(value, rb) || !rb.eof()) + return false; + } + else + { + if (!tryReadIntText(value, rb) || !rb.eof()) + return false; + } + break; } - else if (!accurate::convertNumeric(element.getDouble(), value)) + default: return false; } - else if (element.isBool() && is_integer && convert_bool_to_integer) - { - value = static_cast(element.getBool()); - } - else - return false; auto & col_vec = assert_cast &>(dest); col_vec.insertValue(value); @@ -719,9 +744,25 @@ using JSONExtractInt64Impl = JSONExtractNumericImpl; template using JSONExtractUInt64Impl = JSONExtractNumericImpl; template +using JSONExtractInt128Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt128Impl = JSONExtractNumericImpl; +template +using JSONExtractInt256Impl = JSONExtractNumericImpl; +template +using JSONExtractUInt256Impl = JSONExtractNumericImpl; +template using JSONExtractFloat32Impl = JSONExtractNumericImpl; template using JSONExtractFloat64Impl = JSONExtractNumericImpl; +template +using JSONExtractDecimal32Impl = JSONExtractNumericImpl; +template +using JSONExtractDecimal64Impl = JSONExtractNumericImpl; +template +using JSONExtractDecimal128Impl = JSONExtractNumericImpl; +template +using JSONExtractDecimal256Impl = JSONExtractNumericImpl; template @@ -739,11 +780,22 @@ public: static bool insertResultToColumn(IColumn & dest, const Element & element, std::string_view) { - if (!element.isBool()) - return false; + bool value; + switch (element.type()) + { + case ElementType::BOOL: + value = element.getBool(); + break; + case ElementType::INT64: + case ElementType::UINT64: + value = element.getUInt64() != 0; + break; + default: + return false; + } auto & col_vec = assert_cast &>(dest); - col_vec.insertValue(static_cast(element.getBool())); + col_vec.insertValue(static_cast(value)); return true; } }; @@ -845,12 +897,35 @@ struct JSONExtractTree explicit DecimalNode(DataTypePtr data_type_) : data_type(data_type_) {} bool insertResultToColumn(IColumn & dest, const Element & element) override { - if (!element.isDouble()) - return false; - const auto * type = assert_cast *>(data_type.get()); - auto result = convertToDecimal, DataTypeDecimal>(element.getDouble(), type->getScale()); - assert_cast &>(dest).insert(result); + + DecimalType value{}; + + switch (element.type()) + { + case ElementType::DOUBLE: + value = convertToDecimal, DataTypeDecimal>( + element.getDouble(), type->getScale()); + break; + case ElementType::UINT64: + value = convertToDecimal, DataTypeDecimal>( + element.getUInt64(), type->getScale()); + break; + case ElementType::INT64: + value = convertToDecimal, DataTypeDecimal>( + element.getInt64(), type->getScale()); + break; + case ElementType::STRING: { + auto rb = ReadBufferFromMemory{element.getString()}; + if (!SerializationDecimal::tryReadText(value, rb, DecimalUtils::max_precision, type->getScale())) + return false; + break; + } + default: + return false; + } + + assert_cast &>(dest).insert(value); return true; } private: @@ -1088,10 +1163,14 @@ struct JSONExtractTree case TypeIndex::UInt16: return std::make_unique>(); case TypeIndex::UInt32: return std::make_unique>(); case TypeIndex::UInt64: return std::make_unique>(); + case TypeIndex::UInt128: return std::make_unique>(); + case TypeIndex::UInt256: return std::make_unique>(); case TypeIndex::Int8: return std::make_unique>(); case TypeIndex::Int16: return std::make_unique>(); case TypeIndex::Int32: return std::make_unique>(); case TypeIndex::Int64: return std::make_unique>(); + case TypeIndex::Int128: return std::make_unique>(); + case TypeIndex::Int256: return std::make_unique>(); case TypeIndex::Float32: return std::make_unique>(); case TypeIndex::Float64: return std::make_unique>(); case TypeIndex::String: return std::make_unique(); diff --git a/src/IO/readDecimalText.h b/src/IO/readDecimalText.h index 64374a20574..054990a8c7e 100644 --- a/src/IO/readDecimalText.h +++ b/src/IO/readDecimalText.h @@ -147,23 +147,30 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp return true; } -template -inline void readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_t & scale, bool digits_only = false) +template +inline bool readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_t & scale, bool digits_only = false) { uint32_t digits = precision; int32_t exponent; - readDigits(buf, x, digits, exponent, digits_only); + auto ok = readDigits<_throw_on_error>(buf, x, digits, exponent, digits_only); + + if (!_throw_on_error && !ok) + return false; if (static_cast(digits) + exponent > static_cast(precision - scale)) { - static constexpr const char * pattern = - "Decimal value is too big: {} digits were read: {}e{}." - " Expected to read decimal with scale {} and precision {}"; + if constexpr (_throw_on_error) + { + static constexpr const char * pattern = "Decimal value is too big: {} digits were read: {}e{}." + " Expected to read decimal with scale {} and precision {}"; - if constexpr (is_big_int_v) - throw Exception(fmt::format(pattern, digits, x.value, exponent, scale, precision), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + if constexpr (is_big_int_v) + throw Exception(fmt::format(pattern, digits, x.value, exponent, scale, precision), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + else + throw Exception(fmt::format(pattern, digits, x, exponent, scale, precision), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + } else - throw Exception(fmt::format(pattern, digits, x, exponent, scale, precision), ErrorCodes::ARGUMENT_OUT_OF_BOUND); + return false; } if (static_cast(scale) + exponent < 0) @@ -175,7 +182,7 @@ inline void readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_ /// Too big negative exponent x.value = 0; scale = 0; - return; + return true; } else { @@ -184,26 +191,18 @@ inline void readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_ assert(divisor > 0); /// This is for Clang Static Analyzer. It is not smart enough to infer it automatically. x.value /= divisor; scale = 0; - return; + return true; } } scale += exponent; + return true; } template inline bool tryReadDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_t & scale) { - uint32_t digits = precision; - int32_t exponent; - - if (!readDigits(buf, x, digits, exponent, true) || - static_cast(digits) + exponent > static_cast(precision - scale) || - static_cast(scale) + exponent < 0) - return false; - - scale += exponent; - return true; + return readDecimalText(buf, x, precision, scale, true); } template diff --git a/tests/queries/0_stateless/00700_to_decimal_or_something.reference b/tests/queries/0_stateless/00700_to_decimal_or_something.reference index 89ded7bd6d4..dec36ed5df5 100644 --- a/tests/queries/0_stateless/00700_to_decimal_or_something.reference +++ b/tests/queries/0_stateless/00700_to_decimal_or_something.reference @@ -1,5 +1,5 @@ 1.1 1.1 1.1 -0 +1 0 0.42 0 0.42 0 0.42 @@ -13,7 +13,7 @@ 0 ---- 1.1 1.1 1.1 -\N +1 \N -0.42 \N -0.42 \N -0.42 diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 8e6fc3914e0..7d925c1236d 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -61,11 +61,47 @@ Friday (1,'417ddc5d-e556-4d27-95dd-a34d84e46a50') hello (3333.6,'test') +(3333.6,'test') +(3333.6333333333,'test') (3333.6333333333,'test') 123456.1234 Decimal(20, 4) +123456.1234 Decimal(20, 4) +123456789012345.12 Decimal(30, 4) +(1234567890.1234567890123456789,'test') Tuple(a Decimal(35, 20), b LowCardinality(String)) +(1234567890.12345678901234567890123456789,'test') Tuple(a Decimal(45, 30), b LowCardinality(String)) 123456789012345.1136 123456789012345.1136 1234567890.12345677879616925706 (1234567890.12345677879616925706,'test') 1234567890.123456695758468374595199311875 (1234567890.123456695758468374595199311875,'test') +-1234567890 Int32 +1234567890 UInt32 +-1234567890123456789 Int64 +1234567890123456789 UInt64 +-1234567890123456789 Int128 +1234567890123456789 UInt128 +-1234567890123456789 Int256 +1234567890123456789 UInt256 +-123456789 Int32 +123456789 UInt32 +-123456789012 Int64 +123456789012 UInt64 +-123456789012 Int128 +123456789012 UInt128 +-123456789012 Int256 +123456789012 UInt256 +-123456789 Int32 +123456789 UInt32 +-1234567890123456789 Int64 +1234567890123456789 UInt64 +-12345678901234567890123456789012345678 Int128 +12345678901234567890123456789012345678 UInt128 +-11345678901234567890123456789012345678901234567890123456789012345678901234567 Int256 +11345678901234567890123456789012345678901234567890123456789012345678901234567 UInt256 +0 Int32 +0 UInt32 +0 Int64 +0 UInt64 +false Bool +true Bool --JSONExtractKeysAndValues-- [('a','hello'),('b','[-100,200,300]')] [('b',[-100,200,300])] diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index 87682587c8e..d614d507dda 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -72,11 +72,47 @@ SELECT JSONExtract('{"a":123456, "b":3.55}', 'Tuple(a LowCardinality(Int32), b D SELECT JSONExtract('{"a":1, "b":"417ddc5d-e556-4d27-95dd-a34d84e46a50"}', 'Tuple(a Int8, b UUID)'); SELECT JSONExtract('{"a": "hello", "b": [-100, 200.0, 300]}', 'a', 'LowCardinality(String)'); SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Decimal(10,1), b LowCardinality(String))'); +SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(10,1), b LowCardinality(String))'); SELECT JSONExtract('{"a":3333.6333333333333333333333, "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); +SELECT JSONExtract('{"a":"3333.6333333333333333333333", "b":"test"}', 'Tuple(a Decimal(20,10), b LowCardinality(String))'); SELECT JSONExtract('{"a":123456.123456}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); +SELECT JSONExtract('{"a":"123456.123456"}', 'a', 'Decimal(20, 4)') as a, toTypeName(a); +SELECT JSONExtract('{"a":"123456789012345.12"}', 'a', 'Decimal(30, 4)') as a, toTypeName(a); +SELECT JSONExtract('{"a":"1234567890.12345678901234567890", "b":"test"}', 'Tuple(a Decimal(35,20), b LowCardinality(String))') as a, toTypeName(a); +SELECT JSONExtract('{"a":"1234567890.123456789012345678901234567890", "b":"test"}', 'Tuple(a Decimal(45,30), b LowCardinality(String))') as a, toTypeName(a); SELECT toDecimal64(123456789012345.12, 4), JSONExtract('{"a":123456789012345.12}', 'a', 'Decimal(30, 4)'); SELECT toDecimal128(1234567890.12345678901234567890, 20), JSONExtract('{"a":1234567890.12345678901234567890, "b":"test"}', 'Tuple(a Decimal(35,20), b LowCardinality(String))'); SELECT toDecimal256(1234567890.123456789012345678901234567890, 30), JSONExtract('{"a":1234567890.12345678901234567890, "b":"test"}', 'Tuple(a Decimal(45,30), b LowCardinality(String))'); +SELECT JSONExtract('{"a":-1234567890}', 'a', 'Int32') as a, toTypeName(a); +SELECT JSONExtract('{"a":1234567890}', 'a', 'UInt32') as a, toTypeName(a); +SELECT JSONExtract('{"a":-1234567890123456789}', 'a', 'Int64') as a, toTypeName(a); +SELECT JSONExtract('{"a":1234567890123456789}', 'a', 'UInt64') as a, toTypeName(a); +SELECT JSONExtract('{"a":-1234567890123456789}', 'a', 'Int128') as a, toTypeName(a); +SELECT JSONExtract('{"a":1234567890123456789}', 'a', 'UInt128') as a, toTypeName(a); +SELECT JSONExtract('{"a":-1234567890123456789}', 'a', 'Int256') as a, toTypeName(a); +SELECT JSONExtract('{"a":1234567890123456789}', 'a', 'UInt256') as a, toTypeName(a); +SELECT JSONExtract('{"a":-123456789.345}', 'a', 'Int32') as a, toTypeName(a); +SELECT JSONExtract('{"a":123456789.345}', 'a', 'UInt32') as a, toTypeName(a); +SELECT JSONExtract('{"a":-123456789012.345}', 'a', 'Int64') as a, toTypeName(a); +SELECT JSONExtract('{"a":123456789012.345}', 'a', 'UInt64') as a, toTypeName(a); +SELECT JSONExtract('{"a":-123456789012.345}', 'a', 'Int128') as a, toTypeName(a); +SELECT JSONExtract('{"a":123456789012.345}', 'a', 'UInt128') as a, toTypeName(a); +SELECT JSONExtract('{"a":-123456789012.345}', 'a', 'Int256') as a, toTypeName(a); +SELECT JSONExtract('{"a":123456789012.345}', 'a', 'UInt256') as a, toTypeName(a); +SELECT JSONExtract('{"a":"-123456789"}', 'a', 'Int32') as a, toTypeName(a); +SELECT JSONExtract('{"a":"123456789"}', 'a', 'UInt32') as a, toTypeName(a); +SELECT JSONExtract('{"a":"-1234567890123456789"}', 'a', 'Int64') as a, toTypeName(a); +SELECT JSONExtract('{"a":"1234567890123456789"}', 'a', 'UInt64') as a, toTypeName(a); +SELECT JSONExtract('{"a":"-12345678901234567890123456789012345678"}', 'a', 'Int128') as a, toTypeName(a); +SELECT JSONExtract('{"a":"12345678901234567890123456789012345678"}', 'a', 'UInt128') as a, toTypeName(a); +SELECT JSONExtract('{"a":"-11345678901234567890123456789012345678901234567890123456789012345678901234567"}', 'a', 'Int256') as a, toTypeName(a); +SELECT JSONExtract('{"a":"11345678901234567890123456789012345678901234567890123456789012345678901234567"}', 'a', 'UInt256') as a, toTypeName(a); +SELECT JSONExtract('{"a":"-1234567899999"}', 'a', 'Int32') as a, toTypeName(a); +SELECT JSONExtract('{"a":"1234567899999"}', 'a', 'UInt32') as a, toTypeName(a); +SELECT JSONExtract('{"a":"-1234567890123456789999"}', 'a', 'Int64') as a, toTypeName(a); +SELECT JSONExtract('{"a":"1234567890123456789999"}', 'a', 'UInt64') as a, toTypeName(a); +SELECT JSONExtract('{"a":0}', 'a', 'Bool') as a, toTypeName(a); +SELECT JSONExtract('{"a":1}', 'a', 'Bool') as a, toTypeName(a); SELECT '--JSONExtractKeysAndValues--'; SELECT JSONExtractKeysAndValues('{"a": "hello", "b": [-100, 200.0, 300]}', 'String'); diff --git a/tests/queries/0_stateless/01186_conversion_to_nullable.reference b/tests/queries/0_stateless/01186_conversion_to_nullable.reference index 86fa0afff20..e4c1fd7c40b 100644 --- a/tests/queries/0_stateless/01186_conversion_to_nullable.reference +++ b/tests/queries/0_stateless/01186_conversion_to_nullable.reference @@ -26,7 +26,7 @@ \N 42 \N -\N +3.14 42 \N 3.14159 diff --git a/tests/queries/0_stateless/01685_json_extract_double_as_float.reference b/tests/queries/0_stateless/01685_json_extract_double_as_float.reference index f3f4206b425..a24f6569f44 100644 --- a/tests/queries/0_stateless/01685_json_extract_double_as_float.reference +++ b/tests/queries/0_stateless/01685_json_extract_double_as_float.reference @@ -1,7 +1,7 @@ 1.1 1.1 1.1 1.1 0.01 0.01 0.01 0.01 -0 -\N +1 +1 -1e300 -inf 0 diff --git a/tests/queries/0_stateless/02013_json_function_null_column.reference b/tests/queries/0_stateless/02013_json_function_null_column.reference index b580986f548..ab702ab52cb 100644 --- a/tests/queries/0_stateless/02013_json_function_null_column.reference +++ b/tests/queries/0_stateless/02013_json_function_null_column.reference @@ -11,7 +11,7 @@ b c 1 -0 +1 1 a 1 @@ -20,3 +20,4 @@ true 1 a \N +\N diff --git a/tests/queries/0_stateless/02013_json_function_null_column.sql b/tests/queries/0_stateless/02013_json_function_null_column.sql index ef5ea3e1320..bf680dfe0b5 100644 --- a/tests/queries/0_stateless/02013_json_function_null_column.sql +++ b/tests/queries/0_stateless/02013_json_function_null_column.sql @@ -23,3 +23,4 @@ SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Bool)'); SELECT JSONExtract('[1]', toNullable(1), 'Nullable(Float)'); SELECT JSONExtract('["a"]', toNullable(1), 'Nullable(String)'); SELECT JSONExtract('["a"]', toNullable(1), 'Nullable(Int)'); +SELECT JSONExtract('["-a"]', toNullable(1), 'Nullable(Int)'); From aaf71d157ddae80941487299ef807259e97da30f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Wed, 26 Oct 2022 21:44:17 +0200 Subject: [PATCH 052/152] Use return type template parameter to select throwing behavior --- src/IO/readDecimalText.h | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/src/IO/readDecimalText.h b/src/IO/readDecimalText.h index 054990a8c7e..9d7f8137136 100644 --- a/src/IO/readDecimalText.h +++ b/src/IO/readDecimalText.h @@ -147,19 +147,21 @@ inline bool readDigits(ReadBuffer & buf, T & x, uint32_t & digits, int32_t & exp return true; } -template -inline bool readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_t & scale, bool digits_only = false) +template +inline ReturnType readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_t & scale, bool digits_only = false) { + static constexpr bool throw_exception = std::is_same_v; + uint32_t digits = precision; int32_t exponent; - auto ok = readDigits<_throw_on_error>(buf, x, digits, exponent, digits_only); + auto ok = readDigits(buf, x, digits, exponent, digits_only); - if (!_throw_on_error && !ok) - return false; + if (!throw_exception && !ok) + return ReturnType(false); if (static_cast(digits) + exponent > static_cast(precision - scale)) { - if constexpr (_throw_on_error) + if constexpr (throw_exception) { static constexpr const char * pattern = "Decimal value is too big: {} digits were read: {}e{}." " Expected to read decimal with scale {} and precision {}"; @@ -170,7 +172,7 @@ inline bool readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_ throw Exception(fmt::format(pattern, digits, x, exponent, scale, precision), ErrorCodes::ARGUMENT_OUT_OF_BOUND); } else - return false; + return ReturnType(false); } if (static_cast(scale) + exponent < 0) @@ -182,7 +184,7 @@ inline bool readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_ /// Too big negative exponent x.value = 0; scale = 0; - return true; + return ReturnType(true); } else { @@ -191,18 +193,18 @@ inline bool readDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_ assert(divisor > 0); /// This is for Clang Static Analyzer. It is not smart enough to infer it automatically. x.value /= divisor; scale = 0; - return true; + return ReturnType(true); } } scale += exponent; - return true; + return ReturnType(true); } template inline bool tryReadDecimalText(ReadBuffer & buf, T & x, uint32_t precision, uint32_t & scale) { - return readDecimalText(buf, x, precision, scale, true); + return readDecimalText(buf, x, precision, scale, true); } template From ec1389cbe7514b0ae4c00b8de228fee682cb573a Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Thu, 27 Oct 2022 10:30:00 +0200 Subject: [PATCH 053/152] Update src/Common/Throttler.cpp Co-authored-by: Yakov Olkhovskiy <99031427+yakov-olkhovskiy@users.noreply.github.com> --- src/Common/Throttler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index a3ae966c49b..e0f2b63e8ae 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -65,7 +65,7 @@ void Throttler::add(size_t amount) /// Wait unless there is positive amount of tokens - throttling if (max_speed && tokens_value < 0) { - int64_t sleep_time = static_cast(-tokens_value * max_speed * NS); + int64_t sleep_time = static_cast(-tokens_value / max_speed * NS); accumulated_sleep += sleep_time; sleepForNanoseconds(sleep_time); accumulated_sleep -= sleep_time; From accf78f1fffae8b04ec14f0586e7038f94e59d51 Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 27 Oct 2022 08:39:08 +0000 Subject: [PATCH 054/152] fix the first add() call --- src/Common/Throttler.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index e0f2b63e8ae..169262d30c7 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -48,9 +48,9 @@ void Throttler::add(size_t amount) { std::lock_guard lock(mutex); auto now = clock_gettime_ns_adjusted(prev_ns); - if (max_speed && prev_ns != 0) + if (max_speed) { - double delta_seconds = static_cast(now - prev_ns) / NS; + double delta_seconds = prev_ns ? static_cast(now - prev_ns) / NS : 0; tokens = std::min(tokens + max_speed * delta_seconds - amount, max_burst); } count += amount; From 275d076cb720b132572210fdf3bfe1525b793cc9 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Thu, 27 Oct 2022 11:26:53 +0200 Subject: [PATCH 055/152] Fixed style check --- src/Analyzer/UnionNode.cpp | 1 + 1 file changed, 1 insertion(+) diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 039077bd08a..52ad0102d7c 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -30,6 +30,7 @@ namespace DB namespace ErrorCodes { extern const int TYPE_MISMATCH; + extern const int BAD_ARGUMENTS; } UnionNode::UnionNode(SelectUnionMode union_mode_) From b2ab692d87ec8a8fac20b7d8cc92653504bb6aaf Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Oct 2022 10:03:03 +0000 Subject: [PATCH 056/152] Safer getResult() --- src/Parsers/ExpressionListParsers.cpp | 252 +++++++++++++------------- 1 file changed, 128 insertions(+), 124 deletions(-) diff --git a/src/Parsers/ExpressionListParsers.cpp b/src/Parsers/ExpressionListParsers.cpp index 2f39162e104..c362340d013 100644 --- a/src/Parsers/ExpressionListParsers.cpp +++ b/src/Parsers/ExpressionListParsers.cpp @@ -561,13 +561,10 @@ public: virtual bool getResult(ASTPtr & node) { - if (elements.size() == 1) - { - node = std::move(elements[0]); - return true; - } + if (!finished) + return false; - return false; + return getResultImpl(node); } virtual bool parse(IParser::Pos & /*pos*/, Expected & /*expected*/, Action & /*action*/) = 0; @@ -746,6 +743,17 @@ public: Checkpoint current_checkpoint = Checkpoint::None; protected: + virtual bool getResultImpl(ASTPtr & node) + { + if (elements.size() == 1) + { + node = std::move(elements[0]); + return true; + } + + return false; + } + std::vector operators; ASTs operands; ASTs elements; @@ -766,17 +774,12 @@ public: bool getResult(ASTPtr & node) override { /// We can exit the main cycle outside the parse() function, - /// so we need to merge the element here + /// so we need to merge the element here. + /// Because of this 'finished' flag can also not be set. if (!mergeElement()) return false; - if (elements.size() == 1) - { - node = std::move(elements[0]); - return true; - } - - return false; + return Layer::getResultImpl(node); } bool parse(IParser::Pos & pos, Expected & /*expected*/, Action & /*action*/) override @@ -830,16 +833,6 @@ public: explicit FunctionLayer(String function_name_, bool allow_function_parameters_ = true) : function_name(function_name_), allow_function_parameters(allow_function_parameters_){} - bool getResult(ASTPtr & node) override - { - // FunctionLayer can be the only layer in our Layers stack, - // so we need to check that we exited the main cycle properly - if (!finished) - return false; - - return Layer::getResult(node); - } - bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// | 0 | 1 | 2 | @@ -1039,17 +1032,6 @@ private: class RoundBracketsLayer : public Layer { public: - bool getResult(ASTPtr & node) override - { - // Round brackets can mean priority operator as well as function tuple() - if (!is_tuple && elements.size() == 1) - node = std::move(elements[0]); - else - node = makeASTFunction("tuple", std::move(elements)); - - return true; - } - bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { if (ParserToken(TokenType::Comma).ignore(pos, expected)) @@ -1079,6 +1061,19 @@ public: return true; } + +protected: + bool getResultImpl(ASTPtr & node) override + { + // Round brackets can mean priority operator as well as function tuple() + if (!is_tuple && elements.size() == 1) + node = std::move(elements[0]); + else + node = makeASTFunction("tuple", std::move(elements)); + + return true; + } + private: bool is_tuple = false; }; @@ -1087,16 +1082,17 @@ private: class ArrayLayer : public LayerWithSeparator { public: - bool getResult(ASTPtr & node) override - { - node = makeASTFunction("array", std::move(elements)); - return true; - } - bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { return LayerWithSeparator::parse(pos, expected, action); } + +protected: + bool getResultImpl(ASTPtr & node) override + { + node = makeASTFunction("array", std::move(elements)); + return true; + } }; /// Layer for arrayElement square brackets operator @@ -1216,23 +1212,6 @@ class ExtractLayer : public LayerWithSeparator(interval_kind.toDateDiffUnit()), elements[0], elements[1]); - else if (elements.size() == 3) - node = makeASTFunction("dateDiff", std::make_shared(interval_kind.toDateDiffUnit()), elements[0], elements[1], elements[2]); - else - return false; - } - else - { - node = makeASTFunction("dateDiff", std::move(elements)); - } - return true; - } - bool parse(IParser::Pos & pos, Expected & expected, Action & action) override { /// 0. Try to parse interval_kind (-> 1) @@ -1709,6 +1693,25 @@ public: return true; } +protected: + bool getResultImpl(ASTPtr & node) override + { + if (parsed_interval_kind) + { + if (elements.size() == 2) + node = makeASTFunction("dateDiff", std::make_shared(interval_kind.toDateDiffUnit()), elements[0], elements[1]); + else if (elements.size() == 3) + node = makeASTFunction("dateDiff", std::make_shared(interval_kind.toDateDiffUnit()), elements[0], elements[1], elements[2]); + else + return false; + } + else + { + node = makeASTFunction("dateDiff", std::move(elements)); + } + return true; + } + private: IntervalKind interval_kind; bool parsed_interval_kind = false; @@ -1892,16 +1895,6 @@ class ViewLayer : public Layer public: explicit ViewLayer(bool if_permitted_) : if_permitted(if_permitted_) {} - bool getResult(ASTPtr & node) override - { - if (if_permitted) - node = makeASTFunction("viewIfPermitted", std::move(elements)); - else - node = makeASTFunction("view", std::move(elements)); - - return true; - } - bool parse(IParser::Pos & pos, Expected & expected, Action & /*action*/) override { /// view(SELECT ...) @@ -1958,6 +1951,17 @@ public: return true; } +protected: + bool getResultImpl(ASTPtr & node) override + { + if (if_permitted) + node = makeASTFunction("viewIfPermitted", std::move(elements)); + else + node = makeASTFunction("view", std::move(elements)); + + return true; + } + private: bool if_permitted; }; From d489c46f02b709e01bda01ffe7b3daa26cc63305 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Thu, 27 Oct 2022 10:23:41 +0000 Subject: [PATCH 057/152] Better test --- tests/queries/0_stateless/02474_fix_function_parser_bug.sql | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/queries/0_stateless/02474_fix_function_parser_bug.sql b/tests/queries/0_stateless/02474_fix_function_parser_bug.sql index 12e9e03f151..67d97aa1c25 100644 --- a/tests/queries/0_stateless/02474_fix_function_parser_bug.sql +++ b/tests/queries/0_stateless/02474_fix_function_parser_bug.sql @@ -1 +1 @@ -CREATE DATABASE conv_mian ENGINE QALL(COLUMNS('|T.D'),¸mp} -- { clientError 62 } +CREATE DATABASE conv_mian ENGINE QALL(COLUMNS('|T.D'),¸mp} -- { clientError SYNTAX_ERROR } From 8bd94979ebfa9fa2e90544910e78bee6928f4568 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Thu, 27 Oct 2022 10:46:41 +0000 Subject: [PATCH 058/152] Add test queries for Date32 --- tests/queries/0_stateless/00718_format_datetime.reference | 2 ++ tests/queries/0_stateless/00718_format_datetime.sql | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/tests/queries/0_stateless/00718_format_datetime.reference b/tests/queries/0_stateless/00718_format_datetime.reference index 4f12a46d7c0..6a0fd451e73 100644 --- a/tests/queries/0_stateless/00718_format_datetime.reference +++ b/tests/queries/0_stateless/00718_format_datetime.reference @@ -28,6 +28,8 @@ PM % no formatting pattern 2018-01-01 00:00:00 +2018-01-01 00:00:00 +1927-01-01 00:00:00 2018-01-01 01:00:00 2018-01-01 04:00:00 +0000 -1100 diff --git a/tests/queries/0_stateless/00718_format_datetime.sql b/tests/queries/0_stateless/00718_format_datetime.sql index 7ed1f0abea4..532a0c02e6f 100644 --- a/tests/queries/0_stateless/00718_format_datetime.sql +++ b/tests/queries/0_stateless/00718_format_datetime.sql @@ -40,6 +40,10 @@ SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%%'); SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), 'no formatting pattern'); SELECT formatDateTime(toDate('2018-01-01'), '%F %T'); + +SELECT formatDateTime(toDate32('2018-01-01'), '%F %T'); +SELECT formatDateTime(toDate32('1927-01-01'), '%F %T'); + SELECT formatDateTime(toDateTime('2018-01-01 01:00:00', 'UTC'), '%F %T', 'UTC'), formatDateTime(toDateTime('2018-01-01 01:00:00', 'UTC'), '%F %T', 'Asia/Istanbul'); From 1fa24161e79833ebd333c67a56b0f41ff4ff5d68 Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Thu, 27 Oct 2022 18:34:37 +0200 Subject: [PATCH 059/152] Minor fix implicit cast CaresPTRResolver --- src/Common/CaresPTRResolver.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index c6fe70a09fa..ab4883c104a 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -128,7 +128,7 @@ namespace DB int number_of_fds_ready = 0; if (!readable_sockets.empty()) { - number_of_fds_ready = poll(readable_sockets.data(), readable_sockets.size(), timeout); + number_of_fds_ready = poll(readable_sockets.data(), static_cast(readable_sockets.size()), static_cast(timeout)); } if (number_of_fds_ready > 0) From c4f25228f73a86a626e5491ba7c7b6057ee4d3ed Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 27 Oct 2022 17:18:50 +0000 Subject: [PATCH 060/152] smaller burst to be closer to old behaviour --- src/Common/Throttler.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Common/Throttler.cpp b/src/Common/Throttler.cpp index 169262d30c7..b38777efc03 100644 --- a/src/Common/Throttler.cpp +++ b/src/Common/Throttler.cpp @@ -20,7 +20,7 @@ namespace ErrorCodes /// Just 10^9. static constexpr auto NS = 1000000000UL; -static const size_t default_burst_seconds = 10; +static const size_t default_burst_seconds = 1; Throttler::Throttler(size_t max_speed_, const std::shared_ptr & parent_) : max_speed(max_speed_) From 2038db21f485d8b3de230c58e30429012c4d2b1a Mon Sep 17 00:00:00 2001 From: serxa Date: Thu, 27 Oct 2022 17:26:12 +0000 Subject: [PATCH 061/152] lower limit to avoid hitting another bottleneck --- .../0_stateless/01288_shard_max_network_bandwidth.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql index 969bb0a126c..d2daf48a1cb 100644 --- a/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql +++ b/tests/queries/0_stateless/01288_shard_max_network_bandwidth.sql @@ -1,7 +1,7 @@ -- Tags: shard --- Limit to 10 MB/sec -SET max_network_bandwidth = 10000000; +-- Limit to 100 KB/sec +SET max_network_bandwidth = 100000; -- Lower max_block_size, so we can start throttling sooner. Otherwise query will be executed too quickly. SET max_block_size = 100; @@ -11,7 +11,7 @@ CREATE TEMPORARY TABLE times (t DateTime); -- rand64 is uncompressable data. Each number will take 8 bytes of bandwidth. -- This query should execute in no less than 1.6 seconds if throttled. INSERT INTO times SELECT now(); -SELECT sum(ignore(*)) FROM (SELECT rand64() FROM remote('127.0.0.{2,3}', numbers(2000000))); +SELECT sum(ignore(*)) FROM (SELECT rand64() FROM remote('127.0.0.{2,3}', numbers(20000))); INSERT INTO times SELECT now(); SELECT max(t) - min(t) >= 1 FROM times; From 073d81c31f612ccaba3ba23678966e202ef17dca Mon Sep 17 00:00:00 2001 From: Jordi Villar Date: Thu, 27 Oct 2022 20:25:49 +0200 Subject: [PATCH 062/152] Use nfds_t instead --- src/Common/CaresPTRResolver.cpp | 2 +- utils/iotest/iotest_nonblock.cpp | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Common/CaresPTRResolver.cpp b/src/Common/CaresPTRResolver.cpp index ab4883c104a..99b4c34dfbd 100644 --- a/src/Common/CaresPTRResolver.cpp +++ b/src/Common/CaresPTRResolver.cpp @@ -128,7 +128,7 @@ namespace DB int number_of_fds_ready = 0; if (!readable_sockets.empty()) { - number_of_fds_ready = poll(readable_sockets.data(), static_cast(readable_sockets.size()), static_cast(timeout)); + number_of_fds_ready = poll(readable_sockets.data(), static_cast(readable_sockets.size()), static_cast(timeout)); } if (number_of_fds_ready > 0) diff --git a/utils/iotest/iotest_nonblock.cpp b/utils/iotest/iotest_nonblock.cpp index 6db00045e03..32c86282743 100644 --- a/utils/iotest/iotest_nonblock.cpp +++ b/utils/iotest/iotest_nonblock.cpp @@ -101,7 +101,7 @@ int mainImpl(int argc, char ** argv) size_t ops = 0; while (ops < count) { - if (poll(polls.data(), static_cast(descriptors), -1) <= 0) + if (poll(polls.data(), static_cast(descriptors), -1) <= 0) throwFromErrno("poll failed", ErrorCodes::SYSTEM_ERROR); for (size_t i = 0; i < descriptors; ++i) { From 51e59405bc33ce3b2edddb6a08de5b7795e143d5 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Oct 2022 09:41:29 +0000 Subject: [PATCH 063/152] Update version_date.tsv and changelogs after v22.3.14.18-lts --- docs/changelogs/v22.3.14.18-lts.md | 26 ++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 27 insertions(+) create mode 100644 docs/changelogs/v22.3.14.18-lts.md diff --git a/docs/changelogs/v22.3.14.18-lts.md b/docs/changelogs/v22.3.14.18-lts.md new file mode 100644 index 00000000000..d0c67a2b241 --- /dev/null +++ b/docs/changelogs/v22.3.14.18-lts.md @@ -0,0 +1,26 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.3.14.18-lts (642946f61b2) FIXME as compared to v22.3.13.80-lts (e2708b01fba) + +#### Bug Fix +* Backported in [#42432](https://github.com/ClickHouse/ClickHouse/issues/42432): - Choose correct aggregation method for LowCardinality with BigInt. [#42342](https://github.com/ClickHouse/ClickHouse/pull/42342) ([Duc Canh Le](https://github.com/canhld94)). + +#### Build/Testing/Packaging Improvement +* Backported in [#42328](https://github.com/ClickHouse/ClickHouse/issues/42328): Update cctz to the latest master, update tzdb to 2020e. [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). +* Backported in [#42358](https://github.com/ClickHouse/ClickHouse/issues/42358): Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#42298](https://github.com/ClickHouse/ClickHouse/issues/42298): Fix a bug with projections and the `aggregate_functions_null_for_empty` setting. This bug is very rare and appears only if you enable the `aggregate_functions_null_for_empty` setting in the server's config. This closes [#41647](https://github.com/ClickHouse/ClickHouse/issues/41647). [#42198](https://github.com/ClickHouse/ClickHouse/pull/42198) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#42592](https://github.com/ClickHouse/ClickHouse/issues/42592): This closes [#42453](https://github.com/ClickHouse/ClickHouse/issues/42453). [#42573](https://github.com/ClickHouse/ClickHouse/pull/42573) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Add a warning message to release.py script, require release type [#41975](https://github.com/ClickHouse/ClickHouse/pull/41975) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Revert [#27787](https://github.com/ClickHouse/ClickHouse/issues/27787) [#42136](https://github.com/ClickHouse/ClickHouse/pull/42136) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8977c98eb7e..5e50b998ec4 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -36,6 +36,7 @@ v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.14.18-lts 2022-10-28 v22.3.13.80-lts 2022-09-30 v22.3.12.19-lts 2022-08-29 v22.3.11.12-lts 2022-08-10 From acec4475261df88a9139f2743f4034b6d33b9554 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Fri, 28 Oct 2022 12:35:18 +0200 Subject: [PATCH 064/152] Fix anchor links --- docs/en/sql-reference/statements/alter/partition.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/partition.md b/docs/en/sql-reference/statements/alter/partition.md index da99c52538f..2d89c1d5d18 100644 --- a/docs/en/sql-reference/statements/alter/partition.md +++ b/docs/en/sql-reference/statements/alter/partition.md @@ -39,7 +39,7 @@ ALTER TABLE mt DETACH PARTITION '2020-11-21'; ALTER TABLE mt DETACH PART 'all_2_2_0'; ``` -Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Read about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). After the query is executed, you can do whatever you want with the data in the `detached` directory — delete it from the file system, or just leave it. @@ -53,7 +53,7 @@ ALTER TABLE table_name [ON CLUSTER cluster] DROP PARTITION|PART partition_expr Deletes the specified partition from the table. This query tags the partition as inactive and deletes data completely, approximately in 10 minutes. -Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Read about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). The query is replicated – it deletes data on all replicas. @@ -71,7 +71,7 @@ ALTER TABLE table_name [ON CLUSTER cluster] DROP DETACHED PARTITION|PART partiti ``` Removes the specified part or all parts of the specified partition from `detached`. -Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Read more about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). ## ATTACH PARTITION\|PART @@ -86,7 +86,7 @@ ALTER TABLE visits ATTACH PARTITION 201901; ALTER TABLE visits ATTACH PART 201901_2_2_0; ``` -Read more about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Read more about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). This query is replicated. The replica-initiator checks whether there is data in the `detached` directory. If data exists, the query checks its integrity. If everything is correct, the query adds the data to the table. @@ -166,7 +166,7 @@ This query creates a local backup of a specified partition. If the `PARTITION` c The entire backup process is performed without stopping the server. ::: -Note that for old-styled tables you can specify the prefix of the partition name (for example, `2019`) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Note that for old-styled tables you can specify the prefix of the partition name (for example, `2019`) - then the query creates the backup for all the corresponding partitions. Read about setting the partition expression in a section [How to set the partition expression](#how-to-set-partition-expression). At the time of execution, for a data snapshot, the query creates hardlinks to a table data. Hardlinks are placed in the directory `/var/lib/clickhouse/shadow/N/...`, where: From 4f62bd6ba05f44ab992a4eed53dc55c8ad9727d9 Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Fri, 28 Oct 2022 11:43:06 +0000 Subject: [PATCH 065/152] Update version_date.tsv and changelogs after v22.3.14.23-lts --- docs/changelogs/v22.3.14.23-lts.md | 29 ++++++++++++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 2 files changed, 30 insertions(+) create mode 100644 docs/changelogs/v22.3.14.23-lts.md diff --git a/docs/changelogs/v22.3.14.23-lts.md b/docs/changelogs/v22.3.14.23-lts.md new file mode 100644 index 00000000000..663d8b43f6f --- /dev/null +++ b/docs/changelogs/v22.3.14.23-lts.md @@ -0,0 +1,29 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.3.14.23-lts (74956bfee4d) FIXME as compared to v22.3.13.80-lts (e2708b01fba) + +#### Improvement +* Backported in [#42527](https://github.com/ClickHouse/ClickHouse/issues/42527): Fix issue with passing MySQL timeouts for MySQL database engine and MySQL table function. Closes [#34168](https://github.com/ClickHouse/ClickHouse/issues/34168)?notification_referrer_id=NT_kwDOAzsV57MzMDMxNjAzNTY5OjU0MjAzODc5. [#40751](https://github.com/ClickHouse/ClickHouse/pull/40751) ([Kseniia Sumarokova](https://github.com/kssenii)). + +#### Bug Fix +* Backported in [#42432](https://github.com/ClickHouse/ClickHouse/issues/42432): - Choose correct aggregation method for LowCardinality with BigInt. [#42342](https://github.com/ClickHouse/ClickHouse/pull/42342) ([Duc Canh Le](https://github.com/canhld94)). + +#### Build/Testing/Packaging Improvement +* Backported in [#42328](https://github.com/ClickHouse/ClickHouse/issues/42328): Update cctz to the latest master, update tzdb to 2020e. [#42273](https://github.com/ClickHouse/ClickHouse/pull/42273) ([Dom Del Nano](https://github.com/ddelnano)). +* Backported in [#42358](https://github.com/ClickHouse/ClickHouse/issues/42358): Update tzdata to 2022e to support the new timezone changes. Palestine transitions are now Saturdays at 02:00. Simplify three Ukraine zones into one. Jordan and Syria switch from +02/+03 with DST to year-round +03. (https://data.iana.org/time-zones/tzdb/NEWS). This closes [#42252](https://github.com/ClickHouse/ClickHouse/issues/42252). [#42327](https://github.com/ClickHouse/ClickHouse/pull/42327) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#42298](https://github.com/ClickHouse/ClickHouse/issues/42298): Fix a bug with projections and the `aggregate_functions_null_for_empty` setting. This bug is very rare and appears only if you enable the `aggregate_functions_null_for_empty` setting in the server's config. This closes [#41647](https://github.com/ClickHouse/ClickHouse/issues/41647). [#42198](https://github.com/ClickHouse/ClickHouse/pull/42198) ([Alexey Milovidov](https://github.com/alexey-milovidov)). +* Backported in [#42592](https://github.com/ClickHouse/ClickHouse/issues/42592): This closes [#42453](https://github.com/ClickHouse/ClickHouse/issues/42453). [#42573](https://github.com/ClickHouse/ClickHouse/pull/42573) ([Alexey Milovidov](https://github.com/alexey-milovidov)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Add a warning message to release.py script, require release type [#41975](https://github.com/ClickHouse/ClickHouse/pull/41975) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). +* Revert [#27787](https://github.com/ClickHouse/ClickHouse/issues/27787) [#42136](https://github.com/ClickHouse/ClickHouse/pull/42136) ([Nikolai Kochetov](https://github.com/KochetovNicolai)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 8977c98eb7e..21197713694 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -36,6 +36,7 @@ v22.4.5.9-stable 2022-05-06 v22.4.4.7-stable 2022-04-29 v22.4.3.3-stable 2022-04-26 v22.4.2.1-stable 2022-04-22 +v22.3.14.23-lts 2022-10-28 v22.3.13.80-lts 2022-09-30 v22.3.12.19-lts 2022-08-29 v22.3.11.12-lts 2022-08-10 From 15f3f56812d82243d0ed89a45916eb2bf5e1be70 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 28 Oct 2022 12:44:47 +0000 Subject: [PATCH 066/152] Change 1 argument of function to first argument of function etc --- src/Functions/formatDateTime.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index c5240abf7a1..9634768e68a 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -322,11 +322,11 @@ public: getName(), arguments.size()); if (arguments.size() == 1 && !isInteger(arguments[0].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1 argument of function {} when arguments size is 1. Should be integer", + "Illegal type {} of first argument of function {} when arguments size is 1. Should be integer", arguments[0].type->getName(), getName()); if (arguments.size() > 1 && !(isInteger(arguments[0].type) || isDate(arguments[0].type) || isDateTime(arguments[0].type) || isDate32(arguments[0].type) || isDateTime64(arguments[0].type))) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1 argument of function {} when arguments size is 2 or 3. Should be a integer or a date with time", + "Illegal type {} of first argument of function {} when arguments size is 2 or 3. Should be a integer or a date with time", arguments[0].type->getName(), getName()); } else @@ -337,18 +337,18 @@ public: getName(), arguments.size()); if (!isDate(arguments[0].type) && !isDateTime(arguments[0].type) && !isDate32(arguments[0].type) && !isDateTime64(arguments[0].type)) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 1 argument of function {}. Should be a date or a date with time", + "Illegal type {} of first argument of function {}. Should be a date or a date with time", arguments[0].type->getName(), getName()); } if (arguments.size() == 2 && !WhichDataType(arguments[1].type).isString()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 2 argument of function {}. Must be String.", + "Illegal type {} of second argument of function {}. Must be String.", arguments[1].type->getName(), getName()); if (arguments.size() == 3 && !WhichDataType(arguments[2].type).isString()) throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, - "Illegal type {} of 3 argument of function {}. Must be String.", + "Illegal type {} of third argument of function {}. Must be String.", arguments[2].type->getName(), getName()); if (arguments.size() == 1) From fc16752844eb03adfbcb5c4241a6109243ce4e98 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 28 Oct 2022 12:56:52 +0000 Subject: [PATCH 067/152] Update docs for FROM_UNIXTIME --- docs/en/sql-reference/functions/date-time-functions.md | 4 ++-- docs/ru/sql-reference/functions/date-time-functions.md | 5 ++--- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/functions/date-time-functions.md b/docs/en/sql-reference/functions/date-time-functions.md index 15fc9ef0c89..f7ea2690b21 100644 --- a/docs/en/sql-reference/functions/date-time-functions.md +++ b/docs/en/sql-reference/functions/date-time-functions.md @@ -1068,7 +1068,7 @@ Example: SELECT timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600)); SELECT timeSlots(toDateTime('1980-12-12 21:01:02', 'UTC'), toUInt32(600), 299); SELECT timeSlots(toDateTime64('1980-12-12 21:01:02.1234', 4, 'UTC'), toDecimal64(600.1, 1), toDecimal64(299, 0)); -``` +``` ``` text ┌─timeSlots(toDateTime('2012-01-01 12:20:00'), toUInt32(600))─┐ │ ['2012-01-01 12:00:00','2012-01-01 12:30:00'] │ @@ -1244,7 +1244,7 @@ Result: └──────────────────────────┘ ``` -When there are two arguments: first is an [Integer](../../sql-reference/data-types/int-uint.md) or [DateTime](../../sql-reference/data-types/datetime.md), second is a constant format string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. +When there are two or three arguments, the first an [Integer](../../sql-reference/data-types/int-uint.md), [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) or [DateTime64](../../sql-reference/data-types/datetime64.md), the second a constant format string and the third an optional constant time zone string — it acts in the same way as [formatDateTime](#formatdatetime) and return [String](../../sql-reference/data-types/string.md#string) type. For example: diff --git a/docs/ru/sql-reference/functions/date-time-functions.md b/docs/ru/sql-reference/functions/date-time-functions.md index 80e2561a8d7..4acd706b3db 100644 --- a/docs/ru/sql-reference/functions/date-time-functions.md +++ b/docs/ru/sql-reference/functions/date-time-functions.md @@ -318,7 +318,7 @@ SELECT toStartOfISOYear(toDate('2017-01-01')) AS ISOYear20170101; :::note "Attention" Возвращаемое значение для некорректных дат зависит от реализации. ClickHouse может вернуть нулевую дату, выбросить исключение, или выполнить «естественное» перетекание дат между месяцами. ::: - + ## toMonday {#tomonday} Округляет дату или дату-с-временем вниз до ближайшего понедельника. @@ -1126,8 +1126,7 @@ SELECT FROM_UNIXTIME(423543535); └──────────────────────────┘ ``` -В случае, когда есть два аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md) или [DateTime](../../sql-reference/data-types/datetime.md), а второй является строкой постоянного формата — функция работает также, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). - +В случае, когда есть два или три аргумента: первый типа [Integer](../../sql-reference/data-types/int-uint.md), [Date](../../sql-reference/data-types/date.md), [Date32](../../sql-reference/data-types/date32.md), [DateTime](../../sql-reference/data-types/datetime.md) или [DateTime64](../../sql-reference/data-types/datetime64.md), а второй является строкой постоянного формата и третий является строкой постоянной временной зоны — функция работает также, как [formatDateTime](#formatdatetime), и возвращает значение типа [String](../../sql-reference/data-types/string.md#string). Запрос: From 5755728b33543d472a00c71dc1789d0532d00785 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 28 Oct 2022 13:11:09 +0000 Subject: [PATCH 068/152] Add Date32 and DateTime64 into exception messages --- src/Functions/formatDateTime.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Functions/formatDateTime.cpp b/src/Functions/formatDateTime.cpp index 9634768e68a..4db04d61d84 100644 --- a/src/Functions/formatDateTime.cpp +++ b/src/Functions/formatDateTime.cpp @@ -371,7 +371,7 @@ public: })) { throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of function {}, must be Integer or DateTime when arguments size is 1.", + "Illegal column {} of function {}, must be Integer, Date, Date32, DateTime or DateTime64 when arguments size is 1.", arguments[0].column->getName(), getName()); } } @@ -382,7 +382,7 @@ public: using FromDataType = std::decay_t; if (!(res = executeType(arguments, result_type))) throw Exception(ErrorCodes::ILLEGAL_COLUMN, - "Illegal column {} of function {}, must be Integer or DateTime.", + "Illegal column {} of function {}, must be Integer, Date, Date32, DateTime or DateTime64.", arguments[0].column->getName(), getName()); return true; })) From 7a5432feaa39596be26974ef1d84d6e954deafc1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 28 Oct 2022 15:10:53 +0200 Subject: [PATCH 069/152] Fix possible SIGSEGV for web disks when file does not exists It can be triggered in multiple ways, either when file does not exists and you are trying to create MergeTree table from web (that has special code for UUID handling) or by simply OPTIMIZE TABLE FINAL for MergeTree table that is located on web disk, in both cases you will get the following:
stacktrace 2022.10.28 14:08:40.631226 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} ContextAccess (default): Access granted: OPTIMIZE ON default.data_from_web 2022.10.28 14:08:40.632017 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} default.data_from_web (a3e65e1f-5fd4-47ed-9dbd-307f2586b52d) (MergerMutator): Selected 1 parts from all_1_1_0 to all_1_1_0 2022.10.28 14:08:40.632496 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} default.data_from_web (a3e65e1f-5fd4-47ed-9dbd-307f2586b52d): Trying to reserve 1.00 MiB using storage policy from min volume index 0 2022.10.28 14:08:40.632752 [ 6043 ] {6165bf5f-e76b-4bca-941c-7c7ff5e3b46b} DiskObjectStorage(DiskWebServer): Reserved 1.00 MiB on remote disk `web_disk`, having unreserved 16.00 EiB. 2022.10.28 14:08:40.634155 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} MergeTask::PrepareStage: Merging 1 parts: from all_1_1_0 to all_1_1_0 into Compact 2022.10.28 14:08:40.634498 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} WebObjectStorage: Loading metadata for directory: http://127.0.0.1:8080/store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1 2022.10.28 14:08:40.635025 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} DiskWeb: Adding directory: store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/ 2022.10.28 14:08:40.635355 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} ReadWriteBufferFromHTTP: Sending request to http://127.0.0.1:8080/store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/.index 2022.10.28 14:08:40.639618 [ 6043 ] {a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1} DiskWeb: Cannot load disk metadata. Error: Code: 86. DB::Exception: Received error from remote server /store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/.index. HTTP status code: 404 Not Found, body: 404 Not Found

Not Found

The requested resource /store/a3e/a3e65e1f-5fd4-47ed-9dbd-307f2586b52d/tmp_merge_all_1_1_1/.index was not found on this server.

: while loading disk metadata. (RECEIVED_ERROR_FROM_REMOTE_IO_SERVER) (version 22.11.1.1) 2022.10.28 14:08:40.640527 [ 5488 ] {} BaseDaemon: Received signal 11 2022.10.28 14:08:40.641529 [ 9027 ] {} BaseDaemon: ######################################## 2022.10.28 14:08:40.642759 [ 9027 ] {} BaseDaemon: (version 22.11.1.1, build id: 12145DA78CE5E9EBB10A034177FAE5967EF81A4A) (from thread 6043) (query_id: a3e65e1f-5fd4-47ed-9dbd-307f2586b52d::all_1_1_1) (query: optimize table data_from_web final) Received signal Segmentation fault (11) 2022.10.28 14:08:40.643260 [ 9027 ] {} BaseDaemon: Address: NULL pointer. Access: read. Unknown si_code. 2022.10.28 14:08:40.643769 [ 9027 ] {} BaseDaemon: Stack trace: 0x7ffff416c0f2 0x7ffff7cd1ca8 0x7ffff679ae5e 0x7fffd52e7906 0x7fffd50c65aa 0x7fffca7a0d42 0x7fffcaee79ec 0x7fffcaf242f8 0x7fffcaf242b5 0x7fffcaf2427d 0x7fffcaf24255 0x7fffcaf2421d 0x7ffff65c3686 0x7ffff65c2295 0x7fffcaeee2a9 0x7fffcaef2c43 0x7fffcaee3c0e 0x7fffcc4a7851 0x7fffcc4a768f 0x7fffcc4abb2d 0x7fffcfdce828 0x7fffd03e3eaa 0x7fffd03dfe3b 0x7fffc8ec42d4 0x7fffc8ed51d2 0x7ffff4bdd839 0x7ffff4bde0a8 0x7ffff48ab261 0x7ffff48a769a 0x7ffff48a6335 0x7ffff409f8fd 0x7ffff4121a60 2022.10.28 14:08:40.644411 [ 9027 ] {} BaseDaemon: 4. ? @ 0x7ffff416c0f2 in ? 2022.10.28 14:08:40.676390 [ 9027 ] {} BaseDaemon: 5. /src/ch/clickhouse/src/Common/StringUtils/StringUtils.cpp:9: detail::startsWith(std::__1::basic_string, std::__1::allocator> const&, char const*, unsigned long) @ 0x1ca8 in /src/ch/clickhouse/.cmake/src/Common/StringUtils/libstring_utilsd.so 2022.10.28 14:08:40.730727 [ 9027 ] {} BaseDaemon: 6. /src/ch/clickhouse/src/Common/StringUtils/StringUtils.h:19: startsWith(std::__1::basic_string, std::__1::allocator> const&, std::__1::basic_string, std::__1::allocator> const&) @ 0x59ae5e in /src/ch/clickhouse/.cmake/src/libclickhouse_common_iod.so 2022.10.28 14:08:40.923955 [ 9027 ] {} BaseDaemon: 7. /src/ch/clickhouse/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp:58: DB::MetadataStorageFromStaticFilesWebServer::exists(std::__1::basic_string, std::__1::allocator> const&) const @ 0x6e7906 in /src/ch/clickhouse/.cmake/src/libdbmsd.so 2022.10.28 14:08:41.291996 [ 9027 ] {} BaseDaemon: 8. /src/ch/clickhouse/src/Disks/ObjectStorages/DiskObjectStorage.cpp:181: DB::DiskObjectStorage::exists(std::__1::basic_string, std::__1::allocator> const&) const @ 0x4c65aa in /src/ch/clickhouse/.cmake/src/libdbmsd.so 2022.10.28 14:08:41.704697 [ 9027 ] {} BaseDaemon: 9. /src/ch/clickhouse/src/Storages/MergeTree/DataPartStorageOnDisk.cpp:74: DB::DataPartStorageOnDisk::exists() const @ 0xda0d42 in /src/ch/clickhouse/.cmake/src/libclickhouse_storages_mergetreed.so 2022.10.28 14:08:43.032459 [ 9027 ] {} BaseDaemon: 10. /src/ch/clickhouse/src/Storages/MergeTree/MergeTask.cpp:147: DB::MergeTask::ExecuteAndFinalizeHorizontalPart::prepare() @ 0x14e79ec in /src/ch/clickhouse/.cmake/src/libclickhouse_storages_mergetreed.so ... Segmentation fault (core dumped)
Signed-off-by: Azat Khuzhin --- .../Web/MetadataStorageFromStaticFilesWebServer.cpp | 3 +++ tests/integration/test_disk_over_web_server/test.py | 3 +++ 2 files changed, 6 insertions(+) diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp index 06e36a2ddd8..aa125e93dee 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp @@ -55,6 +55,9 @@ bool MetadataStorageFromStaticFilesWebServer::exists(const std::string & path) c path, [](const auto & file, const std::string & path_) { return file.first < path_; } ); + if (it == object_storage.files.end()) + return false; + if (startsWith(it->first, path) || (it != object_storage.files.begin() && startsWith(std::prev(it)->first, path))) return true; diff --git a/tests/integration/test_disk_over_web_server/test.py b/tests/integration/test_disk_over_web_server/test.py index ea6e407a18f..2ccc17db4f4 100644 --- a/tests/integration/test_disk_over_web_server/test.py +++ b/tests/integration/test_disk_over_web_server/test.py @@ -129,6 +129,9 @@ def test_incorrect_usage(cluster): result = node2.query_and_get_error("TRUNCATE TABLE test0") assert "Table is read-only" in result + result = node2.query_and_get_error("OPTIMIZE TABLE test0 FINAL") + assert "Only read-only operations are supported" in result + node2.query("DROP TABLE test0 SYNC") From 665fcf55aa66c0a19d8c7e4c8b9d7fe6de852aee Mon Sep 17 00:00:00 2001 From: Vitalii S Date: Fri, 28 Oct 2022 10:08:04 -0400 Subject: [PATCH 070/152] Update database.md Minor text correction --- docs/en/sql-reference/statements/create/database.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/database.md b/docs/en/sql-reference/statements/create/database.md index 432f5975cc8..7954d1362f1 100644 --- a/docs/en/sql-reference/statements/create/database.md +++ b/docs/en/sql-reference/statements/create/database.md @@ -31,7 +31,7 @@ By default, ClickHouse uses its own [Atomic](../../../engines/database-engines/a ### COMMENT -You can add a comment to the database when you creating it. +You can add a comment to the database when you are creating it. The comment is supported for all database engines. From 4a4fb20731019f2b1a66c53b9e1b89bc823e93a9 Mon Sep 17 00:00:00 2001 From: Roman Vasin Date: Fri, 28 Oct 2022 14:08:28 +0000 Subject: [PATCH 071/152] Add Date32 to 01411_from_unixtime and improve 00718_format_datetime --- .../00718_format_datetime.reference | 49 +++++++++-------- .../0_stateless/00718_format_datetime.sql | 54 ++++++++++--------- .../0_stateless/01411_from_unixtime.reference | 34 ++++++------ .../0_stateless/01411_from_unixtime.sql | 34 ++++++------ 4 files changed, 86 insertions(+), 85 deletions(-) diff --git a/tests/queries/0_stateless/00718_format_datetime.reference b/tests/queries/0_stateless/00718_format_datetime.reference index 6a0fd451e73..bc98dd59d5f 100644 --- a/tests/queries/0_stateless/00718_format_datetime.reference +++ b/tests/queries/0_stateless/00718_format_datetime.reference @@ -1,33 +1,32 @@ -20 +20 20 +02 02 +01/02/18 01/02/18 + 2 2 +2018-01-02 2018-01-02 +22 00 02 -01/02/18 - 2 -2018-01-02 -22 -02 -10 +10 12 11 12 -001 -366 -01 -33 -\n -AM +001 001 +366 366 +01 01 +33 00 +\n \n +AM AM AM PM -22:33 -44 -\t -22:33:44 -1 7 -01 01 53 52 -1 0 -18 -2018 -% -no formatting pattern -2018-01-01 00:00:00 +22:33 00:00 +44 00 +\t \t +22:33:44 00:00:00 +1 7 1 7 +01 01 53 52 01 01 53 52 +1 0 1 0 +18 18 +2018 2018 +% % +no formatting pattern no formatting pattern 2018-01-01 00:00:00 1927-01-01 00:00:00 2018-01-01 01:00:00 2018-01-01 04:00:00 diff --git a/tests/queries/0_stateless/00718_format_datetime.sql b/tests/queries/0_stateless/00718_format_datetime.sql index 532a0c02e6f..deb5fb96c6c 100644 --- a/tests/queries/0_stateless/00718_format_datetime.sql +++ b/tests/queries/0_stateless/00718_format_datetime.sql @@ -8,40 +8,42 @@ SELECT formatDateTime(now(), 'unescaped %'); -- { serverError 36 } SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%U'); -- { serverError 48 } SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%W'); -- { serverError 48 } -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%C'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%C'), formatDateTime(toDate32('2018-01-02'), '%C'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%d'), formatDateTime(toDate32('2018-01-02'), '%d'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%D'), formatDateTime(toDate32('2018-01-02'), '%D'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%e'), formatDateTime(toDate32('2018-01-02'), '%e'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%F'), formatDateTime(toDate32('2018-01-02'), '%F'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%H'), formatDateTime(toDate32('2018-01-02'), '%H'); SELECT formatDateTime(toDateTime('2018-01-02 02:33:44'), '%H'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%I'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%I'), formatDateTime(toDate32('2018-01-02'), '%I'); SELECT formatDateTime(toDateTime('2018-01-02 11:33:44'), '%I'); SELECT formatDateTime(toDateTime('2018-01-02 00:33:44'), '%I'); -SELECT formatDateTime(toDateTime('2018-01-01 00:33:44'), '%j'); -SELECT formatDateTime(toDateTime('2000-12-31 00:33:44'), '%j'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%m'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%M'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%n'); -SELECT formatDateTime(toDateTime('2018-01-02 00:33:44'), '%p'); +SELECT formatDateTime(toDateTime('2018-01-01 00:33:44'), '%j'), formatDateTime(toDate32('2018-01-01'), '%j'); +SELECT formatDateTime(toDateTime('2000-12-31 00:33:44'), '%j'), formatDateTime(toDate32('2000-12-31'), '%j'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%m'), formatDateTime(toDate32('2018-01-02'), '%m'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%M'), formatDateTime(toDate32('2018-01-02'), '%M'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%n'), formatDateTime(toDate32('2018-01-02'), '%n'); +SELECT formatDateTime(toDateTime('2018-01-02 00:33:44'), '%p'), formatDateTime(toDateTime('2018-01-02'), '%p'); SELECT formatDateTime(toDateTime('2018-01-02 11:33:44'), '%p'); SELECT formatDateTime(toDateTime('2018-01-02 12:33:44'), '%p'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%R'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%S'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%t'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%T'); -SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%u'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%u'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%R'), formatDateTime(toDate32('2018-01-02'), '%R'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%S'), formatDateTime(toDate32('2018-01-02'), '%S'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%t'), formatDateTime(toDate32('2018-01-02'), '%t'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%T'), formatDateTime(toDate32('2018-01-02'), '%T'); +SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%u'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%u'), + formatDateTime(toDate32('2018-01-01'), '%u'), formatDateTime(toDate32('2018-01-07'), '%u'); SELECT formatDateTime(toDateTime('1996-01-01 22:33:44'), '%V'), formatDateTime(toDateTime('1996-12-31 22:33:44'), '%V'), - formatDateTime(toDateTime('1999-01-01 22:33:44'), '%V'), formatDateTime(toDateTime('1999-12-31 22:33:44'), '%V'); -SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%w'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%w'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%y'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%Y'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%%'); -SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), 'no formatting pattern'); + formatDateTime(toDateTime('1999-01-01 22:33:44'), '%V'), formatDateTime(toDateTime('1999-12-31 22:33:44'), '%V'), + formatDateTime(toDate32('1996-01-01'), '%V'), formatDateTime(toDate32('1996-12-31'), '%V'), + formatDateTime(toDate32('1999-01-01'), '%V'), formatDateTime(toDate32('1999-12-31'), '%V'); +SELECT formatDateTime(toDateTime('2018-01-01 22:33:44'), '%w'), formatDateTime(toDateTime('2018-01-07 22:33:44'), '%w'), + formatDateTime(toDate32('2018-01-01'), '%w'), formatDateTime(toDate32('2018-01-07'), '%w'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%y'), formatDateTime(toDate32('2018-01-02'), '%y'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%Y'), formatDateTime(toDate32('2018-01-02'), '%Y'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), '%%'), formatDateTime(toDate32('2018-01-02'), '%%'); +SELECT formatDateTime(toDateTime('2018-01-02 22:33:44'), 'no formatting pattern'), formatDateTime(toDate32('2018-01-02'), 'no formatting pattern'); SELECT formatDateTime(toDate('2018-01-01'), '%F %T'); - -SELECT formatDateTime(toDate32('2018-01-01'), '%F %T'); SELECT formatDateTime(toDate32('1927-01-01'), '%F %T'); SELECT diff --git a/tests/queries/0_stateless/01411_from_unixtime.reference b/tests/queries/0_stateless/01411_from_unixtime.reference index 1bc7519e668..17086e8c58b 100644 --- a/tests/queries/0_stateless/01411_from_unixtime.reference +++ b/tests/queries/0_stateless/01411_from_unixtime.reference @@ -5,25 +5,25 @@ 11 1970-01-15 1970-01-15 06:52:36 -20 +20 20 +02 02 +01/02/18 01/02/18 + 2 2 +2018-01-02 2018-01-02 +22 00 02 -01/02/18 - 2 -2018-01-02 -22 -02 -10 +10 12 11 12 -001 -366 -01 -33 -\n -AM +001 001 +366 366 +01 01 +33 00 +\n \n +AM AM AM PM -22:33 -44 -\t -22:33:44 +22:33 00:00 +44 00 +\t \t +22:33:44 00:00:00 diff --git a/tests/queries/0_stateless/01411_from_unixtime.sql b/tests/queries/0_stateless/01411_from_unixtime.sql index ec7b4d65b57..9a6655768e0 100644 --- a/tests/queries/0_stateless/01411_from_unixtime.sql +++ b/tests/queries/0_stateless/01411_from_unixtime.sql @@ -5,25 +5,25 @@ SELECT FROM_UNIXTIME(5345345, '%C', 'UTC'); SELECT FROM_UNIXTIME(645123, '%H', 'UTC'); SELECT FROM_UNIXTIME(1232456, '%Y-%m-%d', 'UTC'); SELECT FROM_UNIXTIME(1234356, '%Y-%m-%d %R:%S', 'UTC'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%C'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%d'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%D'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%e'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%F'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%H'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%C'), FROM_UNIXTIME(toDate32('2018-01-02'), '%C'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%d'), FROM_UNIXTIME(toDate32('2018-01-02'), '%d'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%D'), FROM_UNIXTIME(toDate32('2018-01-02'), '%D'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%e'), FROM_UNIXTIME(toDate32('2018-01-02'), '%e'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%F'), FROM_UNIXTIME(toDate32('2018-01-02'), '%F'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%H'), FROM_UNIXTIME(toDate32('2018-01-02'), '%H'); SELECT FROM_UNIXTIME(toDateTime('2018-01-02 02:33:44'), '%H'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%I'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%I'), FROM_UNIXTIME(toDate32('2018-01-02'), '%I'); SELECT FROM_UNIXTIME(toDateTime('2018-01-02 11:33:44'), '%I'); SELECT FROM_UNIXTIME(toDateTime('2018-01-02 00:33:44'), '%I'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-01 00:33:44'), '%j'); -SELECT FROM_UNIXTIME(toDateTime('2000-12-31 00:33:44'), '%j'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%m'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%M'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%n'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 00:33:44'), '%p'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-01 00:33:44'), '%j'), FROM_UNIXTIME(toDate32('2018-01-01'), '%j'); +SELECT FROM_UNIXTIME(toDateTime('2000-12-31 00:33:44'), '%j'), FROM_UNIXTIME(toDate32('2000-12-31'), '%j'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%m'), FROM_UNIXTIME(toDate32('2018-01-02'), '%m'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%M'), FROM_UNIXTIME(toDate32('2018-01-02'), '%M'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%n'), FROM_UNIXTIME(toDate32('2018-01-02'), '%n'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 00:33:44'), '%p'), FROM_UNIXTIME(toDate32('2018-01-02'), '%p'); SELECT FROM_UNIXTIME(toDateTime('2018-01-02 11:33:44'), '%p'); SELECT FROM_UNIXTIME(toDateTime('2018-01-02 12:33:44'), '%p'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%R'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%S'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%t'); -SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%T'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%R'), FROM_UNIXTIME(toDate32('2018-01-02'), '%R'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%S'), FROM_UNIXTIME(toDate32('2018-01-02'), '%S'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%t'), FROM_UNIXTIME(toDate32('2018-01-02'), '%t'); +SELECT FROM_UNIXTIME(toDateTime('2018-01-02 22:33:44'), '%T'), FROM_UNIXTIME(toDate32('2018-01-02'), '%T'); From d4c62688069b5d6baad52d328702973860e17ec1 Mon Sep 17 00:00:00 2001 From: UnamedRus Date: Fri, 28 Oct 2022 18:47:32 +0400 Subject: [PATCH 072/152] Fix Polygon dict xml config Missing tag --- .../external-dictionaries/external-dicts-dict-polygon.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md index 912af5b5bce..e5ee48c9166 100644 --- a/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md +++ b/docs/en/sql-reference/dictionaries/external-dictionaries/external-dicts-dict-polygon.md @@ -14,8 +14,10 @@ Example of a polygon dictionary configuration: - key - Array(Array(Array(Array(Float64)))) + + key + Array(Array(Array(Array(Float64)))) + From 0872d5c440ada1cb9966f8e69b832b2c8eb1a026 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 28 Oct 2022 13:33:12 -0400 Subject: [PATCH 073/152] style Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 834f74822a1..eb6e131886c 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -141,7 +141,7 @@ jobs: - name: Download and set up sonar-scanner env: SONAR_SCANNER_DOWNLOAD_URL: https://binaries.sonarsource.com/Distribution/sonar-scanner-cli/sonar-scanner-cli-${{ env.SONAR_SCANNER_VERSION }}-linux.zip - run: | + "run": | mkdir -p $HOME/.sonar curl -sSLo $HOME/.sonar/sonar-scanner.zip ${{ env.SONAR_SCANNER_DOWNLOAD_URL }} unzip -o $HOME/.sonar/sonar-scanner.zip -d $HOME/.sonar/ @@ -149,7 +149,7 @@ jobs: - name: Download and set up build-wrapper env: BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip - run: | + "run": | curl -sSLo $HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }} unzip -o $HOME/.sonar/build-wrapper-linux-x86.zip -d $HOME/.sonar/ echo "$HOME/.sonar/build-wrapper-linux-x86" >> $GITHUB_PATH From 144648a633e90e5f40b196bb4af668afe74c4e2d Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 28 Oct 2022 13:47:37 -0400 Subject: [PATCH 074/152] disable sc2086 Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index eb6e131886c..527e3b1f2af 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -141,7 +141,8 @@ jobs: - name: Download and set up sonar-scanner env: SONAR_SCANNER_DOWNLOAD_URL: https://binaries.sonarsource.com/Distribution/sonar-scanner-cli/sonar-scanner-cli-${{ env.SONAR_SCANNER_VERSION }}-linux.zip - "run": | + # shellcheck disable=SC2086 + run: | mkdir -p $HOME/.sonar curl -sSLo $HOME/.sonar/sonar-scanner.zip ${{ env.SONAR_SCANNER_DOWNLOAD_URL }} unzip -o $HOME/.sonar/sonar-scanner.zip -d $HOME/.sonar/ @@ -149,7 +150,8 @@ jobs: - name: Download and set up build-wrapper env: BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip - "run": | + # shellcheck disable=SC2086 + run: | curl -sSLo $HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }} unzip -o $HOME/.sonar/build-wrapper-linux-x86.zip -d $HOME/.sonar/ echo "$HOME/.sonar/build-wrapper-linux-x86" >> $GITHUB_PATH From 10bee23c584a2effb509e71d4c7aa09b781e853d Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 28 Oct 2022 13:56:12 -0400 Subject: [PATCH 075/152] no newline at end of file. Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 527e3b1f2af..1fa6603c960 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -176,4 +176,4 @@ jobs: --define sonar.host.url="${{ env.SONAR_SERVER_URL }}" \ --define sonar.cfamily.build-wrapper-output="${{ env.BUILD_WRAPPER_OUT_DIR }}" \ --define sonar.projectKey="ClickHouse_ClickHouse" \ - --define sonar.organization="clickhouse-java" \ No newline at end of file + --define sonar.organization="clickhouse-java" From dc10c7a446d2d18823fe3e1c07d8996aaeb99787 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 28 Oct 2022 13:57:47 -0400 Subject: [PATCH 076/152] no newline at end of file. Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 3 --- 1 file changed, 3 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 1fa6603c960..0bada9eb913 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -123,7 +123,6 @@ jobs: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" "$CACHES_PATH" SonarCloud: - name: Sonar Cloud runs-on: [self-hosted, builder] env: SONAR_SCANNER_VERSION: 4.7.0.2747 @@ -141,7 +140,6 @@ jobs: - name: Download and set up sonar-scanner env: SONAR_SCANNER_DOWNLOAD_URL: https://binaries.sonarsource.com/Distribution/sonar-scanner-cli/sonar-scanner-cli-${{ env.SONAR_SCANNER_VERSION }}-linux.zip - # shellcheck disable=SC2086 run: | mkdir -p $HOME/.sonar curl -sSLo $HOME/.sonar/sonar-scanner.zip ${{ env.SONAR_SCANNER_DOWNLOAD_URL }} @@ -150,7 +148,6 @@ jobs: - name: Download and set up build-wrapper env: BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip - # shellcheck disable=SC2086 run: | curl -sSLo $HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }} unzip -o $HOME/.sonar/build-wrapper-linux-x86.zip -d $HOME/.sonar/ From 51f2d6046c1b40edc6d07d3b63abe2d5d4e1697f Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 28 Oct 2022 14:10:23 -0400 Subject: [PATCH 077/152] sc2086 Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 0bada9eb913..81878ccd16f 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -142,14 +142,14 @@ jobs: SONAR_SCANNER_DOWNLOAD_URL: https://binaries.sonarsource.com/Distribution/sonar-scanner-cli/sonar-scanner-cli-${{ env.SONAR_SCANNER_VERSION }}-linux.zip run: | mkdir -p $HOME/.sonar - curl -sSLo $HOME/.sonar/sonar-scanner.zip ${{ env.SONAR_SCANNER_DOWNLOAD_URL }} + curl -sSLo $HOME/.sonar/sonar-scanner.zip "${{ env.SONAR_SCANNER_DOWNLOAD_URL }}" unzip -o $HOME/.sonar/sonar-scanner.zip -d $HOME/.sonar/ echo "$HOME/.sonar/sonar-scanner-${{ env.SONAR_SCANNER_VERSION }}-linux/bin" >> $GITHUB_PATH - name: Download and set up build-wrapper env: BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip run: | - curl -sSLo $HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }} + curl -sSLo "$HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }}" unzip -o $HOME/.sonar/build-wrapper-linux-x86.zip -d $HOME/.sonar/ echo "$HOME/.sonar/build-wrapper-linux-x86" >> $GITHUB_PATH - name: Set Up Build Tools From 53acc23856f83727b53eda5c39e979038f80c52e Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Fri, 28 Oct 2022 14:17:29 -0400 Subject: [PATCH 078/152] sc2086 Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 81878ccd16f..e6da4df7200 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -141,17 +141,17 @@ jobs: env: SONAR_SCANNER_DOWNLOAD_URL: https://binaries.sonarsource.com/Distribution/sonar-scanner-cli/sonar-scanner-cli-${{ env.SONAR_SCANNER_VERSION }}-linux.zip run: | - mkdir -p $HOME/.sonar - curl -sSLo $HOME/.sonar/sonar-scanner.zip "${{ env.SONAR_SCANNER_DOWNLOAD_URL }}" - unzip -o $HOME/.sonar/sonar-scanner.zip -d $HOME/.sonar/ - echo "$HOME/.sonar/sonar-scanner-${{ env.SONAR_SCANNER_VERSION }}-linux/bin" >> $GITHUB_PATH + mkdir -p "$HOME/.sonar" + curl -sSLo "$HOME/.sonar/sonar-scanner.zip" "${{ env.SONAR_SCANNER_DOWNLOAD_URL }}" + unzip -o "$HOME/.sonar/sonar-scanner.zip" -d "$HOME/.sonar/" + echo "$HOME/.sonar/sonar-scanner-${{ env.SONAR_SCANNER_VERSION }}-linux/bin" >> "$GITHUB_PATH" - name: Download and set up build-wrapper env: BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip run: | curl -sSLo "$HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }}" - unzip -o $HOME/.sonar/build-wrapper-linux-x86.zip -d $HOME/.sonar/ - echo "$HOME/.sonar/build-wrapper-linux-x86" >> $GITHUB_PATH + unzip -o "$HOME/.sonar/build-wrapper-linux-x86.zip" -d "$HOME/.sonar/" + echo "$HOME/.sonar/build-wrapper-linux-x86" >> "$GITHUB_PATH" - name: Set Up Build Tools run: | sudo apt-get update From e903efda0a71c9d47d0217d6cbf4e38fa0125a16 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 28 Oct 2022 16:00:37 -0300 Subject: [PATCH 079/152] Update column.md --- docs/en/sql-reference/statements/alter/column.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 067a350dca7..4c0b79dd8f7 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -204,8 +204,9 @@ It is used if it is necessary to add or update a column with a complicated expre Syntax: ```sql -ALTER TABLE table MATERIALIZE COLUMN col; +ALTER TABLE table MATERIALIZE COLUMN col [IN PARTITION partition | IN PARTITION ID 'partition_id']; ``` +- If you specify a PARTITION, a column will be materialized with only the specified partition. **Example** From 7fa71d8c04e20436e232e1f8c55f6fb46a10f279 Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Fri, 28 Oct 2022 16:04:00 -0300 Subject: [PATCH 080/152] Update column.md --- docs/en/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index 4c0b79dd8f7..f36aa1357f4 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -204,7 +204,7 @@ It is used if it is necessary to add or update a column with a complicated expre Syntax: ```sql -ALTER TABLE table MATERIALIZE COLUMN col [IN PARTITION partition | IN PARTITION ID 'partition_id']; +ALTER TABLE [db.]table [ON CLUSTER cluster] MATERIALIZE COLUMN col [IN PARTITION partition | IN PARTITION ID 'partition_id']; ``` - If you specify a PARTITION, a column will be materialized with only the specified partition. From 004ca99712c0037f4fc3e162eb451775965dfaba Mon Sep 17 00:00:00 2001 From: Yuko Takagi <70714860+yukotakagi@users.noreply.github.com> Date: Fri, 28 Oct 2022 13:58:14 -0600 Subject: [PATCH 081/152] Update README.md --- README.md | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index 003b78a3cbb..f90df9686c2 100644 --- a/README.md +++ b/README.md @@ -5,7 +5,7 @@ ClickHouse® is an open-source column-oriented database management system that a ## Useful Links * [Official website](https://clickhouse.com/) has a quick high-level overview of ClickHouse on the main page. -* [ClickHouse Cloud](https://clickhouse.com/cloud) ClickHouse as a service, built by the creators and maintainers. +* [ClickHouse Cloud](https://clickhouse.cloud) ClickHouse as a service, built by the creators and maintainers. * [Tutorial](https://clickhouse.com/docs/en/getting_started/tutorial/) shows how to set up and query a small ClickHouse cluster. * [Documentation](https://clickhouse.com/docs/en/) provides more in-depth information. * [YouTube channel](https://www.youtube.com/c/ClickHouseDB) has a lot of content about ClickHouse in video format. @@ -16,5 +16,6 @@ ClickHouse® is an open-source column-oriented database management system that a * [Contacts](https://clickhouse.com/company/contact) can help to get your questions answered if there are any. ## Upcoming events -* [**v22.10 Release Webinar**](https://clickhouse.com/company/events/v22-10-release-webinar) Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release, provide live demos, and share vision into what is coming in the roadmap. -* [**Introducing ClickHouse Cloud**](https://clickhouse.com/company/events/cloud-beta) Introducing ClickHouse as a service, built by creators and maintainers of the fastest OLAP database on earth. Join Tanya Bragin for a detailed walkthrough of ClickHouse Cloud capabilities, as well as a peek behind the curtain to understand the unique architecture that makes our service tick. +* [**v22.11 Release Webinar**](https://clickhouse.com/company/events/v22-11-release-webinar) Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release, provide live demos, and share vision into what is coming in the roadmap. +* [**ClickHouse Meetup at the Deutsche Bank office in Berlin**](https://www.meetup.com/clickhouse-berlin-user-group/events/289311596/) Hear from Deutsche Bank on why they chose ClickHouse for big sensitive data in a regulated environment. The ClickHouse team will then present how ClickHouse is used for real time financial data analytics, including tick data, trade analytics and risk management. +* [**AWS re:Invent**](https://clickhouse.com/company/events/aws-reinvent) Core members of the ClickHouse team -- including 2 of our founders -- will be at re:Invent from November 29 to December 3. We are available on the show floor, but are also determining interest in holding an event during the time there. From fe0aea2e3a2011ed803894c8e59c83db357e9e26 Mon Sep 17 00:00:00 2001 From: avogar Date: Fri, 28 Oct 2022 18:44:36 +0000 Subject: [PATCH 082/152] Support parallel parsing for LineAsString input format --- src/Formats/newLineSegmentationEngine.cpp | 50 +++++++++++++++++++ src/Formats/newLineSegmentationEngine.h | 9 ++++ src/Formats/registerFormats.cpp | 2 + .../Impl/LineAsStringRowInputFormat.cpp | 8 ++- .../Formats/Impl/RegexpRowInputFormat.cpp | 40 +-------------- tests/performance/line_as_string_parsing.xml | 9 ++++ 6 files changed, 79 insertions(+), 39 deletions(-) create mode 100644 src/Formats/newLineSegmentationEngine.cpp create mode 100644 src/Formats/newLineSegmentationEngine.h create mode 100644 tests/performance/line_as_string_parsing.xml diff --git a/src/Formats/newLineSegmentationEngine.cpp b/src/Formats/newLineSegmentationEngine.cpp new file mode 100644 index 00000000000..a605bba7e5b --- /dev/null +++ b/src/Formats/newLineSegmentationEngine.cpp @@ -0,0 +1,50 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +std::pair newLineFileSegmentationEngine(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows) +{ + char * pos = in.position(); + bool need_more_data = true; + size_t number_of_rows = 0; + + while (loadAtPosition(in, memory, pos) && need_more_data) + { + pos = find_first_symbols<'\r', '\n'>(pos, in.buffer().end()); + if (pos > in.buffer().end()) + throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR); + else if (pos == in.buffer().end()) + continue; + + ++number_of_rows; + if ((memory.size() + static_cast(pos - in.position()) >= min_bytes) || (number_of_rows == max_rows)) + need_more_data = false; + + if (*pos == '\n') + { + ++pos; + if (loadAtPosition(in, memory, pos) && *pos == '\r') + ++pos; + } + else if (*pos == '\r') + { + ++pos; + if (loadAtPosition(in, memory, pos) && *pos == '\n') + ++pos; + } + } + + saveUpToPosition(in, memory, pos); + + return {loadAtPosition(in, memory, pos), number_of_rows}; +} + +} diff --git a/src/Formats/newLineSegmentationEngine.h b/src/Formats/newLineSegmentationEngine.h new file mode 100644 index 00000000000..598f808b798 --- /dev/null +++ b/src/Formats/newLineSegmentationEngine.h @@ -0,0 +1,9 @@ +#pragma once + +#include +#include + +namespace DB +{ + std::pair newLineFileSegmentationEngine(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows); +} diff --git a/src/Formats/registerFormats.cpp b/src/Formats/registerFormats.cpp index 593e4568be1..ba40fe442ab 100644 --- a/src/Formats/registerFormats.cpp +++ b/src/Formats/registerFormats.cpp @@ -18,6 +18,7 @@ void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory); #if USE_HIVE void registerFileSegmentationEngineHiveText(FormatFactory & factory); #endif +void registerFileSegmentationEngineLineAsString(FormatFactory & factory); /// Formats for both input/output. @@ -153,6 +154,7 @@ void registerFormats() #if USE_HIVE registerFileSegmentationEngineHiveText(factory); #endif + registerFileSegmentationEngineLineAsString(factory); registerInputFormatNative(factory); diff --git a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp index 30084804d92..677f8bb28ec 100644 --- a/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/LineAsStringRowInputFormat.cpp @@ -1,5 +1,5 @@ #include -#include +#include #include #include #include @@ -63,6 +63,12 @@ void registerInputFormatLineAsString(FormatFactory & factory) }); } +void registerFileSegmentationEngineLineAsString(FormatFactory & factory) +{ + factory.registerFileSegmentationEngine("LineAsString", &newLineFileSegmentationEngine); +} + + void registerLineAsStringSchemaReader(FormatFactory & factory) { factory.registerExternalSchemaReader("LineAsString", []( diff --git a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp index 6eacfe621e1..2ad2ad6f7a3 100644 --- a/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp +++ b/src/Processors/Formats/Impl/RegexpRowInputFormat.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include namespace DB @@ -178,46 +179,9 @@ void registerInputFormatRegexp(FormatFactory & factory) }); } -static std::pair fileSegmentationEngineRegexpImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_bytes, size_t max_rows) -{ - char * pos = in.position(); - bool need_more_data = true; - size_t number_of_rows = 0; - - while (loadAtPosition(in, memory, pos) && need_more_data) - { - pos = find_first_symbols<'\r', '\n'>(pos, in.buffer().end()); - if (pos > in.buffer().end()) - throw Exception("Position in buffer is out of bounds. There must be a bug.", ErrorCodes::LOGICAL_ERROR); - else if (pos == in.buffer().end()) - continue; - - ++number_of_rows; - if ((memory.size() + static_cast(pos - in.position()) >= min_bytes) || (number_of_rows == max_rows)) - need_more_data = false; - - if (*pos == '\n') - { - ++pos; - if (loadAtPosition(in, memory, pos) && *pos == '\r') - ++pos; - } - else if (*pos == '\r') - { - ++pos; - if (loadAtPosition(in, memory, pos) && *pos == '\n') - ++pos; - } - } - - saveUpToPosition(in, memory, pos); - - return {loadAtPosition(in, memory, pos), number_of_rows}; -} - void registerFileSegmentationEngineRegexp(FormatFactory & factory) { - factory.registerFileSegmentationEngine("Regexp", &fileSegmentationEngineRegexpImpl); + factory.registerFileSegmentationEngine("Regexp", &newLineFileSegmentationEngine); } void registerRegexpSchemaReader(FormatFactory & factory) diff --git a/tests/performance/line_as_string_parsing.xml b/tests/performance/line_as_string_parsing.xml new file mode 100644 index 00000000000..d9fa1d4fa6e --- /dev/null +++ b/tests/performance/line_as_string_parsing.xml @@ -0,0 +1,9 @@ + + +INSERT INTO FUNCTION file(test_line_as_string.tsv) SELECT randomString(1000) FROM numbers(1000000) SETTINGS engine_file_truncate_on_insert=1 + +SELECT * FROM file(test_line_as_string.tsv, LineAsString) FORMAT Null + +INSERT INTO FUNCTION file(test_line_as_string.tsv) SELECT * FROM numbers(0) SETTINGS engine_file_truncate_on_insert=1 + + From ca5dbe88dbbde7bc139b6bfc864a8a1b52167ca7 Mon Sep 17 00:00:00 2001 From: clarkcaoliu Date: Wed, 19 Oct 2022 03:18:12 +0800 Subject: [PATCH 083/152] match function can use index if prefix --- src/Storages/MergeTree/KeyCondition.cpp | 105 +++++++++++++++++- .../02462_match_regexp_pk.reference | 5 + .../0_stateless/02462_match_regexp_pk.sql | 9 ++ 3 files changed, 118 insertions(+), 1 deletion(-) create mode 100644 tests/queries/0_stateless/02462_match_regexp_pk.reference create mode 100644 tests/queries/0_stateless/02462_match_regexp_pk.sql diff --git a/src/Storages/MergeTree/KeyCondition.cpp b/src/Storages/MergeTree/KeyCondition.cpp index 927ca98cc9a..8fc5f13d27d 100644 --- a/src/Storages/MergeTree/KeyCondition.cpp +++ b/src/Storages/MergeTree/KeyCondition.cpp @@ -86,6 +86,88 @@ String extractFixedPrefixFromLikePattern(const String & like_pattern) return fixed_prefix; } +/// for "^prefix..." string it returns "prefix" +static String extractFixedPrefixFromRegularExpression(const String & regexp) +{ + if (regexp.size() <= 1 || regexp[0] != '^') + return {}; + + String fixed_prefix; + const char * begin = regexp.data() + 1; + const char * pos = begin; + const char * end = regexp.data() + regexp.size(); + + while (pos != end) + { + switch (*pos) + { + case '\0': + pos = end; + break; + + case '\\': + { + ++pos; + if (pos == end) + break; + + switch (*pos) + { + case '|': + case '(': + case ')': + case '^': + case '$': + case '.': + case '[': + case '?': + case '*': + case '+': + case '{': + fixed_prefix += *pos; + break; + default: + /// all other escape sequences are not supported + pos = end; + break; + } + + ++pos; + break; + } + + /// non-trivial cases + case '|': + fixed_prefix.clear(); + [[fallthrough]]; + case '(': + case '[': + case '^': + case '$': + case '.': + case '+': + pos = end; + break; + + /// Quantifiers that allow a zero number of occurrences. + case '{': + case '?': + case '*': + if (!fixed_prefix.empty()) + fixed_prefix.pop_back(); + + pos = end; + break; + default: + fixed_prefix += *pos; + pos++; + break; + } + } + + return fixed_prefix; +} + /** For a given string, get a minimum string that is strictly greater than all strings with this prefix, * or return an empty string if there are no such strings. @@ -581,6 +663,27 @@ const KeyCondition::AtomMap KeyCondition::atom_map return true; } }, + { + "match", + [] (RPNElement & out, const Field & value) + { + if (value.getType() != Field::Types::String) + return false; + + String prefix = extractFixedPrefixFromRegularExpression(value.get()); + if (prefix.empty()) + return false; + + String right_bound = firstStringThatIsGreaterThanAllStringsWithPrefix(prefix); + + out.function = RPNElement::FUNCTION_IN_RANGE; + out.range = !right_bound.empty() + ? Range(prefix, true, right_bound, false) + : Range::createLeftBounded(prefix, true); + + return true; + } + }, { "isNotNull", [] (RPNElement & out, const Field &) @@ -1738,7 +1841,7 @@ bool KeyCondition::tryParseAtomFromAST(const Tree & node, ContextPtr context, Bl else if (func_name == "in" || func_name == "notIn" || func_name == "like" || func_name == "notLike" || func_name == "ilike" || func_name == "notIlike" || - func_name == "startsWith") + func_name == "startsWith" || func_name == "match") { /// "const IN data_column" doesn't make sense (unlike "data_column IN const") return false; diff --git a/tests/queries/0_stateless/02462_match_regexp_pk.reference b/tests/queries/0_stateless/02462_match_regexp_pk.reference new file mode 100644 index 00000000000..428d6556f4c --- /dev/null +++ b/tests/queries/0_stateless/02462_match_regexp_pk.reference @@ -0,0 +1,5 @@ +4 +1 +3 +4 +4 diff --git a/tests/queries/0_stateless/02462_match_regexp_pk.sql b/tests/queries/0_stateless/02462_match_regexp_pk.sql new file mode 100644 index 00000000000..1a944b96196 --- /dev/null +++ b/tests/queries/0_stateless/02462_match_regexp_pk.sql @@ -0,0 +1,9 @@ +CREATE TABLE mt_match_pk (v String) ENGINE = MergeTree ORDER BY v SETTINGS index_granularity = 1; +INSERT INTO mt_match_pk VALUES ('a'), ('aaa'), ('aba'), ('bac'), ('acccca'); + +SET force_primary_key = 1; +SELECT count() FROM mt_match_pk WHERE match(v, '^a'); +SELECT count() FROM mt_match_pk WHERE match(v, '^ab'); +SELECT count() FROM mt_match_pk WHERE match(v, '^a.'); +SELECT count() FROM mt_match_pk WHERE match(v, '^ab*'); +SELECT count() FROM mt_match_pk WHERE match(v, '^ac?'); From 8de75bf29c57d58ee430227ecf230c2e186438ba Mon Sep 17 00:00:00 2001 From: Anton Popov Date: Sat, 29 Oct 2022 14:26:34 +0000 Subject: [PATCH 084/152] better code with projections --- src/Storages/MergeTree/DataPartsExchange.cpp | 199 +++++++----------- src/Storages/MergeTree/DataPartsExchange.h | 13 +- .../MergeTree/MergeTreeDataWriter.cpp | 113 ++++------ src/Storages/MergeTree/MergeTreeDataWriter.h | 15 +- .../MergeTree/MergeTreeWriteAheadLog.cpp | 2 +- src/Storages/MergeTree/checkDataPart.cpp | 133 ++++-------- 6 files changed, 173 insertions(+), 302 deletions(-) diff --git a/src/Storages/MergeTree/DataPartsExchange.cpp b/src/Storages/MergeTree/DataPartsExchange.cpp index afef354a2e6..4f9c9ffd596 100644 --- a/src/Storages/MergeTree/DataPartsExchange.cpp +++ b/src/Storages/MergeTree/DataPartsExchange.cpp @@ -180,6 +180,13 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write std::sregex_token_iterator(remote_fs_metadata.begin(), remote_fs_metadata.end(), re, -1), std::sregex_token_iterator()); + bool send_projections = client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION; + if (send_projections) + { + const auto & projections = part->getProjectionParts(); + writeBinary(projections.size(), out); + } + if (data_settings->allow_remote_fs_zero_copy_replication && /// In memory data part does not have metadata yet. !isInMemoryPart(part) && @@ -190,33 +197,15 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write { /// Send metadata if the receiver's capability covers the source disk type. response.addCookie({"remote_fs_metadata", disk_type}); - if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION) - { - const auto & projections = part->getProjectionParts(); - writeBinary(projections.size(), out); - } - - sendPartFromDiskRemoteMeta(part, out, true, part->getProjectionParts()); + sendPartFromDiskRemoteMeta(part, out, true, send_projections); return; } } - if (client_protocol_version >= REPLICATION_PROTOCOL_VERSION_WITH_PARTS_PROJECTION) - { - const auto & projections = part->getProjectionParts(); - writeBinary(projections.size(), out); - if (isInMemoryPart(part)) - sendPartFromMemory(part, out, projections); - else - sendPartFromDisk(part, out, client_protocol_version, projections); - } + if (isInMemoryPart(part)) + sendPartFromMemory(part, out, send_projections); else - { - if (isInMemoryPart(part)) - sendPartFromMemory(part, out); - else - sendPartFromDisk(part, out, client_protocol_version); - } + sendPartFromDisk(part, out, client_protocol_version, send_projections); } catch (const NetException &) { @@ -238,20 +227,23 @@ void Service::processQuery(const HTMLForm & params, ReadBuffer & /*body*/, Write } void Service::sendPartFromMemory( - const MergeTreeData::DataPartPtr & part, WriteBuffer & out, const std::map> & projections) + const MergeTreeData::DataPartPtr & part, WriteBuffer & out, bool send_projections) { auto metadata_snapshot = data.getInMemoryMetadataPtr(); - for (const auto & [name, projection] : projections) + if (send_projections) { - auto projection_sample_block = metadata_snapshot->projections.get(name).sample_block; - auto part_in_memory = asInMemoryPart(projection); - if (!part_in_memory) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection {} of part {} is not stored in memory", name, part->name); + for (const auto & [name, projection] : part->getProjectionParts()) + { + auto projection_sample_block = metadata_snapshot->projections.get(name).sample_block; + auto part_in_memory = asInMemoryPart(projection); + if (!part_in_memory) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Projection {} of part {} is not stored in memory", name, part->name); - writeStringBinary(name, out); - projection->checksums.write(out); - NativeWriter block_out(out, 0, projection_sample_block); - block_out.write(part_in_memory->block); + writeStringBinary(name, out); + projection->checksums.write(out); + NativeWriter block_out(out, 0, projection_sample_block); + block_out.write(part_in_memory->block); + } } auto part_in_memory = asInMemoryPart(part); @@ -269,7 +261,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( const MergeTreeData::DataPartPtr & part, WriteBuffer & out, int client_protocol_version, - const std::map> & projections) + bool send_projections) { /// We'll take a list of files from the list of checksums. MergeTreeData::DataPart::Checksums checksums = part->checksums; @@ -277,7 +269,8 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( auto file_names_without_checksums = part->getFileNamesWithoutChecksums(); for (const auto & file_name : file_names_without_checksums) { - if (client_protocol_version < REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION && file_name == IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME) + if (client_protocol_version < REPLICATION_PROTOCOL_VERSION_WITH_PARTS_DEFAULT_COMPRESSION + && file_name == IMergeTreeDataPart::DEFAULT_COMPRESSION_CODEC_FILE_NAME) continue; checksums.files[file_name] = {}; @@ -288,11 +281,10 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( { // Get rid of projection files checksums.files.erase(name + ".proj"); - auto it = projections.find(name); - if (it != projections.end()) + if (send_projections) { writeStringBinary(name, out); - MergeTreeData::DataPart::Checksums projection_checksum = sendPartFromDisk(it->second, out, client_protocol_version); + MergeTreeData::DataPart::Checksums projection_checksum = sendPartFromDisk(projection, out, client_protocol_version, false); data_checksums.addFile(name + ".proj", projection_checksum.getTotalSizeOnDisk(), projection_checksum.getTotalChecksumUInt128()); } else if (part->checksums.has(name + ".proj")) @@ -337,18 +329,15 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDisk( return data_checksums; } -MergeTreeData::DataPart::Checksums Service::sendPartFromDiskRemoteMeta( +void Service::sendPartFromDiskRemoteMeta( const MergeTreeData::DataPartPtr & part, WriteBuffer & out, bool send_part_id, - const std::map> & projections) + bool send_projections) { - const auto * data_part_storage_on_disk = dynamic_cast(&part->getDataPartStorage()); - if (!data_part_storage_on_disk) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Storage '{}' doesn't support zero-copy replication", part->getDataPartStorage().getDiskName()); - - if (!data_part_storage_on_disk->supportZeroCopyReplication()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk '{}' doesn't support zero-copy replication", data_part_storage_on_disk->getDiskName()); + auto data_part_storage = part->getDataPartStoragePtr(); + if (!data_part_storage->supportZeroCopyReplication()) + throw Exception(ErrorCodes::LOGICAL_ERROR, "Disk '{}' doesn't support zero-copy replication", data_part_storage->getDiskName()); /// We'll take a list of files from the list of checksums. MergeTreeData::DataPart::Checksums checksums = part->checksums; @@ -369,30 +358,20 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDiskRemoteMeta( paths.push_back(fs::path(part->getDataPartStorage().getRelativePath()) / it.first); /// Serialized metadatadatas with zero ref counts. - auto metadatas = data_part_storage_on_disk->getSerializedMetadata(paths); + auto metadatas = data_part_storage->getSerializedMetadata(paths); if (send_part_id) { - String part_id = data_part_storage_on_disk->getUniqueId(); + String part_id = data_part_storage->getUniqueId(); writeStringBinary(part_id, out); } - MergeTreeData::DataPart::Checksums data_checksums; - for (const auto & [name, projection] : part->getProjectionParts()) + if (send_projections) { - auto it = projections.find(name); - if (it != projections.end()) + for (const auto & [name, projection] : part->getProjectionParts()) { - writeStringBinary(name, out); - MergeTreeData::DataPart::Checksums projection_checksum = sendPartFromDiskRemoteMeta(it->second, out, false); - data_checksums.addFile(name + ".proj", projection_checksum.getTotalSizeOnDisk(), projection_checksum.getTotalChecksumUInt128()); - } - else if (part->checksums.has(name + ".proj")) - { - // We don't send this projection, just add out checksum to bypass the following check - const auto & our_checksum = part->checksums.files.find(name + ".proj")->second; - data_checksums.addFile(name + ".proj", our_checksum.file_size, our_checksum.file_hash); + sendPartFromDiskRemoteMeta(projection, out, false, false); } } @@ -403,7 +382,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDiskRemoteMeta( String file_path_prefix = fs::path(part->getDataPartStorage().getRelativePath()) / file_name; /// Just some additional checks - String metadata_file_path = fs::path(data_part_storage_on_disk->getDiskPath()) / file_path_prefix; + String metadata_file_path = fs::path(data_part_storage->getDiskPath()) / file_path_prefix; fs::path metadata(metadata_file_path); if (!fs::exists(metadata)) throw Exception(ErrorCodes::CORRUPTED_DATA, "Remote metadata '{}' is not exists", file_name); @@ -427,12 +406,7 @@ MergeTreeData::DataPart::Checksums Service::sendPartFromDiskRemoteMeta( throw Exception(ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, "Unexpected size of file {}", metadata_file_path); writePODBinary(hashing_out.getHash(), out); - - if (!file_names_without_checksums.contains(file_name)) - data_checksums.addFile(file_name, hashing_out.count(), hashing_out.getHash()); } - - return data_checksums; } MergeTreeData::DataPartPtr Service::findPart(const String & name) @@ -707,68 +681,54 @@ MergeTreeData::MutableDataPartPtr Fetcher::fetchSelectedPart( in->setNextCallback(ReplicatedFetchReadCallback(*entry)); - return part_type == "InMemory" - ? downloadPartToMemory(part_name, part_uuid, metadata_snapshot, context, disk, *in, projections, throttler) - : downloadPartToDisk(part_name, replica_path, to_detached, tmp_prefix, sync, disk, *in, projections, checksums, throttler); + if (part_type == "InMemory") + { + auto volume = std::make_shared("volume_" + part_name, disk, 0); + + auto data_part_storage = std::make_shared( + volume, + data.getRelativeDataPath(), + part_name); + + return downloadPartToMemory( + data_part_storage, part_name, + MergeTreePartInfo::fromPartName(part_name, data.format_version), + part_uuid, metadata_snapshot, context, *in, + projections, false, throttler); + } + + return downloadPartToDisk( + part_name, replica_path, to_detached, tmp_prefix, + sync, disk, *in, projections, checksums, throttler); } MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( + MutableDataPartStoragePtr data_part_storage, const String & part_name, + const MergeTreePartInfo & part_info, const UUID & part_uuid, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - DiskPtr disk, PooledReadWriteBufferFromHTTP & in, size_t projections, + bool is_projection, ThrottlerPtr throttler) { - auto volume = std::make_shared("volume_" + part_name, disk, 0); + auto new_data_part = std::make_shared(data, part_name, part_info, data_part_storage); - auto data_part_storage = std::make_shared( - volume, - data.getRelativeDataPath(), - part_name); - - MergeTreeData::MutableDataPartPtr new_data_part = - std::make_shared(data, part_name, data_part_storage); - - new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr); - - for (auto i = 0ul; i < projections; ++i) + for (size_t i = 0; i < projections; ++i) { String projection_name; readStringBinary(projection_name, in); - MergeTreeData::DataPart::Checksums checksums; - if (!checksums.read(in)) - throw Exception("Cannot deserialize checksums", ErrorCodes::CORRUPTED_DATA); - - NativeReader block_in(in, 0); - auto block = block_in.read(); - throttler->add(block.bytes()); - - auto projection_part_storage = data_part_storage->getProjection(projection_name + ".proj"); MergeTreePartInfo new_part_info("all", 0, 0, 0); - MergeTreeData::MutableDataPartPtr new_projection_part = - std::make_shared(data, projection_name, new_part_info, projection_part_storage, new_data_part.get()); + auto projection_part_storage = data_part_storage->getProjection(projection_name + ".proj"); - new_projection_part->is_temp = false; - new_projection_part->setColumns(block.getNamesAndTypesList(), {}); - MergeTreePartition partition{}; - new_projection_part->partition = std::move(partition); - new_projection_part->minmax_idx = std::make_shared(); + auto new_projection_part = downloadPartToMemory( + projection_part_storage, projection_name, + new_part_info, part_uuid, metadata_snapshot, + context, in, 0, true, throttler); - MergedBlockOutputStream part_out( - new_projection_part, - metadata_snapshot->projections.get(projection_name).metadata, - block.getNamesAndTypesList(), - {}, - CompressionCodecFactory::instance().get("NONE", {}), - NO_TRANSACTION_PTR); - - part_out.write(block); - part_out.finalizePart(new_projection_part, false); - new_projection_part->checksums.checkEqual(checksums, /* have_uncompressed = */ true); new_data_part->addProjectionPart(projection_name, std::move(new_projection_part)); } @@ -780,11 +740,16 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToMemory( auto block = block_in.read(); throttler->add(block.bytes()); - new_data_part->uuid = part_uuid; - new_data_part->is_temp = true; new_data_part->setColumns(block.getNamesAndTypesList(), {}); - new_data_part->minmax_idx->update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); - new_data_part->partition.create(metadata_snapshot, block, 0, context); + + if (!is_projection) + { + new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr); + new_data_part->uuid = part_uuid; + new_data_part->is_temp = true; + new_data_part->minmax_idx->update(block, data.getMinMaxColumnsNames(metadata_snapshot->getPartitionKey())); + new_data_part->partition.create(metadata_snapshot, block, 0, context); + } MergedBlockOutputStream part_out( new_data_part, metadata_snapshot, block.getNamesAndTypesList(), {}, @@ -850,7 +815,6 @@ void Fetcher::downloadBasePartOrProjectionPartToDiskRemoteMeta( checksums.addFile(file_name, file_size, expected_hash); } } - } @@ -966,11 +930,11 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDisk( SyncGuardPtr sync_guard; if (data.getSettings()->fsync_part_directory) - sync_guard = disk->getDirectorySyncGuard(data_part_storage->getRelativePath()); + sync_guard = data_part_storage->getDirectorySyncGuard(); CurrentMetrics::Increment metric_increment{CurrentMetrics::ReplicatedFetch}; - for (auto i = 0ul; i < projections; ++i) + for (size_t i = 0; i < projections; ++i) { String projection_name; readStringBinary(projection_name, in); @@ -1043,7 +1007,7 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDiskRemoteMeta( data_part_storage->createDirectories(); - for (auto i = 0ul; i < projections; ++i) + for (size_t i = 0; i < projections; ++i) { String projection_name; readStringBinary(projection_name, in); @@ -1071,7 +1035,6 @@ MergeTreeData::MutableDataPartPtr Fetcher::downloadPartToDiskRemoteMeta( new_data_part->version.setCreationTID(Tx::PrehistoricTID, nullptr); new_data_part->is_temp = true; new_data_part->modification_time = time(nullptr); - new_data_part->loadColumnsChecksumsIndexes(true, false); } #if USE_AWS_S3 diff --git a/src/Storages/MergeTree/DataPartsExchange.h b/src/Storages/MergeTree/DataPartsExchange.h index 59800756c34..6c92fad4092 100644 --- a/src/Storages/MergeTree/DataPartsExchange.h +++ b/src/Storages/MergeTree/DataPartsExchange.h @@ -1,5 +1,6 @@ #pragma once +#include "Storages/MergeTree/MergeTreePartInfo.h" #include #include #include @@ -42,19 +43,19 @@ private: void sendPartFromMemory( const MergeTreeData::DataPartPtr & part, WriteBuffer & out, - const std::map> & projections = {}); + bool send_projections); MergeTreeData::DataPart::Checksums sendPartFromDisk( const MergeTreeData::DataPartPtr & part, WriteBuffer & out, int client_protocol_version, - const std::map> & projections = {}); + bool send_projections); - MergeTreeData::DataPart::Checksums sendPartFromDiskRemoteMeta( + void sendPartFromDiskRemoteMeta( const MergeTreeData::DataPartPtr & part, WriteBuffer & out, bool send_part_id, - const std::map> & projections = {}); + bool send_projections); /// StorageReplicatedMergeTree::shutdown() waits for all parts exchange handlers to finish, /// so Service will never access dangling reference to storage @@ -120,13 +121,15 @@ private: ThrottlerPtr throttler); MergeTreeData::MutableDataPartPtr downloadPartToMemory( + MutableDataPartStoragePtr data_part_storage, const String & part_name, + const MergeTreePartInfo & part_info, const UUID & part_uuid, const StorageMetadataPtr & metadata_snapshot, ContextPtr context, - DiskPtr disk, PooledReadWriteBufferFromHTTP & in, size_t projections, + bool is_projection, ThrottlerPtr throttler); MergeTreeData::MutableDataPartPtr downloadPartToDiskRemoteMeta( diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 55404324b75..7e064f5164f 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -480,8 +480,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempPart( MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( const String & part_name, - MergeTreeDataPartType part_type, - const String & relative_path, bool is_temp, IMergeTreeDataPart * parent_part, const MergeTreeData & data, @@ -493,6 +491,21 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( const StorageMetadataPtr & metadata_snapshot = projection.metadata; MergeTreePartInfo new_part_info("all", 0, 0, 0); + MergeTreeDataPartType part_type; + if (parent_part->getType() == MergeTreeDataPartType::InMemory) + { + part_type = MergeTreeDataPartType::InMemory; + } + else + { + /// Size of part would not be greater than block.bytes() + epsilon + size_t expected_size = block.bytes(); + // just check if there is enough space on parent volume + data.reserveSpace(expected_size, parent_part->getDataPartStorage()); + part_type = data.choosePartTypeOnDisk(expected_size, block.rows()); + } + + auto relative_path = part_name + (is_temp ? ".tmp_proj" : ".proj"); auto projection_part_storage = parent_part->getDataPartStorage().getProjection(relative_path); auto new_data_part = data.createPart( part_name, @@ -583,77 +596,6 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPartImpl( } MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeProjectionPart( - MergeTreeData & data, - Poco::Logger * log, - Block block, - const ProjectionDescription & projection, - IMergeTreeDataPart * parent_part) -{ - String part_name = projection.name; - MergeTreeDataPartType part_type; - if (parent_part->getType() == MergeTreeDataPartType::InMemory) - { - part_type = MergeTreeDataPartType::InMemory; - } - else - { - /// Size of part would not be greater than block.bytes() + epsilon - size_t expected_size = block.bytes(); - // just check if there is enough space on parent volume - data.reserveSpace(expected_size, parent_part->getDataPartStorage()); - part_type = data.choosePartTypeOnDisk(expected_size, block.rows()); - } - - return writeProjectionPartImpl( - part_name, - part_type, - part_name + ".proj" /* relative_path */, - false /* is_temp */, - parent_part, - data, - log, - block, - projection); -} - -/// This is used for projection materialization process which may contain multiple stages of -/// projection part merges. -MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempProjectionPart( - MergeTreeData & data, - Poco::Logger * log, - Block block, - const ProjectionDescription & projection, - IMergeTreeDataPart * parent_part, - size_t block_num) -{ - String part_name = fmt::format("{}_{}", projection.name, block_num); - MergeTreeDataPartType part_type; - if (parent_part->getType() == MergeTreeDataPartType::InMemory) - { - part_type = MergeTreeDataPartType::InMemory; - } - else - { - /// Size of part would not be greater than block.bytes() + epsilon - size_t expected_size = block.bytes(); - // just check if there is enough space on parent volume - data.reserveSpace(expected_size, parent_part->getDataPartStorage()); - part_type = data.choosePartTypeOnDisk(expected_size, block.rows()); - } - - return writeProjectionPartImpl( - part_name, - part_type, - part_name + ".tmp_proj" /* relative_path */, - true /* is_temp */, - parent_part, - data, - log, - block, - projection); -} - -MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeInMemoryProjectionPart( const MergeTreeData & data, Poco::Logger * log, Block block, @@ -662,13 +604,32 @@ MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeInMemoryProjectionP { return writeProjectionPartImpl( projection.name, - MergeTreeDataPartType::InMemory, - projection.name + ".proj" /* relative_path */, false /* is_temp */, parent_part, data, log, - block, + std::move(block), + projection); +} + +/// This is used for projection materialization process which may contain multiple stages of +/// projection part merges. +MergeTreeDataWriter::TemporaryPart MergeTreeDataWriter::writeTempProjectionPart( + const MergeTreeData & data, + Poco::Logger * log, + Block block, + const ProjectionDescription & projection, + IMergeTreeDataPart * parent_part, + size_t block_num) +{ + String part_name = fmt::format("{}_{}", projection.name, block_num); + return writeProjectionPartImpl( + part_name, + true /* is_temp */, + parent_part, + data, + log, + std::move(block), projection); } diff --git a/src/Storages/MergeTree/MergeTreeDataWriter.h b/src/Storages/MergeTree/MergeTreeDataWriter.h index 8e405016cde..8c2bf66e8f8 100644 --- a/src/Storages/MergeTree/MergeTreeDataWriter.h +++ b/src/Storages/MergeTree/MergeTreeDataWriter.h @@ -73,7 +73,7 @@ public: /// For insertion. static TemporaryPart writeProjectionPart( - MergeTreeData & data, + const MergeTreeData & data, Poco::Logger * log, Block block, const ProjectionDescription & projection, @@ -81,21 +81,13 @@ public: /// For mutation: MATERIALIZE PROJECTION. static TemporaryPart writeTempProjectionPart( - MergeTreeData & data, + const MergeTreeData & data, Poco::Logger * log, Block block, const ProjectionDescription & projection, IMergeTreeDataPart * parent_part, size_t block_num); - /// For WriteAheadLog AddPart. - static TemporaryPart writeInMemoryProjectionPart( - const MergeTreeData & data, - Poco::Logger * log, - Block block, - const ProjectionDescription & projection, - IMergeTreeDataPart * parent_part); - static Block mergeBlock( const Block & block, SortDescription sort_description, @@ -106,8 +98,6 @@ public: private: static TemporaryPart writeProjectionPartImpl( const String & part_name, - MergeTreeDataPartType part_type, - const String & relative_path, bool is_temp, IMergeTreeDataPart * parent_part, const MergeTreeData & data, @@ -116,7 +106,6 @@ private: const ProjectionDescription & projection); MergeTreeData & data; - Poco::Logger * log; }; diff --git a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp index 85b41f6dd83..b3625ba8e93 100644 --- a/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp +++ b/src/Storages/MergeTree/MergeTreeWriteAheadLog.cpp @@ -237,7 +237,7 @@ MergeTreeData::MutableDataPartsVector MergeTreeWriteAheadLog::restore( for (const auto & projection : metadata_snapshot->getProjections()) { auto projection_block = projection.calculate(block, context); - auto temp_part = MergeTreeDataWriter::writeInMemoryProjectionPart(storage, log, projection_block, projection, part.get()); + auto temp_part = MergeTreeDataWriter::writeProjectionPart(storage, log, projection_block, projection, part.get()); temp_part.finalize(); if (projection_block.rows()) part->addProjectionPart(projection.name, std::move(temp_part.part)); diff --git a/src/Storages/MergeTree/checkDataPart.cpp b/src/Storages/MergeTree/checkDataPart.cpp index 4758ccb201a..6f9f16b6155 100644 --- a/src/Storages/MergeTree/checkDataPart.cpp +++ b/src/Storages/MergeTree/checkDataPart.cpp @@ -112,98 +112,17 @@ IMergeTreeDataPart::Checksums checkDataPart( }; /// This function calculates only checksum of file content (compressed or uncompressed). - /// It also calculates checksum of projections. auto checksum_file = [&](const String & file_name) { - if (data_part_storage.isDirectory(file_name) && endsWith(file_name, ".proj")) - { - auto projection_name = file_name.substr(0, file_name.size() - sizeof(".proj") + 1); - auto pit = data_part->getProjectionParts().find(projection_name); - if (pit == data_part->getProjectionParts().end()) - { - if (require_checksums) - throw Exception("Unexpected file " + file_name + " in data part", ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART); - else - return; - } - - const auto & projection = pit->second; - IMergeTreeDataPart::Checksums projection_checksums_data; - - auto projection_part_storage = data_part_storage.getProjection(file_name); - - if (projection->getType() == MergeTreeDataPartType::Compact) - { - auto file_buf = projection_part_storage->readFile(MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION, {}, std::nullopt, std::nullopt); - HashingReadBuffer hashing_buf(*file_buf); - hashing_buf.ignoreAll(); - projection_checksums_data.files[MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION] - = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); - } - else - { - const NamesAndTypesList & projection_columns_list = projection->getColumns(); - for (const auto & projection_column : projection_columns_list) - { - get_serialization(projection_column)->enumerateStreams( - [&](const ISerialization::SubstreamPath & substream_path) - { - String projection_file_name = ISerialization::getFileNameForStream(projection_column, substream_path) + ".bin"; - projection_checksums_data.files[projection_file_name] = checksum_compressed_file(*projection_part_storage, projection_file_name); - }); - } - } - - IMergeTreeDataPart::Checksums projection_checksums_txt; - - if (require_checksums || projection_part_storage->exists("checksums.txt")) - { - auto buf = projection_part_storage->readFile("checksums.txt", {}, std::nullopt, std::nullopt); - projection_checksums_txt.read(*buf); - assertEOF(*buf); - } - - const auto & projection_checksum_files_txt = projection_checksums_txt.files; - for (auto projection_it = projection_part_storage->iterate(); projection_it->isValid(); projection_it->next()) - { - const String & projection_file_name = projection_it->name(); - auto projection_checksum_it = projection_checksums_data.files.find(projection_file_name); - - /// Skip files that we already calculated. Also skip metadata files that are not checksummed. - if (projection_checksum_it == projection_checksums_data.files.end() && !files_without_checksums.contains(projection_file_name)) - { - auto projection_txt_checksum_it = projection_checksum_files_txt.find(file_name); - if (projection_txt_checksum_it == projection_checksum_files_txt.end() - || projection_txt_checksum_it->second.uncompressed_size == 0) - { - auto projection_file_buf = projection_part_storage->readFile(projection_file_name, {}, std::nullopt, std::nullopt); - HashingReadBuffer projection_hashing_buf(*projection_file_buf); - projection_hashing_buf.ignoreAll(); - projection_checksums_data.files[projection_file_name] = IMergeTreeDataPart::Checksums::Checksum( - projection_hashing_buf.count(), projection_hashing_buf.getHash()); - } - else - { - projection_checksums_data.files[projection_file_name] = checksum_compressed_file(*projection_part_storage, projection_file_name); - } - } - } - checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum( - projection_checksums_data.getTotalSizeOnDisk(), projection_checksums_data.getTotalChecksumUInt128()); - - if (require_checksums || !projection_checksums_txt.files.empty()) - projection_checksums_txt.checkEqual(projection_checksums_data, false); - } - else - { - auto file_buf = data_part_storage.readFile(file_name, {}, std::nullopt, std::nullopt); - HashingReadBuffer hashing_buf(*file_buf); - hashing_buf.ignoreAll(); - checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); - } + auto file_buf = data_part_storage.readFile(file_name, {}, std::nullopt, std::nullopt); + HashingReadBuffer hashing_buf(*file_buf); + hashing_buf.ignoreAll(); + checksums_data.files[file_name] = IMergeTreeDataPart::Checksums::Checksum(hashing_buf.count(), hashing_buf.getHash()); }; - bool check_uncompressed = true; + /// Do not check uncompressed for projections. But why? + bool check_uncompressed = !data_part->isProjectionPart(); + /// First calculate checksums for columns data if (part_type == MergeTreeDataPartType::Compact) { @@ -238,10 +157,19 @@ IMergeTreeDataPart::Checksums checkDataPart( assertEOF(*buf); } + NameSet projections_on_disk; const auto & checksum_files_txt = checksums_txt.files; for (auto it = data_part_storage.iterate(); it->isValid(); it->next()) { - const String & file_name = it->name(); + auto file_name = it->name(); + + /// We will check projections later. + if (data_part_storage.isDirectory(file_name) && endsWith(file_name, ".proj")) + { + projections_on_disk.insert(file_name); + continue; + } + auto checksum_it = checksums_data.files.find(file_name); /// Skip files that we already calculated. Also skip metadata files that are not checksummed. @@ -260,11 +188,38 @@ IMergeTreeDataPart::Checksums checkDataPart( } } + for (const auto & [name, projection] : data_part->getProjectionParts()) + { + if (is_cancelled()) + return {}; + + auto projection_file = name + ".proj"; + auto projection_checksums = checkDataPart( + projection, *data_part_storage.getProjection(projection_file), + projection->getColumns(), projection->getType(), + projection->getFileNamesWithoutChecksums(), + require_checksums, is_cancelled); + + checksums_data.files[projection_file] = IMergeTreeDataPart::Checksums::Checksum( + projection_checksums.getTotalSizeOnDisk(), + projection_checksums.getTotalChecksumUInt128()); + + projections_on_disk.erase(projection_file); + } + + if (require_checksums && !projections_on_disk.empty()) + { + throw Exception(ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART, + "Found unexpected projection directories: {}", + fmt::join(projections_on_disk, ",")); + } + if (is_cancelled()) return {}; if (require_checksums || !checksums_txt.files.empty()) checksums_txt.checkEqual(checksums_data, check_uncompressed); + return checksums_data; } From 25d35a97f9a9f19b00cb76834fc28d26e72c8dfd Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Sat, 29 Oct 2022 18:10:33 +0200 Subject: [PATCH 085/152] Update CCTZ to 2022f --- contrib/cctz | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contrib/cctz b/contrib/cctz index 7a454c25c7d..5c8528fb35e 160000 --- a/contrib/cctz +++ b/contrib/cctz @@ -1 +1 @@ -Subproject commit 7a454c25c7d16053bcd327cdd16329212a08fa4a +Subproject commit 5c8528fb35e89ee0b3a7157490423fba0d4dd7b5 From c68ab231f917bc41b354f7d2a1fa769a7fbb8d06 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 30 Oct 2022 17:30:51 +0100 Subject: [PATCH 086/152] fix accessing part in Deleting state --- src/Storages/MergeTree/MergeTreeData.cpp | 11 ++++++--- src/Storages/MergeTree/MergeTreeData.h | 25 ++++++++++++++++++++- src/Storages/StorageReplicatedMergeTree.cpp | 23 ++++++++++--------- src/Storages/StorageReplicatedMergeTree.h | 2 +- 4 files changed, 46 insertions(+), 15 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index bb589161b57..d38c9c465d5 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -3142,7 +3142,7 @@ void MergeTreeData::removePartsInRangeFromWorkingSet(MergeTreeTransaction * txn, removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(txn, drop_range, lock); } -MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper( +MergeTreeData::PartsToRemoveFromZooKeeper MergeTreeData::removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper( MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock) { DataPartsVector parts_to_remove; @@ -3220,15 +3220,20 @@ MergeTreeData::DataPartsVector MergeTreeData::removePartsInRangeFromWorkingSetAn /// FIXME refactor removePartsFromWorkingSet(...), do not remove parts twice removePartsFromWorkingSet(txn, parts_to_remove, clear_without_timeout, lock); + /// Since we can return parts in Deleting state, we have to use a wrapper that restricts access to such parts. + PartsToRemoveFromZooKeeper parts_to_remove_from_zookeeper; + for (auto & part : parts_to_remove) + parts_to_remove_from_zookeeper.emplace_back(std::move(part)); + for (auto & part : inactive_parts_to_remove_immediately) { if (!drop_range.contains(part->info)) continue; part->remove_time.store(0, std::memory_order_relaxed); - parts_to_remove.push_back(std::move(part)); + parts_to_remove_from_zookeeper.emplace_back(std::move(part), /* was_active */ false); } - return parts_to_remove; + return parts_to_remove_from_zookeeper; } void MergeTreeData::restoreAndActivatePart(const DataPartPtr & part, DataPartsLock * acquired_lock) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 2b67face570..60255ce1b16 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -584,10 +584,33 @@ public: /// Used in REPLACE PARTITION command. void removePartsInRangeFromWorkingSet(MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock); + /// This wrapper is required to restrict access to parts in Deleting state + class PartToRemoveFromZooKeeper + { + DataPartPtr part; + bool was_active; + + public: + PartToRemoveFromZooKeeper(DataPartPtr && part_, bool was_active_ = true) + : part(std::move(part_)), was_active(was_active_) + { + } + + /// It's s to get name of any part + const String & getPartName() const { return part->name; } + + DataPartPtr getPartIfItWasActive() const + { + return was_active ? part : nullptr; + } + }; + + using PartsToRemoveFromZooKeeper = std::vector; + /// Same as above, but also returns list of parts to remove from ZooKeeper. /// It includes parts that have been just removed by these method /// and Outdated parts covered by drop_range that were removed earlier for any reason. - DataPartsVector removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper( + PartsToRemoveFromZooKeeper removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper( MergeTreeTransaction * txn, const MergeTreePartInfo & drop_range, DataPartsLock & lock); /// Restores Outdated part and adds it to working set diff --git a/src/Storages/StorageReplicatedMergeTree.cpp b/src/Storages/StorageReplicatedMergeTree.cpp index 8dad5755dab..3c0fbb162bc 100644 --- a/src/Storages/StorageReplicatedMergeTree.cpp +++ b/src/Storages/StorageReplicatedMergeTree.cpp @@ -1827,7 +1827,7 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) /// Therefore, we use all data parts. auto metadata_snapshot = getInMemoryMetadataPtr(); - DataPartsVector parts_to_remove; + PartsToRemoveFromZooKeeper parts_to_remove; { auto data_parts_lock = lockParts(); parts_to_remove = removePartsInRangeFromWorkingSetAndGetPartsToRemoveFromZooKeeper(NO_TRANSACTION_RAW, drop_range_info, data_parts_lock); @@ -1849,8 +1849,11 @@ void StorageReplicatedMergeTree::executeDropRange(const LogEntry & entry) /// If DETACH clone parts to detached/ directory for (const auto & part : parts_to_remove) { - LOG_INFO(log, "Detaching {}", part->getDataPartStorage().getPartDirectory()); - part->makeCloneInDetached("", metadata_snapshot); + if (auto part_to_detach = part.getPartIfItWasActive()) + { + LOG_INFO(log, "Detaching {}", part_to_detach->getDataPartStorage().getPartDirectory()); + part_to_detach->makeCloneInDetached("", metadata_snapshot); + } } } @@ -1941,7 +1944,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) PartDescriptions all_parts; PartDescriptions parts_to_add; - DataPartsVector parts_to_remove; + PartsToRemoveFromZooKeeper parts_to_remove; auto table_lock_holder_dst_table = lockForShare( RWLockImpl::NO_QUERY, getSettings()->lock_acquire_timeout_for_background_operations); @@ -1972,7 +1975,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) String parts_to_remove_str; for (const auto & part : parts_to_remove) { - parts_to_remove_str += part->name; + parts_to_remove_str += part.getPartName(); parts_to_remove_str += " "; } LOG_TRACE(log, "Replacing {} parts {}with empty set", parts_to_remove.size(), parts_to_remove_str); @@ -2248,7 +2251,7 @@ bool StorageReplicatedMergeTree::executeReplaceRange(const LogEntry & entry) String parts_to_remove_str; for (const auto & part : parts_to_remove) { - parts_to_remove_str += part->name; + parts_to_remove_str += part.getPartName(); parts_to_remove_str += " "; } LOG_TRACE(log, "Replacing {} parts {}with {} parts {}", parts_to_remove.size(), parts_to_remove_str, @@ -6230,11 +6233,11 @@ void StorageReplicatedMergeTree::clearOldPartsAndRemoveFromZK() } -void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(DataPartsVector & parts, size_t max_retries) +void StorageReplicatedMergeTree::removePartsFromZooKeeperWithRetries(PartsToRemoveFromZooKeeper & parts, size_t max_retries) { Strings part_names_to_remove; for (const auto & part : parts) - part_names_to_remove.emplace_back(part->name); + part_names_to_remove.emplace_back(part.getPartName()); return removePartsFromZooKeeperWithRetries(part_names_to_remove, max_retries); } @@ -6561,7 +6564,7 @@ void StorageReplicatedMergeTree::replacePartitionFrom( if (replace) clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); - DataPartsVector parts_to_remove; + PartsToRemoveFromZooKeeper parts_to_remove; Coordination::Responses op_results; try @@ -6797,7 +6800,7 @@ void StorageReplicatedMergeTree::movePartitionToTable(const StoragePtr & dest_ta clearBlocksInPartition(*zookeeper, drop_range.partition_id, drop_range.max_block, drop_range.max_block); - DataPartsVector parts_to_remove; + PartsToRemoveFromZooKeeper parts_to_remove; Coordination::Responses op_results; try diff --git a/src/Storages/StorageReplicatedMergeTree.h b/src/Storages/StorageReplicatedMergeTree.h index 5213f963fdf..323b1ce02bf 100644 --- a/src/Storages/StorageReplicatedMergeTree.h +++ b/src/Storages/StorageReplicatedMergeTree.h @@ -549,7 +549,7 @@ private: /// Remove parts from ZooKeeper, throw exception if unable to do so after max_retries. void removePartsFromZooKeeperWithRetries(const Strings & part_names, size_t max_retries = 5); - void removePartsFromZooKeeperWithRetries(DataPartsVector & parts, size_t max_retries = 5); + void removePartsFromZooKeeperWithRetries(PartsToRemoveFromZooKeeper & parts, size_t max_retries = 5); /// Removes a part from ZooKeeper and adds a task to the queue to download it. It is supposed to do this with broken parts. void removePartAndEnqueueFetch(const String & part_name); From 978aa16e29d67cfeed8abead1e99802983448699 Mon Sep 17 00:00:00 2001 From: Nikolay Degterinsky Date: Sun, 30 Oct 2022 16:42:57 +0000 Subject: [PATCH 087/152] Fix a bug in ParserCreateUserQuery --- src/Parsers/Access/ParserCreateUserQuery.cpp | 5 ++--- .../0_stateless/02474_create_user_query_fuzzer_bug.reference | 0 .../0_stateless/02474_create_user_query_fuzzer_bug.sql | 1 + 3 files changed, 3 insertions(+), 3 deletions(-) create mode 100644 tests/queries/0_stateless/02474_create_user_query_fuzzer_bug.reference create mode 100644 tests/queries/0_stateless/02474_create_user_query_fuzzer_bug.sql diff --git a/src/Parsers/Access/ParserCreateUserQuery.cpp b/src/Parsers/Access/ParserCreateUserQuery.cpp index 9e32b3c4618..ed6ecb62667 100644 --- a/src/Parsers/Access/ParserCreateUserQuery.cpp +++ b/src/Parsers/Access/ParserCreateUserQuery.cpp @@ -295,11 +295,11 @@ namespace } - bool parseHosts(IParserBase::Pos & pos, Expected & expected, const String & prefix, AllowedClientHosts & hosts) + bool parseHosts(IParserBase::Pos & pos, Expected & expected, std::string_view prefix, AllowedClientHosts & hosts) { return IParserBase::wrapParseImpl(pos, [&] { - if (!prefix.empty() && !ParserKeyword{prefix.c_str()}.ignore(pos, expected)) + if (!prefix.empty() && !ParserKeyword{prefix}.ignore(pos, expected)) return false; if (!ParserKeyword{"HOST"}.ignore(pos, expected)) @@ -492,7 +492,6 @@ bool ParserCreateUserQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expec if (alter) { - String maybe_new_name; if (!new_name && (names->size() == 1) && parseRenameTo(pos, expected, new_name)) continue; diff --git a/tests/queries/0_stateless/02474_create_user_query_fuzzer_bug.reference b/tests/queries/0_stateless/02474_create_user_query_fuzzer_bug.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/queries/0_stateless/02474_create_user_query_fuzzer_bug.sql b/tests/queries/0_stateless/02474_create_user_query_fuzzer_bug.sql new file mode 100644 index 00000000000..3ef1469cf1b --- /dev/null +++ b/tests/queries/0_stateless/02474_create_user_query_fuzzer_bug.sql @@ -0,0 +1 @@ +EXPLAIN AST ALTER user WITH a; -- { clientError SYNTAX_ERROR } From f53df7870c50dab252969a7bf37e3f776686888d Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Sun, 30 Oct 2022 17:59:47 +0100 Subject: [PATCH 088/152] fix race between drop and failed insert --- src/Storages/MergeTree/MergeTreeData.cpp | 25 +++++++++++++++++++++--- src/Storages/MergeTree/MergeTreeData.h | 3 +++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeData.cpp b/src/Storages/MergeTree/MergeTreeData.cpp index d38c9c465d5..a9e726f25fd 100644 --- a/src/Storages/MergeTree/MergeTreeData.cpp +++ b/src/Storages/MergeTree/MergeTreeData.cpp @@ -2199,6 +2199,7 @@ void MergeTreeData::dropAllData() LOG_TRACE(log, "dropAllData: removing all data parts from memory."); data_parts_indexes.clear(); + all_data_dropped = true; } catch (...) { @@ -5181,9 +5182,27 @@ void MergeTreeData::Transaction::rollback() buf << "."; LOG_DEBUG(data.log, "Undoing transaction.{}", buf.str()); - data.removePartsFromWorkingSet(txn, - DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()), - /* clear_without_timeout = */ true); + auto lock = data.lockParts(); + + if (data.data_parts_indexes.empty()) + { + /// Table was dropped concurrently and all parts (including PreActive parts) were cleared, so there's nothing to rollback + if (!data.all_data_dropped) + { + Strings part_names; + for (const auto & part : precommitted_parts) + part_names.emplace_back(part->name); + throw Exception(ErrorCodes::LOGICAL_ERROR, "There are some PreActive parts ({}) to rollback, " + "but data parts set is empty and table {} was not dropped. It's a bug", + fmt::join(part_names, ", "), data.getStorageID().getNameForLogs()); + } + } + else + { + data.removePartsFromWorkingSet(txn, + DataPartsVector(precommitted_parts.begin(), precommitted_parts.end()), + /* clear_without_timeout = */ true, &lock); + } } clear(); diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 60255ce1b16..40eaa679845 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -663,6 +663,9 @@ public: /// Deletes the data directory and flushes the uncompressed blocks cache and the marks cache. void dropAllData(); + /// This flag is for hardening and assertions. + bool all_data_dropped = false; + /// Drop data directories if they are empty. It is safe to call this method if table creation was unsuccessful. void dropIfEmpty(); From 620caeb07c9a09f855c3deb65e30de2674b4504f Mon Sep 17 00:00:00 2001 From: Gabriel Date: Mon, 31 Oct 2022 13:20:58 +0800 Subject: [PATCH 089/152] Fix typo in comments --- src/Processors/Executors/ExecutingGraph.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Processors/Executors/ExecutingGraph.cpp b/src/Processors/Executors/ExecutingGraph.cpp index 651ede10cfd..c7e89d7547c 100644 --- a/src/Processors/Executors/ExecutingGraph.cpp +++ b/src/Processors/Executors/ExecutingGraph.cpp @@ -71,7 +71,7 @@ bool ExecutingGraph::addEdges(uint64_t node) } } - /// Add direct edges form output ports. + /// Add direct edges from output ports. auto & outputs = from->getOutputs(); auto from_output = nodes[node]->direct_edges.size(); From 32febf5155d55bb63b46b2f04d2ea4a91b85c5d2 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 31 Oct 2022 07:56:59 +0100 Subject: [PATCH 090/152] Remove dead code in LowerUpperUTF8Impl::array() AFAICS it was there before since it was possible to overrun the expected_end, since utf8.convert() was called with "src_end - src" not "expected_end - src". Refs: 5a21f3908b054a0efc90c65a12fbe151c74d90dc:dbms/include/DB/Functions/FunctionsString.h Signed-off-by: Azat Khuzhin --- src/Functions/LowerUpperUTF8Impl.h | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 3a050e2bd6f..1f45772c84a 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -230,16 +230,6 @@ private: while (src < expected_end) toCase(src, expected_end, dst); - - /// adjust src_end_sse by pushing it forward or backward - const auto diff = src - expected_end; - if (diff != 0) - { - if (src_end_sse + diff < src_end) - src_end_sse += diff; - else - src_end_sse -= bytes_sse - diff; - } } } From 5fe44f27368fb27e40fc46ce957ae0da5f37e8f4 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sun, 30 Oct 2022 08:50:05 +0100 Subject: [PATCH 091/152] Fix lowerUTF8()/upperUTF8() in case of symbol was in between 16-byte boundary MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit In lowerUTF8()/upperUTF8() there is an SSE optimization that handles 16 byte at a time, but only for ASCII, for UTF8 symbols converion will be done by symbol. Consider the following example: КВ АМ И СЖ ^ - offset is 15, length of sequence is 2 so first byte of a symbol is in first 16 bytes second byte of a symbol is not ther And in this case it will be handled incorrectly because it will try to process oly these 16 bytes w/o looking forward. This had been broken by #41286, before this patch it does not looks at the row boundaries but only at the string end and so this sutation wasn't possible. Fixes: #42756 Signed-off-by: Azat Khuzhin --- src/Functions/LowerUpperUTF8Impl.h | 22 +++++++++++++++---- ...71_lower_upper_utf8_row_overlaps.reference | 4 ++++ .../02071_lower_upper_utf8_row_overlaps.sql | 4 ++++ 3 files changed, 26 insertions(+), 4 deletions(-) diff --git a/src/Functions/LowerUpperUTF8Impl.h b/src/Functions/LowerUpperUTF8Impl.h index 1f45772c84a..b8fd20d56de 100644 --- a/src/Functions/LowerUpperUTF8Impl.h +++ b/src/Functions/LowerUpperUTF8Impl.h @@ -104,7 +104,7 @@ struct LowerUpperUTF8Impl /** Converts a single code point starting at `src` to desired case, storing result starting at `dst`. * `src` and `dst` are incremented by corresponding sequence lengths. */ - static void toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst) + static bool toCase(const UInt8 *& src, const UInt8 * src_end, UInt8 *& dst, bool partial) { if (src[0] <= ascii_upper_bound) { @@ -136,6 +136,11 @@ struct LowerUpperUTF8Impl static const Poco::UTF8Encoding utf8; size_t src_sequence_length = UTF8::seqLength(*src); + /// In case partial buffer was passed (due to SSE optimization) + /// we cannot convert it with current src_end, but we may have more + /// bytes to convert and eventually got correct symbol. + if (partial && src_sequence_length > static_cast(src_end-src)) + return false; auto src_code_point = UTF8::convertUTF8ToCodePoint(src, src_end - src); if (src_code_point) @@ -152,7 +157,7 @@ struct LowerUpperUTF8Impl { src += dst_sequence_length; dst += dst_sequence_length; - return; + return true; } } } @@ -161,6 +166,8 @@ struct LowerUpperUTF8Impl ++dst; ++src; } + + return true; } private: @@ -229,7 +236,14 @@ private: const UInt8 * expected_end = std::min(src + bytes_sse, row_end); while (src < expected_end) - toCase(src, expected_end, dst); + { + if (!toCase(src, expected_end, dst, /* partial= */ true)) + { + /// Fallback to handling byte by byte. + src_end_sse = src; + break; + } + } } } @@ -245,7 +259,7 @@ private: chassert(row_end >= src); while (src < row_end) - toCase(src, row_end, dst); + toCase(src, row_end, dst, /* partial= */ false); ++offset_it; } } diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference index 2b3f8138c2b..190c7895011 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference @@ -9,3 +9,7 @@ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" 1,"0xE2","0xE2" 15,"foo⚊barbazbam","FOO⚊BARBAZBAM" +-- NOTE: regression test for introduced bug +-- https://github.com/ClickHouse/ClickHouse/issues/42756 +SELECT lowerUTF8('КВ АМ И СЖ'); +кв ам и сж diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql index ee0d29be699..161d8fb512e 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql @@ -8,3 +8,7 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo -- MONOGRAM FOR YANG with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV; + +-- NOTE: regression test for introduced bug +-- https://github.com/ClickHouse/ClickHouse/issues/42756 +SELECT lowerUTF8('КВ АМ И СЖ'); From b62170426a43702bc5f9ad8854a9b468fe4788b3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 31 Oct 2022 10:48:37 +0100 Subject: [PATCH 092/152] Fix compilation of LLVM with cmake cache Simple reproducer: $ cmake $ ninja contrib/llvm-project/llvm/lib/MC/MCParser/CMakeFiles/LLVMMCParser.dir/MasmParser.cpp.o # will have -std=c++14 $ touch CMakeLists.txt $ cmake $ ninja contrib/llvm-project/llvm/lib/MC/MCParser/CMakeFiles/LLVMMCParser.dir/MasmParser.cpp.o # will have -std=c++20 and fail (fails because std::vector cannot work with opaque types anymore) Fixes: #42249 (cc @rschu1ze) --- contrib/llvm-project-cmake/CMakeLists.txt | 3 +++ 1 file changed, 3 insertions(+) diff --git a/contrib/llvm-project-cmake/CMakeLists.txt b/contrib/llvm-project-cmake/CMakeLists.txt index 6a73ae0f0c6..7af4a23bc9d 100644 --- a/contrib/llvm-project-cmake/CMakeLists.txt +++ b/contrib/llvm-project-cmake/CMakeLists.txt @@ -21,6 +21,9 @@ set (LLVM_INCLUDE_DIRS "${ClickHouse_BINARY_DIR}/contrib/llvm-project/llvm/include" ) set (LLVM_LIBRARY_DIRS "${ClickHouse_BINARY_DIR}/contrib/llvm-project/llvm") +# NOTE: You should not remove this line since otherwise it will use default 20, +# and llvm cannot be compiled with bundled libcxx and 20 standard. +set (CMAKE_CXX_STANDARD 14) # This list was generated by listing all LLVM libraries, compiling the binary and removing all libraries while it still compiles. set (REQUIRED_LLVM_LIBRARIES From 115fcaffc558c89cb95bf7fceaee1322f2bb0707 Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 31 Oct 2022 11:30:56 +0100 Subject: [PATCH 093/152] Fixed tests --- .../MergeTree/MergeTreeIndexConditionBloomFilter.cpp | 7 +++++++ src/Storages/MergeTree/RPNBuilder.cpp | 2 +- 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 1413f084536..9856500c65c 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -280,6 +280,13 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo auto arguments_size = function.getArgumentsSize(); auto function_name = function.getFunctionName(); + for (size_t i = 0; i < arguments_size; ++i) + { + auto argument = function.getArgumentAt(i); + if (traverseFunction(argument, out, &node)) + maybe_useful = true; + } + if (arguments_size != 2) return false; diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 4cc311b8b9e..395daab76ba 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -361,7 +361,7 @@ ConstSetPtr RPNBuilderTreeNode::tryGetPreparedSet( RPNBuilderFunctionTreeNode RPNBuilderTreeNode::toFunctionNode() const { if (!isFunction()) - throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a constant"); + throw Exception(ErrorCodes::LOGICAL_ERROR, "RPNBuilderTree node is not a function"); if (this->ast_node) return RPNBuilderFunctionTreeNode(this->ast_node, tree_context); From d760c78754d8e6ff133496dfa1a48a6c6a7683ae Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 31 Oct 2022 11:45:12 +0100 Subject: [PATCH 094/152] Fixed code review issues --- src/Analyzer/Passes/QueryAnalysisPass.cpp | 16 +-------------- src/Analyzer/QueryTreeBuilder.cpp | 16 +-------------- src/Analyzer/TableExpressionModifiers.cpp | 24 +++++++++++++++++++++++ src/Analyzer/TableExpressionModifiers.h | 3 +++ src/Analyzer/UnionNode.cpp | 2 +- 5 files changed, 30 insertions(+), 31 deletions(-) diff --git a/src/Analyzer/Passes/QueryAnalysisPass.cpp b/src/Analyzer/Passes/QueryAnalysisPass.cpp index 019b002c527..9db2d66d99d 100644 --- a/src/Analyzer/Passes/QueryAnalysisPass.cpp +++ b/src/Analyzer/Passes/QueryAnalysisPass.cpp @@ -4646,23 +4646,9 @@ void QueryAnalyzer::initializeQueryJoinTreeNode(QueryTreeNodePtr & join_tree_nod if (table_expression_modifiers.has_value()) { - String table_expression_modifiers_error_message; - - if (table_expression_modifiers->hasFinal()) - { - table_expression_modifiers_error_message += "FINAL"; - - if (table_expression_modifiers->hasSampleSizeRatio()) - table_expression_modifiers_error_message += ", SAMPLE"; - } - else if (table_expression_modifiers->hasSampleSizeRatio()) - { - table_expression_modifiers_error_message += "SAMPLE"; - } - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Table expression modifiers {} are not supported for subquery {}", - table_expression_modifiers_error_message, + table_expression_modifiers->formatForErrorMessage(), resolved_identifier->formatASTForErrorMessage()); } } diff --git a/src/Analyzer/QueryTreeBuilder.cpp b/src/Analyzer/QueryTreeBuilder.cpp index c5bb3ae7074..51745d820e7 100644 --- a/src/Analyzer/QueryTreeBuilder.cpp +++ b/src/Analyzer/QueryTreeBuilder.cpp @@ -673,23 +673,9 @@ QueryTreeNodePtr QueryTreeBuilder::buildJoinTree(const ASTPtr & tables_in_select if (table_expression_modifiers) { - String table_expression_modifiers_error_message; - - if (table_expression_modifiers->hasFinal()) - { - table_expression_modifiers_error_message += "FINAL"; - - if (table_expression_modifiers->hasSampleSizeRatio()) - table_expression_modifiers_error_message += ", SAMPLE"; - } - else if (table_expression_modifiers->hasSampleSizeRatio()) - { - table_expression_modifiers_error_message += "SAMPLE"; - } - throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "Table expression modifiers {} are not supported for subquery {}", - table_expression_modifiers_error_message, + table_expression_modifiers->formatForErrorMessage(), node->formatASTForErrorMessage()); } diff --git a/src/Analyzer/TableExpressionModifiers.cpp b/src/Analyzer/TableExpressionModifiers.cpp index 79b5a8dba41..c8002f44c97 100644 --- a/src/Analyzer/TableExpressionModifiers.cpp +++ b/src/Analyzer/TableExpressionModifiers.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -39,4 +40,27 @@ void TableExpressionModifiers::updateTreeHash(SipHash & hash_state) const } } +String TableExpressionModifiers::formatForErrorMessage() const +{ + WriteBufferFromOwnString buffer; + if (has_final) + buffer << "FINAL"; + + if (sample_size_ratio) + { + if (has_final) + buffer << ' '; + buffer << "SAMPLE " << ASTSampleRatio::toString(*sample_size_ratio); + } + + if (sample_offset_ratio) + { + if (has_final || sample_size_ratio) + buffer << ' '; + buffer << "OFFSET " << ASTSampleRatio::toString(*sample_offset_ratio); + } + + return buffer.str(); +} + } diff --git a/src/Analyzer/TableExpressionModifiers.h b/src/Analyzer/TableExpressionModifiers.h index cc5ac3948bf..f61c2a61610 100644 --- a/src/Analyzer/TableExpressionModifiers.h +++ b/src/Analyzer/TableExpressionModifiers.h @@ -58,6 +58,9 @@ public: /// Update tree hash void updateTreeHash(SipHash & hash_state) const; + /// Format for error message + String formatForErrorMessage() const; + private: bool has_final = false; std::optional sample_size_ratio; diff --git a/src/Analyzer/UnionNode.cpp b/src/Analyzer/UnionNode.cpp index 52ad0102d7c..67860438335 100644 --- a/src/Analyzer/UnionNode.cpp +++ b/src/Analyzer/UnionNode.cpp @@ -40,7 +40,7 @@ UnionNode::UnionNode(SelectUnionMode union_mode_) if (union_mode == SelectUnionMode::UNION_DEFAULT || union_mode == SelectUnionMode::EXCEPT_DEFAULT || union_mode == SelectUnionMode::INTERSECT_DEFAULT) - throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION mode must be normalized"); + throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION mode {} must be normalized", toString(union_mode)); children[queries_child_index] = std::make_shared(); } From f6d0c03ee5dcdf703ca63ebb781f99a22d566a9e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 19:39:02 +0200 Subject: [PATCH 095/152] Fix path to files in MetadataStorageFromPlainObjectStorage Signed-off-by: Azat Khuzhin --- .../MetadataStorageFromPlainObjectStorage.cpp | 21 ++++++++++--------- .../MetadataStorageFromPlainObjectStorage.h | 3 +++ 2 files changed, 14 insertions(+), 10 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 35cd3be15d2..f5b1b4d287a 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -33,10 +33,14 @@ const std::string & MetadataStorageFromPlainObjectStorage::getPath() const { return object_storage_root_path; } +std::filesystem::path MetadataStorageFromPlainObjectStorage::getAbsolutePath(const std::string & path) const +{ + return fs::path(object_storage_root_path) / path; +} bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) const { - auto object = StoredObject::create(*object_storage, fs::path(object_storage_root_path) / path); + auto object = StoredObject::create(*object_storage, getAbsolutePath(path)); return object_storage->exists(object); } @@ -48,7 +52,7 @@ bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) con bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const { - std::string directory = path; + std::string directory = getAbsolutePath(path); trimRight(directory); directory += "/"; @@ -77,7 +81,7 @@ time_t MetadataStorageFromPlainObjectStorage::getLastChanged(const std::string & uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { RelativePathsWithSize children; - object_storage->listPrefix(path, children); + object_storage->listPrefix(getAbsolutePath(path), children); if (children.empty()) return 0; if (children.size() != 1) @@ -88,7 +92,7 @@ uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { RelativePathsWithSize children; - object_storage->listPrefix(path, children); + object_storage->listPrefix(getAbsolutePath(path), children); std::vector result; for (const auto & path_size : children) @@ -120,11 +124,8 @@ std::unordered_map MetadataStorageFromPlainObjectStorage::getSer StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std::string & path) const { std::string blob_name = object_storage->generateBlobNameForPath(path); - - std::string object_path = fs::path(object_storage_root_path) / blob_name; - size_t object_size = getFileSize(object_path); - - auto object = StoredObject::create(*object_storage, object_path, object_size, /* exists */true); + size_t object_size = getFileSize(blob_name); + auto object = StoredObject::create(*object_storage, getAbsolutePath(blob_name), object_size, /* exists */true); return {std::move(object)}; } @@ -150,7 +151,7 @@ void MetadataStorageFromPlainObjectStorageTransaction::setLastModified(const std void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { - auto object = StoredObject::create(*metadata_storage.object_storage, fs::path(metadata_storage.object_storage_root_path) / path); + auto object = StoredObject::create(*metadata_storage.object_storage, metadata_storage.getAbsolutePath(path)); metadata_storage.object_storage->removeObject(object); } diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index bd993918413..c58ee17b495 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -70,6 +70,9 @@ public: StoredObjects getStorageObjects(const std::string & path) const override; std::string getObjectStorageRootPath() const override { return object_storage_root_path; } + +private: + std::filesystem::path getAbsolutePath(const std::string & path) const; }; class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction From b34ffda272fe9be0a014eb71ff6053f537911150 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 19:39:34 +0200 Subject: [PATCH 096/152] Implement MetadataStorageFromPlainObjectStorage::getLastModified() (used by MergeTree) Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index f5b1b4d287a..f99f4eb21f3 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -65,7 +65,8 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path Poco::Timestamp MetadataStorageFromPlainObjectStorage::getLastModified(const std::string &) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getLastModified is not implemented for MetadataStorageFromPlainObjectStorage"); + /// NOTE: This is required for MergeTree + return {}; } struct stat MetadataStorageFromPlainObjectStorage::stat(const std::string &) const From 94d9600fb813ea25f892a461f8ffc3c632123318 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 19:39:50 +0200 Subject: [PATCH 097/152] Implement MetadataStorageFromPlainObjectStorage::getLastChanged() (as for web) Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index f99f4eb21f3..dda7ed98837 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -76,7 +76,8 @@ struct stat MetadataStorageFromPlainObjectStorage::stat(const std::string &) con time_t MetadataStorageFromPlainObjectStorage::getLastChanged(const std::string &) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getLastChanged is not implemented for MetadataStorageFromPlainObjectStorage"); + /// NOTE: by analogy with MetadataStorageFromStaticFilesWebServer::getLastChanged() + return {}; } uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const From 88db8ae7fac9c09cf4e92fc3a9a55d45887682b9 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 20:05:24 +0200 Subject: [PATCH 098/152] Add a comment for IObjectStorage::listPrefix() Signed-off-by: Azat Khuzhin --- src/Disks/ObjectStorages/IObjectStorage.h | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 52e1a2cb270..ec696726ed2 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -66,6 +66,13 @@ public: virtual bool exists(const StoredObject & object) const = 0; /// List on prefix, return children (relative paths) with their sizes. + /// So it should return all "objects" with their sizes. + /// + /// For example if you do this over filesystem, you should skip folders and return files only. + /// + /// NOTE: It makes sense only for real object storages (S3, Azure), since + /// this is used for restoring metadata (see "send_metadata" and + /// DiskObjectStorage::restoreMetadataIfNeeded()). virtual void listPrefix(const std::string & path, RelativePathsWithSize & children) const = 0; /// Get object metadata if supported. It should be possible to receive From ee18140c4843fddd067255b8848a2ef8dc5c5041 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 20:06:18 +0200 Subject: [PATCH 099/152] Remove listPrefix() implementation for disks that does not support send_metadata The reason for removing is it because not compatible with restoring (with send_metadata set) anyway: - HDFS - is not compatible with send_metadata, and besides it's implementaion is not correct, since it is simply `ls -l`, while the following is required: `find . -maxdepth 1 -type f` - Web - is not compatible with send_metadata anyway - Local - is not compatible with send_metadata anyway Signed-off-by: Azat Khuzhin --- src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp | 12 ------------ src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h | 2 -- src/Disks/ObjectStorages/IObjectStorage.cpp | 5 +++++ src/Disks/ObjectStorages/IObjectStorage.h | 13 +++++++++---- src/Disks/ObjectStorages/LocalObjectStorage.cpp | 7 ------- src/Disks/ObjectStorages/LocalObjectStorage.h | 2 -- src/Disks/ObjectStorages/Web/WebObjectStorage.cpp | 11 ----------- src/Disks/ObjectStorages/Web/WebObjectStorage.h | 2 -- 8 files changed, 14 insertions(+), 40 deletions(-) diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp index 2f82458ecd8..80c4bb2bc64 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.cpp @@ -101,18 +101,6 @@ std::unique_ptr HDFSObjectStorage::writeObject( /// NOL } -void HDFSObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize & children) const -{ - const size_t begin_of_path = path.find('/', path.find("//") + 2); - int32_t num_entries; - auto * files_list = hdfsListDirectory(hdfs_fs.get(), path.substr(begin_of_path).c_str(), &num_entries); - if (num_entries == -1) - throw Exception(ErrorCodes::HDFS_ERROR, "HDFSDelete failed with path: " + path); - - for (int32_t i = 0; i < num_entries; ++i) - children.emplace_back(files_list[i].mName, files_list[i].mSize); -} - /// Remove file. Throws exception if file doesn't exists or it's a directory. void HDFSObjectStorage::removeObject(const StoredObject & object) { diff --git a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h index 82cddfb9122..4064a5c5b7f 100644 --- a/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h +++ b/src/Disks/ObjectStorages/HDFS/HDFSObjectStorage.h @@ -85,8 +85,6 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; - /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeObject(const StoredObject & object) override; diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 9d6610ee326..7f1cdcbfa59 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -14,6 +14,11 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } +void IObjectStorage::listPrefix(const std::string &, RelativePathsWithSize &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listPrefix() is not supported"); +} + IAsynchronousReader & IObjectStorage::getThreadPoolReader() { auto context = Context::getGlobalContextInstance(); diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index ec696726ed2..39819e2e58c 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -68,12 +68,17 @@ public: /// List on prefix, return children (relative paths) with their sizes. /// So it should return all "objects" with their sizes. /// - /// For example if you do this over filesystem, you should skip folders and return files only. + /// For example if you do this over filesystem, you should skip folders and + /// return files only, so something like on local filesystem: + /// + /// find . -type f /// /// NOTE: It makes sense only for real object storages (S3, Azure), since - /// this is used for restoring metadata (see "send_metadata" and - /// DiskObjectStorage::restoreMetadataIfNeeded()). - virtual void listPrefix(const std::string & path, RelativePathsWithSize & children) const = 0; + /// it is used only for one of the following: + /// - send_metadata (to restore metadata) + /// - see DiskObjectStorage::restoreMetadataIfNeeded() + /// - MetadataStorageFromPlainObjectStorage - only for s3_plain disk + virtual void listPrefix(const std::string & path, RelativePathsWithSize & children) const; /// Get object metadata if supported. It should be possible to receive /// at least size of object diff --git a/src/Disks/ObjectStorages/LocalObjectStorage.cpp b/src/Disks/ObjectStorages/LocalObjectStorage.cpp index dbb3a7c2aba..67e2cc2d74b 100644 --- a/src/Disks/ObjectStorages/LocalObjectStorage.cpp +++ b/src/Disks/ObjectStorages/LocalObjectStorage.cpp @@ -104,13 +104,6 @@ std::unique_ptr LocalObjectStorage::writeObject( /// NO return std::make_unique(path, buf_size, flags); } -void LocalObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize & children) const -{ - fs::directory_iterator end_it; - for (auto it = fs::directory_iterator(path); it != end_it; ++it) - children.emplace_back(it->path().filename(), it->file_size()); -} - void LocalObjectStorage::removeObject(const StoredObject & object) { /// For local object storage files are actually removed when "metadata" is removed. diff --git a/src/Disks/ObjectStorages/LocalObjectStorage.h b/src/Disks/ObjectStorages/LocalObjectStorage.h index 0e4c71b4a47..b04e3fa6285 100644 --- a/src/Disks/ObjectStorages/LocalObjectStorage.h +++ b/src/Disks/ObjectStorages/LocalObjectStorage.h @@ -45,8 +45,6 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; - void removeObject(const StoredObject & object) override; void removeObjects(const StoredObjects & objects) override; diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp index 71bde110fa6..f97409cfc6c 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.cpp @@ -178,17 +178,6 @@ std::unique_ptr WebObjectStorage::readObject( /// NOLINT } } -void WebObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize & children) const -{ - for (const auto & [file_path, file_info] : files) - { - if (file_info.type == FileType::File && file_path.starts_with(path)) - { - children.emplace_back(file_path, file_info.size); - } - } -} - void WebObjectStorage::throwNotAllowed() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only read-only operations are supported"); diff --git a/src/Disks/ObjectStorages/Web/WebObjectStorage.h b/src/Disks/ObjectStorages/Web/WebObjectStorage.h index 2fda5e576aa..2dab8fdb62d 100644 --- a/src/Disks/ObjectStorages/Web/WebObjectStorage.h +++ b/src/Disks/ObjectStorages/Web/WebObjectStorage.h @@ -55,8 +55,6 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; - void removeObject(const StoredObject & object) override; void removeObjects(const StoredObjects & objects) override; From 18e4fdf40f47cb5ad5cd608e4d0dabdfd6529bdb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 20:32:18 +0200 Subject: [PATCH 100/152] Introduce IObjectStorage::listPrefixInPath() Signed-off-by: Azat Khuzhin --- src/Disks/ObjectStorages/IObjectStorage.cpp | 6 ++++++ src/Disks/ObjectStorages/IObjectStorage.h | 12 ++++++++++++ 2 files changed, 18 insertions(+) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 7f1cdcbfa59..7c187e3dd2f 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -18,6 +18,12 @@ void IObjectStorage::listPrefix(const std::string &, RelativePathsWithSize &) co { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listPrefix() is not supported"); } +void IObjectStorage::listPrefixInPath(const std::string &, + RelativePathsWithSize &, + std::vector &) const +{ + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listPrefixInPath() is not supported"); +} IAsynchronousReader & IObjectStorage::getThreadPoolReader() { diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index 39819e2e58c..a7ad4e5b515 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -80,6 +80,18 @@ public: /// - MetadataStorageFromPlainObjectStorage - only for s3_plain disk virtual void listPrefix(const std::string & path, RelativePathsWithSize & children) const; + /// List on prefix, but it is limited only to current path, something like + /// on local filesystem: + /// + /// find . -maxdepth 1 + /// + /// @param path - path to list prefix in + /// @param children - list of child nodes (for unix filesystem - files goes here) + /// @param common_prefixes - list of common prefixes in this path, (for unix filesystem - subfolders) + virtual void listPrefixInPath(const std::string & path, + RelativePathsWithSize & children, + std::vector & common_prefixes) const; + /// Get object metadata if supported. It should be possible to receive /// at least size of object virtual ObjectMetadata getObjectMetadata(const std::string & path) const = 0; From f8ba24f04046f06af42e756560b10cb8884299d3 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 20:33:12 +0200 Subject: [PATCH 101/152] Implement S3ObjectStorage::listPrefixInPath() Signed-off-by: Azat Khuzhin --- .../ObjectStorages/S3/S3ObjectStorage.cpp | 45 ++++++++++++++++++- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 3 ++ 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index 83908d02f48..af3374f8d3c 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -28,7 +28,7 @@ #include #include - +#include #include #include @@ -279,6 +279,49 @@ void S3ObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize } while (outcome.GetResult().GetIsTruncated()); } +void S3ObjectStorage::listPrefixInPath(const std::string & path, + RelativePathsWithSize & children, + std::vector & common_prefixes) const +{ + auto settings_ptr = s3_settings.get(); + auto client_ptr = client.get(); + + Aws::S3::Model::ListObjectsV2Request request; + request.SetBucket(bucket); + request.SetPrefix(path); + request.SetMaxKeys(settings_ptr->list_object_keys_size); + request.SetDelimiter("/"); + + Aws::S3::Model::ListObjectsV2Outcome outcome; + do + { + ProfileEvents::increment(ProfileEvents::S3ListObjects); + ProfileEvents::increment(ProfileEvents::DiskS3ListObjects); + outcome = client_ptr->ListObjectsV2(request); + throwIfError(outcome); + + auto result = outcome.GetResult(); + auto result_objects = result.GetContents(); + auto result_common_prefixes = result.GetCommonPrefixes(); + + if (result_objects.empty() && result_common_prefixes.empty()) + break; + + for (const auto & object : result_objects) + children.emplace_back(object.GetKey(), object.GetSize()); + + for (const auto & common_prefix : result_common_prefixes) + { + std::string subfolder = common_prefix.GetPrefix(); + /// Make it compatible with std::filesystem::path::filename() + trimRight(subfolder, '/'); + common_prefixes.emplace_back(subfolder); + } + + request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); + } while (outcome.GetResult().GetIsTruncated()); +} + void S3ObjectStorage::removeObjectImpl(const StoredObject & object, bool if_exists) { auto client_ptr = client.get(); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index a193653db9a..9e90efcb9dc 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -106,6 +106,9 @@ public: const WriteSettings & write_settings = {}) override; void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; + void listPrefixInPath(const std::string & path, + RelativePathsWithSize & children, + std::vector & common_prefixes) const override; /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; From 4e42521f44116ca5c9275e7f78259611b8405488 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 21:48:27 +0200 Subject: [PATCH 102/152] Introduce ReadOnlyMetadataStorage And use it for: - MetadataStorageFromPlainObjectStorage - MetadataStorageFromStaticFilesWebServer This will allow to reduce ~100-200 lines of duplicated code, and plus make the code less error prone. Note, for now I tried to make this without behaviour changes. Signed-off-by: Azat Khuzhin --- .../MetadataStorageFromPlainObjectStorage.cpp | 97 ---------- .../MetadataStorageFromPlainObjectStorage.h | 54 +----- .../ObjectStorages/ReadOnlyMetadataStorage.h | 165 ++++++++++++++++++ ...etadataStorageFromStaticFilesWebServer.cpp | 107 ------------ .../MetadataStorageFromStaticFilesWebServer.h | 55 +----- 5 files changed, 176 insertions(+), 302 deletions(-) create mode 100644 src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index dda7ed98837..0722c3622b9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int LOGICAL_ERROR; } @@ -63,23 +62,6 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path return !children.empty(); } -Poco::Timestamp MetadataStorageFromPlainObjectStorage::getLastModified(const std::string &) const -{ - /// NOTE: This is required for MergeTree - return {}; -} - -struct stat MetadataStorageFromPlainObjectStorage::stat(const std::string &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "stat is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -time_t MetadataStorageFromPlainObjectStorage::getLastChanged(const std::string &) const -{ - /// NOTE: by analogy with MetadataStorageFromStaticFilesWebServer::getLastChanged() - return {}; -} - uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { RelativePathsWithSize children; @@ -113,16 +95,6 @@ DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(con return std::make_unique(std::move(fs_paths)); } -std::string MetadataStorageFromPlainObjectStorage::readFileToString(const std::string &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "readFileToString is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -std::unordered_map MetadataStorageFromPlainObjectStorage::getSerializedMetadata(const std::vector &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getSerializedMetadata is not implemented for MetadataStorageFromPlainObjectStorage"); -} - StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std::string & path) const { std::string blob_name = object_storage->generateBlobNameForPath(path); @@ -131,99 +103,30 @@ StoredObjects MetadataStorageFromPlainObjectStorage::getStorageObjects(const std return {std::move(object)}; } -uint32_t MetadataStorageFromPlainObjectStorage::getHardlinkCount(const std::string &) const -{ - return 1; -} - const IMetadataStorage & MetadataStorageFromPlainObjectStorageTransaction::getStorageForNonTransactionalReads() const { return metadata_storage; } -void MetadataStorageFromPlainObjectStorageTransaction::writeStringToFile(const std::string &, const std::string & /* data */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "writeStringToFile is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::setLastModified(const std::string &, const Poco::Timestamp &) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "setLastModified is not implemented for MetadataStorageFromPlainObjectStorage"); -} - void MetadataStorageFromPlainObjectStorageTransaction::unlinkFile(const std::string & path) { auto object = StoredObject::create(*metadata_storage.object_storage, metadata_storage.getAbsolutePath(path)); metadata_storage.object_storage->removeObject(object); } -void MetadataStorageFromPlainObjectStorageTransaction::removeRecursive(const std::string &) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "removeRecursive is not implemented for MetadataStorageFromPlainObjectStorage"); -} - void MetadataStorageFromPlainObjectStorageTransaction::createDirectory(const std::string &) { /// Noop. It is an Object Storage not a filesystem. } - void MetadataStorageFromPlainObjectStorageTransaction::createDirectoryRecursive(const std::string &) { /// Noop. It is an Object Storage not a filesystem. } - -void MetadataStorageFromPlainObjectStorageTransaction::removeDirectory(const std::string &) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "removeDirectory is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::moveFile(const std::string & /* path_from */, const std::string & /* path_to */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "moveFile is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::moveDirectory(const std::string & /* path_from */, const std::string & /* path_to */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "moveDirectory is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::replaceFile(const std::string & /* path_from */, const std::string & /* path_to */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "replaceFile is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::chmod(const String &, mode_t) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "chmod is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::setReadOnly(const std::string &) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "setReadOnly is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::createHardLink(const std::string & /* path_from */, const std::string & /* path_to */) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "createHardLink is not implemented for MetadataStorageFromPlainObjectStorage"); -} - -void MetadataStorageFromPlainObjectStorageTransaction::createEmptyMetadataFile(const std::string &) -{ - /// Noop, no separate metadata. -} - -void MetadataStorageFromPlainObjectStorageTransaction::createMetadataFile( - const std::string &, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) -{ - /// Noop, no separate metadata. -} - void MetadataStorageFromPlainObjectStorageTransaction::addBlobToMetadata( const std::string &, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) { /// Noop, local metadata files is only one file, it is the metadata file itself. } - void MetadataStorageFromPlainObjectStorageTransaction::unlinkMetadata(const std::string &) { /// Noop, no separate metadata. diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index c58ee17b495..2cfa123ecc9 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -20,7 +21,10 @@ namespace DB /// It is used to allow BACKUP/RESTORE to ObjectStorage (S3/...) with the same /// structure as on disk MergeTree, and does not requires metadata from local /// disk to restore. -class MetadataStorageFromPlainObjectStorage final : public IMetadataStorage +/// +/// NOTE: Inheritance from ReadOnlyMetadataStorage is used here to throw +/// NOT_IMPLEMENTED error for lost of unsupported methods (mtime/move/stat/...) +class MetadataStorageFromPlainObjectStorage final : public ReadOnlyMetadataStorage { private: friend class MetadataStorageFromPlainObjectStorageTransaction; @@ -45,26 +49,10 @@ public: uint64_t getFileSize(const String & path) const override; - Poco::Timestamp getLastModified(const std::string & path) const override; - - time_t getLastChanged(const std::string & path) const override; - - bool supportsChmod() const override { return false; } - - bool supportsStat() const override { return false; } - - struct stat stat(const String & path) const override; - std::vector listDirectory(const std::string & path) const override; DirectoryIteratorPtr iterateDirectory(const std::string & path) const override; - std::string readFileToString(const std::string & path) const override; - - std::unordered_map getSerializedMetadata(const std::vector & file_paths) const override; - - uint32_t getHardlinkCount(const std::string & path) const override; - DiskPtr getDisk() const { return {}; } StoredObjects getStorageObjects(const std::string & path) const override; @@ -75,7 +63,7 @@ private: std::filesystem::path getAbsolutePath(const std::string & path) const; }; -class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction +class MetadataStorageFromPlainObjectStorageTransaction final : public ReadOnlyMetadataTransaction { private: const MetadataStorageFromPlainObjectStorage & metadata_storage; @@ -90,41 +78,13 @@ public: const IMetadataStorage & getStorageForNonTransactionalReads() const final; - void commit() final {} - - void writeStringToFile(const std::string & path, const std::string & data) override; - - void createEmptyMetadataFile(const std::string & path) override; - - void createMetadataFile(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; - void addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; - void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override; - - bool supportsChmod() const override { return false; } - - void chmod(const String & path, mode_t mode) override; - - void setReadOnly(const std::string & path) override; - - void unlinkFile(const std::string & path) override; - void createDirectory(const std::string & path) override; void createDirectoryRecursive(const std::string & path) override; - void removeDirectory(const std::string & path) override; - - void removeRecursive(const std::string & path) override; - - void createHardLink(const std::string & path_from, const std::string & path_to) override; - - void moveFile(const std::string & path_from, const std::string & path_to) override; - - void moveDirectory(const std::string & path_from, const std::string & path_to) override; - - void replaceFile(const std::string & path_from, const std::string & path_to) override; + void unlinkFile(const std::string & path) override; void unlinkMetadata(const std::string & path) override; }; diff --git a/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h b/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h new file mode 100644 index 00000000000..e136b0580ea --- /dev/null +++ b/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h @@ -0,0 +1,165 @@ +#pragma once + +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + +class ReadOnlyMetadataStorage; + +/// Readonly, throws NOT_IMPLEMENTED error. +/// Can be used to add limited read-only support of MergeTree. +class ReadOnlyMetadataTransaction : public IMetadataTransaction +{ +public: + /// + /// Noop + /// + void commit() override + { + /// Noop, nothing to commit. + } + void createEmptyMetadataFile(const std::string & /* path */) override + { + /// No metadata, no need to create anything. + } + void createMetadataFile(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) override + { + /// Noop + } + + /// + /// Throws + /// + void writeStringToFile(const std::string & /* path */, const std::string & /* data */) override + { + throwNotAllowed(); + } + void setLastModified(const std::string & /* path */, const Poco::Timestamp & /* timestamp */) override + { + throwNotAllowed(); + } + void chmod(const String & /* path */, mode_t /* mode */) override + { + throwNotAllowed(); + } + void setReadOnly(const std::string & /* path */) override + { + throwNotAllowed(); + } + void unlinkFile(const std::string & /* path */) override + { + throwNotAllowed(); + } + void removeDirectory(const std::string & /* path */) override + { + throwNotAllowed(); + } + void removeRecursive(const std::string & /* path */) override + { + throwNotAllowed(); + } + void createHardLink(const std::string & /* path_from */, const std::string & /* path_to */) override + { + throwNotAllowed(); + } + void moveFile(const std::string & /* path_from */, const std::string & /* path_to */) override + { + throwNotAllowed(); + } + void moveDirectory(const std::string & /* path_from */, const std::string & /* path_to */) override + { + throwNotAllowed(); + } + void replaceFile(const std::string & /* path_from */, const std::string & /* path_to */) override + { + throwNotAllowed(); + } + void createDirectory(const std::string & /* path */) override + { + throwNotAllowed(); + } + void createDirectoryRecursive(const std::string & /* path */) override + { + throwNotAllowed(); + } + void addBlobToMetadata(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) override + { + throwNotAllowed(); + } + void unlinkMetadata(const std::string & /* path */) override + { + throwNotAllowed(); + } + + + /// + /// Others + /// + bool supportsChmod() const override { return false; } + +private: + [[noreturn]] static void throwNotAllowed() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only read-only transaction operations are supported"); + } +}; + +/// Readonly, throws NOT_IMPLEMENTED error. +/// Can be used to add limited read-only support of MergeTree. +class ReadOnlyMetadataStorage : public IMetadataStorage +{ +public: + /// + /// Noop + /// + Poco::Timestamp getLastModified(const std::string & /* path */) const override + { + /// Required by MergeTree + return {}; + } + time_t getLastChanged(const std::string & /* path */) const override + { + return {}; + } + + struct stat stat(const String & /* path */) const override + { + throwNotAllowed(); + } + + uint32_t getHardlinkCount(const std::string & /* path */) const override + { + return 1; + } + + std::string readFileToString(const std::string & /* path */) const override + { + throwNotAllowed(); + } + + std::unordered_map getSerializedMetadata(const std::vector & /* file_paths */) const override + { + throwNotAllowed(); + } + + /// + /// Others + /// + bool supportsChmod() const override { return false; } + bool supportsStat() const override { return false; } + +private: + [[noreturn]] static void throwNotAllowed() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only read-only metadata operations are supported"); + } +}; + +} diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp index aa125e93dee..12c2cd16a9f 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.cpp @@ -12,7 +12,6 @@ namespace DB namespace ErrorCodes { - extern const int NOT_IMPLEMENTED; extern const int FILE_DOESNT_EXIST; extern const int NETWORK_ERROR; } @@ -168,91 +167,11 @@ DirectoryIteratorPtr MetadataStorageFromStaticFilesWebServer::iterateDirectory(c return std::make_unique(std::move(dir_file_paths)); } -std::string MetadataStorageFromStaticFilesWebServer::readFileToString(const std::string &) const -{ - WebObjectStorage::throwNotAllowed(); -} - -Poco::Timestamp MetadataStorageFromStaticFilesWebServer::getLastModified(const std::string &) const -{ - return {}; -} - -time_t MetadataStorageFromStaticFilesWebServer::getLastChanged(const std::string &) const -{ - return {}; -} - -uint32_t MetadataStorageFromStaticFilesWebServer::getHardlinkCount(const std::string &) const -{ - return 1; -} - const IMetadataStorage & MetadataStorageFromStaticFilesWebServerTransaction::getStorageForNonTransactionalReads() const { return metadata_storage; } -void MetadataStorageFromStaticFilesWebServerTransaction::writeStringToFile(const std::string &, const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::setLastModified(const std::string &, const Poco::Timestamp &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::unlinkFile(const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::removeRecursive(const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::removeDirectory(const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::moveFile(const std::string &, const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::moveDirectory(const std::string &, const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::replaceFile(const std::string &, const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::setReadOnly(const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::createHardLink(const std::string &, const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::addBlobToMetadata(const std::string &, const std::string &, uint64_t) -{ - WebObjectStorage::throwNotAllowed(); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::unlinkMetadata(const std::string &) -{ - WebObjectStorage::throwNotAllowed(); -} - void MetadataStorageFromStaticFilesWebServerTransaction::createDirectory(const std::string &) { /// Noop. @@ -263,30 +182,4 @@ void MetadataStorageFromStaticFilesWebServerTransaction::createDirectoryRecursiv /// Noop. } -void MetadataStorageFromStaticFilesWebServerTransaction::createEmptyMetadataFile(const std::string & /* path */) -{ - /// Noop. -} - -void MetadataStorageFromStaticFilesWebServerTransaction::createMetadataFile( - const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) -{ - /// Noop. -} - -void MetadataStorageFromStaticFilesWebServerTransaction::commit() -{ - /// Noop. -} - -std::unordered_map MetadataStorageFromStaticFilesWebServer::getSerializedMetadata(const std::vector &) const -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getSerializedMetadata is not implemented for MetadataStorageFromStaticFilesWebServer"); -} - -void MetadataStorageFromStaticFilesWebServerTransaction::chmod(const String &, mode_t) -{ - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "chmod is not implemented for MetadataStorageFromStaticFilesWebServer"); -} - } diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h index 27a1ae8b8fa..6ef44defad0 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h @@ -1,6 +1,7 @@ #pragma once #include +#include #include #include #include @@ -9,7 +10,7 @@ namespace DB { -class MetadataStorageFromStaticFilesWebServer final : public IMetadataStorage +class MetadataStorageFromStaticFilesWebServer final : public ReadOnlyMetadataStorage { private: friend class MetadataStorageFromStaticFilesWebServerTransaction; @@ -36,32 +37,18 @@ public: uint64_t getFileSize(const String & path) const override; - Poco::Timestamp getLastModified(const std::string & path) const override; - - time_t getLastChanged(const std::string & path) const override; - std::vector listDirectory(const std::string & path) const override; DirectoryIteratorPtr iterateDirectory(const std::string & path) const override; - std::string readFileToString(const std::string & path) const override; - - std::unordered_map getSerializedMetadata(const std::vector & file_paths) const override; - - uint32_t getHardlinkCount(const std::string & path) const override; - StoredObjects getStorageObjects(const std::string & path) const override; std::string getObjectStorageRootPath() const override { return ""; } - bool supportsChmod() const override { return false; } - - bool supportsStat() const override { return false; } - - struct stat stat(const String &) const override { return {}; } + struct stat stat(const String & /* path */) const override { return {}; } }; -class MetadataStorageFromStaticFilesWebServerTransaction final : public IMetadataTransaction +class MetadataStorageFromStaticFilesWebServerTransaction final : public ReadOnlyMetadataTransaction { private: DiskPtr disk; @@ -77,43 +64,9 @@ public: const IMetadataStorage & getStorageForNonTransactionalReads() const override; - void commit() override; - - void writeStringToFile(const std::string & path, const std::string & data) override; - - void createEmptyMetadataFile(const std::string & path) override; - - void createMetadataFile(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; - - void addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; - - void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) override; - - void setReadOnly(const std::string & path) override; - - void unlinkFile(const std::string & path) override; - void createDirectory(const std::string & path) override; void createDirectoryRecursive(const std::string & path) override; - - void removeDirectory(const std::string & path) override; - - void removeRecursive(const std::string & path) override; - - void createHardLink(const std::string & path_from, const std::string & path_to) override; - - void moveFile(const std::string & path_from, const std::string & path_to) override; - - void moveDirectory(const std::string & path_from, const std::string & path_to) override; - - void replaceFile(const std::string & path_from, const std::string & path_to) override; - - void unlinkMetadata(const std::string & path) override; - - bool supportsChmod() const override { return false; } - - void chmod(const String &, mode_t) override; }; } From 95fb2ad3cf3118dfbd9f5183f6fdcecae59fba2f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 22:02:29 +0200 Subject: [PATCH 103/152] Implement ATTACH of MergeTree table for s3_plain disk Signed-off-by: Azat Khuzhin --- .../MetadataStorageFromPlainObjectStorage.cpp | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 0722c3622b9..19f7f107b35 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -76,20 +76,20 @@ uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { RelativePathsWithSize children; - object_storage->listPrefix(getAbsolutePath(path), children); + std::vector common_prefixes; + object_storage->listPrefixInPath(getAbsolutePath(path), children, common_prefixes); std::vector result; for (const auto & path_size : children) - { result.push_back(path_size.relative_path); - } + for (const auto & common_prefix : common_prefixes) + result.push_back(common_prefix); return result; } DirectoryIteratorPtr MetadataStorageFromPlainObjectStorage::iterateDirectory(const std::string & path) const { - /// NOTE: this is not required for BACKUP/RESTORE, but this is a first step - /// towards MergeTree on plain S3. + /// Required for MergeTree auto paths = listDirectory(path); std::vector fs_paths(paths.begin(), paths.end()); return std::make_unique(std::move(fs_paths)); From 82ea67eb51d878d697476385fe2feed5573cb81f Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 22:04:13 +0200 Subject: [PATCH 104/152] Slightly better MetadataStorageFromPlainObjectStorage::isFile() Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index 19f7f107b35..de97203d771 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -46,7 +46,7 @@ bool MetadataStorageFromPlainObjectStorage::exists(const std::string & path) con bool MetadataStorageFromPlainObjectStorage::isFile(const std::string & path) const { /// NOTE: This check is inaccurate and has excessive API calls - return !isDirectory(path) && exists(path); + return exists(path) && !isDirectory(path); } bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path) const From fa7535c90d22f514dac6d8c85c79beab88ebce7e Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 22:05:26 +0200 Subject: [PATCH 105/152] Slightly optimize MetadataStorageFromPlainObjectStorage::isDirectory() Signed-off-by: Azat Khuzhin --- .../ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index de97203d771..aa83effc481 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -58,8 +58,9 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path /// NOTE: This check is far from ideal, since it work only if the directory /// really has files, and has excessive API calls RelativePathsWithSize children; - object_storage->listPrefix(directory, children); - return !children.empty(); + std::vector common_prefixes; + object_storage->listPrefixInPath(directory, children, common_prefixes); + return !children.empty() || !common_prefixes.empty(); } uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const From 8345c5f8812d3a8a1a2d9ea0eed03d12e94e270d Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 24 Oct 2022 22:07:26 +0200 Subject: [PATCH 106/152] tests: cover ATTACH of BACKUP from s3_plain disk Signed-off-by: Azat Khuzhin --- .../__init__.py | 0 .../configs/disk_s3.xml | 34 ++++++++++++++++ .../test_attach_backup_from_s3_plain/test.py | 40 +++++++++++++++++++ 3 files changed, 74 insertions(+) create mode 100644 tests/integration/test_attach_backup_from_s3_plain/__init__.py create mode 100644 tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml create mode 100644 tests/integration/test_attach_backup_from_s3_plain/test.py diff --git a/tests/integration/test_attach_backup_from_s3_plain/__init__.py b/tests/integration/test_attach_backup_from_s3_plain/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml new file mode 100644 index 00000000000..67278694d39 --- /dev/null +++ b/tests/integration/test_attach_backup_from_s3_plain/configs/disk_s3.xml @@ -0,0 +1,34 @@ + + + + + + s3_plain + http://minio1:9001/root/data/disks/disk_s3_plain/ + minio + minio123 + 33554432 + + + s3_plain + + http://minio1:9001/root/data/disks/disk_s3_plain/backup/ + minio + minio123 + 33554432 + + + + + +
+ attach_disk_s3_plain +
+
+
+
+
+ + backup_disk_s3_plain + +
diff --git a/tests/integration/test_attach_backup_from_s3_plain/test.py b/tests/integration/test_attach_backup_from_s3_plain/test.py new file mode 100644 index 00000000000..35d53d5b8bd --- /dev/null +++ b/tests/integration/test_attach_backup_from_s3_plain/test.py @@ -0,0 +1,40 @@ +# pylint: disable=global-statement +# pylint: disable=line-too-long + +import pytest +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance( + "node", + main_configs=["configs/disk_s3.xml"], + with_minio=True, +) + + +@pytest.fixture(scope="module", autouse=True) +def start_cluster(): + try: + cluster.start() + yield + finally: + cluster.shutdown() + + +def test_attach_backup(): + node.query( + f""" + -- BACKUP writes Ordinary like structure + set allow_deprecated_database_ordinary=1; + create database ordinary engine=Ordinary; + + create table ordinary.test_backup_attach engine=MergeTree() order by tuple() as select * from numbers(100); + -- NOTE: name of backup ("backup") is significant. + backup table ordinary.test_backup_attach TO Disk('backup_disk_s3_plain', 'backup'); + + drop table ordinary.test_backup_attach; + attach table ordinary.test_backup_attach (number UInt64) engine=MergeTree() order by tuple() settings storage_policy='attach_policy_s3_plain'; + """ + ) + + assert int(node.query("select count() from ordinary.test_backup_attach")) == 100 From 3b7abbbff437cd6c38ae35bc5c1e04a419f279fb Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 28 Oct 2022 16:17:52 +0200 Subject: [PATCH 107/152] Rename s/listPrefix/findAllFiles, s/listPrefixInPath/getDirectoryContents/ Signed-off-by: Azat Khuzhin --- .../AzureBlobStorage/AzureObjectStorage.cpp | 2 +- .../AzureBlobStorage/AzureObjectStorage.h | 2 +- .../Cached/CachedObjectStorage.cpp | 4 ++-- .../Cached/CachedObjectStorage.h | 2 +- ...jectStorageRemoteMetadataRestoreHelper.cpp | 4 ++-- src/Disks/ObjectStorages/IObjectStorage.cpp | 8 +++---- src/Disks/ObjectStorages/IObjectStorage.h | 24 +++++++++---------- .../MetadataStorageFromPlainObjectStorage.cpp | 24 +++++++++---------- .../ObjectStorages/S3/S3ObjectStorage.cpp | 16 ++++++------- src/Disks/ObjectStorages/S3/S3ObjectStorage.h | 8 +++---- 10 files changed, 47 insertions(+), 47 deletions(-) diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp index b3dcfdafa9e..c3549701ec1 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.cpp @@ -141,7 +141,7 @@ std::unique_ptr AzureObjectStorage::writeObject( /// NO return std::make_unique(std::move(buffer), std::move(finalize_callback), object.absolute_path); } -void AzureObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize & children) const +void AzureObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children) const { auto client_ptr = client.get(); diff --git a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h index 47ac0d6badd..6fd41dae2ec 100644 --- a/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h +++ b/src/Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h @@ -84,7 +84,7 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; + void findAllFiles(const std::string & path, RelativePathsWithSize & children) const override; /// Remove file. Throws exception if file doesn't exists or it's a directory. void removeObject(const StoredObject & object) override; diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp index 7e829847846..f3d3f049dc1 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.cpp @@ -282,9 +282,9 @@ std::unique_ptr CachedObjectStorage::cloneObjectStorage( return object_storage->cloneObjectStorage(new_namespace, config, config_prefix, context); } -void CachedObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize & children) const +void CachedObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children) const { - object_storage->listPrefix(path, children); + object_storage->findAllFiles(path, children); } ObjectMetadata CachedObjectStorage::getObjectMetadata(const std::string & path) const diff --git a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h index 68ded61a9f1..64e6eed45bb 100644 --- a/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h +++ b/src/Disks/ObjectStorages/Cached/CachedObjectStorage.h @@ -72,7 +72,7 @@ public: const std::string & config_prefix, ContextPtr context) override; - void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; + void findAllFiles(const std::string & path, RelativePathsWithSize & children) const override; ObjectMetadata getObjectMetadata(const std::string & path) const override; diff --git a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp index 65f0d24035a..4ea42616ba2 100644 --- a/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp +++ b/src/Disks/ObjectStorages/DiskObjectStorageRemoteMetadataRestoreHelper.cpp @@ -390,7 +390,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFiles(IObjectStorage * }; RelativePathsWithSize children; - source_object_storage->listPrefix(restore_information.source_path, children); + source_object_storage->findAllFiles(restore_information.source_path, children); restore_files(children); @@ -540,7 +540,7 @@ void DiskObjectStorageRemoteMetadataRestoreHelper::restoreFileOperations(IObject }; RelativePathsWithSize children; - source_object_storage->listPrefix(restore_information.source_path + "operations/", children); + source_object_storage->findAllFiles(restore_information.source_path + "operations/", children); restore_file_operations(children); if (restore_information.detached) diff --git a/src/Disks/ObjectStorages/IObjectStorage.cpp b/src/Disks/ObjectStorages/IObjectStorage.cpp index 7c187e3dd2f..3f8ac566603 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.cpp +++ b/src/Disks/ObjectStorages/IObjectStorage.cpp @@ -14,15 +14,15 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -void IObjectStorage::listPrefix(const std::string &, RelativePathsWithSize &) const +void IObjectStorage::findAllFiles(const std::string &, RelativePathsWithSize &) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listPrefix() is not supported"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "findAllFiles() is not supported"); } -void IObjectStorage::listPrefixInPath(const std::string &, +void IObjectStorage::getDirectoryContents(const std::string &, RelativePathsWithSize &, std::vector &) const { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "listPrefixInPath() is not supported"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "getDirectoryContents() is not supported"); } IAsynchronousReader & IObjectStorage::getThreadPoolReader() diff --git a/src/Disks/ObjectStorages/IObjectStorage.h b/src/Disks/ObjectStorages/IObjectStorage.h index a7ad4e5b515..9451ae31b07 100644 --- a/src/Disks/ObjectStorages/IObjectStorage.h +++ b/src/Disks/ObjectStorages/IObjectStorage.h @@ -65,32 +65,32 @@ public: /// Object exists or not virtual bool exists(const StoredObject & object) const = 0; - /// List on prefix, return children (relative paths) with their sizes. - /// So it should return all "objects" with their sizes. + /// List all objects with specific prefix. /// /// For example if you do this over filesystem, you should skip folders and /// return files only, so something like on local filesystem: /// /// find . -type f /// + /// @param children - out files (relative paths) with their sizes. + /// /// NOTE: It makes sense only for real object storages (S3, Azure), since /// it is used only for one of the following: /// - send_metadata (to restore metadata) /// - see DiskObjectStorage::restoreMetadataIfNeeded() /// - MetadataStorageFromPlainObjectStorage - only for s3_plain disk - virtual void listPrefix(const std::string & path, RelativePathsWithSize & children) const; + virtual void findAllFiles(const std::string & path, RelativePathsWithSize & children) const; - /// List on prefix, but it is limited only to current path, something like - /// on local filesystem: + /// Analog of directory content for object storage (object storage does not + /// have "directory" definition, but it can be emulated with usage of + /// "delimiter"), so this is analog of: /// - /// find . -maxdepth 1 + /// find . -maxdepth 1 $path /// - /// @param path - path to list prefix in - /// @param children - list of child nodes (for unix filesystem - files goes here) - /// @param common_prefixes - list of common prefixes in this path, (for unix filesystem - subfolders) - virtual void listPrefixInPath(const std::string & path, - RelativePathsWithSize & children, - std::vector & common_prefixes) const; + /// Return files in @files and directories in @directories + virtual void getDirectoryContents(const std::string & path, + RelativePathsWithSize & files, + std::vector & directories) const; /// Get object metadata if supported. It should be possible to receive /// at least size of object diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp index aa83effc481..259f6e01fd7 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.cpp @@ -57,34 +57,34 @@ bool MetadataStorageFromPlainObjectStorage::isDirectory(const std::string & path /// NOTE: This check is far from ideal, since it work only if the directory /// really has files, and has excessive API calls - RelativePathsWithSize children; - std::vector common_prefixes; - object_storage->listPrefixInPath(directory, children, common_prefixes); - return !children.empty() || !common_prefixes.empty(); + RelativePathsWithSize files; + std::vector directories; + object_storage->getDirectoryContents(directory, files, directories); + return !files.empty() || !directories.empty(); } uint64_t MetadataStorageFromPlainObjectStorage::getFileSize(const String & path) const { RelativePathsWithSize children; - object_storage->listPrefix(getAbsolutePath(path), children); + object_storage->findAllFiles(getAbsolutePath(path), children); if (children.empty()) return 0; if (children.size() != 1) - throw Exception(ErrorCodes::LOGICAL_ERROR, "listPrefix() return multiple paths ({}) for {}", children.size(), path); + throw Exception(ErrorCodes::LOGICAL_ERROR, "findAllFiles() return multiple paths ({}) for {}", children.size(), path); return children.front().bytes_size; } std::vector MetadataStorageFromPlainObjectStorage::listDirectory(const std::string & path) const { - RelativePathsWithSize children; - std::vector common_prefixes; - object_storage->listPrefixInPath(getAbsolutePath(path), children, common_prefixes); + RelativePathsWithSize files; + std::vector directories; + object_storage->getDirectoryContents(getAbsolutePath(path), files, directories); std::vector result; - for (const auto & path_size : children) + for (const auto & path_size : files) result.push_back(path_size.relative_path); - for (const auto & common_prefix : common_prefixes) - result.push_back(common_prefix); + for (const auto & directory : directories) + result.push_back(directory); return result; } diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp index af3374f8d3c..0c421ee03d7 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.cpp @@ -248,7 +248,7 @@ std::unique_ptr S3ObjectStorage::writeObject( /// NOLIN std::move(s3_buffer), std::move(finalize_callback), object.absolute_path); } -void S3ObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize & children) const +void S3ObjectStorage::findAllFiles(const std::string & path, RelativePathsWithSize & children) const { auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); @@ -279,9 +279,9 @@ void S3ObjectStorage::listPrefix(const std::string & path, RelativePathsWithSize } while (outcome.GetResult().GetIsTruncated()); } -void S3ObjectStorage::listPrefixInPath(const std::string & path, - RelativePathsWithSize & children, - std::vector & common_prefixes) const +void S3ObjectStorage::getDirectoryContents(const std::string & path, + RelativePathsWithSize & files, + std::vector & directories) const { auto settings_ptr = s3_settings.get(); auto client_ptr = client.get(); @@ -308,14 +308,14 @@ void S3ObjectStorage::listPrefixInPath(const std::string & path, break; for (const auto & object : result_objects) - children.emplace_back(object.GetKey(), object.GetSize()); + files.emplace_back(object.GetKey(), object.GetSize()); for (const auto & common_prefix : result_common_prefixes) { - std::string subfolder = common_prefix.GetPrefix(); + std::string directory = common_prefix.GetPrefix(); /// Make it compatible with std::filesystem::path::filename() - trimRight(subfolder, '/'); - common_prefixes.emplace_back(subfolder); + trimRight(directory, '/'); + directories.emplace_back(directory); } request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken()); diff --git a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h index 9e90efcb9dc..6b1e8289b15 100644 --- a/src/Disks/ObjectStorages/S3/S3ObjectStorage.h +++ b/src/Disks/ObjectStorages/S3/S3ObjectStorage.h @@ -105,10 +105,10 @@ public: size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, const WriteSettings & write_settings = {}) override; - void listPrefix(const std::string & path, RelativePathsWithSize & children) const override; - void listPrefixInPath(const std::string & path, - RelativePathsWithSize & children, - std::vector & common_prefixes) const override; + void findAllFiles(const std::string & path, RelativePathsWithSize & children) const override; + void getDirectoryContents(const std::string & path, + RelativePathsWithSize & files, + std::vector & directories) const override; /// Uses `DeleteObjectRequest`. void removeObject(const StoredObject & object) override; From bceca73f6f362577ef6f9f8b05ac1a1ae65da1c1 Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Fri, 28 Oct 2022 18:10:00 +0200 Subject: [PATCH 108/152] Throw NOT_IMPLEMENTED form ReadOnlyMetadataStorage::getLastChanged() Signed-off-by: Azat Khuzhin --- .../ObjectStorages/ReadOnlyMetadataStorage.h | 21 ++++++++++--------- 1 file changed, 11 insertions(+), 10 deletions(-) diff --git a/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h b/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h index e136b0580ea..b21d2c3a28b 100644 --- a/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h +++ b/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h @@ -124,26 +124,27 @@ public: /// Required by MergeTree return {}; } - time_t getLastChanged(const std::string & /* path */) const override - { - return {}; - } - - struct stat stat(const String & /* path */) const override - { - throwNotAllowed(); - } uint32_t getHardlinkCount(const std::string & /* path */) const override { return 1; } + /// + /// Throw + /// + struct stat stat(const String & /* path */) const override + { + throwNotAllowed(); + } + time_t getLastChanged(const std::string & /* path */) const override + { + throwNotAllowed(); + } std::string readFileToString(const std::string & /* path */) const override { throwNotAllowed(); } - std::unordered_map getSerializedMetadata(const std::vector & /* file_paths */) const override { throwNotAllowed(); From 51bd0c2ac11f5b36aab2824938d60b60066359cc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Sat, 29 Oct 2022 11:56:12 +0200 Subject: [PATCH 109/152] Make ReadOnlyMetadataStorage really readonly Signed-off-by: Azat Khuzhin --- .../MetadataStorageFromPlainObjectStorage.h | 10 +++++ .../ObjectStorages/ReadOnlyMetadataStorage.h | 44 +++++-------------- .../MetadataStorageFromStaticFilesWebServer.h | 10 +++++ 3 files changed, 32 insertions(+), 32 deletions(-) diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index 2cfa123ecc9..e4c0fb90fb1 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -80,6 +80,16 @@ public: void addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; + void createEmptyMetadataFile(const std::string & /* path */) override + { + /// No metadata, no need to create anything. + } + + void createMetadataFile(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) override + { + /// Noop + } + void createDirectory(const std::string & path) override; void createDirectoryRecursive(const std::string & path) override; diff --git a/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h b/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h index b21d2c3a28b..f59bc0ad77f 100644 --- a/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h +++ b/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h @@ -13,30 +13,24 @@ namespace ErrorCodes class ReadOnlyMetadataStorage; -/// Readonly, throws NOT_IMPLEMENTED error. +/// Transaction for read-only storage, throws NOT_IMPLEMENTED error. /// Can be used to add limited read-only support of MergeTree. class ReadOnlyMetadataTransaction : public IMetadataTransaction { public: - /// - /// Noop - /// void commit() override { /// Noop, nothing to commit. } + void createEmptyMetadataFile(const std::string & /* path */) override { - /// No metadata, no need to create anything. + throwNotAllowed(); } void createMetadataFile(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) override { - /// Noop + throwNotAllowed(); } - - /// - /// Throws - /// void writeStringToFile(const std::string & /* path */, const std::string & /* data */) override { throwNotAllowed(); @@ -98,68 +92,54 @@ public: throwNotAllowed(); } - - /// - /// Others - /// bool supportsChmod() const override { return false; } private: [[noreturn]] static void throwNotAllowed() { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only read-only transaction operations are supported"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transaction for read-only storage is not supported"); } }; -/// Readonly, throws NOT_IMPLEMENTED error. +/// Readonly storage, throws NOT_IMPLEMENTED error. /// Can be used to add limited read-only support of MergeTree. class ReadOnlyMetadataStorage : public IMetadataStorage { public: - /// - /// Noop - /// Poco::Timestamp getLastModified(const std::string & /* path */) const override { /// Required by MergeTree return {}; } - uint32_t getHardlinkCount(const std::string & /* path */) const override { return 1; } - /// - /// Throw - /// struct stat stat(const String & /* path */) const override { - throwNotAllowed(); + throwNotImplemented(); } time_t getLastChanged(const std::string & /* path */) const override { - throwNotAllowed(); + throwNotImplemented(); } std::string readFileToString(const std::string & /* path */) const override { - throwNotAllowed(); + throwNotImplemented(); } std::unordered_map getSerializedMetadata(const std::vector & /* file_paths */) const override { - throwNotAllowed(); + throwNotImplemented(); } - /// - /// Others - /// bool supportsChmod() const override { return false; } bool supportsStat() const override { return false; } private: - [[noreturn]] static void throwNotAllowed() + [[noreturn]] static void throwNotImplemented() { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Only read-only metadata operations are supported"); + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Operation is ont implemented"); } }; diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h index 6ef44defad0..eb70b9c8108 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h @@ -64,6 +64,16 @@ public: const IMetadataStorage & getStorageForNonTransactionalReads() const override; + void createEmptyMetadataFile(const std::string & /* path */) override + { + /// No metadata, no need to create anything. + } + + void createMetadataFile(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) override + { + /// Noop + } + void createDirectory(const std::string & path) override; void createDirectoryRecursive(const std::string & path) override; From c93262170df0c0fb09ccbb56f5f73b7a41bc58fc Mon Sep 17 00:00:00 2001 From: Azat Khuzhin Date: Mon, 31 Oct 2022 11:02:13 +0100 Subject: [PATCH 110/152] Remove ReadOnlyMetadataStorage Throw exceptions from IMetadataStorage instead to avoid introducing extra abstractions. Signed-off-by: Azat Khuzhin --- src/Disks/ObjectStorages/IMetadataStorage.h | 108 ++++++++++--- .../MetadataStorageFromPlainObjectStorage.h | 33 ++-- .../ObjectStorages/ReadOnlyMetadataStorage.h | 146 ------------------ .../MetadataStorageFromStaticFilesWebServer.h | 27 +++- 4 files changed, 136 insertions(+), 178 deletions(-) delete mode 100644 src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h diff --git a/src/Disks/ObjectStorages/IMetadataStorage.h b/src/Disks/ObjectStorages/IMetadataStorage.h index 3d6c772157d..597d7744c78 100644 --- a/src/Disks/ObjectStorages/IMetadataStorage.h +++ b/src/Disks/ObjectStorages/IMetadataStorage.h @@ -11,10 +11,16 @@ #include #include #include +#include namespace DB { +namespace ErrorCodes +{ + extern const int NOT_IMPLEMENTED; +} + class IMetadataStorage; /// Tries to provide some "transactions" interface, which allow @@ -33,32 +39,71 @@ public: /// General purpose methods /// Write metadata string to file - virtual void writeStringToFile(const std::string & path, const std::string & data) = 0; + virtual void writeStringToFile(const std::string & /* path */, const std::string & /* data */) + { + throwNotImplemented(); + } - virtual void setLastModified(const std::string & path, const Poco::Timestamp & timestamp) = 0; + virtual void setLastModified(const std::string & /* path */, const Poco::Timestamp & /* timestamp */) + { + throwNotImplemented(); + } virtual bool supportsChmod() const = 0; - virtual void chmod(const String & path, mode_t mode) = 0; + virtual void chmod(const String & /* path */, mode_t /* mode */) + { + throwNotImplemented(); + } - virtual void setReadOnly(const std::string & path) = 0; + virtual void setReadOnly(const std::string & /* path */) + { + throwNotImplemented(); + } - virtual void unlinkFile(const std::string & path) = 0; + virtual void unlinkFile(const std::string & /* path */) + { + throwNotImplemented(); + } - virtual void createDirectory(const std::string & path) = 0; + virtual void createDirectory(const std::string & /* path */) + { + throwNotImplemented(); + } - virtual void createDirectoryRecursive(const std::string & path) = 0; + virtual void createDirectoryRecursive(const std::string & /* path */) + { + throwNotImplemented(); + } - virtual void removeDirectory(const std::string & path) = 0; + virtual void removeDirectory(const std::string & /* path */) + { + throwNotImplemented(); + } - virtual void removeRecursive(const std::string & path) = 0; + virtual void removeRecursive(const std::string & /* path */) + { + throwNotImplemented(); + } - virtual void createHardLink(const std::string & path_from, const std::string & path_to) = 0; + virtual void createHardLink(const std::string & /* path_from */, const std::string & /* path_to */) + { + throwNotImplemented(); + } - virtual void moveFile(const std::string & path_from, const std::string & path_to) = 0; + virtual void moveFile(const std::string & /* path_from */, const std::string & /* path_to */) + { + throwNotImplemented(); + } - virtual void moveDirectory(const std::string & path_from, const std::string & path_to) = 0; + virtual void moveDirectory(const std::string & /* path_from */, const std::string & /* path_to */) + { + throwNotImplemented(); + } - virtual void replaceFile(const std::string & path_from, const std::string & path_to) = 0; + virtual void replaceFile(const std::string & /* path_from */, const std::string & /* path_to */) + { + throwNotImplemented(); + } /// Metadata related methods @@ -69,7 +114,10 @@ public: virtual void createMetadataFile(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) = 0; /// Add to new blob to metadata file (way to implement appends) - virtual void addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) = 0; + virtual void addBlobToMetadata(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) + { + throwNotImplemented(); + } /// Unlink metadata file and do something special if required /// By default just remove file (unlink file). @@ -79,6 +127,12 @@ public: } virtual ~IMetadataTransaction() = default; + +private: + [[noreturn]] static void throwNotImplemented() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Operation is not implemented"); + } }; using MetadataTransactionPtr = std::shared_ptr; @@ -106,12 +160,18 @@ public: virtual Poco::Timestamp getLastModified(const std::string & path) const = 0; - virtual time_t getLastChanged(const std::string & path) const = 0; + virtual time_t getLastChanged(const std::string & /* path */) const + { + throwNotImplemented(); + } virtual bool supportsChmod() const = 0; virtual bool supportsStat() const = 0; - virtual struct stat stat(const String & path) const = 0; + virtual struct stat stat(const String & /* path */) const + { + throwNotImplemented(); + } virtual std::vector listDirectory(const std::string & path) const = 0; @@ -120,20 +180,32 @@ public: virtual uint32_t getHardlinkCount(const std::string & path) const = 0; /// Read metadata file to string from path - virtual std::string readFileToString(const std::string & path) const = 0; + virtual std::string readFileToString(const std::string & /* path */) const + { + throwNotImplemented(); + } virtual ~IMetadataStorage() = default; /// ==== More specific methods. Previous were almost general purpose. ==== /// Read multiple metadata files into strings and return mapping from file_path -> metadata - virtual std::unordered_map getSerializedMetadata(const std::vector & file_paths) const = 0; + virtual std::unordered_map getSerializedMetadata(const std::vector & /* file_paths */) const + { + throwNotImplemented(); + } /// Return object information (absolute_path, bytes_size, ...) for metadata path. /// object_storage_path is absolute. virtual StoredObjects getStorageObjects(const std::string & path) const = 0; virtual std::string getObjectStorageRootPath() const = 0; + +private: + [[noreturn]] static void throwNotImplemented() + { + throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Operation is not implemented"); + } }; using MetadataStoragePtr = std::shared_ptr; diff --git a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h index e4c0fb90fb1..99cc960b9e4 100644 --- a/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h +++ b/src/Disks/ObjectStorages/MetadataStorageFromPlainObjectStorage.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -21,10 +20,7 @@ namespace DB /// It is used to allow BACKUP/RESTORE to ObjectStorage (S3/...) with the same /// structure as on disk MergeTree, and does not requires metadata from local /// disk to restore. -/// -/// NOTE: Inheritance from ReadOnlyMetadataStorage is used here to throw -/// NOT_IMPLEMENTED error for lost of unsupported methods (mtime/move/stat/...) -class MetadataStorageFromPlainObjectStorage final : public ReadOnlyMetadataStorage +class MetadataStorageFromPlainObjectStorage final : public IMetadataStorage { private: friend class MetadataStorageFromPlainObjectStorageTransaction; @@ -59,11 +55,25 @@ public: std::string getObjectStorageRootPath() const override { return object_storage_root_path; } + Poco::Timestamp getLastModified(const std::string & /* path */) const override + { + /// Required by MergeTree + return {}; + } + + uint32_t getHardlinkCount(const std::string & /* path */) const override + { + return 1; + } + + bool supportsChmod() const override { return false; } + bool supportsStat() const override { return false; } + private: std::filesystem::path getAbsolutePath(const std::string & path) const; }; -class MetadataStorageFromPlainObjectStorageTransaction final : public ReadOnlyMetadataTransaction +class MetadataStorageFromPlainObjectStorageTransaction final : public IMetadataTransaction { private: const MetadataStorageFromPlainObjectStorage & metadata_storage; @@ -74,9 +84,7 @@ public: : metadata_storage(metadata_storage_) {} - ~MetadataStorageFromPlainObjectStorageTransaction() override = default; - - const IMetadataStorage & getStorageForNonTransactionalReads() const final; + const IMetadataStorage & getStorageForNonTransactionalReads() const override; void addBlobToMetadata(const std::string & path, const std::string & blob_name, uint64_t size_in_bytes) override; @@ -97,6 +105,13 @@ public: void unlinkFile(const std::string & path) override; void unlinkMetadata(const std::string & path) override; + + void commit() override + { + /// Nothing to commit. + } + + bool supportsChmod() const override { return false; } }; } diff --git a/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h b/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h deleted file mode 100644 index f59bc0ad77f..00000000000 --- a/src/Disks/ObjectStorages/ReadOnlyMetadataStorage.h +++ /dev/null @@ -1,146 +0,0 @@ -#pragma once - -#include -#include - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int NOT_IMPLEMENTED; -} - -class ReadOnlyMetadataStorage; - -/// Transaction for read-only storage, throws NOT_IMPLEMENTED error. -/// Can be used to add limited read-only support of MergeTree. -class ReadOnlyMetadataTransaction : public IMetadataTransaction -{ -public: - void commit() override - { - /// Noop, nothing to commit. - } - - void createEmptyMetadataFile(const std::string & /* path */) override - { - throwNotAllowed(); - } - void createMetadataFile(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) override - { - throwNotAllowed(); - } - void writeStringToFile(const std::string & /* path */, const std::string & /* data */) override - { - throwNotAllowed(); - } - void setLastModified(const std::string & /* path */, const Poco::Timestamp & /* timestamp */) override - { - throwNotAllowed(); - } - void chmod(const String & /* path */, mode_t /* mode */) override - { - throwNotAllowed(); - } - void setReadOnly(const std::string & /* path */) override - { - throwNotAllowed(); - } - void unlinkFile(const std::string & /* path */) override - { - throwNotAllowed(); - } - void removeDirectory(const std::string & /* path */) override - { - throwNotAllowed(); - } - void removeRecursive(const std::string & /* path */) override - { - throwNotAllowed(); - } - void createHardLink(const std::string & /* path_from */, const std::string & /* path_to */) override - { - throwNotAllowed(); - } - void moveFile(const std::string & /* path_from */, const std::string & /* path_to */) override - { - throwNotAllowed(); - } - void moveDirectory(const std::string & /* path_from */, const std::string & /* path_to */) override - { - throwNotAllowed(); - } - void replaceFile(const std::string & /* path_from */, const std::string & /* path_to */) override - { - throwNotAllowed(); - } - void createDirectory(const std::string & /* path */) override - { - throwNotAllowed(); - } - void createDirectoryRecursive(const std::string & /* path */) override - { - throwNotAllowed(); - } - void addBlobToMetadata(const std::string & /* path */, const std::string & /* blob_name */, uint64_t /* size_in_bytes */) override - { - throwNotAllowed(); - } - void unlinkMetadata(const std::string & /* path */) override - { - throwNotAllowed(); - } - - bool supportsChmod() const override { return false; } - -private: - [[noreturn]] static void throwNotAllowed() - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Transaction for read-only storage is not supported"); - } -}; - -/// Readonly storage, throws NOT_IMPLEMENTED error. -/// Can be used to add limited read-only support of MergeTree. -class ReadOnlyMetadataStorage : public IMetadataStorage -{ -public: - Poco::Timestamp getLastModified(const std::string & /* path */) const override - { - /// Required by MergeTree - return {}; - } - uint32_t getHardlinkCount(const std::string & /* path */) const override - { - return 1; - } - - struct stat stat(const String & /* path */) const override - { - throwNotImplemented(); - } - time_t getLastChanged(const std::string & /* path */) const override - { - throwNotImplemented(); - } - std::string readFileToString(const std::string & /* path */) const override - { - throwNotImplemented(); - } - std::unordered_map getSerializedMetadata(const std::vector & /* file_paths */) const override - { - throwNotImplemented(); - } - - bool supportsChmod() const override { return false; } - bool supportsStat() const override { return false; } - -private: - [[noreturn]] static void throwNotImplemented() - { - throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Operation is ont implemented"); - } -}; - -} diff --git a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h index eb70b9c8108..338a2690b8f 100644 --- a/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h +++ b/src/Disks/ObjectStorages/Web/MetadataStorageFromStaticFilesWebServer.h @@ -1,7 +1,6 @@ #pragma once #include -#include #include #include #include @@ -10,7 +9,7 @@ namespace DB { -class MetadataStorageFromStaticFilesWebServer final : public ReadOnlyMetadataStorage +class MetadataStorageFromStaticFilesWebServer final : public IMetadataStorage { private: friend class MetadataStorageFromStaticFilesWebServerTransaction; @@ -46,9 +45,22 @@ public: std::string getObjectStorageRootPath() const override { return ""; } struct stat stat(const String & /* path */) const override { return {}; } + + Poco::Timestamp getLastModified(const std::string & /* path */) const override + { + /// Required by MergeTree + return {}; + } + uint32_t getHardlinkCount(const std::string & /* path */) const override + { + return 1; + } + + bool supportsChmod() const override { return false; } + bool supportsStat() const override { return false; } }; -class MetadataStorageFromStaticFilesWebServerTransaction final : public ReadOnlyMetadataTransaction +class MetadataStorageFromStaticFilesWebServerTransaction final : public IMetadataTransaction { private: DiskPtr disk; @@ -60,8 +72,6 @@ public: : metadata_storage(metadata_storage_) {} - ~MetadataStorageFromStaticFilesWebServerTransaction() override = default; - const IMetadataStorage & getStorageForNonTransactionalReads() const override; void createEmptyMetadataFile(const std::string & /* path */) override @@ -77,6 +87,13 @@ public: void createDirectory(const std::string & path) override; void createDirectoryRecursive(const std::string & path) override; + + void commit() override + { + /// Nothing to commit. + } + + bool supportsChmod() const override { return false; } }; } From bca22ec5f50b05e4e578da816278379d86ab3a6d Mon Sep 17 00:00:00 2001 From: Maksim Kita Date: Mon, 31 Oct 2022 12:30:00 +0100 Subject: [PATCH 111/152] Fixed code review issues --- .../MergeTreeIndexConditionBloomFilter.cpp | 19 +++++------ .../MergeTree/MergeTreeIndexFullText.cpp | 25 +++++++------- src/Storages/MergeTree/MergeTreeIndexUtils.h | 2 +- src/Storages/MergeTree/RPNBuilder.cpp | 33 ++++++++++++------- src/Storages/MergeTree/RPNBuilder.h | 18 ++++++---- 5 files changed, 54 insertions(+), 43 deletions(-) diff --git a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp index 9856500c65c..be7118066bb 100644 --- a/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexConditionBloomFilter.cpp @@ -293,8 +293,6 @@ bool MergeTreeIndexConditionBloomFilter::traverseFunction(const RPNBuilderTreeNo auto lhs_argument = function.getArgumentAt(0); auto rhs_argument = function.getArgumentAt(1); - auto lhs_argument_column_name = lhs_argument.getColumnName(); - if (functionIsInOrGlobalInOperator(function_name)) { ConstSetPtr prepared_set = rhs_argument.tryGetPreparedSet(); @@ -407,20 +405,21 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeIn( if (set_contain_default_value) return false; - const auto & col_name = key_node_function.getArgumentAt(0).getColumnName(); - auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); - auto map_values_index_column_name = fmt::format("mapValues({})", col_name); + auto first_argument = key_node_function.getArgumentAt(0); + const auto column_name = first_argument.getColumnName(); + auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); + auto map_values_index_column_name = fmt::format("mapValues({})", column_name); if (header.has(map_keys_index_column_name)) { /// For mapKeys we serialize key argument with bloom filter - auto first_argument = key_node_function.getArgumentAt(1); + auto second_argument = key_node_function.getArgumentAt(1); Field constant_value; DataTypePtr constant_type; - if (first_argument.tryGetConstant(constant_value, constant_type)) + if (second_argument.tryGetConstant(constant_value, constant_type)) { size_t position = header.getPositionByName(map_keys_index_column_name); const DataTypePtr & index_type = header.getByPosition(position).type; @@ -688,10 +687,10 @@ bool MergeTreeIndexConditionBloomFilter::traverseTreeEquals( return false; auto first_argument = key_node_function.getArgumentAt(0); - const auto col_name = first_argument.getColumnName(); + const auto column_name = first_argument.getColumnName(); - auto map_keys_index_column_name = fmt::format("mapKeys({})", col_name); - auto map_values_index_column_name = fmt::format("mapValues({})", col_name); + auto map_keys_index_column_name = fmt::format("mapKeys({})", column_name); + auto map_values_index_column_name = fmt::format("mapValues({})", column_name); size_t position = 0; Field const_value = value_field; diff --git a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp index 9074485a020..b96d40f5759 100644 --- a/src/Storages/MergeTree/MergeTreeIndexFullText.cpp +++ b/src/Storages/MergeTree/MergeTreeIndexFullText.cpp @@ -434,7 +434,7 @@ bool MergeTreeConditionFullText::traverseTreeEquals( return false; auto first_argument = key_function_node.getArgumentAt(0); - const auto & map_column_name = first_argument.getColumnName(); + const auto map_column_name = first_argument.getColumnName(); size_t map_keys_key_column_num = 0; auto map_keys_index_column_name = fmt::format("mapKeys({})", map_column_name); @@ -588,23 +588,20 @@ bool MergeTreeConditionFullText::tryPrepareSetBloomFilter( std::vector key_tuple_mapping; DataTypes data_types; - if (left_argument.isFunction()) + auto left_argument_function_node_optional = left_argument.toFunctionNodeOrNull(); + + if (left_argument_function_node_optional && left_argument_function_node_optional->getFunctionName() == "tuple") { - auto left_argument_function_node = left_argument.toFunctionNode(); - auto left_argument_function_node_name = left_argument_function_node.getFunctionName(); + const auto & left_argument_function_node = *left_argument_function_node_optional; + size_t left_argument_function_node_arguments_size = left_argument_function_node.getArgumentsSize(); - if (left_argument_function_node_name == "tuple") + for (size_t i = 0; i < left_argument_function_node_arguments_size; ++i) { - size_t left_argument_function_node_arguments_size = left_argument_function_node.getArgumentsSize(); - - for (size_t i = 0; i < left_argument_function_node_arguments_size; ++i) + size_t key = 0; + if (getKey(left_argument_function_node.getArgumentAt(i).getColumnName(), key)) { - size_t key = 0; - if (getKey(left_argument_function_node.getArgumentAt(i).getColumnName(), key)) - { - key_tuple_mapping.emplace_back(i, key); - data_types.push_back(index_data_types[key]); - } + key_tuple_mapping.emplace_back(i, key); + data_types.push_back(index_data_types[key]); } } } diff --git a/src/Storages/MergeTree/MergeTreeIndexUtils.h b/src/Storages/MergeTree/MergeTreeIndexUtils.h index 23cd92f3d99..6ba9725b564 100644 --- a/src/Storages/MergeTree/MergeTreeIndexUtils.h +++ b/src/Storages/MergeTree/MergeTreeIndexUtils.h @@ -6,7 +6,7 @@ namespace DB { /** Build AST filter node for index analysis from WHERE and PREWHERE sections of select query and additional filters. - * If select query does not have WHERE or PREWHERE and additional filters are empty null is returned. + * If select query does not have WHERE and PREWHERE and additional filters are empty null is returned. */ ASTPtr buildFilterNode(const ASTPtr & select_query, ASTs additional_filters = {}); diff --git a/src/Storages/MergeTree/RPNBuilder.cpp b/src/Storages/MergeTree/RPNBuilder.cpp index 395daab76ba..d7ea68e7d64 100644 --- a/src/Storages/MergeTree/RPNBuilder.cpp +++ b/src/Storages/MergeTree/RPNBuilder.cpp @@ -129,7 +129,7 @@ std::string RPNBuilderTreeNode::getColumnNameWithModuloLegacy() const } else { - return getColumnNameWithoutAlias(*dag_node, true); + return getColumnNameWithoutAlias(*dag_node, true /*legacy*/); } } @@ -203,7 +203,7 @@ bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & outp String column_name = ast_node->getColumnName(); const auto & block_with_constants = tree_context.getBlockWithConstants(); - if (const auto * lit = ast_node->as()) + if (const auto * literal = ast_node->as()) { /// By default block_with_constants has only one column named "_dummy". /// If block contains only constants it's may not be preprocessed by @@ -212,7 +212,7 @@ bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & outp column_name = "_dummy"; /// Simple literal - output_value = lit->value; + output_value = literal->value; output_type = block_with_constants.getByName(column_name).type; /// If constant is not Null, we can assume it's type is not Nullable as well. @@ -225,9 +225,9 @@ bool RPNBuilderTreeNode::tryGetConstant(Field & output_value, DataTypePtr & outp isColumnConst(*block_with_constants.getByName(column_name).column)) { /// An expression which is dependent on constants only - const auto & expr_info = block_with_constants.getByName(column_name); - output_value = (*expr_info.column)[0]; - output_type = expr_info.type; + const auto & constant_column = block_with_constants.getByName(column_name); + output_value = (*constant_column.column)[0]; + output_type = constant_column.type; if (!output_value.isNull()) output_type = removeNullable(output_type); @@ -260,13 +260,13 @@ ConstSetPtr tryGetSetFromDAGNode(const ActionsDAG::Node * dag_node) if (!dag_node->column) return {}; - const IColumn * col = dag_node->column.get(); - if (const auto * col_const = typeid_cast(col)) - col = &col_const->getDataColumn(); + const IColumn * column = dag_node->column.get(); + if (const auto * column_const = typeid_cast(column)) + column = &column_const->getDataColumn(); - if (const auto * col_set = typeid_cast(col)) + if (const auto * column_set = typeid_cast(column)) { - auto set = col_set->getData(); + auto set = column_set->getData(); if (set->isCreated()) return set; @@ -369,6 +369,17 @@ RPNBuilderFunctionTreeNode RPNBuilderTreeNode::toFunctionNode() const return RPNBuilderFunctionTreeNode(this->dag_node, tree_context); } +std::optional RPNBuilderTreeNode::toFunctionNodeOrNull() const +{ + if (!isFunction()) + return {}; + + if (this->ast_node) + return RPNBuilderFunctionTreeNode(this->ast_node, tree_context); + else + return RPNBuilderFunctionTreeNode(this->dag_node, tree_context); +} + std::string RPNBuilderFunctionTreeNode::getFunctionName() const { if (ast_node) diff --git a/src/Storages/MergeTree/RPNBuilder.h b/src/Storages/MergeTree/RPNBuilder.h index f6fc4a70102..132d3aa44e8 100644 --- a/src/Storages/MergeTree/RPNBuilder.h +++ b/src/Storages/MergeTree/RPNBuilder.h @@ -12,7 +12,7 @@ namespace DB /** Context of RPNBuilderTree. * - * For AST tree context, precalculated block with constansts and prepared sets are required for index analysis. + * For AST tree context, precalculated block with constants and prepared sets are required for index analysis. * For DAG tree precalculated block with constants and prepared sets are not required, because constants and sets already * calculated inside COLUMN actions dag node. */ @@ -112,6 +112,9 @@ public: */ RPNBuilderFunctionTreeNode toFunctionNode() const; + /// Convert node to function node or null optional + std::optional toFunctionNodeOrNull() const; + /// Get tree context const RPNBuilderTreeContext & getTreeContext() const { @@ -136,16 +139,16 @@ protected: class RPNBuilderFunctionTreeNode : public RPNBuilderTreeNode { public: + using RPNBuilderTreeNode::RPNBuilderTreeNode; + /// Get function name std::string getFunctionName() const; /// Get function arguments size size_t getArgumentsSize() const; - /// Get argument at index + /// Get function argument at index RPNBuilderTreeNode getArgumentAt(size_t index) const; - - using RPNBuilderTreeNode::RPNBuilderTreeNode; }; /** RPN Builder build stack of reverse polish notation elements (RPNElements) required for index analysis. @@ -240,9 +243,10 @@ private: bool extractLogicalOperatorFromTree(const RPNBuilderFunctionTreeNode & function_node, RPNElement & out) { - /// Functions AND, OR, NOT. - /// Also a special function `indexHint` - works as if instead of calling a function there are just parentheses - /// (or, the same thing - calling the function `and` from one argument). + /** Functions AND, OR, NOT. + * Also a special function `indexHint` - works as if instead of calling a function there are just parentheses + * (or, the same thing - calling the function `and` from one argument). + */ auto function_name = function_node.getFunctionName(); if (function_name == "not") From b677e68c4eadcc7b59e6a6094506cfe23cc19707 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 31 Oct 2022 12:46:14 +0100 Subject: [PATCH 112/152] Update column.md --- docs/en/sql-reference/statements/alter/column.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/alter/column.md b/docs/en/sql-reference/statements/alter/column.md index f36aa1357f4..cc278465437 100644 --- a/docs/en/sql-reference/statements/alter/column.md +++ b/docs/en/sql-reference/statements/alter/column.md @@ -107,7 +107,7 @@ ALTER TABLE visits RENAME COLUMN webBrowser TO browser CLEAR COLUMN [IF EXISTS] name IN PARTITION partition_name ``` -Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to specify the partition expression](#alter-how-to-specify-part-expr). +Resets all data in a column for a specified partition. Read more about setting the partition name in the section [How to set the partition expression](partition.md#how-to-set-partition-expression). If the `IF EXISTS` clause is specified, the query won’t return an error if the column does not exist. From e43ecf9ca0f9c66661b706a2f80896d43264daa1 Mon Sep 17 00:00:00 2001 From: Sergei Trifonov Date: Mon, 31 Oct 2022 12:52:31 +0100 Subject: [PATCH 113/152] Link to proper place in docs --- docs/en/sql-reference/statements/optimize.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/optimize.md b/docs/en/sql-reference/statements/optimize.md index 680ff773992..036d3f0599a 100644 --- a/docs/en/sql-reference/statements/optimize.md +++ b/docs/en/sql-reference/statements/optimize.md @@ -22,7 +22,7 @@ The `OPTIMIZE` query is supported for [MergeTree](../../engines/table-engines/me When `OPTIMIZE` is used with the [ReplicatedMergeTree](../../engines/table-engines/mergetree-family/replication.md) family of table engines, ClickHouse creates a task for merging and waits for execution on all replicas (if the [replication_alter_partitions_sync](../../operations/settings/settings.md#replication-alter-partitions-sync) setting is set to `2`) or on current replica (if the [replication_alter_partitions_sync](../../operations/settings/settings.md#replication-alter-partitions-sync) setting is set to `1`). - If `OPTIMIZE` does not perform a merge for any reason, it does not notify the client. To enable notifications, use the [optimize_throw_if_noop](../../operations/settings/settings.md#setting-optimize_throw_if_noop) setting. -- If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](../../sql-reference/statements/alter/index.md#alter-how-to-specify-part-expr). +- If you specify a `PARTITION`, only the specified partition is optimized. [How to set partition expression](alter/partition.md#how-to-set-partition-expression). - If you specify `FINAL`, optimization is performed even when all the data is already in one part. Also merge is forced even if concurrent merges are performed. - If you specify `DEDUPLICATE`, then completely identical rows (unless by-clause is specified) will be deduplicated (all columns are compared), it makes sense only for the MergeTree engine. From 89a155d95cecf91031633f020851e1fe0b52858e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 31 Oct 2022 12:00:49 +0000 Subject: [PATCH 114/152] test for #28083 --- ...02343_analyzer_lambdas_issue_28083.reference | 2 ++ .../02343_analyzer_lambdas_issue_28083.sql | 17 +++++++++++++++++ 2 files changed, 19 insertions(+) create mode 100644 tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.reference create mode 100644 tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.reference b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.reference new file mode 100644 index 00000000000..a2ed8e55d62 --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.reference @@ -0,0 +1,2 @@ +n [('n',1)] +[('n',1)] diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql new file mode 100644 index 00000000000..3b780e1dec3 --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_28083.sql @@ -0,0 +1,17 @@ +SET allow_experimental_analyzer = 1; + +select so, + r +from + (select [('y',0),('n',1)] as cg, + if( arrayMap( x -> x.1, cg ) != ['y', 'n'], 'y', 'n') as so, + arrayFilter( x -> x.1 = so , cg) as r + ); + +select + r +from + (select [('y',0),('n',1)] as cg, + if( arrayMap( x -> x.1, cg ) != ['y', 'n'], 'y', 'n') as so, + arrayFilter( x -> x.1 = so , cg) as r + ); From 1c9fd1d0c777cc05d4650bb6900bb9f8efbd942e Mon Sep 17 00:00:00 2001 From: Denny Crane Date: Mon, 31 Oct 2022 12:01:04 +0000 Subject: [PATCH 115/152] test for #36677 --- .../02343_analyzer_lambdas_issue_36677.reference | 2 ++ .../02343_analyzer_lambdas_issue_36677.sql | 14 ++++++++++++++ 2 files changed, 16 insertions(+) create mode 100644 tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.reference create mode 100644 tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.reference b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.reference new file mode 100644 index 00000000000..bec52aa46b6 --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.reference @@ -0,0 +1,2 @@ +2.1999999999999997 289.99999999999994 [1,2,3,4] [0.1,0.2,0.1,0.2] +2.1999999999999997 289.99999999999994 [1,2,3,4] [0.1,0.2,0.1,0.2] diff --git a/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql new file mode 100644 index 00000000000..b07f3f33ac3 --- /dev/null +++ b/tests/queries/0_stateless/02343_analyzer_lambdas_issue_36677.sql @@ -0,0 +1,14 @@ +SET allow_experimental_analyzer = 1; + +SELECT + arraySum(x -> ((x.1) / ((x.2) * (x.2))), arrayZip(mag, magerr)) / arraySum(x -> (1. / (x * x)), magerr) AS weightedmeanmag, + arraySum(x -> ((((x.1) - weightedmeanmag) * ((x.1) - weightedmeanmag)) / ((x.2) * (x.2))), arrayZip(mag, magerr)) AS chi2, + [1, 2, 3, 4] AS mag, + [0.1, 0.2, 0.1, 0.2] AS magerr; + +SELECT + arraySum(x -> ((x.1) / ((x.2) * (x.2))), arrayZip(mag, magerr)) / arraySum(x -> (1. / (x * x)), magerr) AS weightedmeanmag, + arraySum(x -> ((((x.1) - weightedmeanmag) * ((x.1) - weightedmeanmag)) / ((x.2) * (x.2))), arrayZip(mag, magerr)) AS chi2, + [1, 2, 3, 4] AS mag, + [0.1, 0.2, 0.1, 0.2] AS magerr +WHERE isFinite(chi2) From b2b9479afa810ce7e69f820ecc4a3e0b5db4e6a7 Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 Oct 2022 16:15:35 +0300 Subject: [PATCH 116/152] Update src/Storages/MergeTree/MergeTreeData.h Co-authored-by: Sergei Trifonov --- src/Storages/MergeTree/MergeTreeData.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index 40eaa679845..cfad11b8d36 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -591,7 +591,7 @@ public: bool was_active; public: - PartToRemoveFromZooKeeper(DataPartPtr && part_, bool was_active_ = true) + explicit PartToRemoveFromZooKeeper(DataPartPtr && part_, bool was_active_ = true) : part(std::move(part_)), was_active(was_active_) { } From 599ccb99396a84151b280bdf809ff54d04e8c38a Mon Sep 17 00:00:00 2001 From: Alexander Tokmakov Date: Mon, 31 Oct 2022 16:18:17 +0300 Subject: [PATCH 117/152] Update MergeTreeData.h --- src/Storages/MergeTree/MergeTreeData.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Storages/MergeTree/MergeTreeData.h b/src/Storages/MergeTree/MergeTreeData.h index cfad11b8d36..99ba6991e43 100644 --- a/src/Storages/MergeTree/MergeTreeData.h +++ b/src/Storages/MergeTree/MergeTreeData.h @@ -596,7 +596,7 @@ public: { } - /// It's s to get name of any part + /// It's safe to get name of any part const String & getPartName() const { return part->name; } DataPartPtr getPartIfItWasActive() const From 399d024c2c58f928c14058f887dd2f635eb06aca Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?M=C3=A1rcio=20Martins?= Date: Mon, 31 Oct 2022 15:02:08 +0100 Subject: [PATCH 118/152] Fix array index argument type check for JSONExtract* functions --- src/Functions/FunctionsJSON.cpp | 2 +- tests/queries/0_stateless/00918_json_functions.reference | 1 + tests/queries/0_stateless/00918_json_functions.sql | 3 +++ 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/src/Functions/FunctionsJSON.cpp b/src/Functions/FunctionsJSON.cpp index 7c221209071..c856419c9e8 100644 --- a/src/Functions/FunctionsJSON.cpp +++ b/src/Functions/FunctionsJSON.cpp @@ -191,7 +191,7 @@ private: for (const auto i : collections::range(first_index_argument, first_index_argument + num_index_arguments)) { const auto & column = columns[i]; - if (!isString(column.type) && !isInteger(column.type)) + if (!isString(column.type) && !isNativeInteger(column.type)) throw Exception{"The argument " + std::to_string(i + 1) + " of function " + String(function_name) + " should be a string specifying key or an integer specifying index, illegal type: " + column.type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; diff --git a/tests/queries/0_stateless/00918_json_functions.reference b/tests/queries/0_stateless/00918_json_functions.reference index 7d925c1236d..fc03457c677 100644 --- a/tests/queries/0_stateless/00918_json_functions.reference +++ b/tests/queries/0_stateless/00918_json_functions.reference @@ -253,3 +253,4 @@ e u v --show error: type should be const string +--show error: index type should be integer diff --git a/tests/queries/0_stateless/00918_json_functions.sql b/tests/queries/0_stateless/00918_json_functions.sql index d614d507dda..3105994ce20 100644 --- a/tests/queries/0_stateless/00918_json_functions.sql +++ b/tests/queries/0_stateless/00918_json_functions.sql @@ -280,3 +280,6 @@ SELECT JSONExtractString(json, 's') FROM (SELECT arrayJoin(['{"s":"u"}', '{"s":" SELECT '--show error: type should be const string'; SELECT JSONExtractKeysAndValues([], JSONLength('^?V{LSwp')); -- { serverError 44 } WITH '{"i": 1, "f": 1.2}' AS json SELECT JSONExtract(json, 'i', JSONType(json, 'i')); -- { serverError 44 } + +SELECT '--show error: index type should be integer'; +SELECT JSONExtract('[]', JSONExtract('0', 'UInt256'), 'UInt256'); -- { serverError 43 } From 3742fd848a11642325bc345ee84f4d8574b3887b Mon Sep 17 00:00:00 2001 From: Alexander Gololobov <440544+davenger@users.noreply.github.com> Date: Mon, 31 Oct 2022 15:31:14 +0100 Subject: [PATCH 119/152] Tests for upperUTF8() and for 32 and 64 byte boundaries --- .../02071_lower_upper_utf8_row_overlaps.reference | 15 +++++++++++++++ .../02071_lower_upper_utf8_row_overlaps.sql | 8 ++++++++ 2 files changed, 23 insertions(+) diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference index 190c7895011..a3bac432482 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.reference @@ -13,3 +13,18 @@ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8 -- https://github.com/ClickHouse/ClickHouse/issues/42756 SELECT lowerUTF8('КВ АМ И СЖ'); кв ам и сж +SELECT upperUTF8('кв ам и сж'); +КВ АМ И СЖ +SELECT lowerUTF8('КВ АМ И СЖ КВ АМ И СЖ'); +кв ам и сж кв ам и сж +SELECT upperUTF8('кв ам и сж кв ам и сж'); +КВ АМ И СЖ КВ АМ И СЖ +-- Test at 32 and 64 byte boundaries +SELECT lowerUTF8(repeat('0', 16) || 'КВ АМ И СЖ'); +0000000000000000кв ам и сж +SELECT upperUTF8(repeat('0', 16) || 'кв ам и сж'); +0000000000000000КВ АМ И СЖ +SELECT lowerUTF8(repeat('0', 48) || 'КВ АМ И СЖ'); +000000000000000000000000000000000000000000000000кв ам и сж +SELECT upperUTF8(repeat('0', 48) || 'кв ам и сж'); +000000000000000000000000000000000000000000000000КВ АМ И СЖ diff --git a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql index 161d8fb512e..8ca0a3f5f75 100644 --- a/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql +++ b/tests/queries/0_stateless/02071_lower_upper_utf8_row_overlaps.sql @@ -12,3 +12,11 @@ select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8 -- NOTE: regression test for introduced bug -- https://github.com/ClickHouse/ClickHouse/issues/42756 SELECT lowerUTF8('КВ АМ И СЖ'); +SELECT upperUTF8('кв ам и сж'); +SELECT lowerUTF8('КВ АМ И СЖ КВ АМ И СЖ'); +SELECT upperUTF8('кв ам и сж кв ам и сж'); +-- Test at 32 and 64 byte boundaries +SELECT lowerUTF8(repeat('0', 16) || 'КВ АМ И СЖ'); +SELECT upperUTF8(repeat('0', 16) || 'кв ам и сж'); +SELECT lowerUTF8(repeat('0', 48) || 'КВ АМ И СЖ'); +SELECT upperUTF8(repeat('0', 48) || 'кв ам и сж'); From 9fb6f5228675add4662c52cacb70d38dd571776d Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Mon, 31 Oct 2022 11:02:08 -0400 Subject: [PATCH 120/152] Fix Missing Quotes - Sonar Nightly Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index e6da4df7200..5a208807c81 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -149,7 +149,7 @@ jobs: env: BUILD_WRAPPER_DOWNLOAD_URL: ${{ env.SONAR_SERVER_URL }}/static/cpp/build-wrapper-linux-x86.zip run: | - curl -sSLo "$HOME/.sonar/build-wrapper-linux-x86.zip ${{ env.BUILD_WRAPPER_DOWNLOAD_URL }}" + curl -sSLo "$HOME/.sonar/build-wrapper-linux-x86.zip" "${{ env.BUILD_WRAPPER_DOWNLOAD_URL }}" unzip -o "$HOME/.sonar/build-wrapper-linux-x86.zip" -d "$HOME/.sonar/" echo "$HOME/.sonar/build-wrapper-linux-x86" >> "$GITHUB_PATH" - name: Set Up Build Tools From f8091424629714ff43094e224f6cf95f79b3f715 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Mon, 31 Oct 2022 17:47:02 +0100 Subject: [PATCH 121/152] Add exclusions from the Snyk scan --- .snyk | 4 ++++ 1 file changed, 4 insertions(+) create mode 100644 .snyk diff --git a/.snyk b/.snyk new file mode 100644 index 00000000000..7acc6b9fbf5 --- /dev/null +++ b/.snyk @@ -0,0 +1,4 @@ +# Snyk (https://snyk.io) policy file +exclude: + global: + - tests/** From 440cc51a7e7aa6df4bde96c895965048a710f3c1 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Mon, 31 Oct 2022 14:11:52 -0400 Subject: [PATCH 122/152] Fix Missing Env Vars - Sonar Nightly Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 5a208807c81..904c0dffe57 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -128,6 +128,8 @@ jobs: SONAR_SCANNER_VERSION: 4.7.0.2747 SONAR_SERVER_URL: "https://sonarcloud.io" BUILD_WRAPPER_OUT_DIR: build_wrapper_output_directory # Directory where build-wrapper output will be placed + CC: clang-15 + CXX: clang++-15 steps: - uses: actions/checkout@v2 with: From d18d08bcc78431977abe735e3bc6793e6e2308c2 Mon Sep 17 00:00:00 2001 From: Zhiguo Zhou Date: Tue, 1 Nov 2022 04:36:15 +0800 Subject: [PATCH 123/152] Remove short-circuit evaluation in AssociativeApplierImpl::apply (#42214) The short-circuit evaluation was implemented when applying the saturable operators (and, or) on a vector of ColumnUInt8. However, its control flow would be compiled as a series of conditional branch instructions which are hard to predict by the hardware and at the same time hinder the vectorization optimization by the compiler. This commit removes the short-circuit and evaluates the whole expression. --- src/Functions/FunctionsLogical.cpp | 5 +---- 1 file changed, 1 insertion(+), 4 deletions(-) diff --git a/src/Functions/FunctionsLogical.cpp b/src/Functions/FunctionsLogical.cpp index 2ac7688737f..7e52c55e5b0 100644 --- a/src/Functions/FunctionsLogical.cpp +++ b/src/Functions/FunctionsLogical.cpp @@ -168,10 +168,7 @@ public: inline ResultValueType apply(const size_t i) const { const auto a = !!vec[i]; - if constexpr (Op::isSaturable()) - return Op::isSaturatedValue(a) ? a : Op::apply(a, next.apply(i)); - else - return Op::apply(a, next.apply(i)); + return Op::apply(a, next.apply(i)); } private: From 439ddc2bf7d07122cd15d8054b7c9400ad29578e Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Mon, 31 Oct 2022 16:53:58 -0400 Subject: [PATCH 124/152] exclude java Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 904c0dffe57..94fe404ea97 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -175,4 +175,5 @@ jobs: --define sonar.host.url="${{ env.SONAR_SERVER_URL }}" \ --define sonar.cfamily.build-wrapper-output="${{ env.BUILD_WRAPPER_OUT_DIR }}" \ --define sonar.projectKey="ClickHouse_ClickHouse" \ - --define sonar.organization="clickhouse-java" + --define sonar.organization="clickhouse-java" \ + --define sonar.exclusions="**/*.java" From baf0e92e60185ade936ec0caa4ff0cd2960d6751 Mon Sep 17 00:00:00 2001 From: Julio Jimenez Date: Mon, 31 Oct 2022 17:58:43 -0400 Subject: [PATCH 125/152] exclude more stuff Signed-off-by: Julio Jimenez --- .github/workflows/nightly.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/nightly.yml b/.github/workflows/nightly.yml index 94fe404ea97..612bb1f8f9b 100644 --- a/.github/workflows/nightly.yml +++ b/.github/workflows/nightly.yml @@ -176,4 +176,4 @@ jobs: --define sonar.cfamily.build-wrapper-output="${{ env.BUILD_WRAPPER_OUT_DIR }}" \ --define sonar.projectKey="ClickHouse_ClickHouse" \ --define sonar.organization="clickhouse-java" \ - --define sonar.exclusions="**/*.java" + --define sonar.exclusions="**/*.java,**/*.ts,**/*.js,**/*.css,**/*.sql" From 70eaf69df176a3cc2dee84f88fbb22fc1185478b Mon Sep 17 00:00:00 2001 From: Alexander Yakovlev <33040934+AlexJameson@users.noreply.github.com> Date: Tue, 1 Nov 2022 02:57:55 +0300 Subject: [PATCH 126/152] Fix a typo in table-engines/integrations/s3.md --- docs/en/engines/table-engines/integrations/s3.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/engines/table-engines/integrations/s3.md b/docs/en/engines/table-engines/integrations/s3.md index 986a29b8307..db983ab9c68 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 - `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`. - `region` — Specifies S3 region name. Optional. - `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. +- `header` — Adds specified HTTP header to a request to given endpoint. Optional, can be specified multiple times. - `server_side_encryption_customer_key_base64` — If specified, required headers for accessing S3 objects with SSE-C encryption will be set. Optional. - `max_single_read_retries` — The maximum number of attempts during single read. Default value is `4`. Optional. From 8b849d4b374db3527d3857b9e88908a5a32179a7 Mon Sep 17 00:00:00 2001 From: 94rain <21193371+94rain@users.noreply.github.com> Date: Mon, 31 Oct 2022 20:45:01 +0000 Subject: [PATCH 127/152] Fix broken links in system tables docs Signed-off-by: 94rain <21193371+94rain@users.noreply.github.com> --- docs/en/operations/system-tables/information_schema.md | 2 +- docs/en/operations/system-tables/replicated_fetches.md | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index a573491282a..811e0dc3c3f 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -178,7 +178,7 @@ Columns: - `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` query for view. - `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, no checking. - `is_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the view is not updated. -- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — Shows whether the created view is [materialized](../../sql-reference/statements/create/view/#materialized). Possible values: +- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — Shows whether the created view is [materialized](../../sql-reference/statements/create/view.md#materialized-view). Possible values: - `NO` — The created view is not materialized. - `YES` — The created view is materialized. - `is_trigger_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the trigger is not updated. diff --git a/docs/en/operations/system-tables/replicated_fetches.md b/docs/en/operations/system-tables/replicated_fetches.md index 3536bbaff4d..734da63a600 100644 --- a/docs/en/operations/system-tables/replicated_fetches.md +++ b/docs/en/operations/system-tables/replicated_fetches.md @@ -68,6 +68,6 @@ thread_id: 54 **See Also** -- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system/#query-language-system-replicated) +- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system.md#managing-replicatedmergetree-tables) -[Original article](https://clickhouse.com/docs/en/operations/system_tables/replicated_fetches) +[Original article](https://clickhouse.com/docs/en/operations/system-tables/replicated_fetches/) From 7234600ec31f511146bd795df91d1936c5a42200 Mon Sep 17 00:00:00 2001 From: shuyang <21193371+94rain@users.noreply.github.com> Date: Mon, 31 Oct 2022 19:14:58 -0500 Subject: [PATCH 128/152] Apply suggestions from code review Co-authored-by: Dan Roscigno --- docs/en/operations/system-tables/information_schema.md | 2 +- docs/en/operations/system-tables/replicated_fetches.md | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/operations/system-tables/information_schema.md b/docs/en/operations/system-tables/information_schema.md index 811e0dc3c3f..a8e516f02a3 100644 --- a/docs/en/operations/system-tables/information_schema.md +++ b/docs/en/operations/system-tables/information_schema.md @@ -178,7 +178,7 @@ Columns: - `view_definition` ([String](../../sql-reference/data-types/string.md)) — `SELECT` query for view. - `check_option` ([String](../../sql-reference/data-types/string.md)) — `NONE`, no checking. - `is_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the view is not updated. -- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — Shows whether the created view is [materialized](../../sql-reference/statements/create/view.md#materialized-view). Possible values: +- `is_insertable_into` ([Enum8](../../sql-reference/data-types/enum.md)) — Shows whether the created view is [materialized](../../sql-reference/statements/create/view.md/#materialized-view). Possible values: - `NO` — The created view is not materialized. - `YES` — The created view is materialized. - `is_trigger_updatable` ([Enum8](../../sql-reference/data-types/enum.md)) — `NO`, the trigger is not updated. diff --git a/docs/en/operations/system-tables/replicated_fetches.md b/docs/en/operations/system-tables/replicated_fetches.md index 734da63a600..74888fd2f13 100644 --- a/docs/en/operations/system-tables/replicated_fetches.md +++ b/docs/en/operations/system-tables/replicated_fetches.md @@ -68,6 +68,5 @@ thread_id: 54 **See Also** -- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system.md#managing-replicatedmergetree-tables) +- [Managing ReplicatedMergeTree Tables](../../sql-reference/statements/system.md/#managing-replicatedmergetree-tables) -[Original article](https://clickhouse.com/docs/en/operations/system-tables/replicated_fetches/) From 2b0cde9ef9f9d77eac3b8045f1683ed693a1fb01 Mon Sep 17 00:00:00 2001 From: GoGoWen2021 Date: Tue, 1 Nov 2022 17:46:44 +0800 Subject: [PATCH 129/152] fix typo --- src/Columns/IColumn.h | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/src/Columns/IColumn.h b/src/Columns/IColumn.h index 19f3dea4f82..eda4ae62b03 100644 --- a/src/Columns/IColumn.h +++ b/src/Columns/IColumn.h @@ -85,8 +85,8 @@ public: [[nodiscard]] virtual MutablePtr cloneEmpty() const { return cloneResized(0); } /// Creates column with the same type and specified size. - /// If size is less current size, then data is cut. - /// If size is greater, than default values are appended. + /// If size is less than current size, then data is cut. + /// If size is greater, then default values are appended. [[nodiscard]] virtual MutablePtr cloneResized(size_t /*size*/) const { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Cannot cloneResized() column {}", getName()); } /// Returns number of values in column. From 29318a1e133a29ff99602f53b3376bd0fa296465 Mon Sep 17 00:00:00 2001 From: Derek Chia Date: Tue, 1 Nov 2022 18:03:17 +0800 Subject: [PATCH 130/152] Update nyc-taxi.md - Add enum 'UNK' = 5 to `payment_type` to avoid `Unknown element 'UNK' for enum` error when loading trips_*.gz - Change url table function to s3 table function as described in the earlier paragraph --- docs/en/getting-started/example-datasets/nyc-taxi.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/getting-started/example-datasets/nyc-taxi.md b/docs/en/getting-started/example-datasets/nyc-taxi.md index e24fb4b01a7..69098f63037 100644 --- a/docs/en/getting-started/example-datasets/nyc-taxi.md +++ b/docs/en/getting-started/example-datasets/nyc-taxi.md @@ -33,7 +33,7 @@ CREATE TABLE trips ( tip_amount Float32, tolls_amount Float32, total_amount Float32, - payment_type Enum('CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4), + payment_type Enum('CSH' = 1, 'CRE' = 2, 'NOC' = 3, 'DIS' = 4, 'UNK' = 5), pickup_ntaname LowCardinality(String), dropoff_ntaname LowCardinality(String) ) @@ -63,7 +63,7 @@ SELECT payment_type, pickup_ntaname, dropoff_ntaname -FROM url( +FROM s3( 'https://datasets-documentation.s3.eu-west-3.amazonaws.com/nyc-taxi/trips_{0..2}.gz', 'TabSeparatedWithNames' ) From e0735e1e67da3c74dd43dabd7147cce2d85d49ed Mon Sep 17 00:00:00 2001 From: vdimir Date: Wed, 26 Oct 2022 14:02:04 +0000 Subject: [PATCH 131/152] Debug TestBugfixCheck --- tests/ci/functional_test_check.py | 13 ++++++++++--- .../0_stateless/01744_fuse_sum_count_aggregate.sql | 1 + 2 files changed, 11 insertions(+), 3 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 388f93f34ec..b4601f766f8 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -7,6 +7,7 @@ import os import subprocess import sys import atexit +import time from github import Github @@ -221,12 +222,18 @@ if __name__ == "__main__": if args.post_commit_status == "file": post_commit_status_to_file( os.path.join(temp_path, "post_commit_status.tsv"), - "Skipped (no pr-bugfix)", + f"Skipped (no pr-bugfix in {pr_info.labels})", "success", "null", ) - logging.info("Skipping '%s' (no pr-bugfix)", check_name) - sys.exit(0) + logging.info("Skipping '%s' (no pr-bugfix in %s)", check_name, pr_info.labels) + + # >>>> + time.sleep(10) + pr_info = PRInfo(need_changed_files=run_changed_tests) + logging.info("pr_info '%s' (%s)", pr_info, pr_info.labels) + sys.exit(1) + # <<<< if "RUN_BY_HASH_NUM" in os.environ: run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM")) diff --git a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql b/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql index 5b6ed440ba4..375662eb405 100644 --- a/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql +++ b/tests/queries/0_stateless/01744_fuse_sum_count_aggregate.sql @@ -10,4 +10,5 @@ EXPLAIN SYNTAX SELECT sum(a), sum(b), count(b) from fuse_tbl; SELECT '---------NOT trigger fuse--------'; SELECT sum(a), avg(b) from fuse_tbl; EXPLAIN SYNTAX SELECT sum(a), avg(b) from fuse_tbl; + DROP TABLE fuse_tbl; From 9f5dcd14af323c9c0063cc585f6186556559d691 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 10:49:02 +0000 Subject: [PATCH 132/152] Request labels from gh api for validate bugfix check --- tests/ci/functional_test_check.py | 11 +++-------- tests/ci/integration_test_check.py | 8 +++++--- 2 files changed, 8 insertions(+), 11 deletions(-) diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index b4601f766f8..9b40d7facbe 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -211,7 +211,8 @@ if __name__ == "__main__": run_changed_tests = flaky_check or validate_bugix_check gh = Github(get_best_robot_token(), per_page=100) - pr_info = PRInfo(need_changed_files=run_changed_tests) + # For validate_bugix_check we need up to date information about labels, so pr_event_from_api is used + pr_info = PRInfo(need_changed_files=run_changed_tests, pr_event_from_api=validate_bugix_check) atexit.register(update_mergeable_check, gh, pr_info, check_name) @@ -227,13 +228,7 @@ if __name__ == "__main__": "null", ) logging.info("Skipping '%s' (no pr-bugfix in %s)", check_name, pr_info.labels) - - # >>>> - time.sleep(10) - pr_info = PRInfo(need_changed_files=run_changed_tests) - logging.info("pr_info '%s' (%s)", pr_info, pr_info.labels) - sys.exit(1) - # <<<< + sys.exit(0) if "RUN_BY_HASH_NUM" in os.environ: run_by_hash_num = int(os.getenv("RUN_BY_HASH_NUM")) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 3709a7271d7..2cdbd553144 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -167,17 +167,19 @@ if __name__ == "__main__": os.makedirs(temp_path) is_flaky_check = "flaky" in check_name - pr_info = PRInfo(need_changed_files=is_flaky_check or validate_bugix_check) + + # For validate_bugix_check we need up to date information about labels, so pr_event_from_api is used + pr_info = PRInfo(need_changed_files=is_flaky_check or validate_bugix_check, pr_event_from_api=validate_bugix_check) if validate_bugix_check and "pr-bugfix" not in pr_info.labels: if args.post_commit_status == "file": post_commit_status_to_file( os.path.join(temp_path, "post_commit_status.tsv"), - "Skipped (no pr-bugfix)", + f"Skipped (no pr-bugfix in {pr_info.labels})", "success", "null", ) - logging.info("Skipping '%s' (no pr-bugfix)", check_name) + logging.info("Skipping '%s' (no pr-bugfix in '%s')", check_name, pr_info.labels) sys.exit(0) gh = Github(get_best_robot_token(), per_page=100) From c18838715da399605df9de0a7d59561f89281dcf Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 11:32:42 +0000 Subject: [PATCH 133/152] Post status for test bugfix check --- tests/ci/bugfix_validate_check.py | 52 ++++++++++++++++++++++++------ tests/ci/functional_test_check.py | 6 ++-- tests/ci/integration_test_check.py | 7 ++-- 3 files changed, 51 insertions(+), 14 deletions(-) diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index 4e6001aaa74..6ab47392dca 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -7,10 +7,18 @@ import os import sys +from github import Github + +from s3_helper import S3Helper +from get_robot_token import get_best_robot_token +from pr_info import PRInfo +from upload_result_helper import upload_results +from commit_status_helper import post_commit_status + + def parse_args(): parser = argparse.ArgumentParser() - parser.add_argument("report1") - parser.add_argument("report2") + parser.add_argument("status", nargs="+", help="Path to status file") return parser.parse_args() @@ -27,19 +35,43 @@ def post_commit_status_from_file(file_path): def process_results(file_path): + test_results = [] state, report_url, description = post_commit_status_from_file(file_path) prefix = os.path.basename(os.path.dirname(file_path)) - print( - f"::notice:: bugfix check: {prefix} - {state}: {description} Report url: {report_url}" - ) - return state == "success" + + test_results.append([f"{prefix}: {description}", state, report_url]) + return state == "success", test_results def main(args): - is_ok = False - is_ok = process_results(args.report1) or is_ok - is_ok = process_results(args.report2) or is_ok - sys.exit(0 if is_ok else 1) + all_ok = False + all_results = [] + for status_path in args.status: + is_ok, test_results = process_results(status_path) + all_ok = all_ok or is_ok + all_results.extend(test_results) + + check_name_with_group = "Bugfix validate check" + + pr_info = PRInfo() + report_url = upload_results( + S3Helper(), + pr_info.number, + pr_info.sha, + all_results, + [], + check_name_with_group, + ) + + gh = Github(get_best_robot_token(), per_page=100) + post_commit_status( + gh, + pr_info.sha, + check_name_with_group, + "", + "success" if is_ok else "error", + report_url, + ) if __name__ == "__main__": diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 9b40d7facbe..1d215920cb5 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -212,7 +212,9 @@ if __name__ == "__main__": gh = Github(get_best_robot_token(), per_page=100) # For validate_bugix_check we need up to date information about labels, so pr_event_from_api is used - pr_info = PRInfo(need_changed_files=run_changed_tests, pr_event_from_api=validate_bugix_check) + pr_info = PRInfo( + need_changed_files=run_changed_tests, pr_event_from_api=validate_bugix_check + ) atexit.register(update_mergeable_check, gh, pr_info, check_name) @@ -322,7 +324,7 @@ if __name__ == "__main__": state, description, test_results, additional_logs = process_results( result_path, server_log_path ) - state = override_status(state, check_name, validate_bugix_check) + state = override_status(state, check_name, invert=validate_bugix_check) ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, check_name, test_results) diff --git a/tests/ci/integration_test_check.py b/tests/ci/integration_test_check.py index 2cdbd553144..cba428cbcf5 100644 --- a/tests/ci/integration_test_check.py +++ b/tests/ci/integration_test_check.py @@ -169,7 +169,10 @@ if __name__ == "__main__": is_flaky_check = "flaky" in check_name # For validate_bugix_check we need up to date information about labels, so pr_event_from_api is used - pr_info = PRInfo(need_changed_files=is_flaky_check or validate_bugix_check, pr_event_from_api=validate_bugix_check) + pr_info = PRInfo( + need_changed_files=is_flaky_check or validate_bugix_check, + pr_event_from_api=validate_bugix_check, + ) if validate_bugix_check and "pr-bugfix" not in pr_info.labels: if args.post_commit_status == "file": @@ -246,7 +249,7 @@ if __name__ == "__main__": subprocess.check_call(f"sudo chown -R ubuntu:ubuntu {temp_path}", shell=True) state, description, test_results, additional_logs = process_results(result_path) - state = override_status(state, check_name, validate_bugix_check) + state = override_status(state, check_name, invert=validate_bugix_check) ch_helper = ClickHouseHelper() mark_flaky_tests(ch_helper, check_name, test_results) From 11a933b0465bedea333ff48dfb90078869d5822f Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 11:34:45 +0000 Subject: [PATCH 134/152] broke style in src/Interpreters/InterpreterSelectQuery.cpp --- src/Interpreters/InterpreterSelectQuery.cpp | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index d8ac263e3d1..225dfd1f607 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -138,8 +138,7 @@ FilterDAGInfoPtr generateFilterActions( expr_list->children.push_back(row_policy_filter->clone()); /// Keep columns that are required after the filter actions. - for (const auto & column_str : prerequisite_columns) - { + for (const auto & column_str : prerequisite_columns) { ParserExpression expr_parser; expr_list->children.push_back(parseQuery(expr_parser, column_str, 0, context->getSettingsRef().max_parser_depth)); } From 1b32c7d21277887b3475ef15a53c15a4c13c16a8 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 13:06:37 +0000 Subject: [PATCH 135/152] Update bugfix_validate_check.py --- tests/ci/bugfix_validate_check.py | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index 6ab47392dca..975c3eba2bb 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -38,9 +38,16 @@ def process_results(file_path): test_results = [] state, report_url, description = post_commit_status_from_file(file_path) prefix = os.path.basename(os.path.dirname(file_path)) + is_ok = state == "success" - test_results.append([f"{prefix}: {description}", state, report_url]) - return state == "success", test_results + test_results.append( + [ + f"{prefix}: {description}", + "Bug reproduced" if is_ok else "Bug is not reproduced", + report_url, + ] + ) + return is_ok, test_results def main(args): @@ -68,7 +75,7 @@ def main(args): gh, pr_info.sha, check_name_with_group, - "", + "" if is_ok else "Changed tests doesn't reproduce the bug", "success" if is_ok else "error", report_url, ) From bc78810f2c73ef91d178c0814805f7ec222c4769 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 15:08:05 +0000 Subject: [PATCH 136/152] Add 02474_dummy.sql --- tests/queries/0_stateless/02474_dummy.reference | 1 + tests/queries/0_stateless/02474_dummy.sql | 2 ++ 2 files changed, 3 insertions(+) create mode 100644 tests/queries/0_stateless/02474_dummy.reference create mode 100644 tests/queries/0_stateless/02474_dummy.sql diff --git a/tests/queries/0_stateless/02474_dummy.reference b/tests/queries/0_stateless/02474_dummy.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/tests/queries/0_stateless/02474_dummy.reference @@ -0,0 +1 @@ +1 diff --git a/tests/queries/0_stateless/02474_dummy.sql b/tests/queries/0_stateless/02474_dummy.sql new file mode 100644 index 00000000000..bf2485aeb69 --- /dev/null +++ b/tests/queries/0_stateless/02474_dummy.sql @@ -0,0 +1,2 @@ +-- Tags: no-fasttest +SELECT 0; From 63b63da214340704269ee881d8129e09725b31af Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 15:11:16 +0000 Subject: [PATCH 137/152] Update bugfix_validate_check.py --- tests/ci/bugfix_validate_check.py | 19 ++++++++++++------- 1 file changed, 12 insertions(+), 7 deletions(-) diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index 975c3eba2bb..97294152941 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -34,7 +34,7 @@ def post_commit_status_from_file(file_path): return res[0] -def process_results(file_path): +def process_result(file_path): test_results = [] state, report_url, description = post_commit_status_from_file(file_path) prefix = os.path.basename(os.path.dirname(file_path)) @@ -50,22 +50,27 @@ def process_results(file_path): return is_ok, test_results -def main(args): - all_ok = False +def process_all_results(file_paths): + all_ok = True all_results = [] - for status_path in args.status: - is_ok, test_results = process_results(status_path) - all_ok = all_ok or is_ok + for status_path in file_paths: + is_ok, test_results = process_result(status_path) + all_ok = all_ok and is_ok all_results.extend(test_results) + return all_ok, all_results + +def main(args): check_name_with_group = "Bugfix validate check" + is_ok, test_results = process_all_results(args.status) + pr_info = PRInfo() report_url = upload_results( S3Helper(), pr_info.number, pr_info.sha, - all_results, + test_results, [], check_name_with_group, ) From e83d61749a7748b347f05cd4c99c9fd69d59c81e Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 16:51:11 +0000 Subject: [PATCH 138/152] update test bugfix report --- tests/ci/bugfix_validate_check.py | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index 97294152941..c9d3886b73b 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -40,24 +40,21 @@ def process_result(file_path): prefix = os.path.basename(os.path.dirname(file_path)) is_ok = state == "success" - test_results.append( - [ - f"{prefix}: {description}", - "Bug reproduced" if is_ok else "Bug is not reproduced", - report_url, - ] - ) + status = f'OK: Bug reproduced (Report' + if not is_ok: + status = f'Bug is not reproduced (Report)' + test_results.append([f"{prefix}: {description}", status]) return is_ok, test_results def process_all_results(file_paths): - all_ok = True + any_ok = False all_results = [] for status_path in file_paths: is_ok, test_results = process_result(status_path) - all_ok = all_ok and is_ok + any_ok = any_ok or is_ok all_results.extend(test_results) - return all_ok, all_results + return any_ok, all_results def main(args): From e7e223ffc635d3f2ac4eec590237a0c486108d09 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 16:51:34 +0000 Subject: [PATCH 139/152] Revert "Add 02474_dummy.sql" This reverts commit c2a8176716746364e559d61d8475407610cb3d3f. --- tests/queries/0_stateless/02474_dummy.reference | 1 - tests/queries/0_stateless/02474_dummy.sql | 2 -- 2 files changed, 3 deletions(-) delete mode 100644 tests/queries/0_stateless/02474_dummy.reference delete mode 100644 tests/queries/0_stateless/02474_dummy.sql diff --git a/tests/queries/0_stateless/02474_dummy.reference b/tests/queries/0_stateless/02474_dummy.reference deleted file mode 100644 index d00491fd7e5..00000000000 --- a/tests/queries/0_stateless/02474_dummy.reference +++ /dev/null @@ -1 +0,0 @@ -1 diff --git a/tests/queries/0_stateless/02474_dummy.sql b/tests/queries/0_stateless/02474_dummy.sql deleted file mode 100644 index bf2485aeb69..00000000000 --- a/tests/queries/0_stateless/02474_dummy.sql +++ /dev/null @@ -1,2 +0,0 @@ --- Tags: no-fasttest -SELECT 0; From 16032fd0b2b120343c0f229984c5beec43fcd843 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 16:51:43 +0000 Subject: [PATCH 140/152] Revert "broke style in src/Interpreters/InterpreterSelectQuery.cpp" This reverts commit 60e625918cd646c8b143d957709c56a0380fa6ab. --- src/Interpreters/InterpreterSelectQuery.cpp | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/src/Interpreters/InterpreterSelectQuery.cpp b/src/Interpreters/InterpreterSelectQuery.cpp index 225dfd1f607..d8ac263e3d1 100644 --- a/src/Interpreters/InterpreterSelectQuery.cpp +++ b/src/Interpreters/InterpreterSelectQuery.cpp @@ -138,7 +138,8 @@ FilterDAGInfoPtr generateFilterActions( expr_list->children.push_back(row_policy_filter->clone()); /// Keep columns that are required after the filter actions. - for (const auto & column_str : prerequisite_columns) { + for (const auto & column_str : prerequisite_columns) + { ParserExpression expr_parser; expr_list->children.push_back(parseQuery(expr_parser, column_str, 0, context->getSettingsRef().max_parser_depth)); } From fba8686db1eaeab90905004a7643fc3a674ba5b5 Mon Sep 17 00:00:00 2001 From: vdimir Date: Thu, 27 Oct 2022 16:52:16 +0000 Subject: [PATCH 141/152] CheckLabels, StyleCheck for TestsBugfixCheck --- .github/workflows/pull_request.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/pull_request.yml b/.github/workflows/pull_request.yml index 3951f99b16b..23245c16374 100644 --- a/.github/workflows/pull_request.yml +++ b/.github/workflows/pull_request.yml @@ -2023,6 +2023,7 @@ jobs: docker ps --all --quiet | xargs --no-run-if-empty docker rm -f ||: sudo rm -fr "$TEMP_PATH" TestsBugfixCheck: + needs: [CheckLabels, StyleCheck] runs-on: [self-hosted, stress-tester] steps: - name: Set envs From 2060c2d9ac6133017a4477b9cd0d61183d27aa18 Mon Sep 17 00:00:00 2001 From: vdimir Date: Fri, 28 Oct 2022 11:23:44 +0000 Subject: [PATCH 142/152] Fix style --- tests/ci/bugfix_validate_check.py | 2 -- tests/ci/functional_test_check.py | 1 - 2 files changed, 3 deletions(-) diff --git a/tests/ci/bugfix_validate_check.py b/tests/ci/bugfix_validate_check.py index c9d3886b73b..9ecada816b6 100644 --- a/tests/ci/bugfix_validate_check.py +++ b/tests/ci/bugfix_validate_check.py @@ -4,8 +4,6 @@ import argparse import csv import itertools import os -import sys - from github import Github diff --git a/tests/ci/functional_test_check.py b/tests/ci/functional_test_check.py index 1d215920cb5..f7d3288c316 100644 --- a/tests/ci/functional_test_check.py +++ b/tests/ci/functional_test_check.py @@ -7,7 +7,6 @@ import os import subprocess import sys import atexit -import time from github import Github From 3f424de0229f944f20204797ac24b8970c88f45c Mon Sep 17 00:00:00 2001 From: "Mikhail f. Shiryaev" Date: Tue, 1 Nov 2022 14:26:06 +0100 Subject: [PATCH 143/152] Adjust cancel and rerun workflow names to the actual --- .github/workflows/cancel.yml | 2 +- tests/ci/cancel_and_rerun_workflow_lambda/app.py | 2 +- tests/ci/workflow_approve_rerun_lambda/app.py | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/.github/workflows/cancel.yml b/.github/workflows/cancel.yml index cb06d853219..3c2be767ad2 100644 --- a/.github/workflows/cancel.yml +++ b/.github/workflows/cancel.yml @@ -6,7 +6,7 @@ env: on: # yamllint disable-line rule:truthy workflow_run: - workflows: ["PullRequestCI", "ReleaseCI", "DocsCheck", "BackportPR"] + workflows: ["PullRequestCI", "ReleaseBranchCI", "DocsCheck", "BackportPR"] types: - requested jobs: diff --git a/tests/ci/cancel_and_rerun_workflow_lambda/app.py b/tests/ci/cancel_and_rerun_workflow_lambda/app.py index 813ee9d1ab7..21a5ce517f6 100644 --- a/tests/ci/cancel_and_rerun_workflow_lambda/app.py +++ b/tests/ci/cancel_and_rerun_workflow_lambda/app.py @@ -15,7 +15,7 @@ import boto3 # type: ignore NEED_RERUN_OR_CANCELL_WORKFLOWS = { "PullRequestCI", "DocsCheck", - "DocsRelease", + "DocsReleaseChecks", "BackportPR", } diff --git a/tests/ci/workflow_approve_rerun_lambda/app.py b/tests/ci/workflow_approve_rerun_lambda/app.py index 39bd9cfb283..f2b785840d8 100644 --- a/tests/ci/workflow_approve_rerun_lambda/app.py +++ b/tests/ci/workflow_approve_rerun_lambda/app.py @@ -61,11 +61,11 @@ TRUSTED_WORKFLOW_IDS = { NEED_RERUN_WORKFLOWS = { "BackportPR", - "Docs", - "DocsRelease", + "DocsCheck", + "DocsReleaseChecks", "MasterCI", "PullRequestCI", - "ReleaseCI", + "ReleaseBranchCI", } # Individual trusted contirbutors who are not in any trusted organization. From fc1ae1d99dbae2fe7bd15d3004b1e2551b8a040a Mon Sep 17 00:00:00 2001 From: robot-clickhouse Date: Tue, 1 Nov 2022 15:50:38 +0000 Subject: [PATCH 144/152] Update version_date.tsv and changelogs after v22.10.2.11-stable --- docker/server/Dockerfile.alpine | 2 +- docker/server/Dockerfile.ubuntu | 2 +- docs/changelogs/v22.10.2.11-stable.md | 18 ++++++++++++++++++ utils/list-versions/version_date.tsv | 1 + 4 files changed, 21 insertions(+), 2 deletions(-) create mode 100644 docs/changelogs/v22.10.2.11-stable.md diff --git a/docker/server/Dockerfile.alpine b/docker/server/Dockerfile.alpine index cf4eb3fe645..8f1cf6ee98b 100644 --- a/docker/server/Dockerfile.alpine +++ b/docker/server/Dockerfile.alpine @@ -33,7 +33,7 @@ RUN arch=${TARGETARCH:-amd64} \ # lts / testing / prestable / etc ARG REPO_CHANNEL="stable" ARG REPOSITORY="https://packages.clickhouse.com/tgz/${REPO_CHANNEL}" -ARG VERSION="22.10.1.1877" +ARG VERSION="22.10.2.11" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # user/group precreated explicitly with fixed uid/gid on purpose. diff --git a/docker/server/Dockerfile.ubuntu b/docker/server/Dockerfile.ubuntu index d26657a7979..d5fc5d8e0d3 100644 --- a/docker/server/Dockerfile.ubuntu +++ b/docker/server/Dockerfile.ubuntu @@ -21,7 +21,7 @@ RUN sed -i "s|http://archive.ubuntu.com|${apt_archive}|g" /etc/apt/sources.list ARG REPO_CHANNEL="stable" ARG REPOSITORY="deb https://packages.clickhouse.com/deb ${REPO_CHANNEL} main" -ARG VERSION="22.10.1.1877" +ARG VERSION="22.10.2.11" ARG PACKAGES="clickhouse-client clickhouse-server clickhouse-common-static" # set non-empty deb_location_url url to create a docker image diff --git a/docs/changelogs/v22.10.2.11-stable.md b/docs/changelogs/v22.10.2.11-stable.md new file mode 100644 index 00000000000..e4507f4e745 --- /dev/null +++ b/docs/changelogs/v22.10.2.11-stable.md @@ -0,0 +1,18 @@ +--- +sidebar_position: 1 +sidebar_label: 2022 +--- + +# 2022 Changelog + +### ClickHouse release v22.10.2.11-stable (d2bfcaba002) FIXME as compared to v22.10.1.1877-stable (98ab5a3c189) + +#### Bug Fix (user-visible misbehavior in official stable or prestable release) + +* Backported in [#42750](https://github.com/ClickHouse/ClickHouse/issues/42750): A segmentation fault related to DNS & c-ares has been reported. The below error ocurred in multiple threads: ``` 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008088 [ 356 ] {} BaseDaemon: ######################################## 2022-09-28 15:41:19.008,"2022.09.28 15:41:19.008147 [ 356 ] {} BaseDaemon: (version 22.8.5.29 (official build), build id: 92504ACA0B8E2267) (from thread 353) (no query) Received signal Segmentation fault (11)" 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008196 [ 356 ] {} BaseDaemon: Address: 0xf Access: write. Address not mapped to object. 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008216 [ 356 ] {} BaseDaemon: Stack trace: 0x188f8212 0x1626851b 0x1626a69e 0x16269b3f 0x16267eab 0x13cf8284 0x13d24afc 0x13c5217e 0x14ec2495 0x15ba440f 0x15b9d13b 0x15bb2699 0x1891ccb3 0x1891e00d 0x18ae0769 0x18ade022 0x7f76aa985609 0x7f76aa8aa133 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008274 [ 356 ] {} BaseDaemon: 2. Poco::Net::IPAddress::family() const @ 0x188f8212 in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008297 [ 356 ] {} BaseDaemon: 3. ? @ 0x1626851b in /usr/bin/clickhouse 2022-09-28 15:41:19.008,2022.09.28 15:41:19.008309 [ 356 ] {} BaseDaemon: 4. ? @ 0x1626a69e in /usr/bin/clickhouse ```. [#42234](https://github.com/ClickHouse/ClickHouse/pull/42234) ([Arthur Passos](https://github.com/arthurpassos)). +* Backported in [#42793](https://github.com/ClickHouse/ClickHouse/issues/42793): Fix a bug in ParserFunction that could have led to a segmentation fault. [#42724](https://github.com/ClickHouse/ClickHouse/pull/42724) ([Nikolay Degterinsky](https://github.com/evillique)). + +#### NOT FOR CHANGELOG / INSIGNIFICANT + +* Always run `BuilderReport` and `BuilderSpecialReport` in all CI types [#42684](https://github.com/ClickHouse/ClickHouse/pull/42684) ([Mikhail f. Shiryaev](https://github.com/Felixoid)). + diff --git a/utils/list-versions/version_date.tsv b/utils/list-versions/version_date.tsv index 21197713694..47dbec5a5f8 100644 --- a/utils/list-versions/version_date.tsv +++ b/utils/list-versions/version_date.tsv @@ -1,3 +1,4 @@ +v22.10.2.11-stable 2022-11-01 v22.10.1.1877-stable 2022-10-26 v22.9.4.32-stable 2022-10-26 v22.9.3.18-stable 2022-09-30 From e5f9c6559016ff26bbd00e528d5c0ca7bc41cecd Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 1 Nov 2022 12:56:41 -0400 Subject: [PATCH 145/152] add to page title for search --- docs/en/sql-reference/statements/create/function.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/create/function.md b/docs/en/sql-reference/statements/create/function.md index 63c006b1e3e..90be007bf43 100644 --- a/docs/en/sql-reference/statements/create/function.md +++ b/docs/en/sql-reference/statements/create/function.md @@ -4,7 +4,7 @@ sidebar_position: 38 sidebar_label: FUNCTION --- -# CREATE FUNCTION +# CREATE FUNCTION — user defined function (UDF) Creates a user defined function from a lambda expression. The expression must consist of function parameters, constants, operators, or other function calls. From 62428989b4024eef45a5437a319f016064d18131 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Tue, 1 Nov 2022 14:06:42 -0400 Subject: [PATCH 146/152] Add link to docs repo contrib writing guide --- docs/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/README.md b/docs/README.md index fa8b6bed85c..3ca87dc03c3 100644 --- a/docs/README.md +++ b/docs/README.md @@ -212,4 +212,4 @@ Templates: ## How to Build Documentation -You can build your documentation manually by following the instructions in [docs/tools/README.md](../docs/tools/README.md). Also, our CI runs the documentation build after the `documentation` label is added to PR. You can see the results of a build in the GitHub interface. If you have no permissions to add labels, a reviewer of your PR will add it. +You can build your documentation manually by following the instructions in the docs repo [contrib-writing-guide](https://github.com/ClickHouse/clickhouse-docs/blob/main/contrib-writing-guide.md). Also, our CI runs the documentation build after the `documentation` label is added to PR. You can see the results of a build in the GitHub interface. If you have no permissions to add labels, a reviewer of your PR will add it. From 32e1c3b1e648795e05e3494093035f79ad213fb4 Mon Sep 17 00:00:00 2001 From: DanRoscigno Date: Tue, 1 Nov 2022 16:25:54 -0400 Subject: [PATCH 147/152] rename update file as it is included in Manage > Updates --- docs/en/operations/{update.md => _update.md} | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) rename docs/en/operations/{update.md => _update.md} (88%) diff --git a/docs/en/operations/update.md b/docs/en/operations/_update.md similarity index 88% rename from docs/en/operations/update.md rename to docs/en/operations/_update.md index 24f7efecc7b..86981da2be6 100644 --- a/docs/en/operations/update.md +++ b/docs/en/operations/_update.md @@ -1,10 +1,7 @@ ---- -slug: /en/operations/update -sidebar_position: 47 -sidebar_label: ClickHouse Upgrade ---- -# ClickHouse Upgrade +[//]: # (This file is included in Manage > Updates) + +## Self-managed ClickHouse Upgrade If ClickHouse was installed from `deb` packages, execute the following commands on the server: From ba34ad37d1f3ab7d5becbf4c3ec699ff02f60009 Mon Sep 17 00:00:00 2001 From: Habibullah Oladepo Date: Wed, 2 Nov 2022 08:02:01 +0100 Subject: [PATCH 148/152] Fix minor typos in intersect statement documentation --- docs/en/sql-reference/statements/select/intersect.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/sql-reference/statements/select/intersect.md b/docs/en/sql-reference/statements/select/intersect.md index d3b2b51b6be..f1eb4738543 100644 --- a/docs/en/sql-reference/statements/select/intersect.md +++ b/docs/en/sql-reference/statements/select/intersect.md @@ -7,7 +7,7 @@ sidebar_label: INTERSECT The `INTERSECT` clause returns only those rows that result from both the first and the second queries. The queries must match the number of columns, order, and type. The result of `INTERSECT` can contain duplicate rows. -Multiple `INTERSECT` statements are executes left to right if parenthesis are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clause. +Multiple `INTERSECT` statements are executed left to right if parentheses are not specified. The `INTERSECT` operator has a higher priority than the `UNION` and `EXCEPT` clauses. ``` sql From 40314e90303b366b2bcdf584170ff1aa2318926e Mon Sep 17 00:00:00 2001 From: vdimir Date: Tue, 1 Nov 2022 10:43:53 +0000 Subject: [PATCH 149/152] Add timezone to 02458_datediff_date32 --- .../02458_datediff_date32.reference | 154 +++++++++--------- .../0_stateless/02458_datediff_date32.sql | 154 +++++++++--------- 2 files changed, 154 insertions(+), 154 deletions(-) diff --git a/tests/queries/0_stateless/02458_datediff_date32.reference b/tests/queries/0_stateless/02458_datediff_date32.reference index 9e93af03896..fdb8273a74b 100644 --- a/tests/queries/0_stateless/02458_datediff_date32.reference +++ b/tests/queries/0_stateless/02458_datediff_date32.reference @@ -1,169 +1,169 @@ -- { echo } -- Date32 vs Date32 -SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('second', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 86400 -SELECT dateDiff('minute', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('minute', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 1440 -SELECT dateDiff('hour', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('hour', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 24 -SELECT dateDiff('day', toDate32('1927-01-01'), toDate32('1927-01-02')); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 1 -SELECT dateDiff('week', toDate32('1927-01-01'), toDate32('1927-01-08')); +SELECT dateDiff('week', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-08', 'UTC'), 'UTC'); 1 -SELECT dateDiff('month', toDate32('1927-01-01'), toDate32('1927-02-01')); +SELECT dateDiff('month', toDate32('1927-01-01', 'UTC'), toDate32('1927-02-01', 'UTC'), 'UTC'); 1 -SELECT dateDiff('quarter', toDate32('1927-01-01'), toDate32('1927-04-01')); +SELECT dateDiff('quarter', toDate32('1927-01-01', 'UTC'), toDate32('1927-04-01', 'UTC'), 'UTC'); 1 -SELECT dateDiff('year', toDate32('1927-01-01'), toDate32('1928-01-01')); +SELECT dateDiff('year', toDate32('1927-01-01', 'UTC'), toDate32('1928-01-01', 'UTC'), 'UTC'); 1 -- With DateTime64 -- Date32 vs DateTime64 -SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('second', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); 86400 -SELECT dateDiff('minute', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('minute', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); 1440 -SELECT dateDiff('hour', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('hour', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); 24 -SELECT dateDiff('day', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); 1 -SELECT dateDiff('week', toDate32('1927-01-01'), toDateTime64('1927-01-08 00:00:00', 3)); +SELECT dateDiff('week', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-08 00:00:00', 3, 'UTC'), 'UTC'); 1 -SELECT dateDiff('month', toDate32('1927-01-01'), toDateTime64('1927-02-01 00:00:00', 3)); +SELECT dateDiff('month', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-02-01 00:00:00', 3, 'UTC'), 'UTC'); 1 -SELECT dateDiff('quarter', toDate32('1927-01-01'), toDateTime64('1927-04-01 00:00:00', 3)); +SELECT dateDiff('quarter', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-04-01 00:00:00', 3, 'UTC'), 'UTC'); 1 -SELECT dateDiff('year', toDate32('1927-01-01'), toDateTime64('1928-01-01 00:00:00', 3)); +SELECT dateDiff('year', toDate32('1927-01-01', 'UTC'), toDateTime64('1928-01-01 00:00:00', 3, 'UTC'), 'UTC'); 1 -- DateTime64 vs Date32 -SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 86400 -SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 1440 -SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 24 -SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); 1 -SELECT dateDiff('week', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-08')); +SELECT dateDiff('week', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-08', 'UTC'), 'UTC'); 1 -SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-02-01')); +SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-02-01', 'UTC'), 'UTC'); 1 -SELECT dateDiff('quarter', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-04-01')); +SELECT dateDiff('quarter', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-04-01', 'UTC'), 'UTC'); 1 -SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1928-01-01')); +SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1928-01-01', 'UTC'), 'UTC'); 1 -- With DateTime -- Date32 vs DateTime -SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); +SELECT dateDiff('second', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); 86400 -SELECT dateDiff('minute', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); +SELECT dateDiff('minute', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); 1440 -SELECT dateDiff('hour', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); +SELECT dateDiff('hour', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); 24 -SELECT dateDiff('day', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); 1 -SELECT dateDiff('week', toDate32('2015-08-18'), toDateTime('2015-08-25 00:00:00')); +SELECT dateDiff('week', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-25 00:00:00', 'UTC'), 'UTC'); 1 -SELECT dateDiff('month', toDate32('2015-08-18'), toDateTime('2015-09-18 00:00:00')); +SELECT dateDiff('month', toDate32('2015-08-18', 'UTC'), toDateTime('2015-09-18 00:00:00', 'UTC'), 'UTC'); 1 -SELECT dateDiff('quarter', toDate32('2015-08-18'), toDateTime('2015-11-18 00:00:00')); +SELECT dateDiff('quarter', toDate32('2015-08-18', 'UTC'), toDateTime('2015-11-18 00:00:00', 'UTC'), 'UTC'); 1 -SELECT dateDiff('year', toDate32('2015-08-18'), toDateTime('2016-08-18 00:00:00')); +SELECT dateDiff('year', toDate32('2015-08-18', 'UTC'), toDateTime('2016-08-18 00:00:00', 'UTC'), 'UTC'); 1 -- DateTime vs Date32 -SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); +SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 86400 -SELECT dateDiff('minute', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); +SELECT dateDiff('minute', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 1440 -SELECT dateDiff('hour', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); +SELECT dateDiff('hour', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 24 -SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); +SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 1 -SELECT dateDiff('week', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-25')); +SELECT dateDiff('week', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-25', 'UTC'), 'UTC'); 1 -SELECT dateDiff('month', toDateTime('2015-08-18 00:00:00'), toDate32('2015-09-18')); +SELECT dateDiff('month', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-09-18', 'UTC'), 'UTC'); 1 -SELECT dateDiff('quarter', toDateTime('2015-08-18 00:00:00'), toDate32('2015-11-18')); +SELECT dateDiff('quarter', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-11-18', 'UTC'), 'UTC'); 1 -SELECT dateDiff('year', toDateTime('2015-08-18 00:00:00'), toDate32('2016-08-18')); +SELECT dateDiff('year', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2016-08-18', 'UTC'), 'UTC'); 1 -- With Date -- Date32 vs Date -SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')); +SELECT dateDiff('second', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); 86400 -SELECT dateDiff('minute', toDate32('2015-08-18'), toDate('2015-08-19')); +SELECT dateDiff('minute', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); 1440 -SELECT dateDiff('hour', toDate32('2015-08-18'), toDate('2015-08-19')); +SELECT dateDiff('hour', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); 24 -SELECT dateDiff('day', toDate32('2015-08-18'), toDate('2015-08-19')); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); 1 -SELECT dateDiff('week', toDate32('2015-08-18'), toDate('2015-08-25')); +SELECT dateDiff('week', toDate32('2015-08-18', 'UTC'), toDate('2015-08-25', 'UTC'), 'UTC'); 1 -SELECT dateDiff('month', toDate32('2015-08-18'), toDate('2015-09-18')); +SELECT dateDiff('month', toDate32('2015-08-18', 'UTC'), toDate('2015-09-18', 'UTC'), 'UTC'); 1 -SELECT dateDiff('quarter', toDate32('2015-08-18'), toDate('2015-11-18')); +SELECT dateDiff('quarter', toDate32('2015-08-18', 'UTC'), toDate('2015-11-18', 'UTC'), 'UTC'); 1 -SELECT dateDiff('year', toDate32('2015-08-18'), toDate('2016-08-18')); +SELECT dateDiff('year', toDate32('2015-08-18', 'UTC'), toDate('2016-08-18', 'UTC'), 'UTC'); 1 -- Date vs Date32 -SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')); +SELECT dateDiff('second', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 86400 -SELECT dateDiff('minute', toDate('2015-08-18'), toDate32('2015-08-19')); +SELECT dateDiff('minute', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 1440 -SELECT dateDiff('hour', toDate('2015-08-18'), toDate32('2015-08-19')); +SELECT dateDiff('hour', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 24 -SELECT dateDiff('day', toDate('2015-08-18'), toDate32('2015-08-19')); +SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); 1 -SELECT dateDiff('week', toDate('2015-08-18'), toDate32('2015-08-25')); +SELECT dateDiff('week', toDate('2015-08-18', 'UTC'), toDate32('2015-08-25', 'UTC'), 'UTC'); 1 -SELECT dateDiff('month', toDate('2015-08-18'), toDate32('2015-09-18')); +SELECT dateDiff('month', toDate('2015-08-18', 'UTC'), toDate32('2015-09-18', 'UTC'), 'UTC'); 1 -SELECT dateDiff('quarter', toDate('2015-08-18'), toDate32('2015-11-18')); +SELECT dateDiff('quarter', toDate('2015-08-18', 'UTC'), toDate32('2015-11-18', 'UTC'), 'UTC'); 1 -SELECT dateDiff('year', toDate('2015-08-18'), toDate32('2016-08-18')); +SELECT dateDiff('year', toDate('2015-08-18', 'UTC'), toDate32('2016-08-18', 'UTC'), 'UTC'); 1 -- Const vs non-const columns -SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDate32('1927-01-02'))); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDateTime64('1927-01-02 00:00:00', 3))); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), materialize(toDateTime64('1927-01-02 00:00:00', 3, 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), materialize(toDate32('1927-01-02'))); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDateTime('2015-08-19 00:00:00'))); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), materialize(toDateTime('2015-08-19 00:00:00', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00'), materialize(toDate32('2015-08-19'))); +SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00', 'UTC'), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDate('2015-08-19'))); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), materialize(toDate('2015-08-19', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', toDate('2015-08-18'), materialize(toDate32('2015-08-19'))); +SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); 1 -- Non-const vs const columns -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDate32('1927-01-02')); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), toDate32('1927-01-02', 'UTC'), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDateTime64('1927-01-02 00:00:00', 3)); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), toDate32('1927-01-02')); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3, 'UTC')), toDate32('1927-01-02', 'UTC'), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDateTime('2015-08-19 00:00:00')); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00')), toDate32('2015-08-19')); +SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), toDate32('2015-08-19', 'UTC'), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDate('2015-08-19')); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), toDate('2015-08-19', 'UTC'), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate('2015-08-18')), toDate32('2015-08-19')); +SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), toDate32('2015-08-19', 'UTC'), 'UTC'); 1 -- Non-const vs non-const columns -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDate32('1927-01-02'))); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDateTime64('1927-01-02 00:00:00', 3))); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), materialize(toDateTime64('1927-01-02 00:00:00', 3, 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), materialize(toDate32('1927-01-02'))); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3, 'UTC')), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), materialize(toDateTime('2015-08-19 00:00:00'))); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDateTime('2015-08-19 00:00:00', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00')), materialize(toDate32('2015-08-19'))); +SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), materialize(toDate('2015-08-19'))); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDate('2015-08-19', 'UTC')), 'UTC'); 1 -SELECT dateDiff('day', materialize(toDate('2015-08-18')), materialize(toDate32('2015-08-19'))); +SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); 1 diff --git a/tests/queries/0_stateless/02458_datediff_date32.sql b/tests/queries/0_stateless/02458_datediff_date32.sql index b4cb203080e..e41070e8146 100644 --- a/tests/queries/0_stateless/02458_datediff_date32.sql +++ b/tests/queries/0_stateless/02458_datediff_date32.sql @@ -1,101 +1,101 @@ -- { echo } -- Date32 vs Date32 -SELECT dateDiff('second', toDate32('1927-01-01'), toDate32('1927-01-02')); -SELECT dateDiff('minute', toDate32('1927-01-01'), toDate32('1927-01-02')); -SELECT dateDiff('hour', toDate32('1927-01-01'), toDate32('1927-01-02')); -SELECT dateDiff('day', toDate32('1927-01-01'), toDate32('1927-01-02')); -SELECT dateDiff('week', toDate32('1927-01-01'), toDate32('1927-01-08')); -SELECT dateDiff('month', toDate32('1927-01-01'), toDate32('1927-02-01')); -SELECT dateDiff('quarter', toDate32('1927-01-01'), toDate32('1927-04-01')); -SELECT dateDiff('year', toDate32('1927-01-01'), toDate32('1928-01-01')); +SELECT dateDiff('second', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('minute', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('hour', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('week', toDate32('1927-01-01', 'UTC'), toDate32('1927-01-08', 'UTC'), 'UTC'); +SELECT dateDiff('month', toDate32('1927-01-01', 'UTC'), toDate32('1927-02-01', 'UTC'), 'UTC'); +SELECT dateDiff('quarter', toDate32('1927-01-01', 'UTC'), toDate32('1927-04-01', 'UTC'), 'UTC'); +SELECT dateDiff('year', toDate32('1927-01-01', 'UTC'), toDate32('1928-01-01', 'UTC'), 'UTC'); -- With DateTime64 -- Date32 vs DateTime64 -SELECT dateDiff('second', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); -SELECT dateDiff('minute', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); -SELECT dateDiff('hour', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); -SELECT dateDiff('day', toDate32('1927-01-01'), toDateTime64('1927-01-02 00:00:00', 3)); -SELECT dateDiff('week', toDate32('1927-01-01'), toDateTime64('1927-01-08 00:00:00', 3)); -SELECT dateDiff('month', toDate32('1927-01-01'), toDateTime64('1927-02-01 00:00:00', 3)); -SELECT dateDiff('quarter', toDate32('1927-01-01'), toDateTime64('1927-04-01 00:00:00', 3)); -SELECT dateDiff('year', toDate32('1927-01-01'), toDateTime64('1928-01-01 00:00:00', 3)); +SELECT dateDiff('second', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('minute', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('hour', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('week', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-01-08 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('month', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-02-01 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('quarter', toDate32('1927-01-01', 'UTC'), toDateTime64('1927-04-01 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('year', toDate32('1927-01-01', 'UTC'), toDateTime64('1928-01-01 00:00:00', 3, 'UTC'), 'UTC'); -- DateTime64 vs Date32 -SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); -SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); -SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); -SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-02')); -SELECT dateDiff('week', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-01-08')); -SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-02-01')); -SELECT dateDiff('quarter', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1927-04-01')); -SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 3), toDate32('1928-01-01')); +SELECT dateDiff('second', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('minute', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('hour', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('week', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-01-08', 'UTC'), 'UTC'); +SELECT dateDiff('month', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-02-01', 'UTC'), 'UTC'); +SELECT dateDiff('quarter', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1927-04-01', 'UTC'), 'UTC'); +SELECT dateDiff('year', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), toDate32('1928-01-01', 'UTC'), 'UTC'); -- With DateTime -- Date32 vs DateTime -SELECT dateDiff('second', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); -SELECT dateDiff('minute', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); -SELECT dateDiff('hour', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); -SELECT dateDiff('day', toDate32('2015-08-18'), toDateTime('2015-08-19 00:00:00')); -SELECT dateDiff('week', toDate32('2015-08-18'), toDateTime('2015-08-25 00:00:00')); -SELECT dateDiff('month', toDate32('2015-08-18'), toDateTime('2015-09-18 00:00:00')); -SELECT dateDiff('quarter', toDate32('2015-08-18'), toDateTime('2015-11-18 00:00:00')); -SELECT dateDiff('year', toDate32('2015-08-18'), toDateTime('2016-08-18 00:00:00')); +SELECT dateDiff('second', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('minute', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('hour', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('week', toDate32('2015-08-18', 'UTC'), toDateTime('2015-08-25 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('month', toDate32('2015-08-18', 'UTC'), toDateTime('2015-09-18 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('quarter', toDate32('2015-08-18', 'UTC'), toDateTime('2015-11-18 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('year', toDate32('2015-08-18', 'UTC'), toDateTime('2016-08-18 00:00:00', 'UTC'), 'UTC'); -- DateTime vs Date32 -SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); -SELECT dateDiff('minute', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); -SELECT dateDiff('hour', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); -SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-19')); -SELECT dateDiff('week', toDateTime('2015-08-18 00:00:00'), toDate32('2015-08-25')); -SELECT dateDiff('month', toDateTime('2015-08-18 00:00:00'), toDate32('2015-09-18')); -SELECT dateDiff('quarter', toDateTime('2015-08-18 00:00:00'), toDate32('2015-11-18')); -SELECT dateDiff('year', toDateTime('2015-08-18 00:00:00'), toDate32('2016-08-18')); +SELECT dateDiff('second', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('minute', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('hour', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('week', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-08-25', 'UTC'), 'UTC'); +SELECT dateDiff('month', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-09-18', 'UTC'), 'UTC'); +SELECT dateDiff('quarter', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2015-11-18', 'UTC'), 'UTC'); +SELECT dateDiff('year', toDateTime('2015-08-18 00:00:00', 'UTC'), toDate32('2016-08-18', 'UTC'), 'UTC'); -- With Date -- Date32 vs Date -SELECT dateDiff('second', toDate32('2015-08-18'), toDate('2015-08-19')); -SELECT dateDiff('minute', toDate32('2015-08-18'), toDate('2015-08-19')); -SELECT dateDiff('hour', toDate32('2015-08-18'), toDate('2015-08-19')); -SELECT dateDiff('day', toDate32('2015-08-18'), toDate('2015-08-19')); -SELECT dateDiff('week', toDate32('2015-08-18'), toDate('2015-08-25')); -SELECT dateDiff('month', toDate32('2015-08-18'), toDate('2015-09-18')); -SELECT dateDiff('quarter', toDate32('2015-08-18'), toDate('2015-11-18')); -SELECT dateDiff('year', toDate32('2015-08-18'), toDate('2016-08-18')); +SELECT dateDiff('second', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('minute', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('hour', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), toDate('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('week', toDate32('2015-08-18', 'UTC'), toDate('2015-08-25', 'UTC'), 'UTC'); +SELECT dateDiff('month', toDate32('2015-08-18', 'UTC'), toDate('2015-09-18', 'UTC'), 'UTC'); +SELECT dateDiff('quarter', toDate32('2015-08-18', 'UTC'), toDate('2015-11-18', 'UTC'), 'UTC'); +SELECT dateDiff('year', toDate32('2015-08-18', 'UTC'), toDate('2016-08-18', 'UTC'), 'UTC'); -- Date vs Date32 -SELECT dateDiff('second', toDate('2015-08-18'), toDate32('2015-08-19')); -SELECT dateDiff('minute', toDate('2015-08-18'), toDate32('2015-08-19')); -SELECT dateDiff('hour', toDate('2015-08-18'), toDate32('2015-08-19')); -SELECT dateDiff('day', toDate('2015-08-18'), toDate32('2015-08-19')); -SELECT dateDiff('week', toDate('2015-08-18'), toDate32('2015-08-25')); -SELECT dateDiff('month', toDate('2015-08-18'), toDate32('2015-09-18')); -SELECT dateDiff('quarter', toDate('2015-08-18'), toDate32('2015-11-18')); -SELECT dateDiff('year', toDate('2015-08-18'), toDate32('2016-08-18')); +SELECT dateDiff('second', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('minute', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('hour', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('week', toDate('2015-08-18', 'UTC'), toDate32('2015-08-25', 'UTC'), 'UTC'); +SELECT dateDiff('month', toDate('2015-08-18', 'UTC'), toDate32('2015-09-18', 'UTC'), 'UTC'); +SELECT dateDiff('quarter', toDate('2015-08-18', 'UTC'), toDate32('2015-11-18', 'UTC'), 'UTC'); +SELECT dateDiff('year', toDate('2015-08-18', 'UTC'), toDate32('2016-08-18', 'UTC'), 'UTC'); -- Const vs non-const columns -SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDate32('1927-01-02'))); -SELECT dateDiff('day', toDate32('1927-01-01'), materialize(toDateTime64('1927-01-02 00:00:00', 3))); -SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3), materialize(toDate32('1927-01-02'))); -SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDateTime('2015-08-19 00:00:00'))); -SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00'), materialize(toDate32('2015-08-19'))); -SELECT dateDiff('day', toDate32('2015-08-18'), materialize(toDate('2015-08-19'))); -SELECT dateDiff('day', toDate('2015-08-18'), materialize(toDate32('2015-08-19'))); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); +SELECT dateDiff('day', toDate32('1927-01-01', 'UTC'), materialize(toDateTime64('1927-01-02 00:00:00', 3, 'UTC')), 'UTC'); +SELECT dateDiff('day', toDateTime64('1927-01-01 00:00:00', 3, 'UTC'), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), materialize(toDateTime('2015-08-19 00:00:00', 'UTC')), 'UTC'); +SELECT dateDiff('day', toDateTime('2015-08-18 00:00:00', 'UTC'), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); +SELECT dateDiff('day', toDate32('2015-08-18', 'UTC'), materialize(toDate('2015-08-19', 'UTC')), 'UTC'); +SELECT dateDiff('day', toDate('2015-08-18', 'UTC'), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); -- Non-const vs const columns -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDate32('1927-01-02')); -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), toDateTime64('1927-01-02 00:00:00', 3)); -SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), toDate32('1927-01-02')); -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDateTime('2015-08-19 00:00:00')); -SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00')), toDate32('2015-08-19')); -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), toDate('2015-08-19')); -SELECT dateDiff('day', materialize(toDate('2015-08-18')), toDate32('2015-08-19')); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), toDateTime64('1927-01-02 00:00:00', 3, 'UTC'), 'UTC'); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3, 'UTC')), toDate32('1927-01-02', 'UTC'), 'UTC'); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), toDateTime('2015-08-19 00:00:00', 'UTC'), 'UTC'); +SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), toDate32('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), toDate('2015-08-19', 'UTC'), 'UTC'); +SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), toDate32('2015-08-19', 'UTC'), 'UTC'); -- Non-const vs non-const columns -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDate32('1927-01-02'))); -SELECT dateDiff('day', materialize(toDate32('1927-01-01')), materialize(toDateTime64('1927-01-02 00:00:00', 3))); -SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3)), materialize(toDate32('1927-01-02'))); -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), materialize(toDateTime('2015-08-19 00:00:00'))); -SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00')), materialize(toDate32('2015-08-19'))); -SELECT dateDiff('day', materialize(toDate32('2015-08-18')), materialize(toDate('2015-08-19'))); -SELECT dateDiff('day', materialize(toDate('2015-08-18')), materialize(toDate32('2015-08-19'))); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); +SELECT dateDiff('day', materialize(toDate32('1927-01-01', 'UTC')), materialize(toDateTime64('1927-01-02 00:00:00', 3, 'UTC')), 'UTC'); +SELECT dateDiff('day', materialize(toDateTime64('1927-01-01 00:00:00', 3, 'UTC')), materialize(toDate32('1927-01-02', 'UTC')), 'UTC'); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDateTime('2015-08-19 00:00:00', 'UTC')), 'UTC'); +SELECT dateDiff('day', materialize(toDateTime('2015-08-18 00:00:00', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); +SELECT dateDiff('day', materialize(toDate32('2015-08-18', 'UTC')), materialize(toDate('2015-08-19', 'UTC')), 'UTC'); +SELECT dateDiff('day', materialize(toDate('2015-08-18', 'UTC')), materialize(toDate32('2015-08-19', 'UTC')), 'UTC'); From 4c77987bf876706f8d0f579d8e33e95a94726239 Mon Sep 17 00:00:00 2001 From: Dan Roscigno Date: Wed, 2 Nov 2022 10:41:55 -0400 Subject: [PATCH 150/152] Update projection.md Experiment with Algolia search --- .../statements/alter/projection.md | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/docs/en/sql-reference/statements/alter/projection.md b/docs/en/sql-reference/statements/alter/projection.md index a1981b4a0bb..ff8ecf3a77f 100644 --- a/docs/en/sql-reference/statements/alter/projection.md +++ b/docs/en/sql-reference/statements/alter/projection.md @@ -7,18 +7,26 @@ title: "Manipulating Projections" The following operations with [projections](../../../engines/table-engines/mergetree-family/mergetree.md#projections) are available: -- `ALTER TABLE [db].name ADD PROJECTION name ( SELECT [GROUP BY] [ORDER BY] )` - Adds projection description to tables metadata. +## ADD PROJECTION -- `ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db].name ADD PROJECTION name ( SELECT [GROUP BY] [ORDER BY] )` - Adds projection description to tables metadata. -- `ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +## DROP PROJECTION -- `ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). +`ALTER TABLE [db].name DROP PROJECTION name` - Removes projection description from tables metadata and deletes projection files from disk. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). + +## MATERIALIZE PROJECTION + +`ALTER TABLE [db.]table MATERIALIZE PROJECTION name IN PARTITION partition_name` - The query rebuilds the projection `name` in the partition `partition_name`. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). + +## CLEAR PROJECTION + +`ALTER TABLE [db.]table CLEAR PROJECTION name IN PARTITION partition_name` - Deletes projection files from disk without removing description. Implemented as a [mutation](../../../sql-reference/statements/alter/index.md#mutations). The commands `ADD`, `DROP` and `CLEAR` are lightweight in a sense that they only change metadata or remove files. -Also, they are replicated, syncing projections metadata via ZooKeeper. +Also, they are replicated, syncing projections metadata via ClickHouse Keeper or ZooKeeper. :::note Projection manipulation is supported only for tables with [`*MergeTree`](../../../engines/table-engines/mergetree-family/mergetree.md) engine (including [replicated](../../../engines/table-engines/mergetree-family/replication.md) variants). From b6d84ef5fa451d3905d09fd8bcc9b5220f70905e Mon Sep 17 00:00:00 2001 From: Camilo Sierra Date: Wed, 2 Nov 2022 15:43:57 +0100 Subject: [PATCH 151/152] range hashed missing parameters On the query we have missing two options: - `LIFETIME` Is not on the example if you copy paste you will have an Exception `DB::Exception: Cannot create dictionary with empty lifetime.` - `SOURCE` was not mentioned and it's important to link to the main/source table. - There was an error on the `dictGetT` function there was an additional T this function do not exist (I have tested and we need to use `dictGet`). - Also in the Dictionary example we have no extra attribute other than the id and the two dates, and for running the queries and the `dicGet` function you need an additional attribute this is why I have added `advertiser_id` (BTW I use advertiser_id as this was use in the example just before) and also add one example, without the example it was not easy to understand what was the 'attr_name' mentioned before. - I add an example as an user did not knew how to cast the date to a Uint64 (Because most of the time the original/raw dates are defined on the range as Date64, so this example will explain them how to cast when doing the query) --- .../external-dicts-dict-layout.md | 14 +++++++++++--- 1 file changed, 11 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 198ff12f1d6..02a4ad57a3b 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 @@ -303,17 +303,25 @@ or CREATE DICTIONARY somedict ( id UInt64, first Date, - last Date + last Date, + advertiser_id UInt64 ) PRIMARY KEY id +SOURCE(CLICKHOUSE(TABLE 'date_table')) +LIFETIME(MIN 1 MAX 1000) LAYOUT(RANGE_HASHED()) RANGE(MIN first MAX last) ``` -To work with these dictionaries, you need to pass an additional argument to the `dictGetT` function, for which a range is selected: +To work with these dictionaries, you need to pass an additional argument to the `dictGet` function, for which a range is selected: ``` sql -dictGetT('dict_name', 'attr_name', id, date) +dictGet('dict_name', 'attr_name', id, date) +``` +Query example: + +``` sql +SELECT dictGet('somedict', 'advertiser_id', 1, '2022-10-20 23:20:10.000'::DateTime64::UInt64); ``` This function returns the value for the specified `id`s and the date range that includes the passed date. From c3466fb52f635f8cbd5d0ae95781ac5a84b9ed87 Mon Sep 17 00:00:00 2001 From: Alexey Milovidov Date: Wed, 2 Nov 2022 22:22:16 +0300 Subject: [PATCH 152/152] Update clickhouse-keeper.md --- docs/en/operations/clickhouse-keeper.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/clickhouse-keeper.md b/docs/en/operations/clickhouse-keeper.md index 82fa5c114ea..aad20da0010 100644 --- a/docs/en/operations/clickhouse-keeper.md +++ b/docs/en/operations/clickhouse-keeper.md @@ -309,7 +309,7 @@ Sessions with Ephemerals (1): /clickhouse/task_queue/ddl ``` -## [experimental] Migration from ZooKeeper {#migration-from-zookeeper} +## Migration from ZooKeeper {#migration-from-zookeeper} Seamlessly migration from ZooKeeper to ClickHouse Keeper is impossible you have to stop your ZooKeeper cluster, convert data and start ClickHouse Keeper. `clickhouse-keeper-converter` tool allows converting ZooKeeper logs and snapshots to ClickHouse Keeper snapshot. It works only with ZooKeeper > 3.4. Steps for migration: