mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 17:41:59 +00:00
Merge branch 'master' into split-cast-overload-resolver
This commit is contained in:
commit
146e401851
@ -33,6 +33,9 @@ ENV TSAN_OPTIONS='halt_on_error=1 abort_on_error=1 history_size=7 memory_limit_m
|
||||
ENV UBSAN_OPTIONS='print_stacktrace=1'
|
||||
ENV MSAN_OPTIONS='abort_on_error=1 poison_in_dtor=1'
|
||||
|
||||
# for external_symbolizer_path
|
||||
RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer
|
||||
|
||||
RUN echo "en_US.UTF-8 UTF-8" > /etc/locale.gen && locale-gen en_US.UTF-8
|
||||
ENV LC_ALL en_US.UTF-8
|
||||
|
||||
|
@ -14,7 +14,6 @@ RUN apt-get update \
|
||||
libclang-${LLVM_VERSION}-dev \
|
||||
libclang-rt-${LLVM_VERSION}-dev \
|
||||
lld-${LLVM_VERSION} \
|
||||
llvm-${LLVM_VERSION} \
|
||||
llvm-${LLVM_VERSION}-dev \
|
||||
lsof \
|
||||
ninja-build \
|
||||
@ -37,8 +36,6 @@ RUN pip3 install numpy==1.26.3 scipy==1.12.0 pandas==1.5.3 Jinja2==3.1.3
|
||||
|
||||
# This symlink is required by gcc to find the lld linker
|
||||
RUN ln -s /usr/bin/lld-${LLVM_VERSION} /usr/bin/ld.lld
|
||||
# for external_symbolizer_path
|
||||
RUN ln -s /usr/bin/llvm-symbolizer-${LLVM_VERSION} /usr/bin/llvm-symbolizer
|
||||
# FIXME: workaround for "The imported target "merge-fdata" references the file" error
|
||||
# https://salsa.debian.org/pkg-llvm-team/llvm-toolchain/-/commit/992e52c0b156a5ba9c6a8a54f8c4857ddd3d371d
|
||||
RUN sed -i '/_IMPORT_CHECK_FILES_FOR_\(mlir-\|llvm-bolt\|merge-fdata\|MLIR\)/ {s|^|#|}' /usr/lib/llvm-${LLVM_VERSION}/lib/cmake/llvm/LLVMExports-*.cmake
|
||||
|
@ -26,6 +26,8 @@ RUN apt-get update \
|
||||
&& export CODENAME="$(lsb_release --codename --short | tr 'A-Z' 'a-z')" \
|
||||
&& echo "deb https://apt.llvm.org/${CODENAME}/ llvm-toolchain-${CODENAME}-${LLVM_VERSION} main" >> \
|
||||
/etc/apt/sources.list \
|
||||
&& apt-get update \
|
||||
&& apt-get install --yes --no-install-recommends --verbose-versions llvm-${LLVM_VERSION} \
|
||||
&& apt-get clean \
|
||||
&& rm -rf /var/lib/apt/lists/* /var/cache/debconf /tmp/*
|
||||
|
||||
|
@ -78,8 +78,8 @@ It is recommended to use official pre-compiled `deb` packages for Debian or Ubun
|
||||
|
||||
#### Setup the Debian repository
|
||||
``` bash
|
||||
sudo apt-get install -y apt-transport-https ca-certificates dirmngr
|
||||
sudo gpg --no-default-keyring --keyring /usr/share/keyrings/clickhouse-keyring.gpg --keyserver hkp://keyserver.ubuntu.com:80 --recv-keys 8919F6BD2B48D754
|
||||
sudo apt-get install -y apt-transport-https ca-certificates curl gnupg
|
||||
curl -fsSL 'https://packages.clickhouse.com/rpm/lts/repodata/repomd.xml.key' | sudo gpg --dearmor -o /usr/share/keyrings/clickhouse-keyring.gpg
|
||||
|
||||
echo "deb [signed-by=/usr/share/keyrings/clickhouse-keyring.gpg] https://packages.clickhouse.com/deb stable main" | sudo tee \
|
||||
/etc/apt/sources.list.d/clickhouse.list
|
||||
|
@ -5,7 +5,7 @@ sidebar_label: cluster
|
||||
title: "cluster, clusterAllReplicas"
|
||||
---
|
||||
|
||||
Allows to access all shards in an existing cluster which configured in `remote_servers` section without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. One replica of each shard is queried.
|
||||
Allows to access all shards (configured in the `remote_servers` section) of a cluster without creating a [Distributed](../../engines/table-engines/special/distributed.md) table. Only one replica of each shard is queried.
|
||||
|
||||
`clusterAllReplicas` function — same as `cluster`, but all replicas are queried. Each replica in a cluster is used as a separate shard/connection.
|
||||
|
||||
|
@ -20,12 +20,12 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int DUPLICATE_COLUMN;
|
||||
extern const int NUMBER_OF_DIMENSIONS_MISMATCHED;
|
||||
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
extern const int EXPERIMENTAL_FEATURE_ERROR;
|
||||
}
|
||||
|
||||
namespace
|
||||
@ -247,7 +247,7 @@ void ColumnObject::Subcolumn::checkTypes() const
|
||||
prefix_types.push_back(current_type);
|
||||
auto prefix_common_type = getLeastSupertype(prefix_types);
|
||||
if (!prefix_common_type->equals(*current_type))
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
|
||||
"Data type {} of column at position {} cannot represent all columns from i-th prefix",
|
||||
current_type->getName(), i);
|
||||
}
|
||||
@ -635,7 +635,7 @@ void ColumnObject::checkConsistency() const
|
||||
{
|
||||
if (num_rows != leaf->data.size())
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of subcolumns are inconsistent in ColumnObject."
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Sizes of subcolumns are inconsistent in ColumnObject."
|
||||
" Subcolumn '{}' has {} rows, but expected size is {}",
|
||||
leaf->path.getPath(), leaf->data.size(), num_rows);
|
||||
}
|
||||
@ -919,7 +919,7 @@ void ColumnObject::addSubcolumn(const PathInData & key, size_t new_size)
|
||||
void ColumnObject::addNestedSubcolumn(const PathInData & key, const FieldInfo & field_info, size_t new_size)
|
||||
{
|
||||
if (!key.hasNested())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
|
||||
"Cannot add Nested subcolumn, because path doesn't contain Nested");
|
||||
|
||||
bool inserted = false;
|
||||
|
@ -598,6 +598,7 @@
|
||||
M(714, UNEXPECTED_CLUSTER) \
|
||||
M(715, CANNOT_DETECT_FORMAT) \
|
||||
M(716, CANNOT_FORGET_PARTITION) \
|
||||
M(717, EXPERIMENTAL_FEATURE_ERROR) \
|
||||
\
|
||||
M(999, KEEPER_EXCEPTION) \
|
||||
M(1000, POCO_EXCEPTION) \
|
||||
|
@ -28,9 +28,9 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int TYPE_MISMATCH;
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int INCOMPATIBLE_COLUMNS;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int EXPERIMENTAL_FEATURE_ERROR;
|
||||
}
|
||||
|
||||
size_t getNumberOfDimensions(const IDataType & type)
|
||||
@ -92,7 +92,7 @@ ColumnPtr createArrayOfColumn(ColumnPtr column, size_t num_dimensions)
|
||||
Array createEmptyArrayField(size_t num_dimensions)
|
||||
{
|
||||
if (num_dimensions == 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create array field with 0 dimensions");
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Cannot create array field with 0 dimensions");
|
||||
|
||||
Array array;
|
||||
Array * current_array = &array;
|
||||
@ -231,7 +231,7 @@ static std::pair<ColumnPtr, DataTypePtr> recursivlyConvertDynamicColumnToTuple(
|
||||
};
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type->getName());
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Type {} unexpectedly has dynamic columns", type->getName());
|
||||
}
|
||||
|
||||
void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & storage_snapshot)
|
||||
@ -247,7 +247,7 @@ void convertDynamicColumnsToTuples(Block & block, const StorageSnapshotPtr & sto
|
||||
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);
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Column '{}' not found in storage", column.name);
|
||||
|
||||
auto storage_column_concrete = storage_snapshot->getColumn(options.withExtendedObjects(), column.name);
|
||||
|
||||
@ -315,7 +315,7 @@ static DataTypePtr getLeastCommonTypeForObject(const DataTypes & types, bool che
|
||||
{
|
||||
const auto * type_tuple = typeid_cast<const DataTypeTuple *>(type.get());
|
||||
if (!type_tuple)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
|
||||
"Least common type for object can be deduced only from tuples, but {} given", type->getName());
|
||||
|
||||
auto [tuple_paths, tuple_types] = flattenTuple(type);
|
||||
@ -427,7 +427,7 @@ static DataTypePtr getLeastCommonTypeForDynamicColumnsImpl(
|
||||
if (const auto * type_tuple = typeid_cast<const DataTypeTuple *>(type_in_storage.get()))
|
||||
return getLeastCommonTypeForTuple(*type_tuple, concrete_types, check_ambiguos_paths);
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName());
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName());
|
||||
}
|
||||
|
||||
DataTypePtr getLeastCommonTypeForDynamicColumns(
|
||||
@ -481,7 +481,7 @@ DataTypePtr createConcreteEmptyDynamicColumn(const DataTypePtr & type_in_storage
|
||||
return recreateTupleWithElements(*type_tuple, new_elements);
|
||||
}
|
||||
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName());
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Type {} unexpectedly has dynamic columns", type_in_storage->getName());
|
||||
}
|
||||
|
||||
bool hasDynamicSubcolumns(const ColumnsDescription & columns)
|
||||
@ -613,7 +613,7 @@ DataTypePtr reduceNumberOfDimensions(DataTypePtr type, size_t dimensions_to_redu
|
||||
{
|
||||
const auto * type_array = typeid_cast<const DataTypeArray *>(type.get());
|
||||
if (!type_array)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not enough dimensions to reduce");
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Not enough dimensions to reduce");
|
||||
|
||||
type = type_array->getNestedType();
|
||||
}
|
||||
@ -627,7 +627,7 @@ ColumnPtr reduceNumberOfDimensions(ColumnPtr column, size_t dimensions_to_reduce
|
||||
{
|
||||
const auto * column_array = typeid_cast<const ColumnArray *>(column.get());
|
||||
if (!column_array)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not enough dimensions to reduce");
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Not enough dimensions to reduce");
|
||||
|
||||
column = column_array->getDataPtr();
|
||||
}
|
||||
@ -653,7 +653,7 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
|
||||
auto collect_tuple_elemets = [](const auto & children)
|
||||
{
|
||||
if (children.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot create type from empty Tuple or Nested node");
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Cannot create type from empty Tuple or Nested node");
|
||||
|
||||
std::vector<std::tuple<String, ColumnWithTypeAndDimensions>> tuple_elements;
|
||||
tuple_elements.reserve(children.size());
|
||||
@ -705,6 +705,7 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
|
||||
size_t num_elements = tuple_columns.size();
|
||||
Columns tuple_elements_columns(num_elements);
|
||||
DataTypes tuple_elements_types(num_elements);
|
||||
size_t last_offset = assert_cast<const ColumnArray::ColumnOffsets &>(*offsets_columns.back()).getData().back();
|
||||
|
||||
/// Reduce extra array dimensions to get columns and types of Nested elements.
|
||||
for (size_t i = 0; i < num_elements; ++i)
|
||||
@ -712,6 +713,14 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
|
||||
assert(tuple_columns[i].array_dimensions == tuple_columns[0].array_dimensions);
|
||||
tuple_elements_columns[i] = reduceNumberOfDimensions(tuple_columns[i].column, tuple_columns[i].array_dimensions);
|
||||
tuple_elements_types[i] = reduceNumberOfDimensions(tuple_columns[i].type, tuple_columns[i].array_dimensions);
|
||||
if (tuple_elements_columns[i]->size() != last_offset)
|
||||
throw Exception(
|
||||
ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
|
||||
"Cannot create a type for subcolumn {} in Object data type: offsets_column has data inconsistent with nested_column. "
|
||||
"Data size: {}, last offset: {}",
|
||||
node.path.getPath(),
|
||||
tuple_elements_columns[i]->size(),
|
||||
last_offset);
|
||||
}
|
||||
|
||||
auto result_column = ColumnArray::create(ColumnTuple::create(tuple_elements_columns), offsets_columns.back());
|
||||
@ -720,6 +729,16 @@ ColumnWithTypeAndDimensions createTypeFromNode(const Node & node)
|
||||
/// Recreate result Array type and Array column.
|
||||
for (auto it = offsets_columns.rbegin() + 1; it != offsets_columns.rend(); ++it)
|
||||
{
|
||||
last_offset = assert_cast<const ColumnArray::ColumnOffsets &>((**it)).getData().back();
|
||||
if (result_column->size() != last_offset)
|
||||
throw Exception(
|
||||
ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
|
||||
"Cannot create a type for subcolumn {} in Object data type: offsets_column has data inconsistent with nested_column. "
|
||||
"Data size: {}, last offset: {}",
|
||||
node.path.getPath(),
|
||||
result_column->size(),
|
||||
last_offset);
|
||||
|
||||
result_column = ColumnArray::create(result_column, *it);
|
||||
result_type = std::make_shared<DataTypeArray>(result_type);
|
||||
}
|
||||
@ -822,7 +841,7 @@ std::pair<ColumnPtr, DataTypePtr> unflattenTuple(
|
||||
assert(paths.size() == tuple_columns.size());
|
||||
|
||||
if (paths.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot unflatten empty Tuple");
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Cannot unflatten empty Tuple");
|
||||
|
||||
/// We add all paths to the subcolumn tree and then create a type from it.
|
||||
/// The tree stores column, type and number of array dimensions
|
||||
@ -841,7 +860,7 @@ std::pair<ColumnPtr, DataTypePtr> unflattenTuple(
|
||||
tree.add(paths[i], [&](Node::Kind kind, bool exists) -> std::shared_ptr<Node>
|
||||
{
|
||||
if (pos >= num_parts)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
|
||||
"Not enough name parts for path {}. Expected at least {}, got {}",
|
||||
paths[i].getPath(), pos + 1, num_parts);
|
||||
|
||||
|
@ -27,8 +27,8 @@ 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;
|
||||
extern const int CANNOT_PARSE_TEXT;
|
||||
extern const int EXPERIMENTAL_FEATURE_ERROR;
|
||||
}
|
||||
|
||||
template <typename Parser>
|
||||
@ -176,7 +176,7 @@ void SerializationObject<Parser>::serializeBinaryBulkStatePrefix(
|
||||
auto * stream = settings.getter(settings.path);
|
||||
|
||||
if (!stream)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Missing stream for kind of binary serialization");
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR, "Missing stream for kind of binary serialization");
|
||||
|
||||
auto [tuple_column, tuple_type] = unflattenObjectToTuple(column_object);
|
||||
|
||||
@ -287,7 +287,7 @@ void SerializationObject<Parser>::serializeBinaryBulkWithMultipleStreams(
|
||||
|
||||
if (!state_object->nested_type->equals(*tuple_type))
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
throw Exception(ErrorCodes::EXPERIMENTAL_FEATURE_ERROR,
|
||||
"Types of internal column of Object mismatched. Expected: {}, Got: {}",
|
||||
state_object->nested_type->getName(), tuple_type->getName());
|
||||
}
|
||||
|
@ -99,7 +99,6 @@ struct ReadSettings
|
||||
bool enable_filesystem_cache = true;
|
||||
bool read_from_filesystem_cache_if_exists_otherwise_bypass_cache = false;
|
||||
bool enable_filesystem_cache_log = false;
|
||||
bool force_read_through_cache_merges = false;
|
||||
size_t filesystem_cache_segments_batch_size = 20;
|
||||
|
||||
bool use_page_cache_for_disks_without_file_cache = false;
|
||||
|
@ -275,13 +275,7 @@ private:
|
||||
if (only_replace_current_database_function)
|
||||
return;
|
||||
|
||||
for (ASTRenameQuery::Element & elem : node.elements)
|
||||
{
|
||||
if (!elem.from.database)
|
||||
elem.from.database = std::make_shared<ASTIdentifier>(database_name);
|
||||
if (!elem.to.database)
|
||||
elem.to.database = std::make_shared<ASTIdentifier>(database_name);
|
||||
}
|
||||
node.setDatabaseIfNotExists(database_name);
|
||||
}
|
||||
|
||||
void visitDDL(ASTAlterQuery & node, ASTPtr &) const
|
||||
|
@ -1612,7 +1612,6 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
||||
executeTrivialBlockIO(fill_io, getContext());
|
||||
|
||||
/// Replace target table with created one
|
||||
auto ast_rename = std::make_shared<ASTRenameQuery>();
|
||||
ASTRenameQuery::Element elem
|
||||
{
|
||||
ASTRenameQuery::Table
|
||||
@ -1627,7 +1626,7 @@ BlockIO InterpreterCreateQuery::doCreateOrReplaceTable(ASTCreateQuery & create,
|
||||
}
|
||||
};
|
||||
|
||||
ast_rename->elements.push_back(std::move(elem));
|
||||
auto ast_rename = std::make_shared<ASTRenameQuery>(ASTRenameQuery::Elements{std::move(elem)});
|
||||
ast_rename->dictionary = create.is_dictionary;
|
||||
if (create.create_or_replace)
|
||||
{
|
||||
|
@ -47,12 +47,12 @@ BlockIO InterpreterRenameQuery::execute()
|
||||
*/
|
||||
|
||||
RenameDescriptions descriptions;
|
||||
descriptions.reserve(rename.elements.size());
|
||||
descriptions.reserve(rename.getElements().size());
|
||||
|
||||
/// Don't allow to drop tables (that we are renaming); don't allow to create tables in places where tables will be renamed.
|
||||
TableGuards table_guards;
|
||||
|
||||
for (const auto & elem : rename.elements)
|
||||
for (const auto & elem : rename.getElements())
|
||||
{
|
||||
descriptions.emplace_back(elem, current_database);
|
||||
const auto & description = descriptions.back();
|
||||
@ -186,7 +186,7 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRename
|
||||
{
|
||||
AccessRightsElements required_access;
|
||||
const auto & rename = query_ptr->as<const ASTRenameQuery &>();
|
||||
for (const auto & elem : rename.elements)
|
||||
for (const auto & elem : rename.getElements())
|
||||
{
|
||||
if (type == RenameType::RenameTable)
|
||||
{
|
||||
@ -214,7 +214,7 @@ AccessRightsElements InterpreterRenameQuery::getRequiredAccess(InterpreterRename
|
||||
void InterpreterRenameQuery::extendQueryLogElemImpl(QueryLogElement & elem, const ASTPtr & ast, ContextPtr) const
|
||||
{
|
||||
const auto & rename = ast->as<const ASTRenameQuery &>();
|
||||
for (const auto & element : rename.elements)
|
||||
for (const auto & element : rename.getElements())
|
||||
{
|
||||
{
|
||||
String database = backQuoteIfNeed(!element.from.database ? getContext()->getCurrentDatabase() : element.from.getDatabase());
|
||||
|
@ -579,7 +579,7 @@ ASTs InterpreterRenameImpl::getRewrittenQueries(
|
||||
const InterpreterRenameImpl::TQuery & rename_query, ContextPtr context, const String & mapped_to_database, const String & mysql_database)
|
||||
{
|
||||
ASTRenameQuery::Elements elements;
|
||||
for (const auto & rename_element : rename_query.elements)
|
||||
for (const auto & rename_element : rename_query.getElements())
|
||||
{
|
||||
const auto & to_database = resolveDatabase(rename_element.to.getDatabase(), mysql_database, mapped_to_database, context);
|
||||
const auto & from_database = resolveDatabase(rename_element.from.getDatabase(), mysql_database, mapped_to_database, context);
|
||||
@ -600,8 +600,7 @@ ASTs InterpreterRenameImpl::getRewrittenQueries(
|
||||
if (elements.empty())
|
||||
return ASTs{};
|
||||
|
||||
auto rewritten_query = std::make_shared<ASTRenameQuery>();
|
||||
rewritten_query->elements = elements;
|
||||
auto rewritten_query = std::make_shared<ASTRenameQuery>(std::move(elements));
|
||||
return ASTs{rewritten_query};
|
||||
}
|
||||
|
||||
@ -616,7 +615,8 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
return {};
|
||||
|
||||
auto rewritten_alter_query = std::make_shared<ASTAlterQuery>();
|
||||
auto rewritten_rename_query = std::make_shared<ASTRenameQuery>();
|
||||
ASTRenameQuery::Elements rename_elements;
|
||||
|
||||
rewritten_alter_query->setDatabase(mapped_to_database);
|
||||
rewritten_alter_query->setTable(alter_query.table);
|
||||
rewritten_alter_query->alter_object = ASTAlterQuery::AlterObjectType::TABLE;
|
||||
@ -749,13 +749,13 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
|
||||
/// For ALTER TABLE table_name RENAME TO new_table_name_1, RENAME TO new_table_name_2;
|
||||
/// We just need to generate RENAME TABLE table_name TO new_table_name_2;
|
||||
if (rewritten_rename_query->elements.empty())
|
||||
rewritten_rename_query->elements.push_back(ASTRenameQuery::Element());
|
||||
if (rename_elements.empty())
|
||||
rename_elements.push_back(ASTRenameQuery::Element());
|
||||
|
||||
rewritten_rename_query->elements.back().from.database = std::make_shared<ASTIdentifier>(mapped_to_database);
|
||||
rewritten_rename_query->elements.back().from.table = std::make_shared<ASTIdentifier>(alter_query.table);
|
||||
rewritten_rename_query->elements.back().to.database = std::make_shared<ASTIdentifier>(mapped_to_database);
|
||||
rewritten_rename_query->elements.back().to.table = std::make_shared<ASTIdentifier>(alter_command->new_table_name);
|
||||
rename_elements.back().from.database = std::make_shared<ASTIdentifier>(mapped_to_database);
|
||||
rename_elements.back().from.table = std::make_shared<ASTIdentifier>(alter_query.table);
|
||||
rename_elements.back().to.database = std::make_shared<ASTIdentifier>(mapped_to_database);
|
||||
rename_elements.back().to.table = std::make_shared<ASTIdentifier>(alter_command->new_table_name);
|
||||
}
|
||||
}
|
||||
|
||||
@ -765,8 +765,11 @@ ASTs InterpreterAlterImpl::getRewrittenQueries(
|
||||
if (!rewritten_alter_query->command_list->children.empty())
|
||||
rewritten_queries.push_back(rewritten_alter_query);
|
||||
|
||||
if (!rewritten_rename_query->elements.empty())
|
||||
if (!rename_elements.empty())
|
||||
{
|
||||
auto rewritten_rename_query = std::make_shared<ASTRenameQuery>(std::move(rename_elements));
|
||||
rewritten_queries.push_back(rewritten_rename_query);
|
||||
}
|
||||
|
||||
return rewritten_queries;
|
||||
}
|
||||
|
@ -563,7 +563,6 @@ void SystemLog<LogElement>::prepareTable()
|
||||
{table_id.database_name, table_id.table_name + "_" + toString(suffix)}, getContext()))
|
||||
++suffix;
|
||||
|
||||
auto rename = std::make_shared<ASTRenameQuery>();
|
||||
ASTRenameQuery::Element elem
|
||||
{
|
||||
ASTRenameQuery::Table
|
||||
@ -586,7 +585,7 @@ void SystemLog<LogElement>::prepareTable()
|
||||
old_create_query,
|
||||
create_query);
|
||||
|
||||
rename->elements.emplace_back(std::move(elem));
|
||||
auto rename = std::make_shared<ASTRenameQuery>(ASTRenameQuery::Elements{std::move(elem)});
|
||||
|
||||
ActionLock merges_lock;
|
||||
if (DatabaseCatalog::instance().getDatabase(table_id.database_name)->getUUID() == UUIDHelpers::Nil)
|
||||
|
@ -45,7 +45,6 @@ public:
|
||||
};
|
||||
|
||||
using Elements = std::vector<Element>;
|
||||
Elements elements;
|
||||
|
||||
bool exchange{false}; /// For EXCHANGE TABLES
|
||||
bool database{false}; /// For RENAME DATABASE
|
||||
@ -54,12 +53,48 @@ public:
|
||||
/// Special flag for CREATE OR REPLACE. Do not throw if the second table does not exist.
|
||||
bool rename_if_cannot_exchange{false};
|
||||
|
||||
explicit ASTRenameQuery(Elements elements_ = {})
|
||||
: elements(std::move(elements_))
|
||||
{
|
||||
for (const auto & elem : elements)
|
||||
{
|
||||
if (elem.from.database)
|
||||
children.push_back(elem.from.database);
|
||||
if (elem.from.table)
|
||||
children.push_back(elem.from.table);
|
||||
if (elem.to.database)
|
||||
children.push_back(elem.to.database);
|
||||
if (elem.to.table)
|
||||
children.push_back(elem.to.table);
|
||||
}
|
||||
}
|
||||
|
||||
void setDatabaseIfNotExists(const String & database_name)
|
||||
{
|
||||
for (auto & elem : elements)
|
||||
{
|
||||
if (!elem.from.database)
|
||||
{
|
||||
elem.from.database = std::make_shared<ASTIdentifier>(database_name);
|
||||
children.push_back(elem.from.database);
|
||||
}
|
||||
if (!elem.to.database)
|
||||
{
|
||||
elem.to.database = std::make_shared<ASTIdentifier>(database_name);
|
||||
children.push_back(elem.to.database);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
const Elements & getElements() const { return elements; }
|
||||
|
||||
/** Get the text that identifies this element. */
|
||||
String getID(char) const override { return "Rename"; }
|
||||
|
||||
ASTPtr clone() const override
|
||||
{
|
||||
auto res = std::make_shared<ASTRenameQuery>(*this);
|
||||
res->cloneChildren();
|
||||
cloneOutputOptions(*res);
|
||||
return res;
|
||||
}
|
||||
@ -145,6 +180,8 @@ protected:
|
||||
|
||||
formatOnCluster(settings);
|
||||
}
|
||||
|
||||
Elements elements;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -44,15 +44,14 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
if (!ASTQueryWithOnCluster::parse(pos, cluster_str, expected))
|
||||
return false;
|
||||
}
|
||||
ASTRenameQuery::Elements rename_elements;
|
||||
rename_elements.emplace_back();
|
||||
rename_elements.back().if_exists = if_exists;
|
||||
rename_elements.back().from.database = from_db;
|
||||
rename_elements.back().to.database = to_db;
|
||||
|
||||
auto query = std::make_shared<ASTRenameQuery>();
|
||||
auto query = std::make_shared<ASTRenameQuery>(std::move(rename_elements));
|
||||
query->database = true;
|
||||
query->elements.emplace({});
|
||||
query->elements.front().if_exists = if_exists;
|
||||
query->elements.front().from.database = from_db;
|
||||
query->elements.front().to.database = to_db;
|
||||
query->children.push_back(std::move(from_db));
|
||||
query->children.push_back(std::move(to_db));
|
||||
query->cluster = cluster_str;
|
||||
node = query;
|
||||
return true;
|
||||
@ -75,9 +74,8 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|
||||
const auto ignore_delim = [&] { return exchange ? s_and.ignore(pos) : s_to.ignore(pos); };
|
||||
|
||||
auto query = std::make_shared<ASTRenameQuery>();
|
||||
|
||||
ASTRenameQuery::Elements & elements = query->elements;
|
||||
ASTRenameQuery::Elements elements;
|
||||
|
||||
while (true)
|
||||
{
|
||||
@ -93,15 +91,6 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
|| !ignore_delim()
|
||||
|| !parseDatabaseAndTableAsAST(pos, expected, ref.to.database, ref.to.table))
|
||||
return false;
|
||||
|
||||
if (ref.from.database)
|
||||
query->children.push_back(ref.from.database);
|
||||
if (ref.from.table)
|
||||
query->children.push_back(ref.from.table);
|
||||
if (ref.to.database)
|
||||
query->children.push_back(ref.to.database);
|
||||
if (ref.to.table)
|
||||
query->children.push_back(ref.to.table);
|
||||
}
|
||||
|
||||
String cluster_str;
|
||||
@ -111,6 +100,7 @@ bool ParserRenameQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
return false;
|
||||
}
|
||||
|
||||
auto query = std::make_shared<ASTRenameQuery>(std::move(elements));
|
||||
query->cluster = cluster_str;
|
||||
query->exchange = exchange;
|
||||
query->dictionary = dictionary;
|
||||
|
@ -9,6 +9,7 @@
|
||||
#include <Parsers/ParserAlterQuery.h>
|
||||
#include <Parsers/ParserCreateQuery.h>
|
||||
#include <Parsers/ParserOptimizeQuery.h>
|
||||
#include <Parsers/ParserRenameQuery.h>
|
||||
#include <Parsers/ParserQueryWithOutput.h>
|
||||
#include <Parsers/ParserAttachAccessEntity.h>
|
||||
#include <Parsers/formatAST.h>
|
||||
@ -61,12 +62,31 @@ TEST_P(ParserTest, parseQuery)
|
||||
ASSERT_NO_THROW(ast = parseQuery(*parser, input_text.begin(), input_text.end(), 0, 0));
|
||||
if (std::string("CREATE USER or ALTER USER query") != parser->getName()
|
||||
&& std::string("ATTACH access entity query") != parser->getName())
|
||||
{
|
||||
ASTPtr ast_clone = ast->clone();
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*ast->clone(), buf, false, false);
|
||||
formatAST(*ast_clone, buf, false, false);
|
||||
String formatted_ast = buf.str();
|
||||
EXPECT_EQ(expected_ast, formatted_ast);
|
||||
}
|
||||
|
||||
|
||||
ASTPtr ast_clone2 = ast_clone->clone();
|
||||
/// Break `ast_clone2`, it should not affect `ast_clone` if `clone()` implemented properly
|
||||
for (auto & child : ast_clone2->children)
|
||||
{
|
||||
if (auto * identifier = dynamic_cast<ASTIdentifier *>(child.get()))
|
||||
identifier->setShortName("new_name");
|
||||
}
|
||||
|
||||
{
|
||||
WriteBufferFromOwnString buf;
|
||||
formatAST(*ast_clone, buf, false, false);
|
||||
String formatted_ast = buf.str();
|
||||
EXPECT_EQ(expected_ast, formatted_ast);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
if (input_text.starts_with("ATTACH"))
|
||||
@ -299,6 +319,16 @@ INSTANTIATE_TEST_SUITE_P(ParserAttachUserQuery, ParserTest,
|
||||
}
|
||||
})));
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(ParserRenameQuery, ParserTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(std::make_shared<ParserRenameQuery>()),
|
||||
::testing::ValuesIn(std::initializer_list<ParserTestCase>{
|
||||
{
|
||||
"RENAME TABLE eligible_test TO eligible_test2",
|
||||
"RENAME TABLE eligible_test TO eligible_test2"
|
||||
}
|
||||
})));
|
||||
|
||||
INSTANTIATE_TEST_SUITE_P(ParserKQLQuery, ParserKQLTest,
|
||||
::testing::Combine(
|
||||
::testing::Values(std::make_shared<ParserKQLQuery>()),
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Processors/Port.h>
|
||||
#include <Processors/IProcessor.h>
|
||||
#include <Common/SharedMutex.h>
|
||||
#include <Common/AllocatorWithMemoryTracking.h>
|
||||
#include <mutex>
|
||||
#include <queue>
|
||||
#include <stack>
|
||||
@ -117,7 +118,11 @@ public:
|
||||
}
|
||||
};
|
||||
|
||||
using Queue = std::queue<Node *>;
|
||||
/// This queue can grow a lot and lead to OOM. That is why we use non-default
|
||||
/// allocator for container which throws exceptions in operator new
|
||||
using DequeWithMemoryTracker = std::deque<ExecutingGraph::Node *, AllocatorWithMemoryTracking<ExecutingGraph::Node *>>;
|
||||
using Queue = std::queue<ExecutingGraph::Node *, DequeWithMemoryTracker>;
|
||||
|
||||
using NodePtr = std::unique_ptr<Node>;
|
||||
using Nodes = std::vector<NodePtr>;
|
||||
Nodes nodes;
|
||||
|
@ -47,7 +47,10 @@ class ExecutorTasks
|
||||
|
||||
public:
|
||||
using Stack = std::stack<UInt64>;
|
||||
using Queue = std::queue<ExecutingGraph::Node *>;
|
||||
/// This queue can grow a lot and lead to OOM. That is why we use non-default
|
||||
/// allocator for container which throws exceptions in operator new
|
||||
using DequeWithMemoryTracker = std::deque<ExecutingGraph::Node *, AllocatorWithMemoryTracking<ExecutingGraph::Node *>>;
|
||||
using Queue = std::queue<ExecutingGraph::Node *, DequeWithMemoryTracker>;
|
||||
|
||||
void finish();
|
||||
bool isFinished() const { return finished; }
|
||||
|
@ -5,7 +5,9 @@
|
||||
#include <Common/EventCounter.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Common/ConcurrencyControl.h>
|
||||
#include <Common/AllocatorWithMemoryTracking.h>
|
||||
|
||||
#include <deque>
|
||||
#include <queue>
|
||||
#include <mutex>
|
||||
#include <memory>
|
||||
@ -90,7 +92,10 @@ private:
|
||||
|
||||
ReadProgressCallbackPtr read_progress_callback;
|
||||
|
||||
using Queue = std::queue<ExecutingGraph::Node *>;
|
||||
/// This queue can grow a lot and lead to OOM. That is why we use non-default
|
||||
/// allocator for container which throws exceptions in operator new
|
||||
using DequeWithMemoryTracker = std::deque<ExecutingGraph::Node *, AllocatorWithMemoryTracking<ExecutingGraph::Node *>>;
|
||||
using Queue = std::queue<ExecutingGraph::Node *, DequeWithMemoryTracker>;
|
||||
|
||||
void initializeExecution(size_t num_threads, bool concurrency_control); /// Initialize executor contexts and task_queue.
|
||||
void finalizeExecution(); /// Check all processors are finished.
|
||||
|
@ -465,8 +465,8 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id)
|
||||
if (!from_atomic_to_atomic_database && has_inner_table && tryGetTargetTable())
|
||||
{
|
||||
auto new_target_table_name = generateInnerTableName(new_table_id);
|
||||
auto rename = std::make_shared<ASTRenameQuery>();
|
||||
|
||||
ASTRenameQuery::Elements rename_elements;
|
||||
assert(inner_table_id.database_name == old_table_id.database_name);
|
||||
|
||||
ASTRenameQuery::Element elem
|
||||
@ -482,8 +482,9 @@ void StorageMaterializedView::renameInMemory(const StorageID & new_table_id)
|
||||
std::make_shared<ASTIdentifier>(new_target_table_name)
|
||||
}
|
||||
};
|
||||
rename->elements.emplace_back(std::move(elem));
|
||||
rename_elements.emplace_back(std::move(elem));
|
||||
|
||||
auto rename = std::make_shared<ASTRenameQuery>(std::move(rename_elements));
|
||||
InterpreterRenameQuery(rename, getContext()).execute();
|
||||
updateTargetTableId(new_table_id.database_name, new_target_table_name);
|
||||
}
|
||||
|
@ -51,7 +51,7 @@ class Queue:
|
||||
label: str
|
||||
|
||||
|
||||
def get_scales() -> Tuple[int, int]:
|
||||
def get_scales(runner_type: str) -> Tuple[int, int]:
|
||||
"returns the multipliers for scaling down and up ASG by types"
|
||||
# Scaling down is quicker on the lack of running jobs than scaling up on
|
||||
# queue
|
||||
@ -63,8 +63,12 @@ def get_scales() -> Tuple[int, int]:
|
||||
# 10. I am trying 7 now.
|
||||
# 7 still looks a bit slow, so I try 6
|
||||
# Let's have it the same as the other ASG
|
||||
#
|
||||
# All type of style-checkers should be added very quickly to not block the workflows
|
||||
# UPDATE THE COMMENT ON CHANGES
|
||||
scale_up = 3
|
||||
if "style" in runner_type:
|
||||
scale_up = 1
|
||||
return scale_down, scale_up
|
||||
|
||||
|
||||
@ -95,7 +99,7 @@ def set_capacity(
|
||||
continue
|
||||
raise ValueError("Queue status is not in ['in_progress', 'queued']")
|
||||
|
||||
scale_down, scale_up = get_scales()
|
||||
scale_down, scale_up = get_scales(runner_type)
|
||||
# With lyfecycle hooks some instances are actually free because some of
|
||||
# them are in 'Terminating:Wait' state
|
||||
effective_capacity = max(
|
||||
@ -138,7 +142,7 @@ def set_capacity(
|
||||
|
||||
logging.info(
|
||||
"The ASG %s capacity will be increased to %s, current capacity=%s, "
|
||||
"effective capacity=%sm maximum capacity=%s, running jobs=%s, queue size=%s",
|
||||
"effective capacity=%s, maximum capacity=%s, running jobs=%s, queue size=%s",
|
||||
asg["AutoScalingGroupName"],
|
||||
desired_capacity,
|
||||
effective_capacity,
|
||||
|
@ -80,7 +80,7 @@ class TestSetCapacity(unittest.TestCase):
|
||||
),
|
||||
TestCase("increase-1", 1, 13, 20, [Queue("queued", 23, "increase-1")], 17),
|
||||
TestCase(
|
||||
"style-checker", 1, 13, 20, [Queue("queued", 33, "style-checker")], 20
|
||||
"style-checker", 1, 13, 20, [Queue("queued", 19, "style-checker")], 19
|
||||
),
|
||||
TestCase("increase-2", 1, 13, 20, [Queue("queued", 18, "increase-2")], 15),
|
||||
TestCase("increase-3", 1, 13, 20, [Queue("queued", 183, "increase-3")], 20),
|
||||
|
@ -629,7 +629,9 @@ class CIConfig:
|
||||
|
||||
assert result, f"BUG, no runner for [{check_name}]"
|
||||
|
||||
if ("aarch" in check_name or "arm" in check_name) and "aarch" not in result:
|
||||
if (
|
||||
"aarch" in check_name.lower() or "arm64" in check_name.lower()
|
||||
) and "aarch" not in result:
|
||||
if result == Runners.STRESS_TESTER:
|
||||
# FIXME: no arm stress tester group atm
|
||||
result = Runners.FUNC_TESTER_ARM
|
||||
|
@ -67,7 +67,7 @@ def test_distributed_background_insert_split_batch_on_failure_OFF(started_cluste
|
||||
f"insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}",
|
||||
settings={
|
||||
# max_memory_usage is the limit for the batch on the remote node
|
||||
# (local query should not be affected since 30MB is enough for 100K rows)
|
||||
# (local query should not be affected since 20MB is enough for 100K rows)
|
||||
"max_memory_usage": "20Mi",
|
||||
"max_untracked_memory": "0",
|
||||
},
|
||||
@ -76,7 +76,8 @@ def test_distributed_background_insert_split_batch_on_failure_OFF(started_cluste
|
||||
if setting == "background_insert_batch" and setting_value == 1:
|
||||
with pytest.raises(
|
||||
QueryRuntimeException,
|
||||
match=r"DB::Exception: Received from.*Memory limit \(for query\) exceeded: .*while pushing to view default\.mv",
|
||||
# no DOTALL in pytest.raises, use '(.|\n)'
|
||||
match=r"DB::Exception: Received from.*Memory limit \(for query\) exceeded: (.|\n)*While sending a batch",
|
||||
):
|
||||
node2.query("system flush distributed dist")
|
||||
assert int(node2.query("select count() from dist_data")) == 0
|
||||
@ -94,8 +95,8 @@ def test_distributed_background_insert_split_batch_on_failure_ON(started_cluster
|
||||
f"insert into dist select number/100, number from system.numbers limit {limit} offset {limit*i}",
|
||||
settings={
|
||||
# max_memory_usage is the limit for the batch on the remote node
|
||||
# (local query should not be affected since 30MB is enough for 100K rows)
|
||||
"max_memory_usage": "30Mi",
|
||||
# (local query should not be affected since 20MB is enough for 100K rows)
|
||||
"max_memory_usage": "20Mi",
|
||||
"max_untracked_memory": "0",
|
||||
},
|
||||
)
|
||||
|
@ -2,7 +2,7 @@ DROP TABLE IF EXISTS ttl_group_by_bug;
|
||||
|
||||
CREATE TABLE ttl_group_by_bug
|
||||
(key UInt32, ts DateTime, value UInt32, min_value UInt32 default value, max_value UInt32 default value)
|
||||
ENGINE = MergeTree() PARTITION BY toYYYYMM(ts)
|
||||
ENGINE = MergeTree()
|
||||
ORDER BY (key, toStartOfInterval(ts, toIntervalMinute(3)), ts)
|
||||
TTL ts + INTERVAL 5 MINUTE GROUP BY key, toStartOfInterval(ts, toIntervalMinute(3))
|
||||
SET value = sum(value), min_value = min(min_value), max_value = max(max_value), ts=min(toStartOfInterval(ts, toIntervalMinute(3)));
|
||||
|
Loading…
Reference in New Issue
Block a user