Merge branch 'master' into pr-local-plan

This commit is contained in:
Igor Nikonov 2024-09-05 16:19:47 +02:00 committed by GitHub
commit 536e0808e3
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
90 changed files with 2111 additions and 841 deletions

View File

@ -66,13 +66,11 @@ TRAP(gethostbyname)
TRAP(gethostbyname2)
TRAP(gethostent)
TRAP(getlogin)
TRAP(getmntent)
TRAP(getnetbyaddr)
TRAP(getnetbyname)
TRAP(getnetent)
TRAP(getnetgrent)
TRAP(getnetgrent_r)
TRAP(getopt)
TRAP(getopt_long)
TRAP(getopt_long_only)
TRAP(getpass)
@ -133,7 +131,6 @@ TRAP(nrand48)
TRAP(__ppc_get_timebase_freq)
TRAP(ptsname)
TRAP(putchar_unlocked)
TRAP(putenv)
TRAP(pututline)
TRAP(pututxline)
TRAP(putwchar_unlocked)
@ -148,7 +145,6 @@ TRAP(sethostent)
TRAP(sethostid)
TRAP(setkey)
//TRAP(setlocale) // Used by replxx at startup
TRAP(setlogmask)
TRAP(setnetent)
TRAP(setnetgrent)
TRAP(setprotoent)
@ -203,7 +199,6 @@ TRAP(lgammal)
TRAP(nftw)
TRAP(nl_langinfo)
TRAP(putc_unlocked)
TRAP(rand)
/** In the current POSIX.1 specification (POSIX.1-2008), readdir() is not required to be thread-safe. However, in modern
* implementations (including the glibc implementation), concurrent calls to readdir() that specify different directory streams
* are thread-safe. In cases where multiple threads must read from the same directory stream, using readdir() with external
@ -288,4 +283,14 @@ TRAP(tss_get)
TRAP(tss_set)
TRAP(tss_delete)
#ifndef USE_MUSL
/// These produce duplicate symbol errors when statically linking with musl.
/// Maybe we can remove them from the musl fork.
TRAP(getopt)
TRAP(putenv)
TRAP(setlogmask)
TRAP(rand)
TRAP(getmntent)
#endif
#endif

View File

@ -48,25 +48,17 @@ std::string PathImpl::currentImpl()
std::string PathImpl::homeImpl()
{
std::string path;
#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE)
size_t buf_size = 1024; // Same as glibc use for getpwuid
std::vector<char> buf(buf_size);
struct passwd res;
struct passwd* pwd = nullptr;
getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd);
#else
struct passwd* pwd = getpwuid(getuid());
#endif
if (pwd)
path = pwd->pw_dir;
else
{
#if defined(_POSIX_C_SOURCE) || defined(_BSD_SOURCE) || defined(_POSIX_C_SOURCE)
getpwuid_r(getuid(), &res, buf.data(), buf_size, &pwd);
#else
pwd = getpwuid(geteuid());
#endif
if (pwd)
path = pwd->pw_dir;
else

View File

@ -140,6 +140,12 @@ if (CMAKE_CROSSCOMPILING)
message (STATUS "CROSS COMPILING SET LLVM HOST TRIPLE ${LLVM_HOST_TRIPLE}")
endif()
# llvm-project/llvm/cmake/config-ix.cmake does a weird thing: it defines _LARGEFILE64_SOURCE,
# then checks if lseek64() function exists, then undefines _LARGEFILE64_SOURCE.
# Then the actual code that uses this function *doesn't* define _LARGEFILE64_SOURCE, so lseek64()
# may not exist and compilation fails. This happens with musl.
add_compile_definitions("_LARGEFILE64_SOURCE")
add_subdirectory ("${LLVM_SOURCE_DIR}" "${LLVM_BINARY_DIR}")
set_directory_properties (PROPERTIES

2
contrib/sysroot vendored

@ -1 +1 @@
Subproject commit cc385041b226d1fc28ead14dbab5d40a5f821dd8
Subproject commit 5be834147d5b5dd77ca2b821f356982029320513

View File

@ -111,15 +111,16 @@ ANN indexes are built during column insertion and merge. As a result, `INSERT` a
tables. ANNIndexes are ideally used only with immutable or rarely changed data, respectively when are far more read requests than write
requests.
ANN indexes support these queries:
ANN indexes support this type of query:
``` sql
SELECT *
FROM table
[WHERE ...]
ORDER BY Distance(vectors, Point)
LIMIT N
```
``` sql
WITH [...] AS reference_vector
SELECT *
FROM table
WHERE ... -- WHERE clause is optional
ORDER BY Distance(vectors, reference_vector)
LIMIT N
```
:::tip
To avoid writing out large vectors, you can use [query

View File

@ -3226,7 +3226,7 @@ Default value: `0`.
## lightweight_deletes_sync {#lightweight_deletes_sync}
The same as 'mutation_sync', but controls only execution of lightweight deletes.
The same as [`mutations_sync`](#mutations_sync), but controls only execution of lightweight deletes.
Possible values:

View File

@ -2035,6 +2035,7 @@ Query:
SELECT arrayZip(['a', 'b', 'c'], [5, 2, 1]);
```
Result:
``` text
@ -2043,6 +2044,43 @@ Result:
└──────────────────────────────────────┘
```
## arrayZipUnaligned
Combines multiple arrays into a single array, allowing for unaligned arrays. The resulting array contains the corresponding elements of the source arrays grouped into tuples in the listed order of arguments.
**Syntax**
``` sql
arrayZipUnaligned(arr1, arr2, ..., arrN)
```
**Arguments**
- `arrN` — [Array](../data-types/array.md).
The function can take any number of arrays of different types.
**Returned value**
- Array with elements from the source arrays grouped into [tuples](../data-types/tuple.md). Data types in the tuple are the same as types of the input arrays and in the same order as arrays are passed. [Array](../data-types/array.md). If the arrays have different sizes, the shorter arrays will be padded with `null` values.
**Example**
Query:
``` sql
SELECT arrayZipUnaligned(['a'], [1, 2, 3]);
```
Result:
``` text
┌─arrayZipUnaligned(['a'], [1, 2, 3])─┐
│ [('a',1),(NULL,2),(NULL,3)] │
└─────────────────────────────────────┘
```
## arrayAUC
Calculate AUC (Area Under the Curve, which is a concept in machine learning, see more details: <https://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve>).

View File

@ -265,8 +265,6 @@ SELECT now() AS current_date_time, current_date_time + INTERVAL '4' day + INTERV
└─────────────────────┴────────────────────────────────────────────────────────────┘
```
You can work with dates without using `INTERVAL`, just by adding or subtracting seconds, minutes, and hours. For example, an interval of one day can be set by adding `60*60*24`.
:::note
The `INTERVAL` syntax or `addDays` function are always preferred. Simple addition or subtraction (syntax like `now() + ...`) doesn't consider time settings. For example, daylight saving time.
:::

View File

@ -13,7 +13,7 @@ The lightweight `DELETE` statement removes rows from the table `[db.]table` that
DELETE FROM [db.]table [ON CLUSTER cluster] [IN PARTITION partition_expr] WHERE expr;
```
It is called "lightweight `DELETE`" to contrast it to the [ALTER table DELETE](/en/sql-reference/statements/alter/delete) command, which is a heavyweight process.
It is called "lightweight `DELETE`" to contrast it to the [ALTER TABLE ... DELETE](/en/sql-reference/statements/alter/delete) command, which is a heavyweight process.
## Examples
@ -22,23 +22,25 @@ It is called "lightweight `DELETE`" to contrast it to the [ALTER table DELETE](/
DELETE FROM hits WHERE Title LIKE '%hello%';
```
## Lightweight `DELETE` does not delete data from storage immediately
## Lightweight `DELETE` does not delete data immediately
With lightweight `DELETE`, deleted rows are internally marked as deleted immediately and will be automatically filtered out of all subsequent queries. However, cleanup of data happens during the next merge. As a result, it is possible that for an unspecified period, data is not actually deleted from storage and is only marked as deleted.
Lightweight `DELETE` is implemented as a [mutation](/en/sql-reference/statements/alter#mutations), which is executed asynchronously in the background by default. The statement is going to return almost immediately, but the data can still be visible to queries until the mutation is finished.
If you need to guarantee that your data is deleted from storage in a predictable time, consider using the [ALTER table DELETE](/en/sql-reference/statements/alter/delete) command. Note that deleting data using `ALTER table DELETE` may consume significant resources as it recreates all affected parts.
The mutation marks rows as deleted, and at that point, they will no longer show up in query results. It does not physically delete the data, this will happen during the next merge. As a result, it is possible that for an unspecified period, data is not actually deleted from storage and is only marked as deleted.
If you need to guarantee that your data is deleted from storage in a predictable time, consider using the table setting [`min_age_to_force_merge_seconds`](https://clickhouse.com/docs/en/operations/settings/merge-tree-settings#min_age_to_force_merge_seconds). Or you can use the [ALTER TABLE ... DELETE](/en/sql-reference/statements/alter/delete) command. Note that deleting data using `ALTER TABLE ... DELETE` may consume significant resources as it recreates all affected parts.
## Deleting large amounts of data
Large deletes can negatively affect ClickHouse performance. If you are attempting to delete all rows from a table, consider using the [`TRUNCATE TABLE`](/en/sql-reference/statements/truncate) command.
If you anticipate frequent deletes, consider using a [custom partitioning key](/en/engines/table-engines/mergetree-family/custom-partitioning-key). You can then use the [`ALTER TABLE...DROP PARTITION`](/en/sql-reference/statements/alter/partition#drop-partitionpart) command to quickly drop all rows associated with that partition.
If you anticipate frequent deletes, consider using a [custom partitioning key](/en/engines/table-engines/mergetree-family/custom-partitioning-key). You can then use the [`ALTER TABLE ... DROP PARTITION`](/en/sql-reference/statements/alter/partition#drop-partitionpart) command to quickly drop all rows associated with that partition.
## Limitations of lightweight `DELETE`
### Lightweight `DELETE`s with projections
By default, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation. But there is a [MergeTree setting](https://clickhouse.com/docs/en/operations/settings/merge-tree-settings) `lightweight_mutation_projection_mode` can change the behavior.
By default, `DELETE` does not work for tables with projections. This is because rows in a projection may be affected by a `DELETE` operation. But there is a [MergeTree setting](https://clickhouse.com/docs/en/operations/settings/merge-tree-settings) `lightweight_mutation_projection_mode` to change the behavior.
## Performance considerations when using lightweight `DELETE`
@ -48,7 +50,7 @@ The following can also negatively impact lightweight `DELETE` performance:
- A heavy `WHERE` condition in a `DELETE` query.
- If the mutations queue is filled with many other mutations, this can possibly lead to performance issues as all mutations on a table are executed sequentially.
- The affected table having a very large number of data parts.
- The affected table has a very large number of data parts.
- Having a lot of data in compact parts. In a Compact part, all columns are stored in one file.
## Delete permissions
@ -61,31 +63,31 @@ GRANT ALTER DELETE ON db.table to username;
## How lightweight DELETEs work internally in ClickHouse
1. A "mask" is applied to affected rows
1. **A "mask" is applied to affected rows**
When a `DELETE FROM table ...` query is executed, ClickHouse saves a mask where each row is marked as either “existing” or as “deleted”. Those “deleted” rows are omitted for subsequent queries. However, rows are actually only removed later by subsequent merges. Writing this mask is much more lightweight than what is done by an `ALTER table DELETE` query.
When a `DELETE FROM table ...` query is executed, ClickHouse saves a mask where each row is marked as either “existing” or as “deleted”. Those “deleted” rows are omitted for subsequent queries. However, rows are actually only removed later by subsequent merges. Writing this mask is much more lightweight than what is done by an `ALTER TABLE ... DELETE` query.
The mask is implemented as a hidden `_row_exists` system column that stores `True` for all visible rows and `False` for deleted ones. This column is only present in a part if some rows in the part were deleted. This column does not exist when a part has all values equal to `True`.
The mask is implemented as a hidden `_row_exists` system column that stores `True` for all visible rows and `False` for deleted ones. This column is only present in a part if some rows in the part were deleted. This column does not exist when a part has all values equal to `True`.
2. `SELECT` queries are transformed to include the mask
2. **`SELECT` queries are transformed to include the mask**
When a masked column is used in a query, the `SELECT ... FROM table WHERE condition` query internally is extended by the predicate on `_row_exists` and is transformed to:
```sql
SELECT ... FROM table PREWHERE _row_exists WHERE condition
```
At execution time, the column `_row_exists` is read to determine which rows should not be returned. If there are many deleted rows, ClickHouse can determine which granules can be fully skipped when reading the rest of the columns.
When a masked column is used in a query, the `SELECT ... FROM table WHERE condition` query internally is extended by the predicate on `_row_exists` and is transformed to:
```sql
SELECT ... FROM table PREWHERE _row_exists WHERE condition
```
At execution time, the column `_row_exists` is read to determine which rows should not be returned. If there are many deleted rows, ClickHouse can determine which granules can be fully skipped when reading the rest of the columns.
3. `DELETE` queries are transformed to `ALTER table UPDATE` queries
3. **`DELETE` queries are transformed to `ALTER TABLE ... UPDATE` queries**
The `DELETE FROM table WHERE condition` is translated into an `ALTER table UPDATE _row_exists = 0 WHERE condition` mutation.
The `DELETE FROM table WHERE condition` is translated into an `ALTER TABLE table UPDATE _row_exists = 0 WHERE condition` mutation.
Internally, this mutation is executed in two steps:
Internally, this mutation is executed in two steps:
1. A `SELECT count() FROM table WHERE condition` command is executed for each individual part to determine if the part is affected.
1. A `SELECT count() FROM table WHERE condition` command is executed for each individual part to determine if the part is affected.
2. Based on the commands above, affected parts are then mutated, and hardlinks are created for unaffected parts. In the case of wide parts, the `_row_exists` column for each row is updated and all other columns' files are hardlinked. For compact parts, all columns are re-written because they are all stored together in one file.
2. Based on the commands above, affected parts are then mutated, and hardlinks are created for unaffected parts. In the case of wide parts, the `_row_exists` column for each row is updated, and all other columns' files are hardlinked. For compact parts, all columns are re-written because they are all stored together in one file.
From the steps above, we can see that lightweight deletes using the masking technique improves performance over traditional `ALTER table DELETE` commands because `ALTER table DELETE` reads and re-writes all the columns' files for affected parts.
From the steps above, we can see that lightweight `DELETE` using the masking technique improves performance over traditional `ALTER TABLE ... DELETE` because it does not re-write all the columns' files for affected parts.
## Related content

View File

@ -302,8 +302,12 @@ DataTypePtr tryInferDataTypeByEscapingRule(const String & field, const FormatSet
/// Try to determine the type of value inside quotes
auto type = tryInferDataTypeForSingleField(data, format_settings);
/// If we couldn't infer any type or it's a number and csv.try_infer_numbers_from_strings = 0, we determine it as a string.
if (!type || (format_settings.csv.try_infer_strings_from_quoted_tuples && isTuple(type)) || (!format_settings.csv.try_infer_numbers_from_strings && isNumber(type)))
/// Return String type if one of the following conditions apply
/// - we couldn't infer any type
/// - it's a number and csv.try_infer_numbers_from_strings = 0
/// - it's a tuple and try_infer_strings_from_quoted_tuples = 0
/// - it's a Bool type (we don't allow reading bool values from strings)
if (!type || (format_settings.csv.try_infer_strings_from_quoted_tuples && isTuple(type)) || (!format_settings.csv.try_infer_numbers_from_strings && isNumber(type)) || isBool(type))
return std::make_shared<DataTypeString>();
return type;

View File

@ -4134,6 +4134,29 @@ private:
};
}
/// Create wrapper only if we support this conversion.
WrapperType createWrapperIfCanConvert(const DataTypePtr & from, const DataTypePtr & to) const
{
try
{
/// We can avoid try/catch here if we will implement check that 2 types can be casted, but it
/// requires quite a lot of work. By now let's simply use try/catch.
/// First, check that we can create a wrapper.
WrapperType wrapper = prepareUnpackDictionaries(from, to);
/// Second, check if we can perform a conversion on column with default value.
/// (we cannot just check empty column as we do some checks only during iteration over rows).
auto test_col = from->createColumn();
test_col->insertDefault();
ColumnsWithTypeAndName column_from = {{test_col->getPtr(), from, "" }};
wrapper(column_from, to, nullptr, 1);
return wrapper;
}
catch (const Exception &)
{
return {};
}
}
WrapperType createVariantToColumnWrapper(const DataTypeVariant & from_variant, const DataTypePtr & to_type) const
{
const auto & variant_types = from_variant.getVariants();
@ -4142,7 +4165,19 @@ private:
/// Create conversion wrapper for each variant.
for (const auto & variant_type : variant_types)
variant_wrappers.push_back(prepareUnpackDictionaries(variant_type, to_type));
{
WrapperType wrapper;
if (cast_type == CastType::accurateOrNull)
{
/// Create wrapper only if we support conversion from variant to the resulting type.
wrapper = createWrapperIfCanConvert(variant_type, to_type);
}
else
{
wrapper = prepareUnpackDictionaries(variant_type, to_type);
}
variant_wrappers.push_back(wrapper);
}
return [variant_wrappers, variant_types, to_type]
(ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, const ColumnNullable *, size_t input_rows_count) -> ColumnPtr
@ -4157,7 +4192,11 @@ private:
auto variant_col = column_variant.getVariantPtrByGlobalDiscriminator(i);
ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], "" }};
const auto & variant_wrapper = variant_wrappers[i];
casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size()));
ColumnPtr casted_variant;
/// Check if we have wrapper for this variant.
if (variant_wrapper)
casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size());
casted_variant_columns.push_back(std::move(casted_variant));
}
/// Second, construct resulting column from casted variant columns according to discriminators.
@ -4167,7 +4206,7 @@ private:
for (size_t i = 0; i != input_rows_count; ++i)
{
auto global_discr = column_variant.globalDiscriminatorByLocal(local_discriminators[i]);
if (global_discr == ColumnVariant::NULL_DISCRIMINATOR)
if (global_discr == ColumnVariant::NULL_DISCRIMINATOR || !casted_variant_columns[global_discr])
res->insertDefault();
else
res->insertFrom(*casted_variant_columns[global_discr], column_variant.offsetAt(i));
@ -4357,10 +4396,27 @@ private:
casted_variant_columns.reserve(variant_types.size());
for (size_t i = 0; i != variant_types.size(); ++i)
{
/// Skip shared variant, it will be processed later.
if (i == column_dynamic.getSharedVariantDiscriminator())
{
casted_variant_columns.push_back(nullptr);
continue;
}
const auto & variant_col = variant_column.getVariantPtrByGlobalDiscriminator(i);
ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], ""}};
auto variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type);
casted_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_col->size()));
WrapperType variant_wrapper;
if (cast_type == CastType::accurateOrNull)
/// Create wrapper only if we support conversion from variant to the resulting type.
variant_wrapper = createWrapperIfCanConvert(variant_types[i], result_type);
else
variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type);
ColumnPtr casted_variant;
/// Check if we have wrapper for this variant.
if (variant_wrapper)
casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size());
casted_variant_columns.push_back(casted_variant);
}
/// Second, collect all variants stored in shared variant and cast them to result type.
@ -4416,8 +4472,18 @@ private:
for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i)
{
ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}};
auto variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type);
casted_shared_variant_columns.push_back(variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size()));
WrapperType variant_wrapper;
if (cast_type == CastType::accurateOrNull)
/// Create wrapper only if we support conversion from variant to the resulting type.
variant_wrapper = createWrapperIfCanConvert(variant_types_from_shared_variant[i], result_type);
else
variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type);
ColumnPtr casted_variant;
/// Check if we have wrapper for this variant.
if (variant_wrapper)
casted_variant = variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size());
casted_shared_variant_columns.push_back(casted_variant);
}
/// Construct result column from all casted variants.
@ -4427,11 +4493,23 @@ private:
{
auto global_discr = variant_column.globalDiscriminatorByLocal(local_discriminators[i]);
if (global_discr == ColumnVariant::NULL_DISCRIMINATOR)
{
res->insertDefault();
}
else if (global_discr == shared_variant_discr)
{
if (casted_shared_variant_columns[shared_variant_indexes[i]])
res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]);
else
res->insertDefault();
}
else
{
if (casted_variant_columns[global_discr])
res->insertFrom(*casted_variant_columns[global_discr], offsets[i]);
else
res->insertDefault();
}
}
return res;

View File

@ -1,7 +1,8 @@
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnTuple.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <IO/WriteHelpers.h>
@ -12,23 +13,22 @@ namespace DB
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_COLUMN;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int SIZES_OF_ARRAYS_DONT_MATCH;
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
extern const int ILLEGAL_COLUMN;
}
/// arrayZip(['a', 'b', 'c'], ['d', 'e', 'f']) = [('a', 'd'), ('b', 'e'), ('c', 'f')]
/// arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e']) = [('a', 'd'), ('b', 'e'), ('c', null)]
template <bool allow_unaligned>
class FunctionArrayZip : public IFunction
{
public:
static constexpr auto name = "arrayZip";
static constexpr auto name = allow_unaligned ? "arrayZipUnaligned" : "arrayZip";
static FunctionPtr create(ContextPtr) { return std::make_shared<FunctionArrayZip>(); }
String getName() const override
{
return name;
}
String getName() const override { return name; }
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
@ -39,8 +39,11 @@ public:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
{
if (arguments.empty())
throw Exception(ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Function {} needs at least one argument; passed {}." , getName(), arguments.size());
throw Exception(
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION,
"Function {} needs at least one argument; passed {}.",
getName(),
arguments.size());
DataTypes arguments_types;
for (size_t index = 0; index < arguments.size(); ++index)
@ -48,56 +51,142 @@ public:
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[index].type.get());
if (!array_type)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Argument {} of function {} must be array. Found {} instead.",
toString(index + 1), getName(), arguments[0].type->getName());
throw Exception(
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument {} of function {} must be array. Found {} instead.",
toString(index + 1),
getName(),
arguments[0].type->getName());
arguments_types.emplace_back(array_type->getNestedType());
auto nested_type = array_type->getNestedType();
if constexpr (allow_unaligned)
nested_type = makeNullable(nested_type);
arguments_types.emplace_back(nested_type);
}
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeTuple>(arguments_types));
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t /*input_rows_count*/) const override
ColumnPtr
executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & /*result_type*/, size_t input_rows_count) const override
{
size_t num_arguments = arguments.size();
ColumnPtr first_array_column;
Columns holders(num_arguments);
Columns tuple_columns(num_arguments);
bool has_unaligned = false;
size_t unaligned_index = 0;
for (size_t i = 0; i < num_arguments; ++i)
{
/// Constant columns cannot be inside tuple. It's only possible to have constant tuple as a whole.
ColumnPtr holder = arguments[i].column->convertToFullColumnIfConst();
holders[i] = holder;
const ColumnArray * column_array = checkAndGetColumn<ColumnArray>(holder.get());
if (!column_array)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Argument {} of function {} must be array. Found column {} instead.",
i + 1, getName(), holder->getName());
if (i == 0)
{
first_array_column = holder;
}
else if (!column_array->hasEqualOffsets(static_cast<const ColumnArray &>(*first_array_column)))
{
throw Exception(ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"The argument 1 and argument {} of function {} have different array sizes",
i + 1, getName());
}
throw Exception(
ErrorCodes::ILLEGAL_COLUMN,
"Argument {} of function {} must be array. Found column {} instead.",
i + 1,
getName(),
holder->getName());
tuple_columns[i] = column_array->getDataPtr();
if (i && !column_array->hasEqualOffsets(static_cast<const ColumnArray &>(*holders[0])))
{
has_unaligned = true;
unaligned_index = i;
}
}
if constexpr (!allow_unaligned)
{
if (has_unaligned)
throw Exception(
ErrorCodes::SIZES_OF_ARRAYS_DONT_MATCH,
"The argument 1 and argument {} of function {} have different array sizes",
unaligned_index + 1,
getName());
else
return ColumnArray::create(
ColumnTuple::create(tuple_columns), static_cast<const ColumnArray &>(*first_array_column).getOffsetsPtr());
ColumnTuple::create(std::move(tuple_columns)), static_cast<const ColumnArray &>(*holders[0]).getOffsetsPtr());
}
else
return executeUnaligned(holders, tuple_columns, input_rows_count, has_unaligned);
}
private:
ColumnPtr executeUnaligned(const Columns & holders, Columns & tuple_columns, size_t input_rows_count, bool has_unaligned) const
{
std::vector<const ColumnArray *> array_columns(holders.size());
for (size_t i = 0; i < holders.size(); ++i)
array_columns[i] = checkAndGetColumn<ColumnArray>(holders[i].get());
for (auto & tuple_column : tuple_columns)
tuple_column = makeNullable(tuple_column);
if (!has_unaligned)
return ColumnArray::create(ColumnTuple::create(std::move(tuple_columns)), array_columns[0]->getOffsetsPtr());
MutableColumns res_tuple_columns(tuple_columns.size());
for (size_t i = 0; i < tuple_columns.size(); ++i)
{
res_tuple_columns[i] = tuple_columns[i]->cloneEmpty();
res_tuple_columns[i]->reserve(tuple_columns[i]->size());
}
auto res_offsets_column = ColumnArray::ColumnOffsets::create(input_rows_count);
auto & res_offsets = assert_cast<ColumnArray::ColumnOffsets &>(*res_offsets_column).getData();
size_t curr_offset = 0;
for (size_t row_i = 0; row_i < input_rows_count; ++row_i)
{
size_t max_size = 0;
for (size_t arg_i = 0; arg_i < holders.size(); ++arg_i)
{
const auto * array_column = array_columns[arg_i];
const auto & offsets = array_column->getOffsets();
size_t array_offset = offsets[row_i - 1];
size_t array_size = offsets[row_i] - array_offset;
res_tuple_columns[arg_i]->insertRangeFrom(*tuple_columns[arg_i], array_offset, array_size);
max_size = std::max(max_size, array_size);
}
for (size_t arg_i = 0; arg_i < holders.size(); ++arg_i)
{
const auto * array_column = array_columns[arg_i];
const auto & offsets = array_column->getOffsets();
size_t array_offset = offsets[row_i - 1];
size_t array_size = offsets[row_i] - array_offset;
res_tuple_columns[arg_i]->insertManyDefaults(max_size - array_size);
}
curr_offset += max_size;
res_offsets[row_i] = curr_offset;
}
return ColumnArray::create(ColumnTuple::create(std::move(res_tuple_columns)), std::move(res_offsets_column));
}
};
REGISTER_FUNCTION(ArrayZip)
{
factory.registerFunction<FunctionArrayZip>();
factory.registerFunction<FunctionArrayZip<false>>(
{.description = R"(
Combines multiple arrays into a single array. The resulting array contains the corresponding elements of the source arrays grouped into tuples in the listed order of arguments.
)",
.categories{"String"}});
factory.registerFunction<FunctionArrayZip<true>>(
{.description = R"(
Combines multiple arrays into a single array, allowing for unaligned arrays. The resulting array contains the corresponding elements of the source arrays grouped into tuples in the listed order of arguments.
If the arrays have different sizes, the shorter arrays will be padded with `null` values.
)",
.categories{"String"}}
);
}
}

View File

@ -315,7 +315,18 @@ void AsynchronousInsertQueue::preprocessInsertQuery(const ASTPtr & query, const
auto sample_block = InterpreterInsertQuery::getSampleBlock(insert_query, table, table->getInMemoryMetadataPtr(), query_context);
if (!FormatFactory::instance().isInputFormat(insert_query.format))
{
if (insert_query.format.empty() && insert_query.infile)
{
const auto & in_file_node = insert_query.infile->as<ASTLiteral &>();
const auto in_file = in_file_node.value.safeGet<std::string>();
const auto in_file_format = FormatFactory::instance().getFormatFromFileName(in_file);
if (!FormatFactory::instance().isInputFormat(in_file_format))
throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown input INFILE format {}", in_file_format);
}
else
throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown input format {}", insert_query.format);
}
/// For table functions we check access while executing
/// InterpreterInsertQuery::getTable() -> ITableFunction::execute().

View File

@ -821,19 +821,6 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
{
properties.indices = as_storage_metadata->getSecondaryIndices();
properties.projections = as_storage_metadata->getProjections().clone();
/// CREATE TABLE AS should copy PRIMARY KEY, ORDER BY, and similar clauses.
if (!create.storage->primary_key && as_storage_metadata->isPrimaryKeyDefined() && as_storage_metadata->hasPrimaryKey())
create.storage->set(create.storage->primary_key, as_storage_metadata->getPrimaryKeyAST()->clone());
if (!create.storage->partition_by && as_storage_metadata->isPartitionKeyDefined() && as_storage_metadata->hasPartitionKey())
create.storage->set(create.storage->partition_by, as_storage_metadata->getPartitionKeyAST()->clone());
if (!create.storage->order_by && as_storage_metadata->isSortingKeyDefined() && as_storage_metadata->hasSortingKey())
create.storage->set(create.storage->order_by, as_storage_metadata->getSortingKeyAST()->clone());
if (!create.storage->sample_by && as_storage_metadata->isSamplingKeyDefined() && as_storage_metadata->hasSamplingKey())
create.storage->set(create.storage->sample_by, as_storage_metadata->getSamplingKeyAST()->clone());
}
else
{

View File

@ -33,25 +33,19 @@ public:
}
};
/// Duplicate of code from StringUtils.h. Copied here for less dependencies.
static bool startsWith(const std::string & s, const char * prefix)
{
return s.size() >= strlen(prefix) && 0 == memcmp(s.data(), prefix, strlen(prefix));
}
Lemmatizers::Lemmatizers(const Poco::Util::AbstractConfiguration & config)
{
String prefix = "lemmatizers";
Poco::Util::AbstractConfiguration::Keys keys;
const String prefix = "lemmatizers";
if (!config.has(prefix))
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "No lemmatizers specified in server config on prefix '{}'", prefix);
return;
Poco::Util::AbstractConfiguration::Keys keys;
config.keys(prefix, keys);
for (const auto & key : keys)
{
if (startsWith(key, "lemmatizer"))
if (key.starts_with("lemmatizer"))
{
const auto & lemm_name = config.getString(prefix + "." + key + ".lang", "");
const auto & lemm_path = config.getString(prefix + "." + key + ".path", "");
@ -81,13 +75,13 @@ Lemmatizers::LemmPtr Lemmatizers::getLemmatizer(const String & name)
if (paths.find(name) != paths.end())
{
if (!std::filesystem::exists(paths[name]))
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Incorrect path to lemmatizer: {}", paths[name]);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Path to lemmatizer does not exist: {}", paths[name]);
lemmatizers[name] = std::make_shared<Lemmatizer>(paths[name]);
return lemmatizers[name];
}
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Lemmatizer named: '{}' is not found", name);
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Lemmatizer with the name '{}' was not found in the configuration", name);
}
}

View File

@ -147,6 +147,7 @@ ColumnDependencies getAllColumnDependencies(
bool isStorageTouchedByMutations(
MergeTreeData::DataPartPtr source_part,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MutationCommand> & commands,
ContextPtr context)
@ -154,7 +155,7 @@ bool isStorageTouchedByMutations(
if (commands.empty())
return false;
auto storage_from_part = std::make_shared<StorageFromMergeTreeDataPart>(source_part);
auto storage_from_part = std::make_shared<StorageFromMergeTreeDataPart>(source_part, mutations_snapshot);
bool all_commands_can_be_skipped = true;
for (const auto & command : commands)
@ -285,8 +286,13 @@ MutationsInterpreter::Source::Source(StoragePtr storage_) : storage(std::move(st
{
}
MutationsInterpreter::Source::Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_)
: data(&storage_), part(std::move(source_part_))
MutationsInterpreter::Source::Source(
MergeTreeData & storage_,
MergeTreeData::DataPartPtr source_part_,
AlterConversionsPtr alter_conversions_)
: data(&storage_)
, part(std::move(source_part_))
, alter_conversions(std::move(alter_conversions_))
{
}
@ -386,13 +392,14 @@ MutationsInterpreter::MutationsInterpreter(
MutationsInterpreter::MutationsInterpreter(
MergeTreeData & storage_,
MergeTreeData::DataPartPtr source_part_,
AlterConversionsPtr alter_conversions_,
StorageMetadataPtr metadata_snapshot_,
MutationCommands commands_,
Names available_columns_,
ContextPtr context_,
Settings settings_)
: MutationsInterpreter(
Source(storage_, std::move(source_part_)),
Source(storage_, std::move(source_part_), std::move(alter_conversions_)),
std::move(metadata_snapshot_), std::move(commands_),
std::move(available_columns_), std::move(context_), std::move(settings_))
{
@ -1218,7 +1225,7 @@ void MutationsInterpreter::Source::read(
createReadFromPartStep(
MergeTreeSequentialSourceType::Mutation,
plan, *data, storage_snapshot,
part, required_columns,
part, alter_conversions, required_columns,
apply_deleted_mask_, std::move(filter), context_,
getLogger("MutationsInterpreter"));
}

View File

@ -20,6 +20,7 @@ using QueryPipelineBuilderPtr = std::unique_ptr<QueryPipelineBuilder>;
/// Return false if the data isn't going to be changed by mutations.
bool isStorageTouchedByMutations(
MergeTreeData::DataPartPtr source_part,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const StorageMetadataPtr & metadata_snapshot,
const std::vector<MutationCommand> & commands,
ContextPtr context
@ -70,6 +71,7 @@ public:
MutationsInterpreter(
MergeTreeData & storage_,
MergeTreeData::DataPartPtr source_part_,
AlterConversionsPtr alter_conversions_,
StorageMetadataPtr metadata_snapshot_,
MutationCommands commands_,
Names available_columns_,
@ -137,7 +139,7 @@ public:
bool can_execute_) const;
explicit Source(StoragePtr storage_);
Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_);
Source(MergeTreeData & storage_, MergeTreeData::DataPartPtr source_part_, AlterConversionsPtr alter_conversions_);
private:
StoragePtr storage;
@ -145,6 +147,7 @@ public:
/// Special case for *MergeTree.
MergeTreeData * data = nullptr;
MergeTreeData::DataPartPtr part;
AlterConversionsPtr alter_conversions;
};
private:

View File

@ -1,6 +1,8 @@
#include <Common/typeid_cast.h>
#include <Parsers/ParserStringAndSubstitution.h>
#include <Parsers/ParserAlterQuery.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
@ -9,14 +11,19 @@
#include <Parsers/ParserRefreshStrategy.h>
#include <Parsers/ParserSelectWithUnionQuery.h>
#include <Parsers/ParserSetQuery.h>
#include <Parsers/ASTAlterQuery.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ParserStringAndSubstitution.h>
#include <Parsers/parseDatabaseAndTableName.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
}
bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto command = std::make_shared<ASTAlterCommand>();
@ -122,7 +129,6 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
ParserCompoundIdentifier parser_name;
ParserStringLiteral parser_string_literal;
ParserStringAndSubstitution parser_string_and_substituion;
ParserIdentifier parser_remove_property;
ParserCompoundColumnDeclaration parser_col_decl;
ParserIndexDeclaration parser_idx_decl;
ParserStatisticsDeclaration parser_stat_decl;
@ -725,8 +731,21 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
if (!parser_modify_col_decl.parse(pos, command_col_decl, expected))
return false;
auto check_no_type = [&](const std::string_view keyword)
{
const auto & column_decl = command_col_decl->as<const ASTColumnDeclaration &>();
if (!column_decl.children.empty() || column_decl.null_modifier.has_value() || !column_decl.default_specifier.empty()
|| column_decl.ephemeral_default || column_decl.primary_key_specifier)
{
throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot specify column properties before '{}'", keyword);
}
};
if (s_remove.ignore(pos, expected))
{
check_no_type(s_remove.getName());
if (s_default.ignore(pos, expected))
command->remove_property = toStringView(Keyword::DEFAULT);
else if (s_materialized.ignore(pos, expected))
@ -746,11 +765,15 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
else if (s_modify_setting.ignore(pos, expected))
{
check_no_type(s_modify_setting.getName());
if (!parser_settings.parse(pos, command_settings_changes, expected))
return false;
}
else if (s_reset_setting.ignore(pos, expected))
{
check_no_type(s_reset_setting.getName());
if (!parser_reset_setting.parse(pos, command_settings_resets, expected))
return false;
}
@ -765,6 +788,11 @@ bool ParserAlterCommand::parseImpl(Pos & pos, ASTPtr & node, Expected & expected
}
}
command->type = ASTAlterCommand::MODIFY_COLUMN;
/// Make sure that type is not populated when REMOVE/MODIFY SETTING/RESET SETTING is used, because we wouldn't modify the type, which can be confusing
chassert(
nullptr == command_col_decl->as<const ASTColumnDeclaration &>().type
|| (command->remove_property.empty() && nullptr == command_settings_changes && nullptr == command_settings_resets));
}
else if (s_modify_order_by.ignore(pos, expected))
{

View File

@ -757,7 +757,7 @@ std::optional<String> optimizeUseAggregateProjections(QueryPlan::Node & node, Qu
projection_reading = reader.readFromParts(
/* parts = */ {},
/* alter_conversions = */ {},
reading->getMutationsSnapshot()->cloneEmpty(),
best_candidate->dag.getRequiredColumnsNames(),
proj_snapshot,
projection_query_info,

View File

@ -199,7 +199,7 @@ std::optional<String> optimizeUseNormalProjections(Stack & stack, QueryPlan::Nod
auto projection_reading = reader.readFromParts(
/*parts=*/ {},
/*alter_conversions=*/ {},
reading->getMutationsSnapshot()->cloneEmpty(),
required_columns,
proj_snapshot,
query_info_copy,

View File

@ -41,12 +41,19 @@ bool canUseProjectionForReadingStep(ReadFromMergeTree * reading)
if (reading->readsInOrder())
return false;
const auto & query_settings = reading->getContext()->getSettingsRef();
// Currently projection don't support deduplication when moving parts between shards.
if (reading->getContext()->getSettingsRef().allow_experimental_query_deduplication)
if (query_settings.allow_experimental_query_deduplication)
return false;
// Currently projection don't support settings which implicitly modify aggregate functions.
if (reading->getContext()->getSettingsRef().aggregate_functions_null_for_empty)
if (query_settings.aggregate_functions_null_for_empty)
return false;
/// Don't use projections if have mutations to apply
/// because we need to apply them on original data.
if (query_settings.apply_mutations_on_fly && reading->getMutationsSnapshot()->hasDataMutations())
return false;
return true;
@ -215,20 +222,15 @@ bool analyzeProjectionCandidate(
{
MergeTreeData::DataPartsVector projection_parts;
MergeTreeData::DataPartsVector normal_parts;
std::vector<AlterConversionsPtr> alter_conversions;
for (const auto & part_with_ranges : parts_with_ranges)
{
const auto & created_projections = part_with_ranges.data_part->getProjectionParts();
auto it = created_projections.find(candidate.projection->name);
if (it != created_projections.end() && !it->second->is_broken)
{
projection_parts.push_back(it->second);
}
else
{
normal_parts.push_back(part_with_ranges.data_part);
alter_conversions.push_back(part_with_ranges.alter_conversions);
}
}
if (projection_parts.empty())
@ -241,6 +243,7 @@ bool analyzeProjectionCandidate(
auto projection_result_ptr = reader.estimateNumMarksToRead(
std::move(projection_parts),
reading.getMutationsSnapshot()->cloneEmpty(),
required_column_names,
candidate.projection->metadata,
projection_query_info,
@ -254,7 +257,7 @@ bool analyzeProjectionCandidate(
if (!normal_parts.empty())
{
/// TODO: We can reuse existing analysis_result by filtering out projection parts
auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts), std::move(alter_conversions));
auto normal_result_ptr = reading.selectRangesToRead(std::move(normal_parts));
if (normal_result_ptr->selected_marks != 0)
{

View File

@ -229,7 +229,6 @@ public:
{
ranges_in_data_parts.emplace_back(
initial_ranges_in_data_parts[part_index].data_part,
initial_ranges_in_data_parts[part_index].alter_conversions,
initial_ranges_in_data_parts[part_index].part_index_in_query,
MarkRanges{mark_range});
part_index_to_initial_ranges_in_data_parts_index[it->second] = part_index;

View File

@ -266,7 +266,7 @@ void ReadFromMergeTree::AnalysisResult::checkLimits(const Settings & settings, c
ReadFromMergeTree::ReadFromMergeTree(
MergeTreeData::DataPartsVector parts_,
std::vector<AlterConversionsPtr> alter_conversions_,
MergeTreeData::MutationsSnapshotPtr mutations_,
Names all_column_names_,
const MergeTreeData & data_,
const SelectQueryInfo & query_info_,
@ -286,7 +286,7 @@ ReadFromMergeTree::ReadFromMergeTree(
query_info_.prewhere_info)}, all_column_names_, query_info_, storage_snapshot_, context_)
, reader_settings(getMergeTreeReaderSettings(context_, query_info_))
, prepared_parts(std::move(parts_))
, alter_conversions_for_parts(std::move(alter_conversions_))
, mutations_snapshot(std::move(mutations_))
, all_column_names(std::move(all_column_names_))
, data(data_)
, actions_settings(ExpressionActionsSettings::fromContext(context_))
@ -405,6 +405,7 @@ Pipe ReadFromMergeTree::readFromPoolParallelReplicas(RangesInDataParts parts_wit
auto pool = std::make_shared<MergeTreeReadPoolParallelReplicas>(
std::move(extension),
std::move(parts_with_range),
mutations_snapshot,
shared_virtual_fields,
storage_snapshot,
prewhere_info,
@ -486,6 +487,7 @@ Pipe ReadFromMergeTree::readFromPool(
{
pool = std::make_shared<MergeTreePrefetchedReadPool>(
std::move(parts_with_range),
mutations_snapshot,
shared_virtual_fields,
storage_snapshot,
prewhere_info,
@ -499,6 +501,7 @@ Pipe ReadFromMergeTree::readFromPool(
{
pool = std::make_shared<MergeTreeReadPool>(
std::move(parts_with_range),
mutations_snapshot,
shared_virtual_fields,
storage_snapshot,
prewhere_info,
@ -587,6 +590,7 @@ Pipe ReadFromMergeTree::readInOrder(
std::move(extension),
mode,
parts_with_ranges,
mutations_snapshot,
shared_virtual_fields,
storage_snapshot,
prewhere_info,
@ -602,6 +606,7 @@ Pipe ReadFromMergeTree::readInOrder(
has_limit_below_one_block,
read_type,
parts_with_ranges,
mutations_snapshot,
shared_virtual_fields,
storage_snapshot,
prewhere_info,
@ -1072,7 +1077,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsWithOrder(
}
ranges_to_get_from_part = split_ranges(ranges_to_get_from_part, input_order_info->direction);
new_parts.emplace_back(part.data_part, part.alter_conversions, part.part_index_in_query, std::move(ranges_to_get_from_part));
new_parts.emplace_back(part.data_part, part.part_index_in_query, std::move(ranges_to_get_from_part));
}
splitted_parts_and_ranges.emplace_back(std::move(new_parts));
@ -1299,7 +1304,7 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
RangesInDataParts new_parts;
for (auto part_it = parts_to_merge_ranges[range_index]; part_it != parts_to_merge_ranges[range_index + 1]; ++part_it)
new_parts.emplace_back(part_it->data_part, part_it->alter_conversions, part_it->part_index_in_query, part_it->ranges);
new_parts.emplace_back(part_it->data_part, part_it->part_index_in_query, part_it->ranges);
if (new_parts.empty())
continue;
@ -1412,15 +1417,14 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreamsFinal(
ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(bool find_exact_ranges) const
{
return selectRangesToRead(prepared_parts, alter_conversions_for_parts, find_exact_ranges);
return selectRangesToRead(prepared_parts, find_exact_ranges);
}
ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
MergeTreeData::DataPartsVector parts, std::vector<AlterConversionsPtr> alter_conversions, bool find_exact_ranges) const
ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges) const
{
return selectRangesToRead(
std::move(parts),
std::move(alter_conversions),
mutations_snapshot,
storage_snapshot->metadata,
query_info,
context,
@ -1438,9 +1442,11 @@ static void buildIndexes(
const ActionsDAG * filter_actions_dag,
const MergeTreeData & data,
const MergeTreeData::DataPartsVector & parts,
const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot,
const ContextPtr & context,
const SelectQueryInfo & query_info,
const StorageMetadataPtr & metadata_snapshot)
const StorageMetadataPtr & metadata_snapshot,
const LoggerPtr & log)
{
indexes.reset();
@ -1463,19 +1469,21 @@ static void buildIndexes(
indexes->partition_pruner.emplace(metadata_snapshot, filter_actions_dag, context, false /* strict */);
}
indexes->part_values
= MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context);
indexes->part_values = MergeTreeDataSelectExecutor::filterPartsByVirtualColumns(metadata_snapshot, data, parts, filter_actions_dag, context);
MergeTreeDataSelectExecutor::buildKeyConditionFromPartOffset(indexes->part_offset_condition, filter_actions_dag, context);
indexes->use_skip_indexes = settings.use_skip_indexes;
bool final = query_info.isFinal();
if (final && !settings.use_skip_indexes_if_final)
if (query_info.isFinal() && !settings.use_skip_indexes_if_final)
indexes->use_skip_indexes = false;
if (!indexes->use_skip_indexes)
return;
const auto & all_indexes = metadata_snapshot->getSecondaryIndices();
if (all_indexes.empty())
return;
std::unordered_set<std::string> ignored_index_names;
if (settings.ignore_data_skipping_indices.changed)
@ -1500,15 +1508,34 @@ static void buildIndexes(
throw Exception(ErrorCodes::CANNOT_PARSE_TEXT, "Cannot parse ignore_data_skipping_indices ('{}')", indices);
}
auto all_updated_columns = mutations_snapshot->getAllUpdatedColumns();
UsefulSkipIndexes skip_indexes;
using Key = std::pair<String, size_t>;
std::map<Key, size_t> merged;
for (const auto & index : metadata_snapshot->getSecondaryIndices())
{
if (!ignored_index_names.contains(index.name))
for (const auto & index : all_indexes)
{
if (ignored_index_names.contains(index.name))
continue;
auto index_helper = MergeTreeIndexFactory::instance().get(index);
if (!all_updated_columns.empty())
{
auto required_columns = index_helper->getColumnsRequiredForIndexCalc();
auto it = std::ranges::find_if(required_columns, [&](const auto & column_name)
{
return all_updated_columns.contains(column_name);
});
if (it != required_columns.end())
{
LOG_TRACE(log, "Index {} is not used because it depends on column {} which will be updated on fly", index.name, *it);
continue;
}
}
if (index_helper->isMergeable())
{
auto [it, inserted] = merged.emplace(Key{index_helper->index.type, index_helper->getGranularity()}, skip_indexes.merged_indices.size());
@ -1519,10 +1546,11 @@ static void buildIndexes(
}
skip_indexes.merged_indices[it->second].addIndex(index_helper);
continue;
}
else
{
MergeTreeIndexConditionPtr condition;
if (index_helper->isVectorSimilarityIndex())
{
#if USE_USEARCH
@ -1531,17 +1559,18 @@ static void buildIndexes(
#endif
if (const auto * legacy_vector_similarity_index = typeid_cast<const MergeTreeIndexLegacyVectorSimilarity *>(index_helper.get()))
condition = legacy_vector_similarity_index->createIndexCondition(query_info, context);
if (!condition)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unknown vector search index {}", index_helper->index.name);
}
else
{
condition = index_helper->createIndexCondition(filter_actions_dag, context);
}
if (!condition->alwaysUnknownOrTrue())
skip_indexes.useful_indices.emplace_back(index_helper, condition);
}
}
}
// move minmax indices to first positions, so they will be applied first as cheapest ones
std::stable_sort(begin(skip_indexes.useful_indices), end(skip_indexes.useful_indices), [](const auto & l, const auto & r)
@ -1578,15 +1607,17 @@ void ReadFromMergeTree::applyFilters(ActionDAGNodes added_filter_nodes)
query_info.filter_actions_dag.get(),
data,
prepared_parts,
mutations_snapshot,
context,
query_info,
storage_snapshot->metadata);
storage_snapshot->metadata,
log);
}
}
ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
MergeTreeData::DataPartsVector parts,
std::vector<AlterConversionsPtr> alter_conversions,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info_,
ContextPtr context_,
@ -1617,7 +1648,7 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
const Names & primary_key_column_names = primary_key.column_names;
if (!indexes)
buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, context_, query_info_, metadata_snapshot);
buildIndexes(indexes, query_info_.filter_actions_dag.get(), data, parts, mutations_snapshot, context_, query_info_, metadata_snapshot, log);
if (indexes->part_values && indexes->part_values->empty())
return std::make_shared<AnalysisResult>(std::move(result));
@ -1648,10 +1679,9 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
{
MergeTreeDataSelectExecutor::filterPartsByPartition(
parts,
indexes->partition_pruner,
indexes->minmax_idx_condition,
parts,
alter_conversions,
indexes->part_values,
metadata_snapshot,
data,
@ -1680,7 +1710,6 @@ ReadFromMergeTree::AnalysisResultPtr ReadFromMergeTree::selectRangesToRead(
auto reader_settings = getMergeTreeReaderSettings(context_, query_info_);
result.parts_with_ranges = MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes(
std::move(parts),
std::move(alter_conversions),
metadata_snapshot,
context_,
indexes->key_condition,

View File

@ -110,7 +110,7 @@ public:
ReadFromMergeTree(
MergeTreeData::DataPartsVector parts_,
std::vector<AlterConversionsPtr> alter_conversions_,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot_,
Names all_column_names_,
const MergeTreeData & data_,
const SelectQueryInfo & query_info_,
@ -168,7 +168,7 @@ public:
static AnalysisResultPtr selectRangesToRead(
MergeTreeData::DataPartsVector parts,
std::vector<AlterConversionsPtr> alter_conversions,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
ContextPtr context,
@ -180,8 +180,7 @@ public:
std::optional<Indexes> & indexes,
bool find_exact_ranges);
AnalysisResultPtr selectRangesToRead(
MergeTreeData::DataPartsVector parts, std::vector<AlterConversionsPtr> alter_conversions, bool find_exact_ranges = false) const;
AnalysisResultPtr selectRangesToRead(MergeTreeData::DataPartsVector parts, bool find_exact_ranges = false) const;
AnalysisResultPtr selectRangesToRead(bool find_exact_ranges = false) const;
@ -202,7 +201,7 @@ public:
void setAnalyzedResult(AnalysisResultPtr analyzed_result_ptr_) { analyzed_result_ptr = std::move(analyzed_result_ptr_); }
const MergeTreeData::DataPartsVector & getParts() const { return prepared_parts; }
const std::vector<AlterConversionsPtr> & getAlterConvertionsForParts() const { return alter_conversions_for_parts; }
MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot() const { return mutations_snapshot; }
const MergeTreeData & getMergeTreeData() const { return data; }
size_t getMaxBlockSize() const { return block_size.max_block_size_rows; }
@ -223,7 +222,7 @@ private:
MergeTreeReaderSettings reader_settings;
MergeTreeData::DataPartsVector prepared_parts;
std::vector<AlterConversionsPtr> alter_conversions_for_parts;
MergeTreeData::MutationsSnapshotPtr mutations_snapshot;
Names all_column_names;

View File

@ -271,6 +271,7 @@ public:
/// Return true if the trivial count query could be optimized without reading the data at all
/// in totalRows() or totalRowsByPartitionPredicate() methods or with optimized reading in read() method.
/// 'storage_snapshot' may be nullptr.
virtual bool supportsTrivialCountOptimization(const StorageSnapshotPtr & /*storage_snapshot*/, ContextPtr /*query_context*/) const
{
return false;

View File

@ -9,9 +9,15 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
bool AlterConversions::supportsMutationCommandType(MutationCommand::Type t)
bool AlterConversions::isSupportedDataMutation(MutationCommand::Type)
{
return t == MutationCommand::Type::RENAME_COLUMN;
/// Currently there is no such mutations. See setting 'apply_mutations_on_fly'.
return false;
}
bool AlterConversions::isSupportedMetadataMutation(MutationCommand::Type type)
{
return type == MutationCommand::Type::RENAME_COLUMN;
}
void AlterConversions::addMutationCommand(const MutationCommand & command)

View File

@ -1,8 +1,8 @@
#pragma once
#include <Storages/MutationCommands.h>
#include <string>
#include <unordered_map>
#include <Interpreters/Context_fwd.h>
#include <Storages/StorageInMemoryMetadata.h>
namespace DB
@ -11,11 +11,17 @@ namespace DB
/// Alter conversions which should be applied on-fly for part.
/// Built from of the most recent mutation commands for part.
/// Now only ALTER RENAME COLUMN is applied.
class AlterConversions : private boost::noncopyable
class AlterConversions : private WithContext, boost::noncopyable
{
public:
AlterConversions() = default;
AlterConversions(StorageMetadataPtr metadata_snapshot_, ContextPtr context_)
: WithContext(context_)
, metadata_snapshot(std::move(metadata_snapshot_))
{
}
struct RenamePair
{
std::string rename_to;
@ -34,11 +40,13 @@ public:
/// Get column old name before rename (lookup by key in rename_map)
std::string getColumnOldName(const std::string & new_name) const;
static bool supportsMutationCommandType(MutationCommand::Type);
static bool isSupportedDataMutation(MutationCommand::Type type);
static bool isSupportedMetadataMutation(MutationCommand::Type type);
private:
/// Rename map new_name -> old_name.
std::vector<RenamePair> rename_map;
StorageMetadataPtr metadata_snapshot;
};
using AlterConversionsPtr = std::shared_ptr<const AlterConversions>;

View File

@ -2052,6 +2052,7 @@ DataPartStoragePtr IMergeTreeDataPart::makeCloneInDetached(const String & prefix
IDataPartStorage::ClonePartParams params
{
.copy_instead_of_hardlink = isStoredOnRemoteDiskWithZeroCopySupport() && storage.supportsReplication() && storage_settings->allow_remote_fs_zero_copy_replication,
.keep_metadata_version = prefix == "covered-by-broken",
.make_source_readonly = true,
.external_transaction = disk_transaction
};

View File

@ -286,6 +286,14 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
if (enabledBlockOffsetColumn(global_ctx))
addGatheringColumn(global_ctx, BlockOffsetColumn::name, BlockOffsetColumn::type);
MergeTreeData::IMutationsSnapshot::Params params
{
.metadata_version = global_ctx->metadata_snapshot->getMetadataVersion(),
.min_part_metadata_version = MergeTreeData::getMinMetadataVersion(global_ctx->future_part->parts),
};
auto mutations_snapshot = global_ctx->data->getMutationsSnapshot(params);
SerializationInfo::Settings info_settings =
{
.ratio_of_defaults_for_sparse = global_ctx->data->getSettings()->ratio_of_defaults_for_sparse_serialization,
@ -293,10 +301,12 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
};
SerializationInfoByName infos(global_ctx->storage_columns, info_settings);
global_ctx->alter_conversions.reserve(global_ctx->future_part->parts.size());
for (const auto & part : global_ctx->future_part->parts)
{
global_ctx->new_data_part->ttl_infos.update(part->ttl_infos);
if (global_ctx->metadata_snapshot->hasAnyTTL() && !part->checkAllTTLCalculated(global_ctx->metadata_snapshot))
{
LOG_INFO(ctx->log, "Some TTL values were not calculated for part {}. Will calculate them forcefully during merge.", part->name);
@ -317,6 +327,8 @@ bool MergeTask::ExecuteAndFinalizeHorizontalPart::prepare()
infos.add(part_infos);
}
global_ctx->alter_conversions.push_back(MergeTreeData::getAlterConversionsForPart(part, mutations_snapshot, global_ctx->metadata_snapshot, global_ctx->context));
}
const auto & local_part_min_ttl = global_ctx->new_data_part->ttl_infos.part_min_ttl;
@ -815,6 +827,7 @@ Pipe MergeTask::VerticalMergeStage::createPipeForReadingOneColumn(const String &
*global_ctx->data,
global_ctx->storage_snapshot,
global_ctx->future_part->parts[part_num],
global_ctx->alter_conversions[part_num],
Names{column_name},
/*mark_ranges=*/ {},
global_ctx->input_rows_filtered,
@ -1238,13 +1251,14 @@ void MergeTask::ExecuteAndFinalizeHorizontalPart::createMergedStream()
global_ctx->horizontal_stage_progress = std::make_unique<MergeStageProgress>(
ctx->column_sizes ? ctx->column_sizes->keyColumnsWeight() : 1.0);
for (const auto & part : global_ctx->future_part->parts)
for (size_t i = 0; i < global_ctx->future_part->parts.size(); ++i)
{
Pipe pipe = createMergeTreeSequentialSource(
MergeTreeSequentialSourceType::Merge,
*global_ctx->data,
global_ctx->storage_snapshot,
part,
global_ctx->future_part->parts[i],
global_ctx->alter_conversions[i],
global_ctx->merging_columns.getNames(),
/*mark_ranges=*/ {},
global_ctx->input_rows_filtered,

View File

@ -166,6 +166,7 @@ private:
StorageSnapshotPtr storage_snapshot{nullptr};
StorageMetadataPtr metadata_snapshot{nullptr};
FutureMergedMutatedPartPtr future_part{nullptr};
std::vector<AlterConversionsPtr> alter_conversions;
/// This will be either nullptr or new_data_part, so raw pointer is ok.
IMergeTreeDataPart * parent_part{nullptr};
ContextPtr context{nullptr};

View File

@ -7156,11 +7156,16 @@ UInt64 MergeTreeData::estimateNumberOfRowsToRead(
ContextPtr query_context, const StorageSnapshotPtr & storage_snapshot, const SelectQueryInfo & query_info) const
{
const auto & snapshot_data = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data);
const auto & parts = snapshot_data.parts;
MergeTreeDataSelectExecutor reader(*this);
auto result_ptr = reader.estimateNumMarksToRead(
parts, {}, storage_snapshot->metadata, query_info, query_context, query_context->getSettingsRef().max_threads);
snapshot_data.parts,
snapshot_data.mutations_snapshot,
storage_snapshot->metadata->getColumns().getAll().getNames(),
storage_snapshot->metadata,
query_info,
query_context,
query_context->getSettingsRef().max_threads);
UInt64 total_rows = result_ptr->selected_rows;
if (query_info.trivial_limit > 0 && query_info.trivial_limit < total_rows)
@ -8174,11 +8179,15 @@ bool MergeTreeData::canUsePolymorphicParts(const MergeTreeSettings & settings, S
return true;
}
AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(MergeTreeDataPartPtr part) const
AlterConversionsPtr MergeTreeData::getAlterConversionsForPart(
const MergeTreeDataPartPtr & part,
const MutationsSnapshotPtr & mutations,
const StorageMetadataPtr & metadata,
const ContextPtr & query_context)
{
auto commands = getAlterMutationCommandsForPart(part);
auto commands = mutations->getAlterMutationCommandsForPart(part);
auto result = std::make_shared<AlterConversions>(metadata, query_context);
auto result = std::make_shared<AlterConversions>();
for (const auto & command : commands | std::views::reverse)
result->addMutationCommand(command);
@ -8470,9 +8479,28 @@ void MergeTreeData::updateObjectColumns(const DataPartPtr & part, const DataPart
DB::updateObjectColumns(object_columns, columns, part->getColumns());
}
bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const
bool MergeTreeData::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const
{
return !hasLightweightDeletedMask();
if (hasLightweightDeletedMask())
return false;
if (!storage_snapshot)
return !query_context->getSettingsRef().apply_mutations_on_fly;
const auto & snapshot_data = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data);
return !snapshot_data.mutations_snapshot->hasDataMutations();
}
Int64 MergeTreeData::getMinMetadataVersion(const DataPartsVector & parts)
{
Int64 version = -1;
for (const auto & part : parts)
{
Int64 part_version = part->getMetadataVersion();
if (version == -1 || part_version < version)
version = part_version;
}
return version;
}
StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const
@ -8486,10 +8514,14 @@ StorageSnapshotPtr MergeTreeData::getStorageSnapshot(const StorageMetadataPtr &
object_columns_copy = object_columns;
}
snapshot_data->alter_conversions.reserve(snapshot_data->parts.size());
for (const auto & part : snapshot_data->parts)
snapshot_data->alter_conversions.push_back(getAlterConversionsForPart(part));
IMutationsSnapshot::Params params
{
.metadata_version = metadata_snapshot->getMetadataVersion(),
.min_part_metadata_version = getMinMetadataVersion(snapshot_data->parts),
.need_data_mutations = query_context->getSettingsRef().apply_mutations_on_fly,
};
snapshot_data->mutations_snapshot = getMutationsSnapshot(params);
return std::make_shared<StorageSnapshot>(*this, metadata_snapshot, std::move(object_columns_copy), std::move(snapshot_data));
}
@ -8707,28 +8739,57 @@ void MergeTreeData::verifySortingKey(const KeyDescription & sorting_key)
}
}
bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic<ssize_t> & alter_conversions_mutations, bool remove)
static void updateMutationsCounters(
Int64 & num_data_mutations_to_apply,
Int64 & num_metadata_mutations_to_apply,
const MutationCommands & commands,
Int64 increment)
{
if (num_data_mutations_to_apply < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply);
if (num_metadata_mutations_to_apply < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply);
bool has_data_mutation = false;
bool has_metadata_mutation = false;
for (const auto & command : commands)
{
if (AlterConversions::supportsMutationCommandType(command.type))
if (!has_data_mutation && AlterConversions::isSupportedDataMutation(command.type))
{
if (remove)
num_data_mutations_to_apply += increment;
has_data_mutation = true;
if (num_data_mutations_to_apply < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly data mutations counter is negative ({})", num_data_mutations_to_apply);
}
if (!has_metadata_mutation && AlterConversions::isSupportedMetadataMutation(command.type))
{
--alter_conversions_mutations;
if (alter_conversions_mutations < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations);
}
else
{
if (alter_conversions_mutations < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly mutations counter is negative ({})", alter_conversions_mutations);
++alter_conversions_mutations;
}
return true;
num_metadata_mutations_to_apply += increment;
has_metadata_mutation = true;
if (num_metadata_mutations_to_apply < 0)
throw Exception(ErrorCodes::LOGICAL_ERROR, "On-fly metadata mutations counter is negative ({})", num_metadata_mutations_to_apply);
}
}
return false;
}
void incrementMutationsCounters(
Int64 & num_data_mutations_to_apply,
Int64 & num_metadata_mutations_to_apply,
const MutationCommands & commands)
{
updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, 1);
}
void decrementMutationsCounters(
Int64 & num_data_mutations_to_apply,
Int64 & num_metadata_mutations_to_apply,
const MutationCommands & commands)
{
updateMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands, -1);
}
}

View File

@ -443,14 +443,53 @@ public:
bool areAsynchronousInsertsEnabled() const override;
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override;
bool supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr query_context) const override;
/// A snapshot of pending mutations that weren't applied to some of the parts yet
/// and should be applied on the fly (i.e. when reading from the part).
/// Mutations not supported by AlterConversions (supportsMutationCommandType()) can be omitted.
struct IMutationsSnapshot
{
/// Contains info that doesn't depend on state of mutations.
struct Params
{
Int64 metadata_version = -1;
Int64 min_part_metadata_version = -1;
bool need_data_mutations = false;
};
/// Contains info that depends on state of mutations.
struct Info
{
Int64 num_data_mutations = 0;
Int64 num_metadata_mutations = 0;
};
Params params;
Info info;
IMutationsSnapshot() = default;
IMutationsSnapshot(Params params_, Info info_): params(std::move(params_)), info(std::move(info_)) {}
/// Returns mutation commands that are required to be applied to the `part`.
/// @return list of mutation commands, in *reverse* order (newest to oldest)
virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0;
virtual std::shared_ptr<IMutationsSnapshot> cloneEmpty() const = 0;
virtual NameSet getAllUpdatedColumns() const = 0;
bool hasDataMutations() const { return params.need_data_mutations && info.num_data_mutations > 0; }
virtual ~IMutationsSnapshot() = default;
};
using MutationsSnapshotPtr = std::shared_ptr<const IMutationsSnapshot>;
/// Snapshot for MergeTree contains the current set of data parts
/// at the moment of the start of query.
/// and mutations required to be applied at the moment of the start of query.
struct SnapshotData : public StorageSnapshot::Data
{
DataPartsVector parts;
std::vector<AlterConversionsPtr> alter_conversions;
MutationsSnapshotPtr mutations_snapshot;
};
StorageSnapshotPtr getStorageSnapshot(const StorageMetadataPtr & metadata_snapshot, ContextPtr query_context) const override;
@ -929,8 +968,18 @@ public:
Disks getDisks() const { return getStoragePolicy()->getDisks(); }
/// Returns a snapshot of mutations that probably will be applied on the fly to parts during reading.
virtual MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const = 0;
/// Returns the minimum version of metadata among parts.
static Int64 getMinMetadataVersion(const DataPartsVector & parts);
/// Return alter conversions for part which must be applied on fly.
AlterConversionsPtr getAlterConversionsForPart(MergeTreeDataPartPtr part) const;
static AlterConversionsPtr getAlterConversionsForPart(
const MergeTreeDataPartPtr & part,
const MutationsSnapshotPtr & mutations,
const StorageMetadataPtr & metadata,
const ContextPtr & query_context);
/// Returns destination disk or volume for the TTL rule according to current storage policy.
SpacePtr getDestinationForMoveTTL(const TTLDescription & move_ttl) const;
@ -1450,13 +1499,6 @@ protected:
/// mechanisms for parts locking
virtual bool partIsAssignedToBackgroundOperation(const DataPartPtr & part) const = 0;
/// Return pending mutations that weren't applied to `part` yet and should be applied on the fly
/// (i.e. when reading from the part). Mutations not supported by AlterConversions
/// (supportsMutationCommandType()) can be omitted.
///
/// @return list of mutations, in *reverse* order (newest to oldest)
virtual MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const = 0;
struct PartBackupEntries
{
String part_name;
@ -1731,7 +1773,14 @@ struct CurrentlySubmergingEmergingTagger
};
/// Look at MutationCommands if it contains mutations for AlterConversions, update the counter.
/// Return true if the counter had been updated
bool updateAlterConversionsMutations(const MutationCommands & commands, std::atomic<ssize_t> & alter_conversions_mutations, bool remove);
void incrementMutationsCounters(
Int64 & num_data_mutations_to_apply,
Int64 & num_metadata_mutations_to_apply,
const MutationCommands & commands);
void decrementMutationsCounters(
Int64 & num_data_mutations_to_apply,
Int64 & num_metadata_mutations_to_apply,
const MutationCommands & commands);
}

View File

@ -133,12 +133,10 @@ QueryPlanPtr MergeTreeDataSelectExecutor::read(
bool enable_parallel_reading) const
{
const auto & snapshot_data = assert_cast<const MergeTreeData::SnapshotData &>(*storage_snapshot->data);
const auto & parts = snapshot_data.parts;
const auto & alter_conversions = snapshot_data.alter_conversions;
auto step = readFromParts(
parts,
alter_conversions,
snapshot_data.parts,
snapshot_data.mutations_snapshot,
column_names_to_return,
storage_snapshot,
query_info,
@ -500,10 +498,9 @@ std::optional<std::unordered_set<String>> MergeTreeDataSelectExecutor::filterPar
}
void MergeTreeDataSelectExecutor::filterPartsByPartition(
MergeTreeData::DataPartsVector & parts,
const std::optional<PartitionPruner> & partition_pruner,
const std::optional<KeyCondition> & minmax_idx_condition,
MergeTreeData::DataPartsVector & parts,
std::vector<AlterConversionsPtr> & alter_conversions,
const std::optional<std::unordered_set<String>> & part_values,
const StorageMetadataPtr & metadata_snapshot,
const MergeTreeData & data,
@ -512,8 +509,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
LoggerPtr log,
ReadFromMergeTree::IndexStats & index_stats)
{
chassert(alter_conversions.empty() || parts.size() == alter_conversions.size());
const Settings & settings = context->getSettingsRef();
DataTypes minmax_columns_types;
@ -537,7 +532,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
if (query_context->getSettingsRef().allow_experimental_query_deduplication)
selectPartsToReadWithUUIDFilter(
parts,
alter_conversions,
part_values,
data.getPinnedPartUUIDs(),
minmax_idx_condition,
@ -550,7 +544,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
else
selectPartsToRead(
parts,
alter_conversions,
part_values,
minmax_idx_condition,
minmax_columns_types,
@ -589,7 +582,6 @@ void MergeTreeDataSelectExecutor::filterPartsByPartition(
RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipIndexes(
MergeTreeData::DataPartsVector && parts,
std::vector<AlterConversionsPtr> && alter_conversions,
StorageMetadataPtr metadata_snapshot,
const ContextPtr & context,
const KeyCondition & key_condition,
@ -602,8 +594,6 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
bool use_skip_indexes,
bool find_exact_ranges)
{
chassert(alter_conversions.empty() || parts.size() == alter_conversions.size());
RangesInDataParts parts_with_ranges;
parts_with_ranges.resize(parts.size());
const Settings & settings = context->getSettingsRef();
@ -662,11 +652,8 @@ RangesInDataParts MergeTreeDataSelectExecutor::filterPartsByPrimaryKeyAndSkipInd
auto process_part = [&](size_t part_index)
{
auto & part = parts[part_index];
auto alter_conversions_for_part = !alter_conversions.empty()
? alter_conversions[part_index]
: std::make_shared<AlterConversions>();
RangesInDataPart ranges(part, alter_conversions_for_part, part_index);
RangesInDataPart ranges(part, part_index);
size_t total_marks_count = part->index_granularity.getMarksCountWithoutFinal();
if (metadata_snapshot->hasPrimaryKey() || part_offset_condition)
@ -904,6 +891,7 @@ std::shared_ptr<QueryIdHolder> MergeTreeDataSelectExecutor::checkLimits(
ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMarksToRead(
MergeTreeData::DataPartsVector parts,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const Names & column_names_to_return,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -916,11 +904,9 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar
return std::make_shared<ReadFromMergeTree::AnalysisResult>();
std::optional<ReadFromMergeTree::Indexes> indexes;
/// NOTE: We don't need alter_conversions because the returned analysis_result is only used for:
/// 1. estimate the number of rows to read; 2. projection reading, which doesn't have alter_conversions.
return ReadFromMergeTree::selectRangesToRead(
std::move(parts),
/*alter_conversions=*/{},
mutations_snapshot,
metadata_snapshot,
query_info,
context,
@ -935,7 +921,7 @@ ReadFromMergeTree::AnalysisResultPtr MergeTreeDataSelectExecutor::estimateNumMar
QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
MergeTreeData::DataPartsVector parts,
std::vector<AlterConversionsPtr> alter_conversions,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const Names & column_names_to_return,
const StorageSnapshotPtr & storage_snapshot,
const SelectQueryInfo & query_info,
@ -957,7 +943,7 @@ QueryPlanStepPtr MergeTreeDataSelectExecutor::readFromParts(
return std::make_unique<ReadFromMergeTree>(
std::move(parts),
std::move(alter_conversions),
std::move(mutations_snapshot),
column_names_to_return,
data,
query_info,
@ -1554,7 +1540,6 @@ MarkRanges MergeTreeDataSelectExecutor::filterMarksUsingMergedIndex(
void MergeTreeDataSelectExecutor::selectPartsToRead(
MergeTreeData::DataPartsVector & parts,
std::vector<AlterConversionsPtr> & alter_conversions,
const std::optional<std::unordered_set<String>> & part_values,
const std::optional<KeyCondition> & minmax_idx_condition,
const DataTypes & minmax_columns_types,
@ -1563,14 +1548,11 @@ void MergeTreeDataSelectExecutor::selectPartsToRead(
PartFilterCounters & counters)
{
MergeTreeData::DataPartsVector prev_parts;
std::vector<AlterConversionsPtr> prev_conversions;
std::swap(prev_parts, parts);
std::swap(prev_conversions, alter_conversions);
for (size_t i = 0; i < prev_parts.size(); ++i)
for (const auto & part_or_projection : prev_parts)
{
const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get();
const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get();
if (part_values && part_values->find(part->name) == part_values->end())
continue;
@ -1607,15 +1589,12 @@ void MergeTreeDataSelectExecutor::selectPartsToRead(
counters.num_parts_after_partition_pruner += 1;
counters.num_granules_after_partition_pruner += num_granules;
parts.push_back(prev_parts[i]);
if (!prev_conversions.empty())
alter_conversions.push_back(prev_conversions[i]);
parts.push_back(part_or_projection);
}
}
void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
MergeTreeData::DataPartsVector & parts,
std::vector<AlterConversionsPtr> & alter_conversions,
const std::optional<std::unordered_set<String>> & part_values,
MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids,
const std::optional<KeyCondition> & minmax_idx_condition,
@ -1628,22 +1607,17 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
{
/// process_parts prepare parts that have to be read for the query,
/// returns false if duplicated parts' UUID have been met
auto select_parts = [&] (
MergeTreeData::DataPartsVector & selected_parts,
std::vector<AlterConversionsPtr> & selected_conversions) -> bool
auto select_parts = [&](MergeTreeData::DataPartsVector & selected_parts) -> bool
{
auto ignored_part_uuids = query_context->getIgnoredPartUUIDs();
std::unordered_set<UUID> temp_part_uuids;
MergeTreeData::DataPartsVector prev_parts;
std::vector<AlterConversionsPtr> prev_conversions;
std::swap(prev_parts, selected_parts);
std::swap(prev_conversions, selected_conversions);
for (size_t i = 0; i < prev_parts.size(); ++i)
for (const auto & part_or_projection : prev_parts)
{
const auto * part = prev_parts[i]->isProjectionPart() ? prev_parts[i]->getParentPart() : prev_parts[i].get();
const auto * part = part_or_projection->isProjectionPart() ? part_or_projection->getParentPart() : part_or_projection.get();
if (part_values && part_values->find(part->name) == part_values->end())
continue;
@ -1693,9 +1667,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
throw Exception(ErrorCodes::LOGICAL_ERROR, "Found a part with the same UUID on the same replica.");
}
selected_parts.push_back(prev_parts[i]);
if (!prev_conversions.empty())
selected_conversions.push_back(prev_conversions[i]);
selected_parts.push_back(part_or_projection);
}
if (!temp_part_uuids.empty())
@ -1714,7 +1686,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
};
/// Process parts that have to be read for a query.
auto needs_retry = !select_parts(parts, alter_conversions);
auto needs_retry = !select_parts(parts);
/// If any duplicated part UUIDs met during the first step, try to ignore them in second pass.
/// This may happen when `prefer_localhost_replica` is set and "distributed" stage runs in the same process with "remote" stage.
@ -1725,7 +1697,7 @@ void MergeTreeDataSelectExecutor::selectPartsToReadWithUUIDFilter(
counters = PartFilterCounters();
/// Second attempt didn't help, throw an exception
if (!select_parts(parts, alter_conversions))
if (!select_parts(parts))
throw Exception(ErrorCodes::DUPLICATED_PART_UUIDS, "Found duplicate UUIDs while processing query.");
}
}

View File

@ -40,7 +40,7 @@ public:
/// The same as read, but with specified set of parts.
QueryPlanStepPtr readFromParts(
MergeTreeData::DataPartsVector parts,
std::vector<AlterConversionsPtr> alter_conversions,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
const SelectQueryInfo & query_info,
@ -56,6 +56,7 @@ public:
/// This method is used to select best projection for table.
ReadFromMergeTree::AnalysisResultPtr estimateNumMarksToRead(
MergeTreeData::DataPartsVector parts,
MergeTreeData::MutationsSnapshotPtr mutations_snapshot,
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
const SelectQueryInfo & query_info,
@ -120,7 +121,6 @@ private:
/// as well as `max_block_number_to_read`.
static void selectPartsToRead(
MergeTreeData::DataPartsVector & parts,
std::vector<AlterConversionsPtr> & alter_conversions,
const std::optional<std::unordered_set<String>> & part_values,
const std::optional<KeyCondition> & minmax_idx_condition,
const DataTypes & minmax_columns_types,
@ -131,7 +131,6 @@ private:
/// Same as previous but also skip parts uuids if any to the query context, or skip parts which uuids marked as excluded.
static void selectPartsToReadWithUUIDFilter(
MergeTreeData::DataPartsVector & parts,
std::vector<AlterConversionsPtr> & alter_conversions,
const std::optional<std::unordered_set<String>> & part_values,
MergeTreeData::PinnedPartUUIDsPtr pinned_part_uuids,
const std::optional<KeyCondition> & minmax_idx_condition,
@ -175,10 +174,9 @@ public:
/// Filter parts using minmax index and partition key.
static void filterPartsByPartition(
MergeTreeData::DataPartsVector & parts,
const std::optional<PartitionPruner> & partition_pruner,
const std::optional<KeyCondition> & minmax_idx_condition,
MergeTreeData::DataPartsVector & parts,
std::vector<AlterConversionsPtr> & alter_conversions,
const std::optional<std::unordered_set<String>> & part_values,
const StorageMetadataPtr & metadata_snapshot,
const MergeTreeData & data,
@ -192,7 +190,6 @@ public:
/// If 'check_limits = true' it will throw exception if the amount of data exceed the limits from settings.
static RangesInDataParts filterPartsByPrimaryKeyAndSkipIndexes(
MergeTreeData::DataPartsVector && parts,
std::vector<AlterConversionsPtr> && alter_conversions,
StorageMetadataPtr metadata_snapshot,
const ContextPtr & context,
const KeyCondition & key_condition,

View File

@ -50,7 +50,7 @@ UInt64 MergeTreeMutationEntry::parseFileName(const String & file_name_)
MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskPtr disk_, const String & path_prefix_, UInt64 tmp_number,
const TransactionID & tid_, const WriteSettings & settings)
: create_time(time(nullptr))
, commands(std::move(commands_))
, commands(std::make_shared<MutationCommands>(std::move(commands_)))
, disk(std::move(disk_))
, path_prefix(path_prefix_)
, file_name("tmp_mutation_" + toString(tmp_number) + ".txt")
@ -63,7 +63,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(MutationCommands commands_, DiskP
*out << "format version: 1\n"
<< "create time: " << LocalDateTime(create_time, DateLUT::serverTimezoneInstance()) << "\n";
*out << "commands: ";
commands.writeText(*out, /* with_pure_metadata_commands = */ false);
commands->writeText(*out, /* with_pure_metadata_commands = */ false);
*out << "\n";
if (tid.isPrehistoric())
{
@ -116,7 +116,8 @@ void MergeTreeMutationEntry::writeCSN(CSN csn_)
}
MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & path_prefix_, const String & file_name_)
: disk(std::move(disk_))
: commands(std::make_shared<MutationCommands>())
, disk(std::move(disk_))
, path_prefix(path_prefix_)
, file_name(file_name_)
, is_temp(false)
@ -133,7 +134,7 @@ MergeTreeMutationEntry::MergeTreeMutationEntry(DiskPtr disk_, const String & pat
create_time_dt.hour(), create_time_dt.minute(), create_time_dt.second());
*buf >> "commands: ";
commands.readText(*buf);
commands->readText(*buf);
*buf >> "\n";
if (buf->eof())
@ -177,7 +178,7 @@ std::shared_ptr<const IBackupEntry> MergeTreeMutationEntry::backup() const
out << "block number: " << block_number << "\n";
out << "commands: ";
commands.writeText(out, /* with_pure_metadata_commands = */ false);
commands->writeText(out, /* with_pure_metadata_commands = */ false);
out << "\n";
return std::make_shared<BackupEntryFromMemory>(out.str());

View File

@ -16,7 +16,7 @@ class IBackupEntry;
struct MergeTreeMutationEntry
{
time_t create_time = 0;
MutationCommands commands;
std::shared_ptr<MutationCommands> commands;
DiskPtr disk;
String path_prefix;

View File

@ -85,6 +85,7 @@ MergeTreeReadTask::Readers MergeTreePrefetchedReadPool::PrefetchedReaders::get()
MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool(
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,
@ -95,6 +96,7 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool(
const ContextPtr & context_)
: MergeTreeReadPoolBase(
std::move(parts_),
std::move(mutations_snapshot_),
std::move(shared_virtual_fields_),
storage_snapshot_,
prewhere_info_,
@ -103,7 +105,6 @@ MergeTreePrefetchedReadPool::MergeTreePrefetchedReadPool(
column_names_,
settings_,
context_)
, WithContext(context_)
, prefetch_threadpool(getContext()->getPrefetchThreadpool())
, log(getLogger("MergeTreePrefetchedReadPool(" + (parts_ranges.empty() ? "" : parts_ranges.front().data_part->storage.getStorageID().getNameForLogs()) + ")"))
{

View File

@ -14,11 +14,12 @@ using MergeTreeReaderPtr = std::unique_ptr<IMergeTreeReader>;
/// A class which is responsible for creating read tasks
/// which are later taken by readers via getTask method.
/// Does prefetching for the read tasks it creates.
class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase, private WithContext
class MergeTreePrefetchedReadPool : public MergeTreeReadPoolBase
{
public:
MergeTreePrefetchedReadPool(
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,

View File

@ -29,6 +29,7 @@ extern const int BAD_ARGUMENTS;
MergeTreeReadPool::MergeTreeReadPool(
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,
@ -39,6 +40,7 @@ MergeTreeReadPool::MergeTreeReadPool(
const ContextPtr & context_)
: MergeTreeReadPoolBase(
std::move(parts_),
std::move(mutations_snapshot_),
std::move(shared_virtual_fields_),
storage_snapshot_,
prewhere_info_,

View File

@ -26,6 +26,7 @@ public:
MergeTreeReadPool(
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,

View File

@ -4,9 +4,6 @@
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
#include <Storages/MergeTree/MergeTreeBlockReadUtils.h>
#include <cmath>
namespace DB
{
@ -17,6 +14,7 @@ namespace ErrorCodes
MergeTreeReadPoolBase::MergeTreeReadPoolBase(
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,
@ -25,7 +23,9 @@ MergeTreeReadPoolBase::MergeTreeReadPoolBase(
const Names & column_names_,
const PoolSettings & pool_settings_,
const ContextPtr & context_)
: parts_ranges(std::move(parts_))
: WithContext(context_)
, parts_ranges(std::move(parts_))
, mutations_snapshot(std::move(mutations_snapshot_))
, shared_virtual_fields(std::move(shared_virtual_fields_))
, storage_snapshot(storage_snapshot_)
, prewhere_info(prewhere_info_)
@ -120,9 +120,9 @@ void MergeTreeReadPoolBase::fillPerPartInfos(const Settings & settings)
}
read_task_info.part_index_in_query = part_with_ranges.part_index_in_query;
read_task_info.alter_conversions = part_with_ranges.alter_conversions;
read_task_info.alter_conversions = MergeTreeData::getAlterConversionsForPart(part_with_ranges.data_part, mutations_snapshot, storage_snapshot->metadata, getContext());
LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, part_with_ranges.alter_conversions);
LoadedMergeTreeDataPartInfoForReader part_info(part_with_ranges.data_part, read_task_info.alter_conversions);
read_task_info.task_columns = getReadTaskColumns(
part_info,

View File

@ -6,9 +6,11 @@
namespace DB
{
class MergeTreeReadPoolBase : public IMergeTreeReadPool
class MergeTreeReadPoolBase : public IMergeTreeReadPool, protected WithContext
{
public:
using MutationsSnapshotPtr = MergeTreeData::MutationsSnapshotPtr;
struct PoolSettings
{
size_t threads = 0;
@ -23,6 +25,7 @@ public:
MergeTreeReadPoolBase(
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,
@ -37,6 +40,7 @@ public:
protected:
/// Initialized in constructor
const RangesInDataParts parts_ranges;
const MutationsSnapshotPtr mutations_snapshot;
const VirtualFields shared_virtual_fields;
const StorageSnapshotPtr storage_snapshot;
const PrewhereInfoPtr prewhere_info;

View File

@ -12,6 +12,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder(
bool has_limit_below_one_block_,
MergeTreeReadType read_type_,
RangesInDataParts parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,
@ -22,6 +23,7 @@ MergeTreeReadPoolInOrder::MergeTreeReadPoolInOrder(
const ContextPtr & context_)
: MergeTreeReadPoolBase(
std::move(parts_),
std::move(mutations_snapshot_),
std::move(shared_virtual_fields_),
storage_snapshot_,
prewhere_info_,

View File

@ -11,6 +11,7 @@ public:
bool has_limit_below_one_block_,
MergeTreeReadType read_type_,
RangesInDataParts parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,

View File

@ -14,6 +14,7 @@ namespace ErrorCodes
MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
ParallelReadingExtension extension_,
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,
@ -24,6 +25,7 @@ MergeTreeReadPoolParallelReplicas::MergeTreeReadPoolParallelReplicas(
const ContextPtr & context_)
: MergeTreeReadPoolBase(
std::move(parts_),
std::move(mutations_snapshot_),
std::move(shared_virtual_fields_),
storage_snapshot_,
prewhere_info_,

View File

@ -11,6 +11,7 @@ public:
MergeTreeReadPoolParallelReplicas(
ParallelReadingExtension extension_,
RangesInDataParts && parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,

View File

@ -13,6 +13,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd
ParallelReadingExtension extension_,
CoordinationMode mode_,
RangesInDataParts parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,
@ -23,6 +24,7 @@ MergeTreeReadPoolParallelReplicasInOrder::MergeTreeReadPoolParallelReplicasInOrd
const ContextPtr & context_)
: MergeTreeReadPoolBase(
std::move(parts_),
std::move(mutations_snapshot_),
std::move(shared_virtual_fields_),
storage_snapshot_,
prewhere_info_,

View File

@ -12,6 +12,7 @@ public:
ParallelReadingExtension extension_,
CoordinationMode mode_,
RangesInDataParts parts_,
MutationsSnapshotPtr mutations_snapshot_,
VirtualFields shared_virtual_fields_,
const StorageSnapshotPtr & storage_snapshot_,
const PrewhereInfoPtr & prewhere_info_,

View File

@ -34,6 +34,7 @@ public:
const MergeTreeData & storage_,
const StorageSnapshotPtr & storage_snapshot_,
MergeTreeData::DataPartPtr data_part_,
AlterConversionsPtr alter_conversions_,
Names columns_to_read_,
std::optional<MarkRanges> mark_ranges_,
bool apply_deleted_mask,
@ -58,6 +59,9 @@ private:
/// Data part will not be removed if the pointer owns it
MergeTreeData::DataPartPtr data_part;
/// Alter and mutation commands that are required to be applied to the part on-fly.
AlterConversionsPtr alter_conversions;
/// Columns we have to read (each Block from read will contain them)
Names columns_to_read;
@ -87,6 +91,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
const MergeTreeData & storage_,
const StorageSnapshotPtr & storage_snapshot_,
MergeTreeData::DataPartPtr data_part_,
AlterConversionsPtr alter_conversions_,
Names columns_to_read_,
std::optional<MarkRanges> mark_ranges_,
bool apply_deleted_mask,
@ -96,6 +101,7 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
, storage(storage_)
, storage_snapshot(storage_snapshot_)
, data_part(std::move(data_part_))
, alter_conversions(std::move(alter_conversions_))
, columns_to_read(std::move(columns_to_read_))
, read_with_direct_io(read_with_direct_io_)
, mark_ranges(std::move(mark_ranges_))
@ -109,8 +115,6 @@ MergeTreeSequentialSource::MergeTreeSequentialSource(
LOG_DEBUG(log, "Reading {} marks from part {}, total {} rows starting from the beginning of the part",
data_part->getMarksCount(), data_part->name, data_part->rows_count);
auto alter_conversions = storage.getAlterConversionsForPart(data_part);
/// Note, that we don't check setting collaborate_with_coordinator presence, because this source
/// is only used in background merges.
addTotalRowsApprox(data_part->rows_count);
@ -299,6 +303,7 @@ Pipe createMergeTreeSequentialSource(
const MergeTreeData & storage,
const StorageSnapshotPtr & storage_snapshot,
MergeTreeData::DataPartPtr data_part,
AlterConversionsPtr alter_conversions,
Names columns_to_read,
std::optional<MarkRanges> mark_ranges,
std::shared_ptr<std::atomic<size_t>> filtered_rows_count,
@ -315,7 +320,8 @@ Pipe createMergeTreeSequentialSource(
columns_to_read.emplace_back(RowExistsColumn::name);
auto column_part_source = std::make_shared<MergeTreeSequentialSource>(type,
storage, storage_snapshot, data_part, columns_to_read, std::move(mark_ranges),
storage, storage_snapshot, data_part, alter_conversions,
columns_to_read, std::move(mark_ranges),
/*apply_deleted_mask=*/ false, read_with_direct_io, prefetch);
Pipe pipe(std::move(column_part_source));
@ -346,6 +352,7 @@ public:
const MergeTreeData & storage_,
const StorageSnapshotPtr & storage_snapshot_,
MergeTreeData::DataPartPtr data_part_,
AlterConversionsPtr alter_conversions_,
Names columns_to_read_,
bool apply_deleted_mask_,
std::optional<ActionsDAG> filter_,
@ -356,6 +363,7 @@ public:
, storage(storage_)
, storage_snapshot(storage_snapshot_)
, data_part(std::move(data_part_))
, alter_conversions(std::move(alter_conversions_))
, columns_to_read(std::move(columns_to_read_))
, apply_deleted_mask(apply_deleted_mask_)
, filter(std::move(filter_))
@ -399,6 +407,7 @@ public:
storage,
storage_snapshot,
data_part,
alter_conversions,
columns_to_read,
std::move(mark_ranges),
/*filtered_rows_count=*/ nullptr,
@ -414,6 +423,7 @@ private:
const MergeTreeData & storage;
StorageSnapshotPtr storage_snapshot;
MergeTreeData::DataPartPtr data_part;
AlterConversionsPtr alter_conversions;
Names columns_to_read;
bool apply_deleted_mask;
std::optional<ActionsDAG> filter;
@ -427,6 +437,7 @@ void createReadFromPartStep(
const MergeTreeData & storage,
const StorageSnapshotPtr & storage_snapshot,
MergeTreeData::DataPartPtr data_part,
AlterConversionsPtr alter_conversions,
Names columns_to_read,
bool apply_deleted_mask,
std::optional<ActionsDAG> filter,
@ -434,7 +445,8 @@ void createReadFromPartStep(
LoggerPtr log)
{
auto reading = std::make_unique<ReadFromPart>(type,
storage, storage_snapshot, std::move(data_part),
storage, storage_snapshot,
std::move(data_part), std::move(alter_conversions),
std::move(columns_to_read), apply_deleted_mask,
std::move(filter), std::move(context), log);

View File

@ -21,6 +21,7 @@ Pipe createMergeTreeSequentialSource(
const MergeTreeData & storage,
const StorageSnapshotPtr & storage_snapshot,
MergeTreeData::DataPartPtr data_part,
AlterConversionsPtr alter_conversions,
Names columns_to_read,
std::optional<MarkRanges> mark_ranges,
std::shared_ptr<std::atomic<size_t>> filtered_rows_count,
@ -36,6 +37,7 @@ void createReadFromPartStep(
const MergeTreeData & storage,
const StorageSnapshotPtr & storage_snapshot,
MergeTreeData::DataPartPtr data_part,
AlterConversionsPtr alter_conversions,
Names columns_to_read,
bool apply_deleted_mask,
std::optional<ActionsDAG> filter,

View File

@ -115,6 +115,7 @@ static UInt64 getExistingRowsCount(const Block & block)
static void splitAndModifyMutationCommands(
MergeTreeData::DataPartPtr part,
StorageMetadataPtr metadata_snapshot,
AlterConversionsPtr alter_conversions,
const MutationCommands & commands,
MutationCommands & for_interpreter,
MutationCommands & for_file_renames,
@ -180,8 +181,6 @@ static void splitAndModifyMutationCommands(
}
auto alter_conversions = part->storage.getAlterConversionsForPart(part);
/// We don't add renames from commands, instead we take them from rename_map.
/// It's important because required renames depend not only on part's data version (i.e. mutation version)
/// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter
@ -297,7 +296,6 @@ static void splitAndModifyMutationCommands(
}
}
auto alter_conversions = part->storage.getAlterConversionsForPart(part);
/// We don't add renames from commands, instead we take them from rename_map.
/// It's important because required renames depend not only on part's data version (i.e. mutation version)
/// but also on part's metadata version. Why we have such logic only for renames? Because all other types of alter
@ -2042,6 +2040,15 @@ bool MutateTask::prepare()
ctx->num_mutations = std::make_unique<CurrentMetrics::Increment>(CurrentMetrics::PartMutation);
MergeTreeData::IMutationsSnapshot::Params params
{
.metadata_version = ctx->metadata_snapshot->getMetadataVersion(),
.min_part_metadata_version = ctx->source_part->getMetadataVersion(),
};
auto mutations_snapshot = ctx->data->getMutationsSnapshot(params);
auto alter_conversions = MergeTreeData::getAlterConversionsForPart(ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->context);
auto context_for_reading = Context::createCopy(ctx->context);
/// Allow mutations to work when force_index_by_date or force_primary_key is on.
@ -2056,7 +2063,7 @@ bool MutateTask::prepare()
ctx->commands_for_part.emplace_back(command);
if (ctx->source_part->isStoredOnDisk() && !isStorageTouchedByMutations(
ctx->source_part, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading))
ctx->source_part, mutations_snapshot, ctx->metadata_snapshot, ctx->commands_for_part, context_for_reading))
{
NameSet files_to_copy_instead_of_hardlinks;
auto settings_ptr = ctx->data->getSettings();
@ -2116,8 +2123,13 @@ bool MutateTask::prepare()
context_for_reading->setSetting("read_from_filesystem_cache_if_exists_otherwise_bypass_cache", 1);
MutationHelpers::splitAndModifyMutationCommands(
ctx->source_part, ctx->metadata_snapshot,
ctx->commands_for_part, ctx->for_interpreter, ctx->for_file_renames, ctx->log);
ctx->source_part,
ctx->metadata_snapshot,
alter_conversions,
ctx->commands_for_part,
ctx->for_interpreter,
ctx->for_file_renames,
ctx->log);
ctx->stage_progress = std::make_unique<MergeStageProgress>(1.0);
@ -2131,7 +2143,8 @@ bool MutateTask::prepare()
settings.apply_deleted_mask = false;
ctx->interpreter = std::make_unique<MutationsInterpreter>(
*ctx->data, ctx->source_part, ctx->metadata_snapshot, ctx->for_interpreter,
*ctx->data, ctx->source_part, alter_conversions,
ctx->metadata_snapshot, ctx->for_interpreter,
ctx->metadata_snapshot->getColumns().getNamesOfPhysical(), context_for_reading, settings);
ctx->materialized_indices = ctx->interpreter->grabMaterializedIndices();

View File

@ -42,7 +42,6 @@ struct RangesInDataPartsDescription: public std::deque<RangesInDataPartDescripti
struct RangesInDataPart
{
DataPartPtr data_part;
AlterConversionsPtr alter_conversions;
size_t part_index_in_query;
MarkRanges ranges;
MarkRanges exact_ranges;
@ -51,14 +50,13 @@ struct RangesInDataPart
RangesInDataPart(
const DataPartPtr & data_part_,
const AlterConversionsPtr & alter_conversions_,
const size_t part_index_in_query_,
const MarkRanges & ranges_ = MarkRanges{})
: data_part{data_part_}
, alter_conversions{alter_conversions_}
, part_index_in_query{part_index_in_query_}
, ranges{ranges_}
{}
{
}
RangesInDataPartDescription getDescription() const;

View File

@ -135,7 +135,6 @@ struct ReplicatedMergeTreeLogEntryData
int alter_version = -1; /// May be equal to -1, if it's normal mutation, not metadata update.
/// only ALTER METADATA command
/// NOTE It's never used
bool have_mutation = false; /// If this alter requires additional mutation step, for data update
String columns_str; /// New columns data corresponding to alter_version

View File

@ -950,7 +950,7 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper
{
const auto commands = entry.commands;
it = mutations_by_znode.erase(it);
updateAlterConversionsMutations(commands, alter_conversions_mutations, /* remove= */ true);
decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, commands);
}
else
it = mutations_by_znode.erase(it);
@ -999,10 +999,9 @@ int32_t ReplicatedMergeTreeQueue::updateMutations(zkutil::ZooKeeperPtr zookeeper
for (const ReplicatedMergeTreeMutationEntryPtr & entry : new_mutations)
{
auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version))
.first->second;
auto & mutation = mutations_by_znode.emplace(entry->znode_name, MutationStatus(entry, format_version)).first->second;
incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands);
updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ false);
NOEXCEPT_SCOPE({
for (const auto & pair : entry->block_numbers)
{
@ -1076,7 +1075,7 @@ ReplicatedMergeTreeMutationEntryPtr ReplicatedMergeTreeQueue::removeMutation(
}
mutations_by_znode.erase(it);
/// updateAlterConversionsMutations() will be called in updateMutations()
/// decrementMutationsCounters() will be called in updateMutations()
LOG_DEBUG(log, "Removed mutation {} from local state.", entry->znode_name);
}
@ -1901,50 +1900,47 @@ ReplicatedMergeTreeMergePredicate ReplicatedMergeTreeQueue::getMergePredicate(zk
}
MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const
MutationCommands ReplicatedMergeTreeQueue::MutationsSnapshot::getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const
{
int32_t part_metadata_version = part->getMetadataVersion();
int32_t metadata_version = storage.getInMemoryMetadataPtr()->getMetadataVersion();
chassert(alter_conversions_mutations >= 0);
/// NOTE: that just checking part_metadata_version is not enough, since we
/// need to check for non-metadata mutations as well.
if (alter_conversions_mutations == 0 && metadata_version == part_metadata_version)
return {};
std::unique_lock lock(state_mutex);
auto in_partition = mutations_by_partition.find(part->info.partition_id);
if (in_partition == mutations_by_partition.end())
return {};
Int64 part_data_version = part->info.getDataVersion();
Int64 part_metadata_version = part->getMetadataVersion();
MutationCommands result;
bool seen_all_data_mutations = false;
bool seen_all_metadata_mutations = false;
bool seen_all_data_mutations = !hasDataMutations();
bool seen_all_metadata_mutations = part_metadata_version >= params.metadata_version;
if (seen_all_data_mutations && seen_all_metadata_mutations)
return {};
auto add_to_result = [&](const ReplicatedMergeTreeMutationEntryPtr & entry)
{
for (const auto & command : entry->commands | std::views::reverse)
if (AlterConversions::supportsMutationCommandType(command.type))
result.emplace_back(command);
{
if (AlterConversions::isSupportedMetadataMutation(command.type))
result.push_back(command);
else if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type))
result.push_back(command);
}
};
/// Here we return mutation commands for part which has bigger alter version than part metadata version.
/// Please note, we don't use getDataVersion(). It's because these alter commands are used for in-fly conversions
/// of part's metadata.
for (const auto & [mutation_version, mutation_status] : in_partition->second | std::views::reverse)
for (const auto & [mutation_version, entry] : in_partition->second | std::views::reverse)
{
if (seen_all_data_mutations && seen_all_metadata_mutations)
break;
auto & entry = mutation_status->entry;
auto alter_version = entry->alter_version;
if (alter_version != -1)
{
if (alter_version > metadata_version)
if (seen_all_metadata_mutations || alter_version > params.metadata_version)
continue;
/// We take commands with bigger metadata version
@ -1953,7 +1949,7 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const
else
seen_all_metadata_mutations = true;
}
else
else if (!seen_all_data_mutations)
{
if (mutation_version > part_data_version)
add_to_result(entry);
@ -1965,6 +1961,104 @@ MutationCommands ReplicatedMergeTreeQueue::getAlterMutationCommandsForPart(const
return result;
}
NameSet ReplicatedMergeTreeQueue::MutationsSnapshot::getAllUpdatedColumns() const
{
if (!hasDataMutations())
return {};
NameSet res;
for (const auto & [partition_id, mutations] : mutations_by_partition)
{
for (const auto & [version, entry] : mutations)
{
auto names = entry->commands.getAllUpdatedColumns();
std::move(names.begin(), names.end(), std::inserter(res, res.end()));
}
}
return res;
}
MergeTreeData::MutationsSnapshotPtr ReplicatedMergeTreeQueue::getMutationsSnapshot(const MutationsSnapshot::Params & params) const
{
std::lock_guard lock(state_mutex);
MutationsSnapshot::Info info
{
.num_data_mutations = num_data_mutations_to_apply,
.num_metadata_mutations = num_metadata_mutations_to_apply,
};
auto res = std::make_shared<MutationsSnapshot>(params, std::move(info));
bool need_data_mutations = res->hasDataMutations();
bool need_metatadata_mutations = params.min_part_metadata_version < params.metadata_version;
if (!need_data_mutations && !need_metatadata_mutations)
return res;
auto is_supported_command = [&](const auto & command)
{
if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type))
return true;
if (need_metatadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type))
return true;
return false;
};
for (const auto & [partition_id, mutations] : mutations_by_partition)
{
auto & in_partition = res->mutations_by_partition[partition_id];
bool seen_all_data_mutations = !need_data_mutations;
bool seen_all_metadata_mutations = !need_metatadata_mutations;
for (const auto & [mutation_version, status] : mutations | std::views::reverse)
{
if (seen_all_data_mutations && seen_all_metadata_mutations)
break;
auto alter_version = status->entry->alter_version;
if (alter_version != -1)
{
if (seen_all_metadata_mutations || alter_version > params.metadata_version)
continue;
/// We take commands with bigger metadata version
if (alter_version > params.min_part_metadata_version)
{
/// Copy a pointer to the whole entry to avoid extracting and copying commands.
/// Required commands will be copied later only for specific parts.
if (std::ranges::any_of(status->entry->commands, is_supported_command))
in_partition.emplace(mutation_version, status->entry);
}
else
{
seen_all_metadata_mutations = true;
}
}
else if (!seen_all_data_mutations)
{
if (!status->is_done)
{
/// Copy a pointer to the whole entry to avoid extracting and copying commands.
/// Required commands will be copied later only for specific parts.
if (std::ranges::any_of(status->entry->commands, is_supported_command))
in_partition.emplace(mutation_version, status->entry);
}
else
{
seen_all_data_mutations = true;
}
}
}
}
return res;
}
MutationCommands ReplicatedMergeTreeQueue::getMutationCommands(
const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version, Strings & mutation_ids) const
{
@ -2045,7 +2139,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep
mutation.parts_to_do.clear();
}
updateAlterConversionsMutations(mutation.entry->commands, alter_conversions_mutations, /* remove= */ true);
decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, mutation.entry->commands);
}
else if (mutation.parts_to_do.size() == 0)
{
@ -2102,7 +2196,7 @@ bool ReplicatedMergeTreeQueue::tryFinalizeMutations(zkutil::ZooKeeperPtr zookeep
LOG_TRACE(log, "Finishing data alter with version {} for entry {}", entry->alter_version, entry->znode_name);
alter_sequence.finishDataAlter(entry->alter_version, lock);
}
updateAlterConversionsMutations(entry->commands, alter_conversions_mutations, /* remove= */ true);
decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, entry->commands);
}
}
}

View File

@ -1,5 +1,6 @@
#pragma once
#include <cstdint>
#include <optional>
#include <Common/ActionBlocker.h>
@ -151,8 +152,11 @@ private:
/// Mapping from znode path to Mutations Status
std::map<String, MutationStatus> mutations_by_znode;
/// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart())
std::atomic<ssize_t> alter_conversions_mutations = 0;
/// Unfinished mutations that are required for AlterConversions.
Int64 num_data_mutations_to_apply = 0;
Int64 num_metadata_mutations_to_apply = 0;
/// Partition -> (block_number -> MutationStatus)
std::unordered_map<String, std::map<Int64, MutationStatus *>> mutations_by_partition;
/// Znode ID of the latest mutation that is done.
@ -409,10 +413,26 @@ public:
MutationCommands getMutationCommands(const MergeTreeData::DataPartPtr & part, Int64 desired_mutation_version,
Strings & mutation_ids) const;
struct MutationsSnapshot : public MergeTreeData::IMutationsSnapshot
{
public:
MutationsSnapshot() = default;
MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {}
using Params = MergeTreeData::IMutationsSnapshot::Params;
using MutationsByPartititon = std::unordered_map<String, std::map<Int64, ReplicatedMergeTreeMutationEntryPtr>>;
MutationsByPartititon mutations_by_partition;
MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override;
std::shared_ptr<MergeTreeData::IMutationsSnapshot> cloneEmpty() const override { return std::make_shared<MutationsSnapshot>(); }
NameSet getAllUpdatedColumns() const override;
};
/// Return mutation commands for part which could be not applied to
/// it according to part mutation version. Used when we apply alter commands on fly,
/// without actual data modification on disk.
MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const;
MergeTreeData::MutationsSnapshotPtr getMutationsSnapshot(const MutationsSnapshot::Params & params) const;
/// Mark finished mutations as done. If the function needs to be called again at some later time
/// (because some mutations are probably done but we are not sure yet), returns true.

View File

@ -30,7 +30,7 @@ void StorageFromMergeTreeDataPart::read(
{
query_plan.addStep(MergeTreeDataSelectExecutor(storage).readFromParts(
parts,
alter_conversions,
mutations_snapshot,
column_names,
storage_snapshot,
query_info,

View File

@ -18,10 +18,12 @@ class StorageFromMergeTreeDataPart final : public IStorage
{
public:
/// Used in part mutation.
explicit StorageFromMergeTreeDataPart(const MergeTreeData::DataPartPtr & part_)
explicit StorageFromMergeTreeDataPart(
const MergeTreeData::DataPartPtr & part_,
const MergeTreeData::MutationsSnapshotPtr & mutations_snapshot_)
: IStorage(getIDFromPart(part_))
, parts({part_})
, alter_conversions({part_->storage.getAlterConversionsForPart(part_)})
, mutations_snapshot(mutations_snapshot_)
, storage(part_->storage)
, partition_id(part_->info.partition_id)
{
@ -81,7 +83,7 @@ public:
private:
const MergeTreeData::DataPartsVector parts;
const std::vector<AlterConversionsPtr> alter_conversions;
const MergeTreeData::MutationsSnapshotPtr mutations_snapshot;
const MergeTreeData & storage;
const String partition_id;
const ReadFromMergeTree::AnalysisResultPtr analysis_result_ptr;

View File

@ -22,6 +22,7 @@ namespace ErrorCodes
{
extern const int UNKNOWN_MUTATION_COMMAND;
extern const int MULTIPLE_ASSIGNMENTS_TO_COLUMN;
extern const int LOGICAL_ERROR;
}
@ -115,12 +116,17 @@ std::optional<MutationCommand> MutationCommand::parse(ASTAlterCommand * command,
res.column_name = getIdentifierName(command->column);
return res;
}
else if (parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN)
/// MODIFY COLUMN x REMOVE MATERIALIZED/RESET SETTING/MODIFY SETTING is a valid alter command, but doesn't have any specified column type,
/// thus no mutation is needed
else if (
parse_alter_commands && command->type == ASTAlterCommand::MODIFY_COLUMN && command->remove_property.empty() && nullptr == command->settings_changes && nullptr == command->settings_resets)
{
MutationCommand res;
res.ast = command->ptr();
res.type = MutationCommand::Type::READ_COLUMN;
const auto & ast_col_decl = command->col_decl->as<ASTColumnDeclaration &>();
if (nullptr == ast_col_decl.type)
throw Exception(ErrorCodes::LOGICAL_ERROR, "MODIFY COLUMN mutation command doesn't specify type: {}", serializeAST(*command));
res.column_name = ast_col_decl.name;
res.data_type = DataTypeFactory::instance().get(ast_col_decl.type);
return res;
@ -268,4 +274,13 @@ bool MutationCommands::containBarrierCommand() const
return false;
}
NameSet MutationCommands::getAllUpdatedColumns() const
{
NameSet res;
for (const auto & command : *this)
for (const auto & [column_name, _] : command.column_to_update_expression)
res.insert(column_name);
return res;
}
}

View File

@ -92,6 +92,7 @@ public:
/// stick with other commands. Commands from one set have already been validated
/// to be executed without issues on the creation state.
bool containBarrierCommand() const;
NameSet getAllUpdatedColumns() const;
};
using MutationCommandsConstPtr = std::shared_ptr<MutationCommands>;

View File

@ -1625,9 +1625,11 @@ std::tuple<bool /* is_regexp */, ASTPtr> StorageMerge::evaluateDatabaseName(cons
return {false, ast};
}
bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr & storage_snapshot, ContextPtr ctx) const
bool StorageMerge::supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr ctx) const
{
return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(storage_snapshot, ctx); }) == nullptr;
/// Here we actually need storage snapshot of all nested tables.
/// But to avoid complexity pass nullptr to make more lightweight check in MergeTreeData.
return getFirstTable([&](const auto & table) { return !table->supportsTrivialCountOptimization(nullptr, ctx); }) == nullptr;
}
std::optional<UInt64> StorageMerge::totalRows(const Settings & settings) const

View File

@ -512,21 +512,14 @@ Int64 StorageMergeTree::startMutation(const MutationCommands & commands, Context
if (txn)
txn->addMutation(shared_from_this(), mutation_id);
bool alter_conversions_mutations_updated = updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false);
{
std::lock_guard lock(currently_processing_in_background_mutex);
bool inserted = current_mutations_by_version.try_emplace(version, std::move(entry)).second;
auto [it, inserted] = current_mutations_by_version.try_emplace(version, std::move(entry));
if (!inserted)
{
if (alter_conversions_mutations_updated)
{
--alter_conversions_mutations;
chassert(alter_conversions_mutations >= 0);
}
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", version);
}
incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands);
}
LOG_INFO(log, "Added mutation: {}{}", mutation_id, additional_info);
@ -563,7 +556,7 @@ void StorageMergeTree::updateMutationEntriesErrors(FutureMergedMutatedPartPtr re
if (static_cast<UInt64>(result_part->part_info.mutation) == it->first)
mutation_backoff_policy.removePartFromFailed(failed_part->name);
updateAlterConversionsMutations(it->second.commands, alter_conversions_mutations, /* remove= */ true);
decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry.commands);
}
}
else
@ -763,17 +756,15 @@ std::map<std::string, MutationCommands> StorageMergeTree::getUnfinishedMutationC
std::map<std::string, MutationCommands> result;
for (const auto & kv : current_mutations_by_version)
for (const auto & [mutation_version, entry] : current_mutations_by_version)
{
Int64 mutation_version = kv.first;
const MergeTreeMutationEntry & entry = kv.second;
const PartVersionWithName needle{mutation_version, ""};
const PartVersionWithName needle{static_cast<Int64>(mutation_version), ""};
auto versions_it = std::lower_bound(
part_versions_with_names.begin(), part_versions_with_names.end(), needle, comparator);
size_t parts_to_do = versions_it - part_versions_with_names.begin();
if (parts_to_do > 0)
result.emplace(entry.file_name, entry.commands);
result.emplace(entry.file_name, *entry.commands);
}
return result;
}
@ -806,7 +797,7 @@ std::vector<MergeTreeMutationStatus> StorageMergeTree::getMutationsStatus() cons
std::map<String, Int64> block_numbers_map({{"", entry.block_number}});
for (const MutationCommand & command : entry.commands)
for (const MutationCommand & command : *entry.commands)
{
WriteBufferFromOwnString buf;
formatAST(*command.ast, buf, false, true);
@ -843,19 +834,14 @@ CancellationCode StorageMergeTree::killMutation(const String & mutation_id)
auto it = current_mutations_by_version.find(mutation_version);
if (it != current_mutations_by_version.end())
{
bool mutation_finished = true;
if (std::optional<Int64> min_version = getMinPartDataVersion())
mutation_finished = *min_version > static_cast<Int64>(mutation_version);
{
bool mutation_finished = *min_version > static_cast<Int64>(mutation_version);
if (!mutation_finished)
decrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *it->second.commands);
}
to_kill.emplace(std::move(it->second));
if (!mutation_finished)
{
const auto commands = it->second.commands;
current_mutations_by_version.erase(it);
updateAlterConversionsMutations(commands, alter_conversions_mutations, /* remove= */ true);
}
else
current_mutations_by_version.erase(it);
}
}
@ -904,6 +890,8 @@ void StorageMergeTree::loadDeduplicationLog()
void StorageMergeTree::loadMutations()
{
std::lock_guard lock(currently_processing_in_background_mutex);
for (const auto & disk : getDisks())
{
for (auto it = disk->iterateDirectory(relative_data_path); it->isValid(); it->next())
@ -912,7 +900,7 @@ void StorageMergeTree::loadMutations()
{
MergeTreeMutationEntry entry(disk, relative_data_path, it->name());
UInt64 block_number = entry.block_number;
LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands.size());
LOG_DEBUG(log, "Loading mutation: {} entry, commands size: {}", it->name(), entry.commands->size());
if (!entry.tid.isPrehistoric() && !entry.csn)
{
@ -931,10 +919,11 @@ void StorageMergeTree::loadMutations()
}
}
auto inserted = current_mutations_by_version.try_emplace(block_number, std::move(entry)).second;
auto [entry_it, inserted] = current_mutations_by_version.try_emplace(block_number, std::move(entry));
if (!inserted)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Mutation {} already exists, it's a bug", block_number);
updateAlterConversionsMutations(entry.commands, alter_conversions_mutations, /* remove= */ false);
incrementMutationsCounters(num_data_mutations_to_apply, num_metadata_mutations_to_apply, *entry_it->second.commands);
}
else if (startsWith(it->name(), "tmp_mutation_"))
{
@ -1282,7 +1271,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate(
size_t commands_size = 0;
MutationCommands commands_for_size_validation;
for (const auto & command : it->second.commands)
for (const auto & command : *it->second.commands)
{
if (command.type != MutationCommand::Type::DROP_COLUMN
&& command.type != MutationCommand::Type::DROP_INDEX
@ -1326,11 +1315,11 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate(
const auto & single_mutation_commands = it->second.commands;
if (single_mutation_commands.containBarrierCommand())
if (single_mutation_commands->containBarrierCommand())
{
if (commands->empty())
{
commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end());
commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end());
last_mutation_to_apply = it;
}
break;
@ -1338,7 +1327,7 @@ MergeMutateSelectedEntryPtr StorageMergeTree::selectPartsToMutate(
else
{
current_ast_elements += commands_size;
commands->insert(commands->end(), single_mutation_commands.begin(), single_mutation_commands.end());
commands->insert(commands->end(), single_mutation_commands->begin(), single_mutation_commands->end());
last_mutation_to_apply = it;
}
@ -2455,34 +2444,82 @@ void StorageMergeTree::attachRestoredParts(MutableDataPartsVector && parts)
}
}
MutationCommands StorageMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const
MutationCommands StorageMergeTree::MutationsSnapshot::getAlterMutationCommandsForPart(const DataPartPtr & part) const
{
/// NOTE: there is no need to check part metadata_version, since
/// ALTER_METADATA cannot be done asynchronously, like in
/// ReplicatedMergeTree.
chassert(alter_conversions_mutations >= 0);
if (alter_conversions_mutations == 0)
return {};
std::lock_guard lock(currently_processing_in_background_mutex);
UInt64 part_data_version = part->info.getDataVersion();
MutationCommands result;
UInt64 part_data_version = part->info.getDataVersion();
for (const auto & [mutation_version, entry] : current_mutations_by_version | std::views::reverse)
for (const auto & [mutation_version, commands] : mutations_by_version | std::views::reverse)
{
if (mutation_version <= part_data_version)
break;
for (const auto & command : entry.commands | std::views::reverse)
if (AlterConversions::supportsMutationCommandType(command.type))
result.emplace_back(command);
for (const auto & command : *commands | std::views::reverse)
{
if (params.need_data_mutations && AlterConversions::isSupportedDataMutation(command.type))
result.push_back(command);
else if (AlterConversions::isSupportedMetadataMutation(command.type))
result.push_back(command);
}
}
return result;
}
NameSet StorageMergeTree::MutationsSnapshot::getAllUpdatedColumns() const
{
if (!hasDataMutations())
return {};
NameSet res;
for (const auto & [version, commands] : mutations_by_version)
{
auto names = commands->getAllUpdatedColumns();
std::move(names.begin(), names.end(), std::inserter(res, res.end()));
}
return res;
}
MergeTreeData::MutationsSnapshotPtr StorageMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const
{
std::lock_guard lock(currently_processing_in_background_mutex);
MutationsSnapshot::Info info
{
.num_data_mutations = num_data_mutations_to_apply,
.num_metadata_mutations = num_metadata_mutations_to_apply,
};
auto res = std::make_shared<MutationsSnapshot>(params, std::move(info));
bool need_data_mutations = res->hasDataMutations();
bool need_metadata_mutations = num_metadata_mutations_to_apply > 0;
if (!need_data_mutations && !need_metadata_mutations)
return res;
for (const auto & [version, entry] : current_mutations_by_version)
{
bool has_required_command = std::ranges::any_of(*entry.commands, [&](const auto & command)
{
if (need_data_mutations && AlterConversions::isSupportedDataMutation(command.type))
return true;
if (need_metadata_mutations && AlterConversions::isSupportedMetadataMutation(command.type))
return true;
return false;
});
/// Copy a pointer to all commands to avoid extracting and copying them.
/// Required commands will be copied later only for specific parts.
if (has_required_command)
res->mutations_by_version.emplace(version, entry.commands);
}
return res;
}
void StorageMergeTree::startBackgroundMovesIfNeeded()
{
if (areBackgroundMovesNeeded())

View File

@ -147,8 +147,10 @@ private:
DataParts currently_merging_mutating_parts;
std::map<UInt64, MergeTreeMutationEntry> current_mutations_by_version;
/// Unfinished mutations that is required AlterConversions (see getAlterMutationCommandsForPart())
std::atomic<ssize_t> alter_conversions_mutations = 0;
/// Unfinished mutations that are required for AlterConversions.
Int64 num_data_mutations_to_apply = 0;
Int64 num_metadata_mutations_to_apply = 0;
std::atomic<bool> shutdown_called {false};
std::atomic<bool> flush_called {false};
@ -308,9 +310,20 @@ private:
ContextPtr context;
};
protected:
/// Collect mutations that have to be applied on the fly: currently they are only RENAME COLUMN.
MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override;
struct MutationsSnapshot : public IMutationsSnapshot
{
MutationsSnapshot() = default;
MutationsSnapshot(Params params_, Info info_) : IMutationsSnapshot(std::move(params_), std::move(info_)) {}
using MutationsByVersion = std::map<UInt64, std::shared_ptr<const MutationCommands>>;
MutationsByVersion mutations_by_version;
MutationCommands getAlterMutationCommandsForPart(const MergeTreeData::DataPartPtr & part) const override;
std::shared_ptr<MergeTreeData::IMutationsSnapshot> cloneEmpty() const override { return std::make_shared<MutationsSnapshot>(); }
NameSet getAllUpdatedColumns() const override;
};
MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override;
};
}

View File

@ -9208,13 +9208,11 @@ bool StorageReplicatedMergeTree::canUseAdaptiveGranularity() const
(!has_non_adaptive_index_granularity_parts && !other_replicas_fixed_granularity));
}
MutationCommands StorageReplicatedMergeTree::getAlterMutationCommandsForPart(const DataPartPtr & part) const
MergeTreeData::MutationsSnapshotPtr StorageReplicatedMergeTree::getMutationsSnapshot(const IMutationsSnapshot::Params & params) const
{
return queue.getAlterMutationCommandsForPart(part);
return queue.getMutationsSnapshot(params);
}
void StorageReplicatedMergeTree::startBackgroundMovesIfNeeded()
{
if (areBackgroundMovesNeeded())

View File

@ -932,7 +932,7 @@ private:
void waitMutationToFinishOnReplicas(
const Strings & replicas, const String & mutation_id) const;
MutationCommands getAlterMutationCommandsForPart(const DataPartPtr & part) const override;
MutationsSnapshotPtr getMutationsSnapshot(const IMutationsSnapshot::Params & params) const override;
void startBackgroundMovesIfNeeded() override;

View File

@ -417,15 +417,13 @@ class Backport:
f"v{branch}-must-backport" for branch in self.release_branches
]
else:
fetch_release_prs = self.gh.get_release_pulls(self._fetch_from)
fetch_release_branches = [pr.head.ref for pr in fetch_release_prs]
self.labels_to_backport = [
(
f"v{branch}-must-backport"
if self._repo_name == "ClickHouse/ClickHouse"
else f"v{branch.replace('release/','')}-must-backport"
)
for branch in fetch_release_branches
for branch in self.release_branches
]
logging.info("Fetching from %s", self._fetch_from)

View File

@ -62,7 +62,6 @@ class Runners(metaclass=WithIter):
STYLE_CHECKER_ARM = "style-checker-aarch64"
FUNC_TESTER = "func-tester"
FUNC_TESTER_ARM = "func-tester-aarch64"
STRESS_TESTER = "stress-tester"
FUZZER_UNIT_TESTER = "fuzzer-unit-tester"
@ -456,7 +455,7 @@ class CommonJobConfigs:
docker=["clickhouse/stress-test"],
),
run_command="stress_check.py",
runner_type=Runners.STRESS_TESTER,
runner_type=Runners.FUNC_TESTER,
timeout=9000,
)
UPGRADE_TEST = JobConfig(
@ -467,7 +466,7 @@ class CommonJobConfigs:
docker=["clickhouse/stress-test"],
),
run_command="upgrade_check.py",
runner_type=Runners.STRESS_TESTER,
runner_type=Runners.FUNC_TESTER,
timeout=3600,
)
INTEGRATION_TEST = JobConfig(
@ -482,7 +481,7 @@ class CommonJobConfigs:
docker=IMAGES.copy(),
),
run_command='integration_test_check.py "$CHECK_NAME"',
runner_type=Runners.STRESS_TESTER,
runner_type=Runners.FUNC_TESTER,
)
ASTFUZZER_TEST = JobConfig(
job_name_keyword="ast",
@ -517,7 +516,7 @@ class CommonJobConfigs:
docker=["clickhouse/performance-comparison"],
),
run_command="performance_comparison_check.py",
runner_type=Runners.STRESS_TESTER,
runner_type=Runners.FUNC_TESTER,
)
SQLLANCER_TEST = JobConfig(
job_name_keyword="lancer",

View File

@ -1,33 +0,0 @@
#!/usr/bin/env bash
set -xeuo pipefail
bash /usr/local/share/scripts/init-network.sh
# tune sysctl for network performance
cat > /etc/sysctl.d/10-network-memory.conf << EOF
net.core.netdev_max_backlog=2000
net.core.rmem_max=1048576
net.core.wmem_max=1048576
net.ipv4.tcp_max_syn_backlog=1024
net.ipv4.tcp_rmem=4096 131072 16777216
net.ipv4.tcp_wmem=4096 87380 16777216
net.ipv4.tcp_mem=4096 131072 16777216
EOF
sysctl -p /etc/sysctl.d/10-network-memory.conf
mkdir /home/ubuntu/registrystorage
sed -i 's/preserve_hostname: false/preserve_hostname: true/g' /etc/cloud/cloud.cfg
REGISTRY_PROXY_USERNAME=robotclickhouse
REGISTRY_PROXY_PASSWORD=$(aws ssm get-parameter --name dockerhub_robot_password --with-decryption | jq '.Parameter.Value' -r)
docker run -d --network=host -p 5000:5000 -v /home/ubuntu/registrystorage:/var/lib/registry \
-e REGISTRY_STORAGE_CACHE='' \
-e REGISTRY_HTTP_ADDR=0.0.0.0:5000 \
-e REGISTRY_STORAGE_DELETE_ENABLED=true \
-e REGISTRY_PROXY_REMOTEURL=https://registry-1.docker.io \
-e REGISTRY_PROXY_PASSWORD="$REGISTRY_PROXY_PASSWORD" \
-e REGISTRY_PROXY_USERNAME="$REGISTRY_PROXY_USERNAME" \
--restart=always --name registry registry:2

View File

@ -1,254 +0,0 @@
#!/usr/bin/env bash
# The script is downloaded the AWS image builder Task Orchestrator and Executor (AWSTOE)
# We can't use `user data script` because cloud-init does not check the exit code
# The script is downloaded in the component named ci-infrastructure-prepare in us-east-1
# The link there must be adjusted to a particular RAW link, e.g.
# https://github.com/ClickHouse/ClickHouse/raw/653da5f00219c088af66d97a8f1ea3e35e798268/tests/ci/worker/prepare-ci-ami.sh
set -xeuo pipefail
echo "Running prepare script"
export DEBIAN_FRONTEND=noninteractive
export RUNNER_VERSION=2.317.0
export RUNNER_HOME=/home/ubuntu/actions-runner
deb_arch() {
case $(uname -m) in
x86_64 )
echo amd64;;
aarch64 )
echo arm64;;
esac
}
runner_arch() {
case $(uname -m) in
x86_64 )
echo x64;;
aarch64 )
echo arm64;;
esac
}
# We have test for cgroups, and it's broken with cgroups v2
# Ubuntu 22.04 has it enabled by default
sed -r '/GRUB_CMDLINE_LINUX=/ s/"(.*)"/"\1 systemd.unified_cgroup_hierarchy=0"/' -i /etc/default/grub
update-grub
apt-get update
apt-get install --yes --no-install-recommends \
apt-transport-https \
at \
atop \
binfmt-support \
build-essential \
ca-certificates \
curl \
gnupg \
jq \
lsb-release \
pigz \
ripgrep \
zstd \
python3-dev \
python3-pip \
qemu-user-static \
unzip \
gh
# Install docker
curl -fsSL https://download.docker.com/linux/ubuntu/gpg | gpg --dearmor -o /usr/share/keyrings/docker-archive-keyring.gpg
echo "deb [arch=$(deb_arch) signed-by=/usr/share/keyrings/docker-archive-keyring.gpg] https://download.docker.com/linux/ubuntu $(lsb_release -cs) stable" | tee /etc/apt/sources.list.d/docker.list > /dev/null
apt-get update
apt-get install --yes --no-install-recommends docker-ce docker-buildx-plugin docker-ce-cli containerd.io
usermod -aG docker ubuntu
# enable ipv6 in containers (fixed-cidr-v6 is some random network mask)
cat <<EOT > /etc/docker/daemon.json
{
"ipv6": true,
"fixed-cidr-v6": "2001:db8:1::/64",
"log-driver": "json-file",
"log-opts": {
"max-file": "5",
"max-size": "1000m"
},
"insecure-registries" : ["dockerhub-proxy.dockerhub-proxy-zone:5000"],
"registry-mirrors" : ["http://dockerhub-proxy.dockerhub-proxy-zone:5000"]
}
EOT
# Install azure-cli
curl -sLS https://packages.microsoft.com/keys/microsoft.asc | gpg --dearmor -o /etc/apt/keyrings/microsoft.gpg
AZ_DIST=$(lsb_release -cs)
echo "deb [arch=$(dpkg --print-architecture) signed-by=/etc/apt/keyrings/microsoft.gpg] https://packages.microsoft.com/repos/azure-cli/ $AZ_DIST main" | tee /etc/apt/sources.list.d/azure-cli.list
apt-get update
apt-get install --yes --no-install-recommends azure-cli
# Increase the limit on number of virtual memory mappings to aviod 'Cannot mmap' error
echo "vm.max_map_count = 2097152" > /etc/sysctl.d/01-increase-map-counts.conf
# Workarond for sanitizers uncompatibility with some kernels, see https://github.com/google/sanitizers/issues/856
echo "vm.mmap_rnd_bits=28" > /etc/sysctl.d/02-vm-mmap_rnd_bits.conf
systemctl restart docker
# buildx builder is user-specific
sudo -u ubuntu docker buildx version
sudo -u ubuntu docker buildx rm default-builder || : # if it's the second attempt
sudo -u ubuntu docker buildx create --use --name default-builder
pip install boto3 pygithub requests urllib3 unidiff dohq-artifactory jwt
rm -rf $RUNNER_HOME # if it's the second attempt
mkdir -p $RUNNER_HOME && cd $RUNNER_HOME
RUNNER_ARCHIVE="actions-runner-linux-$(runner_arch)-$RUNNER_VERSION.tar.gz"
curl -O -L "https://github.com/actions/runner/releases/download/v$RUNNER_VERSION/$RUNNER_ARCHIVE"
tar xzf "./$RUNNER_ARCHIVE"
rm -f "./$RUNNER_ARCHIVE"
./bin/installdependencies.sh
chown -R ubuntu:ubuntu $RUNNER_HOME
cd /home/ubuntu
curl "https://awscli.amazonaws.com/awscli-exe-linux-$(uname -m).zip" -o "awscliv2.zip"
unzip -q awscliv2.zip
./aws/install
rm -rf /home/ubuntu/awscliv2.zip /home/ubuntu/aws
# SSH keys of core team
mkdir -p /home/ubuntu/.ssh
# ~/.ssh/authorized_keys is cleaned out, so we use deprecated but working ~/.ssh/authorized_keys2
TEAM_KEYS_URL=$(aws ssm get-parameter --region us-east-1 --name team-keys-url --query 'Parameter.Value' --output=text)
curl "${TEAM_KEYS_URL}" > /home/ubuntu/.ssh/authorized_keys2
chown ubuntu: /home/ubuntu/.ssh -R
chmod 0700 /home/ubuntu/.ssh
# Download cloudwatch agent and install config for it
wget --directory-prefix=/tmp https://s3.amazonaws.com/amazoncloudwatch-agent/ubuntu/"$(deb_arch)"/latest/amazon-cloudwatch-agent.deb{,.sig}
gpg --recv-key --keyserver keyserver.ubuntu.com D58167303B789C72
gpg --verify /tmp/amazon-cloudwatch-agent.deb.sig
dpkg -i /tmp/amazon-cloudwatch-agent.deb
aws ssm get-parameter --region us-east-1 --name AmazonCloudWatch-github-runners --query 'Parameter.Value' --output text > /opt/aws/amazon-cloudwatch-agent/etc/amazon-cloudwatch-agent.json
systemctl enable amazon-cloudwatch-agent.service
echo "Install tailscale"
# Build get-authkey for tailscale
docker run --rm -v /usr/local/bin/:/host-local-bin -i golang:alpine sh -ex <<'EOF'
CGO_ENABLED=0 go install -tags tag:svc-core-ci-github tailscale.com/cmd/get-authkey@main
mv /go/bin/get-authkey /host-local-bin
EOF
# install tailscale
curl -fsSL "https://pkgs.tailscale.com/stable/ubuntu/$(lsb_release -cs).noarmor.gpg" > /usr/share/keyrings/tailscale-archive-keyring.gpg
curl -fsSL "https://pkgs.tailscale.com/stable/ubuntu/$(lsb_release -cs).tailscale-keyring.list" > /etc/apt/sources.list.d/tailscale.list
apt-get update
apt-get install tailscale --yes --no-install-recommends
# Create a common script for the instances
mkdir /usr/local/share/scripts -p
setup_cloudflare_dns() {
# Add cloudflare DNS as a fallback
# Get default gateway interface
local IFACE ETH_DNS CLOUDFLARE_NS new_dns
IFACE=$(ip --json route list | jq '.[]|select(.dst == "default").dev' --raw-output)
# `Link 2 (eth0): 172.31.0.2`
ETH_DNS=$(resolvectl dns "$IFACE") || :
CLOUDFLARE_NS=1.1.1.1
if [[ "$ETH_DNS" ]] && [[ "${ETH_DNS#*: }" != *"$CLOUDFLARE_NS"* ]]; then
# Cut the leading legend
ETH_DNS=${ETH_DNS#*: }
# shellcheck disable=SC2206
new_dns=(${ETH_DNS} "$CLOUDFLARE_NS")
resolvectl dns "$IFACE" "${new_dns[@]}"
fi
}
setup_tailscale() {
# Setup tailscale, the very first action
local TS_API_CLIENT_ID TS_API_CLIENT_SECRET TS_AUTHKEY RUNNER_TYPE
TS_API_CLIENT_ID=$(aws ssm get-parameter --region us-east-1 --name /tailscale/api-client-id --query 'Parameter.Value' --output text --with-decryption)
TS_API_CLIENT_SECRET=$(aws ssm get-parameter --region us-east-1 --name /tailscale/api-client-secret --query 'Parameter.Value' --output text --with-decryption)
RUNNER_TYPE=$(/usr/local/bin/aws ec2 describe-tags --filters "Name=resource-id,Values=$INSTANCE_ID" --query "Tags[?Key=='github:runner-type'].Value" --output text)
RUNNER_TYPE=${RUNNER_TYPE:-unknown}
# Clean possible garbage from the runner type
RUNNER_TYPE=${RUNNER_TYPE//[^0-9a-z]/-}
TS_AUTHKEY=$(TS_API_CLIENT_ID="$TS_API_CLIENT_ID" TS_API_CLIENT_SECRET="$TS_API_CLIENT_SECRET" \
get-authkey -tags tag:svc-core-ci-github -ephemeral)
tailscale up --ssh --auth-key="$TS_AUTHKEY" --hostname="ci-runner-$RUNNER_TYPE-$INSTANCE_ID"
}
cat > /usr/local/share/scripts/init-network.sh << EOF
!/usr/bin/env bash
$(declare -f setup_cloudflare_dns)
$(declare -f setup_tailscale)
# If the script is sourced, it will return now and won't execute functions
return 0 &>/dev/null || :
echo Setup Cloudflare DNS
setup_cloudflare_dns
echo Setup Tailscale VPN
setup_tailscale
EOF
chmod +x /usr/local/share/scripts/init-network.sh
# The following line is used in aws TOE check.
touch /var/tmp/clickhouse-ci-ami.success
# END OF THE SCRIPT
# TOE (Task Orchestrator and Executor) description
# name: CIInfrastructurePrepare
# description: installs the infrastructure for ClickHouse CI runners
# schemaVersion: 1.0
#
# phases:
# - name: build
# steps:
# - name: DownloadRemoteScript
# maxAttempts: 3
# action: WebDownload
# onFailure: Abort
# inputs:
# - source: https://github.com/ClickHouse/ClickHouse/raw/653da5f00219c088af66d97a8f1ea3e35e798268/tests/ci/worker/prepare-ci-ami.sh
# destination: /tmp/prepare-ci-ami.sh
# - name: RunScript
# maxAttempts: 3
# action: ExecuteBash
# onFailure: Abort
# inputs:
# commands:
# - bash -x '{{build.DownloadRemoteScript.inputs[0].destination}}'
#
#
# - name: validate
# steps:
# - name: RunScript
# maxAttempts: 3
# action: ExecuteBash
# onFailure: Abort
# inputs:
# commands:
# - ls /var/tmp/clickhouse-ci-ami.success
# - name: Cleanup
# action: DeleteFile
# onFailure: Abort
# maxAttempts: 3
# inputs:
# - path: /var/tmp/clickhouse-ci-ami.success

View File

@ -0,0 +1,103 @@
import pytest
import logging
import time
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
from helpers.test_tools import assert_eq_with_retry
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance("node1", stay_alive=True, with_zookeeper=True)
node2 = cluster.add_instance("node2", with_zookeeper=True)
instance = node1
q = node1.query
path_to_data = "/var/lib/clickhouse/"
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def wait_merged_part(table, part_name, retries=100):
q("OPTIMIZE TABLE {} FINAL".format(table))
for i in range(retries):
result = q(
"SELECT name FROM system.parts where table='{}' AND name='{}'".format(
table, part_name
)
)
if result:
return True
time.sleep(0.5)
else:
return False
def test_make_clone_covered_by_broken_detached_dir_exists(started_cluster):
q("DROP TABLE IF EXISTS test_make_clone_cvbdde SYNC")
q(
"CREATE TABLE test_make_clone_cvbdde(n int, m String) ENGINE=ReplicatedMergeTree('/test_make_clone_cvbdde', '1') ORDER BY n SETTINGS old_parts_lifetime=3600, min_age_to_force_merge_seconds=1, min_age_to_force_merge_on_partition_only=0"
)
path = path_to_data + "data/default/test_make_clone_cvbdde/"
q("INSERT INTO test_make_clone_cvbdde VALUES (0, 'hbl')")
q("INSERT INTO test_make_clone_cvbdde VALUES (1, 'hbl')")
if not (wait_merged_part("test_make_clone_cvbdde", "all_0_1_1")):
assert False, "Part all_0_1_1 doesn't appeared in system.parts"
q("INSERT INTO test_make_clone_cvbdde VALUES (2, 'hbl')")
if not (wait_merged_part("test_make_clone_cvbdde", "all_0_2_2")):
assert False, "Part all_0_2_2 doesn't appeared in system.parts"
q("INSERT INTO test_make_clone_cvbdde VALUES (3, 'hbl')")
if not (wait_merged_part("test_make_clone_cvbdde", "all_0_3_3")):
assert False, "Part all_0_3_3 doesn't appeared in system.parts"
res = str(instance.exec_in_container(["ls", path]).strip().split("\n"))
# broke the merged parts
instance.exec_in_container(
[
"bash",
"-c",
"echo 'broken' > {}".format(path + "all_0_1_1/data.bin"),
]
)
instance.exec_in_container(
[
"bash",
"-c",
"echo 'broken' > {}".format(path + "all_0_2_2/data.bin"),
]
)
instance.exec_in_container(
[
"bash",
"-c",
"echo 'broken' > {}".format(path + "all_0_3_3/data.bin"),
]
)
instance.restart_clickhouse(kill=True)
assert [
"broken-on-start_all_0_1_1",
"broken-on-start_all_0_2_2",
"broken-on-start_all_0_3_3",
"covered-by-broken_all_0_0_0",
"covered-by-broken_all_1_1_0",
"covered-by-broken_all_2_2_0",
"covered-by-broken_all_3_3_0",
] == sorted(
instance.exec_in_container(["ls", path + "detached/"]).strip().split("\n")
)

View File

@ -1,70 +0,0 @@
-------------- Test copy sorting clauses from source table --------------
CREATE TABLE default.x
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS index_granularity = 8192
-------------------------------------------------------------------------
CREATE TABLE default.x_as
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192
-------------- Test copy sorting clauses from destination table (source table without the same type clauses) --------------
CREATE TABLE default.x
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PRIMARY KEY (CounterID, EventDate, intHash32(UserID))
ORDER BY (CounterID, EventDate, intHash32(UserID))
SETTINGS index_granularity = 8192
-------------------------------------------------------------------------
CREATE TABLE default.x_as
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
PRIMARY KEY (CounterID, EventDate, intHash32(UserID))
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192
-------------- Test copy sorting clauses from destination table (source table with the same type clauses) --------------
CREATE TABLE default.x
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
ORDER BY CounterID
SETTINGS index_granularity = 8192
-------------------------------------------------------------------------
CREATE TABLE default.x_as
(
`CounterID` UInt32,
`EventDate` Date,
`UserID` UInt64
)
ENGINE = MergeTree
PARTITION BY toYYYYMM(EventDate)
ORDER BY (CounterID, EventDate, intHash32(UserID))
SAMPLE BY intHash32(UserID)
SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1, index_granularity = 8192

View File

@ -1,37 +0,0 @@
DROP TABLE IF EXISTS x;
DROP TABLE IF EXISTS x_as;
SELECT '-------------- Test copy sorting clauses from source table --------------';
CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID);
CREATE TABLE x_as AS x ENGINE = MergeTree SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1;
SHOW CREATE TABLE x FORMAT TSVRaw;
SELECT '-------------------------------------------------------------------------';
SHOW CREATE TABLE x_as FORMAT TSVRaw;
DROP TABLE x;
DROP TABLE x_as;
SELECT '-------------- Test copy sorting clauses from destination table (source table without the same type clauses) --------------';
CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree PRIMARY KEY (CounterID, EventDate, intHash32(UserID));
CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1;
SHOW CREATE TABLE x FORMAT TSVRaw;
SELECT '-------------------------------------------------------------------------';
SHOW CREATE TABLE x_as FORMAT TSVRaw;
DROP TABLE x;
DROP TABLE x_as;
SELECT '-------------- Test copy sorting clauses from destination table (source table with the same type clauses) --------------';
CREATE TABLE x (`CounterID` UInt32, `EventDate` Date, `UserID` UInt64) ENGINE = MergeTree ORDER BY (CounterID);
CREATE TABLE x_as AS x ENGINE = MergeTree PARTITION BY toYYYYMM(EventDate) ORDER BY (CounterID, EventDate, intHash32(UserID)) SAMPLE BY intHash32(UserID) SETTINGS enable_block_number_column = 1, enable_block_offset_column = 1;
SHOW CREATE TABLE x FORMAT TSVRaw;
SELECT '-------------------------------------------------------------------------';
SHOW CREATE TABLE x_as FORMAT TSVRaw;
DROP TABLE x;
DROP TABLE x_as;

View File

@ -51,8 +51,8 @@ $CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW
$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='${DATABASE_2}'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g"
RANDOM_COMMENT="$RANDOM"
$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 33s (1.5s * 22 rows per partition [Using 5 threads in parallel]), result: 110, 5995
$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(1.5) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 45s (1.5s * 30 rows)
$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 120000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(3) AS col FROM ${DATABASE_1}.mt) -- ${RANDOM_COMMENT}" & # 66s (3s * 22 rows per partition [Using 5 threads in parallel]), result: 110, 5995
$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 120000000 -q "INSERT INTO ${DATABASE_2}.mt SELECT number + sleepEachRow(2.2) FROM numbers(30) -- ${RANDOM_COMMENT}" & # 66s (2.2s * 30 rows)
it=0
while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'") -ne 2 ]]; do
@ -87,7 +87,7 @@ SELECT count() FROM ${DATABASE_1}.mt
" # result: 5
RANDOM_TUPLE="${RANDOM}_tuple"
$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 15s (3s * 5 rows)
$CLICKHOUSE_CLIENT --max-threads 5 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(4)) FROM ${DATABASE_1}.mt -- ${RANDOM_TUPLE}" > /dev/null & # 20s (4s * 5 rows)
it=0
while [[ $($CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'") -ne 1 ]]; do
it=$((it+1))

View File

@ -4,3 +4,39 @@ It is possible to create parts with different Array vector sizes but there will
Correctness of index with > 1 mark
1 [1,0] 0
9000 [9000,0] 0
Issue #69085: Reference vector computed by a subquery
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 2/4
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4

View File

@ -53,3 +53,50 @@ ORDER BY L2Distance(vec, reference_vec)
LIMIT 1;
DROP TABLE tab;
SELECT 'Issue #69085: Reference vector computed by a subquery';
CREATE TABLE tab(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'cosineDistance', 'f16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab VALUES (0, [4.6, 2.3]), (1, [2.0, 3.2]), (2, [4.2, 3.4]), (3, [5.3, 2.9]), (4, [2.4, 5.2]), (5, [5.3, 2.3]), (6, [1.0, 9.3]), (7, [5.5, 4.7]), (8, [6.4, 3.5]), (9, [5.3, 2.5]), (10, [6.4, 3.4]), (11, [6.4, 3.2]);
-- works
EXPLAIN indexes = 1
WITH [0., 2.] AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1
SETTINGS enable_analyzer = 0;
-- does not work
EXPLAIN indexes = 1
WITH (
SELECT vec
FROM tab
LIMIT 1
) AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1
SETTINGS enable_analyzer = 0;
-- does not work as well
EXPLAIN indexes = 1
WITH (
SELECT [0., 2.]
) AS reference_vec
SELECT
id,
vec,
cosineDistance(vec, reference_vec) AS distance
FROM tab
ORDER BY distance
LIMIT 1
SETTINGS enable_analyzer = 0;

View File

@ -143,7 +143,6 @@ arrayStringConcat
arraySum
arrayUniq
arrayWithConstant
arrayZip
asinh
assumeNotNull
atan

View File

@ -1,6 +1,6 @@
c1 Nullable(Int64)
c2 Nullable(Float64)
c3 Nullable(Bool)
c3 Nullable(String)
c1 Nullable(String)
c2 Nullable(String)
c3 Nullable(String)

View File

@ -0,0 +1,516 @@
0 \N
1 1
0 str_2
0 [0,1,2]
0 \N
5 5
0 str_6
0 [0,1,2,3,4,5,6]
\N \N
1 1
\N str_2
\N [0,1,2]
\N \N
5 5
\N str_6
\N [0,1,2,3,4,5,6]
0 \N
1 1
0 str_2
0 [0,1,2]
0 \N
5 5
0 str_6
0 [0,1,2,3,4,5,6]
\N \N
1 1
\N str_2
\N [0,1,2]
\N \N
5 5
\N str_6
\N [0,1,2,3,4,5,6]
-128
-127
-1
0
1
2
3
126
127
0
1
2
3
126
127
254
255
-32768
-32767
-128
-127
-1
0
1
2
3
126
127
254
255
32766
32767
0
1
2
3
126
127
254
255
32766
32767
65534
65535
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
3232235521
4294967294
4294967295
-9223372036854775808
-9223372036854775807
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
4294967294
4294967295
9223372036854775806
9223372036854775807
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
3232235521
4294967294
4294967295
9223372036854775806
9223372036854775807
18446744073709551614
18446744073709551615
-170141183460469231731687303715884105728
-170141183460469231731687303715884105727
-9223372036854775808
-9223372036854775807
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
4294967294
4294967295
9223372036854775806
9223372036854775807
18446744073709551614
18446744073709551615
170141183460469231731687303715884105726
170141183460469231731687303715884105727
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
3232235521
4294967294
4294967295
9223372036854775806
9223372036854775807
18446744073709551614
18446744073709551615
170141183460469231731687303715884105726
170141183460469231731687303715884105727
296245801836096677496328508227807879401
340282366920938463463374607431768211454
340282366920938463463374607431768211455
-57896044618658097711785492504343953926634992332820282019728792003956564819968
-57896044618658097711785492504343953926634992332820282019728792003956564819967
-170141183460469231731687303715884105728
-170141183460469231731687303715884105727
-9223372036854775808
-9223372036854775807
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
4294967294
4294967295
9223372036854775806
9223372036854775807
18446744073709551614
18446744073709551615
170141183460469231731687303715884105726
170141183460469231731687303715884105727
340282366920938463463374607431768211454
340282366920938463463374607431768211455
57896044618658097711785492504343953926634992332820282019728792003956564819966
57896044618658097711785492504343953926634992332820282019728792003956564819967
0
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
3232235521
4294967294
4294967295
9223372036854775806
9223372036854775807
18446744073709551614
18446744073709551615
170141183460469231731687303715884105726
170141183460469231731687303715884105727
340282366920938463463374607431768211454
340282366920938463463374607431768211455
57896044618658097711785492504343953926634992332820282019728792003956564819966
57896044618658097711785492504343953926634992332820282019728792003956564819967
115792089237316195423570985008687907853269984665640564039457584007913129639934
115792089237316195423570985008687907853269984665640564039457584007913129639935
-inf
-3.4028233e38
-1.7014118e38
-9223372000000000000
-2147483600
-32768
-32767
-128
-127
-1
-1.1754942e-38
-1e-45
0
1e-45
1.1754942e-38
1
2
3
126
127
254
255
32766
32767
65534
65535
3.4028233e38
inf
nan
-inf
-1.7976931348623157e308
-5.78960446186581e76
-3.40282347e38
-3.4028232635611926e38
-1.7014118346046923e38
-9223372036854776000
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
-1.1754943499999998e-38
-1.1754942106924411e-38
-1.401298464324817e-45
-1.3999999999999999e-45
-2.2250738585072014e-308
0
2.2250738585072014e-308
1.3999999999999999e-45
1.401298464324817e-45
1.1754942106924411e-38
1.1754943499999998e-38
1
2
3
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
4294967294
4294967295
3.4028232635611926e38
3.40282347e38
1.7976931348623157e308
inf
nan
-32768
-32767
-128
-127
-1
0
1
126
127
254
255
32766
32767
65534
65535
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
0
1
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
4294967294
4294967295
-9223372036854775808
-9223372036854775807
-18446744073709551.616
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
0
1
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
4294967294
4294967295
9223372036854775806
9223372036854775807
18446744073709551614
18446744073709551615
-340282347000000000977176926486249829565.415
-9223372036854775808
-9223372036854775807
-18446744073709551.616
-2147483648
-2147483647
-32768
-32767
-128
-127
-1
0
1
126
127
254
255
32766
32767
65534
65535
2147483646
2147483647
4294967294
4294967295
9223372036854775806
9223372036854775807
18446744073709551614
18446744073709551615
340282347000000000977176926486249829565.415
1970-01-01
1970-01-02
1970-01-03
1970-01-04
1970-05-07
1970-05-08
1970-09-12
1970-09-13
2038-01-19
2059-09-17
2059-09-18
2106-02-07
2149-06-05
2149-06-06
2299-12-31
2299-12-31
1900-01-01
1969-08-26
1969-08-27
1969-12-30
1969-12-31
1970-01-01
1970-01-02
1970-01-03
1970-01-04
1970-05-07
1970-05-08
1970-09-12
1970-09-13
2038-01-19
2059-09-17
2059-09-18
2106-02-07
2149-06-05
2149-06-06
2299-12-31
1970-01-01 00:00:00
1970-01-01 00:00:01
1970-01-01 00:00:02
1970-01-01 00:00:03
1970-01-01 00:02:06
1970-01-01 00:02:07
1970-01-01 00:04:14
1970-01-01 00:04:15
1970-01-01 09:06:06
1970-01-01 09:06:07
1970-01-01 18:12:14
1970-01-01 18:12:15
2038-01-19 03:14:06
2038-01-19 03:14:07
2106-02-07 06:28:14
2106-02-07 06:28:15
0.0.0.0
192.168.0.1
::
::1
::ffff:192.168.0.1
00000000-0000-0000-0000-000000000000
dededdb6-7835-4ce4-8d11-b5de6f2820e9

View File

@ -0,0 +1,117 @@
set allow_experimental_variant_type = 1;
set use_variant_as_common_type = 1;
set allow_experimental_dynamic_type = 1;
set allow_suspicious_low_cardinality_types = 1;
set session_timezone = 'UTC';
select accurateCastOrDefault(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8);
select accurateCastOrNull(variant, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number)) as variant from numbers(8);
select accurateCastOrDefault(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8);
select accurateCastOrNull(dynamic, 'UInt32'), multiIf(number % 4 == 0, NULL, number % 4 == 1, number, number % 4 == 2, 'str_' || toString(number), range(number))::Dynamic as dynamic from numbers(8);
drop table if exists t;
create table t (d Dynamic) engine=MergeTree order by tuple();
-- Integer types: signed and unsigned integers (UInt8, UInt16, UInt32, UInt64, UInt128, UInt256, Int8, Int16, Int32, Int64, Int128, Int256)
INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8);
INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8);
INSERT INTO t VALUES (-128::Int8), (-127::Int8), (-1::Int8), (0::Int8), (1::Int8), (126::Int8), (127::Int8);
INSERT INTO t VALUES (-32768::Int16), (-32767::Int16), (-1::Int16), (0::Int16), (1::Int16), (32766::Int16), (32767::Int16);
INSERT INTO t VALUES (-2147483648::Int32), (-2147483647::Int32), (-1::Int32), (0::Int32), (1::Int32), (2147483646::Int32), (2147483647::Int32);
INSERT INTO t VALUES (-9223372036854775808::Int64), (-9223372036854775807::Int64), (-1::Int64), (0::Int64), (1::Int64), (9223372036854775806::Int64), (9223372036854775807::Int64);
INSERT INTO t VALUES (-170141183460469231731687303715884105728::Int128), (-170141183460469231731687303715884105727::Int128), (-1::Int128), (0::Int128), (1::Int128), (170141183460469231731687303715884105726::Int128), (170141183460469231731687303715884105727::Int128);
INSERT INTO t VALUES (-57896044618658097711785492504343953926634992332820282019728792003956564819968::Int256), (-57896044618658097711785492504343953926634992332820282019728792003956564819967::Int256), (-1::Int256), (0::Int256), (1::Int256), (57896044618658097711785492504343953926634992332820282019728792003956564819966::Int256), (57896044618658097711785492504343953926634992332820282019728792003956564819967::Int256);
INSERT INTO t VALUES (0::UInt8), (1::UInt8), (254::UInt8), (255::UInt8);
INSERT INTO t VALUES (0::UInt16), (1::UInt16), (65534::UInt16), (65535::UInt16);
INSERT INTO t VALUES (0::UInt32), (1::UInt32), (4294967294::UInt32), (4294967295::UInt32);
INSERT INTO t VALUES (0::UInt64), (1::UInt64), (18446744073709551614::UInt64), (18446744073709551615::UInt64);
INSERT INTO t VALUES (0::UInt128), (1::UInt128), (340282366920938463463374607431768211454::UInt128), (340282366920938463463374607431768211455::UInt128);
INSERT INTO t VALUES (0::UInt256), (1::UInt256), (115792089237316195423570985008687907853269984665640564039457584007913129639934::UInt256), (115792089237316195423570985008687907853269984665640564039457584007913129639935::UInt256);
-- Floating-point numbers: floats(Float32 and Float64) values
INSERT INTO t VALUES (1.17549435e-38::Float32), (3.40282347e+38::Float32), (-3.40282347e+38::Float32), (-1.17549435e-38::Float32), (1.4e-45::Float32), (-1.4e-45::Float32);
INSERT INTO t VALUES (inf::Float32), (-inf::Float32), (nan::Float32);
INSERT INTO t VALUES (inf::FLOAT(12)), (-inf::FLOAT(12)), (nan::FLOAT(12));
INSERT INTO t VALUES (inf::FLOAT(15,22)), (-inf::FLOAT(15,22)), (nan::FLOAT(15,22));
INSERT INTO t VALUES (1.17549435e-38::Float64), (3.40282347e+38::Float64), (-3.40282347e+38::Float64), (-1.17549435e-38::Float64), (1.4e-45::Float64), (-1.4e-45::Float64);
INSERT INTO t VALUES (2.2250738585072014e-308::Float64), (1.7976931348623157e+308::Float64), (-1.7976931348623157e+308::Float64), (-2.2250738585072014e-308::Float64);
INSERT INTO t VALUES (inf::Float64), (-inf::Float64), (nan::Float64);
INSERT INTO t VALUES (inf::DOUBLE(12)), (-inf::DOUBLE(12)), (nan::DOUBLE(12));
INSERT INTO t VALUES (inf::DOUBLE(15,22)), (-inf::DOUBLE(15,22)), (nan::DOUBLE(15,22));
-- Strings: String and FixedString
INSERT INTO t VALUES ('string'::String), ('1'::FixedString(1)), ('1'::FixedString(2)), ('1'::FixedString(10)); --(''::String),
-- Boolean
INSERT INTO t VALUES ('1'::Bool), (0::Bool);
-- UUID
INSERT INTO t VALUES ('dededdb6-7835-4ce4-8d11-b5de6f2820e9'::UUID);
INSERT INTO t VALUES ('00000000-0000-0000-0000-000000000000'::UUID);
-- LowCardinality
INSERT INTO t VALUES ('1'::LowCardinality(String)), ('1'::LowCardinality(String)), (0::LowCardinality(UInt16));
-- Arrays
INSERT INTO t VALUES ([]::Array(Dynamic)), ([[]]::Array(Array(Dynamic))), ([[[]]]::Array(Array(Array(Dynamic))));
-- Tuple
INSERT INTO t VALUES (()::Tuple(Dynamic)), ((())::Tuple(Tuple(Dynamic))), (((()))::Tuple(Tuple(Tuple(Dynamic))));
-- Map.
INSERT INTO t VALUES (map(11::Dynamic, 'v1'::Dynamic, '22'::Dynamic, 1::Dynamic));
-- SimpleAggregateFunction
INSERT INTO t VALUES ([1,2]::SimpleAggregateFunction(anyLast, Array(Int16)));
-- IPs
INSERT INTO t VALUES (toIPv4('192.168.0.1')), (toIPv6('::1'));
-- Geo
INSERT INTO t VALUES ((1.23, 4.56)::Point), (([(1.23, 4.56)::Point, (2.34, 5.67)::Point])::Ring);
INSERT INTO t VALUES ([[[(0, 0), (10, 0), (10, 10), (0, 10)]], [[(20, 20), (50, 20), (50, 50), (20, 50)],[(30, 30), (50, 50), (50, 30)]]]::MultiPolygon);
-- Interval
INSERT INTO t VALUES (interval '1' day), (interval '2' month), (interval '3' year);
-- Nested
INSERT INTO t VALUES ([(1, 'aa'), (2, 'bb')]::Nested(x UInt32, y String));
INSERT INTO t VALUES ([(1, (2, ['aa', 'bb']), [(3, 'cc'), (4, 'dd')]), (5, (6, ['ee', 'ff']), [(7, 'gg'), (8, 'hh')])]::Nested(x UInt32, y Tuple(y1 UInt32, y2 Array(String)), z Nested(z1 UInt32, z2 String)));
optimize table t final;
select distinct toInt8OrDefault(d) as res from t order by res;
select distinct toUInt8OrDefault(d) as res from t order by res;
select distinct toInt16OrDefault(d) as res from t order by res;
select distinct toUInt16OrDefault(d) as res from t order by res;
select distinct toInt32OrDefault(d) as res from t order by res;
select distinct toUInt32OrDefault(d) as res from t order by res;
select distinct toInt64OrDefault(d) as res from t order by res;
select distinct toUInt64OrDefault(d) as res from t order by res;
select distinct toInt128OrDefault(d) as res from t order by res;
select distinct toUInt128OrDefault(d) as res from t order by res;
select distinct toInt256OrDefault(d) as res from t order by res;
select distinct toUInt256OrDefault(d) as res from t order by res;
select distinct toFloat32OrDefault(d) as res from t order by res;
select distinct toFloat64OrDefault(d) as res from t order by res;
select distinct toDecimal32OrDefault(d, 3) as res from t order by res;
select distinct toDecimal64OrDefault(d, 3) as res from t order by res;
select distinct toDecimal128OrDefault(d, 3) as res from t order by res;
select distinct toDecimal256OrDefault(d, 3) as res from t order by res;
select distinct toDateOrDefault(d) as res from t order by res;
select distinct toDate32OrDefault(d) as res from t order by res;
select distinct toDateTimeOrDefault(d) as res from t order by res;
select distinct toIPv4OrDefault(d) as res from t order by res;
select distinct toIPv6OrDefault(d) as res from t order by res;
select distinct toUUIDOrDefault(d) as res from t order by res;
drop table t;

View File

@ -0,0 +1,4 @@
BEFORE a x String
BEFORE a y String MATERIALIZED \'str\'
AFTER a x String
AFTER a y String

View File

@ -0,0 +1,13 @@
DROP TABLE IF EXISTS a SYNC;
CREATE TABLE a (x String, y String MATERIALIZED 'str') ENGINE = ReplicatedMergeTree('/clickhouse/{database}/a', 'r1') ORDER BY x;
INSERT INTO a SELECT toString(number) FROM numbers(100);
SELECT 'BEFORE', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table = 'a' ORDER BY table, name;
-- DROP INDEX is important to make the mutation not a pure metadata mutation
ALTER TABLE a
DROP INDEX IF EXISTS some_index,
MODIFY COLUMN y REMOVE MATERIALIZED
SETTINGS alter_sync = 2, mutations_sync = 2;
SELECT 'AFTER', table, name, type, default_kind, default_expression FROM system.columns WHERE database = currentDatabase() AND table = 'a' ORDER BY table, name;

View File

@ -0,0 +1,8 @@
[('a','d'),('b','e'),('c','f')] Array(Tuple(Nullable(String), Nullable(String)))
[('a','d','g'),('b','e','h'),('c','f','i')]
[('a','d'),('b','e'),('c','f'),(NULL,'g')]
[('a',1),(NULL,2),(NULL,3)]
[('a',1,1.1),('b',2,2.2),('c',NULL,3.3),(NULL,NULL,4.4)]
[('g'),('h'),('i')]
[('g'),('h'),('i')]
[('g'),('h'),('i')]

View File

@ -0,0 +1,15 @@
SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f']) as x, toTypeName(x);
SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f'], ['g', 'h', 'i']);
SELECT arrayZipUnaligned(); -- { serverError TOO_FEW_ARGUMENTS_FOR_FUNCTION }
SELECT arrayZipUnaligned('a', 'b', 'c'); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT arrayZipUnaligned(['a', 'b', 'c'], ['d', 'e', 'f', 'g']);
SELECT arrayZipUnaligned(['a'], [1, 2, 3]);
SELECT arrayZipUnaligned(['a', 'b', 'c'], [1, 2], [1.1, 2.2, 3.3, 4.4]);
SELECT arrayZipUnaligned(materialize(['g', 'h', 'i'])) from numbers(3);

View File

@ -0,0 +1,13 @@
REMOVE
The same, but with type
MODIFY SETTING
The same, but with type
RESET SETTING
The same, but with type
All the above, but on server side
REMOVE
The same, but with type
MODIFY SETTING
The same, but with type
RESET SETTING
The same, but with type

View File

@ -0,0 +1,151 @@
DROP TABLE IF EXISTS a SYNC;
CREATE TABLE a (x Int64, y Int64 MATERIALIZED 1 SETTINGS (max_compress_block_size = 30000)) ENGINE = MergeTree ORDER BY x;
-- In cases when the type is not present in column declaration, the parser interprets TTL/COLLATE/SETTINGS as a data type,
-- thus such queries doesn't throw syntax error on client side, just fails to parse. For server side validation these
-- queries still result in an exception of syntax error. Even though the exception is throw for a different reason, they
-- are good safe guards for the future where the parsing of such properties might change.
SELECT 'REMOVE';
ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y COMMENT '5' REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
SELECT 'The same, but with type';
ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 COMMENT '5' REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED; -- { clientError SYNTAX_ERROR }
SELECT 'MODIFY SETTING';
ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y COMMENT '5' MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
SELECT 'The same, but with type';
ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 COMMENT '5' MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000; -- { clientError SYNTAX_ERROR }
SELECT 'RESET SETTING';
ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y COMMENT '5' RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
SELECT 'The same, but with type';
ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 COMMENT '5' RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate('2025-01-01') + toIntervalDay(x) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY RESET SETTING max_compress_block_size; -- { clientError SYNTAX_ERROR }
SELECT 'All the above, but on server side';
SELECT 'REMOVE';
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT \'5\' REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT 'The same, but with type';
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT \'5\' REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (max_compress_block_size = 20000) REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY REMOVE MATERIALIZED'); -- { serverError SYNTAX_ERROR }
SELECT 'MODIFY SETTING';
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT \'5\' MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT 'The same, but with type';
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT \'5\' MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY MODIFY SETTING max_compress_block_size = 20000'); -- { serverError SYNTAX_ERROR }
SELECT 'RESET SETTING';
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COMMENT \'5\' RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y CODEC(ZSTD) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y STATISTICS(tdigest) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y TTL toDate(\'2025-01-01\') + toIntervalDay(x) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y COLLATE binary RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT 'The same, but with type';
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 DEFAULT 2 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 MATERIALIZED 3 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 EPHEMERAL 4 RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COMMENT \'5\' RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 CODEC(ZSTD) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 STATISTICS(tdigest) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 TTL toDate(\'2025-01-01\') + toIntervalDay(x) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 COLLATE binary RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 SETTINGS (some_setting = 2) RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }
SELECT formatQuery('ALTER TABLE a MODIFY COLUMN y Int64 PRIMARY KEY RESET SETTING max_compress_block_size'); -- { serverError SYNTAX_ERROR }

View File

@ -0,0 +1,4 @@
c1 Nullable(Int64)
c2 Nullable(Float64)
c3 Nullable(String)
42 42.42 True

View File

@ -0,0 +1,4 @@
set input_format_csv_try_infer_numbers_from_strings = 1;
desc format(CSV, '"42","42.42","True"');
select * from format(CSV, '"42","42.42","True"');

View File

@ -0,0 +1,3 @@
1 ClickHouse
2 HelloWorld
OK

View File

@ -0,0 +1,25 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
function cleanup()
{
[ -e "${CLICKHOUSE_TMP}"/test_infile.csv ] && rm "${CLICKHOUSE_TMP}"/test_infile.csv
}
trap cleanup EXIT
cleanup
echo -e "id,\"word\"\n1,\"ClickHouse\"\n2,\"HelloWorld\"" > "${CLICKHOUSE_TMP}"/test_infile.csv
${CLICKHOUSE_CLIENT} --query "DROP TABLE IF EXISTS async_insert_infile_data;"
${CLICKHOUSE_CLIENT} --query "CREATE TABLE async_insert_infile_data (id UInt32, word String) ENGINE=Memory();"
${CLICKHOUSE_CLIENT} --query "INSERT INTO async_insert_infile_data FROM INFILE '${CLICKHOUSE_TMP}/test_infile.csv' SETTINGS async_insert=1;"
${CLICKHOUSE_CLIENT} --query "SELECT * FROM async_insert_infile_data ORDER BY id;"
${CLICKHOUSE_CLIENT} --query "INSERT INTO async_insert_infile_data FROM INFILE '${CLICKHOUSE_TMP}/test_infile.csv' SETTINGS async_insert=1 FORMAT NotExists;" 2>&1 | grep -q "UNKNOWN_FORMAT" && echo OK || echo FAIL
${CLICKHOUSE_CLIENT} --query "DROP TABLE async_insert_infile_data SYNC;"

View File

@ -1209,6 +1209,7 @@ arraySum
arrayUniq
arrayWithConstant
arrayZip
arrayZipUnaligned
ascii
asin
asinh