Merge branch 'master' into multi_auth_methods

This commit is contained in:
Arthur Passos 2024-08-29 13:12:19 -03:00
commit f9b845486a
74 changed files with 1784 additions and 485 deletions

View File

@ -30,7 +30,6 @@
* Support more variants of JOIN strictness (`LEFT/RIGHT SEMI/ANTI/ANY JOIN`) with inequality conditions which involve columns from both left and right table. e.g. `t1.y < t2.y` (see the setting `allow_experimental_join_condition`). [#64281](https://github.com/ClickHouse/ClickHouse/pull/64281) ([lgbo](https://github.com/lgbo-ustc)).
* Intrpret Hive-style partitioning for different engines (`File`, `URL`, `S3`, `AzureBlobStorage`, `HDFS`). Hive-style partitioning organizes data into partitioned sub-directories, making it efficient to query and manage large datasets. Currently, it only creates virtual columns with the appropriate name and data. The follow-up PR will introduce the appropriate data filtering (performance speedup). [#65997](https://github.com/ClickHouse/ClickHouse/pull/65997) ([Yarik Briukhovetskyi](https://github.com/yariks5s)).
* Add function `printf` for Spark compatiability (but you can use the existing `format` function). [#66257](https://github.com/ClickHouse/ClickHouse/pull/66257) ([李扬](https://github.com/taiyang-li)).
* Added a new server setting, `disable_insertion_and_mutation`. If it is enabled, the server will deny all insertions and mutations. This includes asynchronous INSERTs. This setting can be used to create read-only replicas. [#66519](https://github.com/ClickHouse/ClickHouse/pull/66519) ([Xu Jia](https://github.com/XuJia0210)).
* Add options `restore_replace_external_engines_to_null` and `restore_replace_external_table_functions_to_null` to replace external engines and table_engines to `Null` engine that can be useful for testing. It should work for RESTORE and explicit table creation. [#66536](https://github.com/ClickHouse/ClickHouse/pull/66536) ([Ilya Yatsishin](https://github.com/qoega)).
* Added support for reading `MULTILINESTRING` geometry in `WKT` format using function `readWKTLineString`. [#67647](https://github.com/ClickHouse/ClickHouse/pull/67647) ([Jacob Reckhard](https://github.com/jacobrec)).
* Add a new table function `fuzzQuery`. This function allows the modification of a given query string with random variations. Example: `SELECT query FROM fuzzQuery('SELECT 1') LIMIT 5;`. [#67655](https://github.com/ClickHouse/ClickHouse/pull/67655) ([pufit](https://github.com/pufit)).

View File

@ -56,6 +56,14 @@ Other upcoming meetups
* [Toronto Meetup (Shopify)](https://www.meetup.com/clickhouse-toronto-user-group/events/301490855/) - September 10
* [Austin Meetup](https://www.meetup.com/clickhouse-austin-user-group/events/302558689/) - September 17
* [London Meetup](https://www.meetup.com/clickhouse-london-user-group/events/302977267) - September 17
* [Tel Aviv Meetup](https://www.meetup.com/clickhouse-meetup-israel/events/303095121) - September 22
* [Madrid Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096564/) - October 22
* [Barcelona Meetup](https://www.meetup.com/clickhouse-spain-user-group/events/303096876/) - October 29
* [Oslo Meetup](https://www.meetup.com/open-source-real-time-data-warehouse-real-time-analytics/events/302938622) - October 31
* [Ghent Meetup](https://www.meetup.com/clickhouse-belgium-user-group/events/303049405/) - November 19
* [Dubai Meetup](https://www.meetup.com/clickhouse-dubai-meetup-group/events/303096989/) - November 21
## Recent Recordings
* **Recent Meetup Videos**: [Meetup Playlist](https://www.youtube.com/playlist?list=PL0Z2YDlm0b3iNDUzpY1S3L_iV4nARda_U) Whenever possible recordings of the ClickHouse Community Meetups are edited and presented as individual talks. Current featuring "Modern SQL in 2023", "Fast, Concurrent, and Consistent Asynchronous INSERTS in ClickHouse", and "Full-Text Indices: Design and Experiments"

2
contrib/usearch vendored

@ -1 +1 @@
Subproject commit e21a5778a0d4469ddaf38c94b7be0196bb701ee4
Subproject commit 7a8967cb442b08ca20c3dd781414378e65957d37

View File

@ -54,7 +54,7 @@ Parameters:
- `distance_function`: either `L2Distance` (the [Euclidean distance](https://en.wikipedia.org/wiki/Euclidean_distance) - the length of a
line between two points in Euclidean space), or `cosineDistance` (the [cosine
distance](https://en.wikipedia.org/wiki/Cosine_similarity#Cosine_distance)- the angle between two non-zero vectors).
- `quantization`: either `f32`, `f16`, or `i8` for storing the vector with reduced precision (optional, default: `f32`)
- `quantization`: either `f64`, `f32`, `f16`, `bf16`, or `i8` for storing the vector with reduced precision (optional, default: `bf16`)
- `m`: the number of neighbors per graph node (optional, default: 16)
- `ef_construction`: (optional, default: 128)
- `ef_search`: (optional, default: 64)

View File

@ -5633,7 +5633,6 @@ Default value: `1GiB`.
## use_json_alias_for_old_object_type
When enabled, `JSON` data type alias will be used to create an old [Object('json')](../../sql-reference/data-types/json.md) type instead of the new [JSON](../../sql-reference/data-types/newjson.md) type.
This setting requires server restart to take effect when changed.
Default value: `false`.

View File

@ -0,0 +1,41 @@
---
slug: /en/operations/system-tables/projections
---
# projections
Contains information about existing projections in all the tables.
Columns:
- `database` ([String](../../sql-reference/data-types/string.md)) — Database name.
- `table` ([String](../../sql-reference/data-types/string.md)) — Table name.
- `name` ([String](../../sql-reference/data-types/string.md)) — Projection name.
- `type` ([Enum](../../sql-reference/data-types/enum.md)) — Projection type ('Normal' = 0, 'Aggregate' = 1).
- `sorting_key` ([Array(String)](../../sql-reference/data-types/array.md)) — Projection sorting key.
- `query` ([String](../../sql-reference/data-types/string.md)) — Projection query.
**Example**
```sql
SELECT * FROM system.projections LIMIT 2 FORMAT Vertical;
```
```text
Row 1:
──────
database: default
table: landing
name: improved_sorting_key
type: Normal
sorting_key: ['user_id','date']
query: SELECT * ORDER BY user_id, date
Row 2:
──────
database: default
table: landing
name: agg_no_key
type: Aggregate
sorting_key: []
query: SELECT count()
```

View File

@ -49,7 +49,7 @@ Result:
## multiIf
Allows to write the [CASE](../../sql-reference/operators/index.md#operator_case) operator more compactly in the query.
Allows to write the [CASE](../../sql-reference/operators/index.md#conditional-expression) operator more compactly in the query.
**Syntax**
@ -264,4 +264,4 @@ SELECT clamp(1, 2, 3) result, toTypeName(result) type;
┌─result─┬─type────┐
│ 2 │ Float64 │
└────────┴─────────┘
```
```

View File

@ -8,7 +8,7 @@ title: "CREATE ROW POLICY"
Creates a [row policy](../../../guides/sre/user-management/index.md#row-policy-management), i.e. a filter used to determine which rows a user can read from a table.
:::tip
Row policies makes sense only for users with readonly access. If user can modify table or copy partitions between tables, it defeats the restrictions of row policies.
Row policies make sense only for users with readonly access. If a user can modify a table or copy partitions between tables, it defeats the restrictions of row policies.
:::
Syntax:
@ -24,40 +24,40 @@ CREATE [ROW] POLICY [IF NOT EXISTS | OR REPLACE] policy_name1 [ON CLUSTER cluste
## USING Clause
Allows to specify a condition to filter rows. An user will see a row if the condition is calculated to non-zero for the row.
Allows specifying a condition to filter rows. A user will see a row if the condition is calculated to non-zero for the row.
## TO Clause
In the section `TO` you can provide a list of users and roles this policy should work for. For example, `CREATE ROW POLICY ... TO accountant, john@localhost`.
In the `TO` section you can provide a list of users and roles this policy should work for. For example, `CREATE ROW POLICY ... TO accountant, john@localhost`.
Keyword `ALL` means all the ClickHouse users including current user. Keyword `ALL EXCEPT` allow to exclude some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost`
Keyword `ALL` means all the ClickHouse users, including current user. Keyword `ALL EXCEPT` allows excluding some users from the all users list, for example, `CREATE ROW POLICY ... TO ALL EXCEPT accountant, john@localhost`
:::note
If there are no row policies defined for a table then any user can `SELECT` all the row from the table. Defining one or more row policies for the table makes the access to the table depending on the row policies no matter if those row policies are defined for the current user or not. For example, the following policy
If there are no row policies defined for a table, then any user can `SELECT` all the rows from the table. Defining one or more row policies for the table makes access to the table dependent on the row policies, no matter if those row policies are defined for the current user or not. For example, the following policy:
`CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter`
forbids the users `mira` and `peter` to see the rows with `b != 1`, and any non-mentioned user (e.g., the user `paul`) will see no rows from `mydb.table1` at all.
forbids the users `mira` and `peter` from seeing the rows with `b != 1`, and any non-mentioned user (e.g., the user `paul`) will see no rows from `mydb.table1` at all.
If that's not desirable it can't be fixed by adding one more row policy, like the following:
If that's not desirable, it can be fixed by adding one more row policy, like the following:
`CREATE ROW POLICY pol2 ON mydb.table1 USING 1 TO ALL EXCEPT mira, peter`
:::
## AS Clause
It's allowed to have more than one policy enabled on the same table for the same user at the one time. So we need a way to combine the conditions from multiple policies.
It's allowed to have more than one policy enabled on the same table for the same user at one time. So we need a way to combine the conditions from multiple policies.
By default policies are combined using the boolean `OR` operator. For example, the following policies
By default, policies are combined using the boolean `OR` operator. For example, the following policies:
``` sql
CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter
CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 TO peter, antonio
```
enables the user `peter` to see rows with either `b=1` or `c=2`.
enable the user `peter` to see rows with either `b=1` or `c=2`.
The `AS` clause specifies how policies should be combined with other policies. Policies can be either permissive or restrictive. By default policies are permissive, which means they are combined using the boolean `OR` operator.
The `AS` clause specifies how policies should be combined with other policies. Policies can be either permissive or restrictive. By default, policies are permissive, which means they are combined using the boolean `OR` operator.
A policy can be defined as restrictive as an alternative. Restrictive policies are combined using the boolean `AND` operator.
@ -68,25 +68,25 @@ row_is_visible = (one or more of the permissive policies' conditions are non-zer
(all of the restrictive policies's conditions are non-zero)
```
For example, the following policies
For example, the following policies:
``` sql
CREATE ROW POLICY pol1 ON mydb.table1 USING b=1 TO mira, peter
CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 AS RESTRICTIVE TO peter, antonio
```
enables the user `peter` to see rows only if both `b=1` AND `c=2`.
enable the user `peter` to see rows only if both `b=1` AND `c=2`.
Database policies are combined with table policies.
For example, the following policies
For example, the following policies:
``` sql
CREATE ROW POLICY pol1 ON mydb.* USING b=1 TO mira, peter
CREATE ROW POLICY pol2 ON mydb.table1 USING c=2 AS RESTRICTIVE TO peter, antonio
```
enables the user `peter` to see table1 rows only if both `b=1` AND `c=2`, although
enable the user `peter` to see table1 rows only if both `b=1` AND `c=2`, although
any other table in mydb would have only `b=1` policy applied for the user.

View File

@ -280,7 +280,7 @@ SYSTEM START REPLICATION QUEUES [ON CLUSTER cluster_name] [[db.]replicated_merge
Ждет когда таблица семейства `ReplicatedMergeTree` будет синхронизирована с другими репликами в кластере, но не более `receive_timeout` секунд:
``` sql
SYSTEM SYNC REPLICA [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT [FROM 'srcReplica1'[, 'srcReplica2'[, ...]]] | PULL]
SYSTEM SYNC REPLICA [ON CLUSTER cluster_name] [db.]replicated_merge_tree_family_table_name [STRICT | LIGHTWEIGHT [FROM 'srcReplica1'[, 'srcReplica2'[, ...]]] | PULL]
```
После выполнения этого запроса таблица `[db.]replicated_merge_tree_family_table_name` загружает команды из общего реплицированного лога в свою собственную очередь репликации. Затем запрос ждет, пока реплика не обработает все загруженные команды. Поддерживаются следующие модификаторы:

View File

@ -209,7 +209,7 @@ std::map<std::pair<TypeIndex, String>, NodeToSubcolumnTransformer> node_transfor
},
};
std::tuple<FunctionNode *, ColumnNode *, TableNode *> getTypedNodesForOptimization(const QueryTreeNodePtr & node)
std::tuple<FunctionNode *, ColumnNode *, TableNode *> getTypedNodesForOptimization(const QueryTreeNodePtr & node, const ContextPtr & context)
{
auto * function_node = node->as<FunctionNode>();
if (!function_node)
@ -232,6 +232,12 @@ std::tuple<FunctionNode *, ColumnNode *, TableNode *> getTypedNodesForOptimizati
const auto & storage_snapshot = table_node->getStorageSnapshot();
auto column = first_argument_column_node->getColumn();
/// If view source is set we cannot optimize because it doesn't support moving functions to subcolumns.
/// The storage is replaced to the view source but it happens only after building a query tree and applying passes.
auto view_source = context->getViewSource();
if (view_source && view_source->getStorageID().getFullNameNotQuoted() == storage->getStorageID().getFullNameNotQuoted())
return {};
if (!storage->supportsOptimizationToSubcolumns() || storage->isVirtualColumn(column.name, storage_snapshot->metadata))
return {};
@ -266,7 +272,7 @@ public:
return;
}
auto [function_node, first_argument_node, table_node] = getTypedNodesForOptimization(node);
auto [function_node, first_argument_node, table_node] = getTypedNodesForOptimization(node, getContext());
if (function_node && first_argument_node && table_node)
{
enterImpl(*function_node, *first_argument_node, *table_node);
@ -416,7 +422,7 @@ public:
if (!getSettings().optimize_functions_to_subcolumns)
return;
auto [function_node, first_argument_column_node, table_node] = getTypedNodesForOptimization(node);
auto [function_node, first_argument_column_node, table_node] = getTypedNodesForOptimization(node, getContext());
if (!function_node || !first_argument_column_node || !table_node)
return;

View File

@ -300,7 +300,7 @@ void ColumnDynamic::get(size_t n, Field & res) const
auto value_data = shared_variant.getDataAt(variant_col.offsetAt(n));
ReadBufferFromMemory buf(value_data.data, value_data.size);
auto type = decodeDataType(buf);
getVariantSerialization(type)->deserializeBinary(res, buf, getFormatSettings());
type->getDefaultSerialization()->deserializeBinary(res, buf, getFormatSettings());
}
@ -736,8 +736,7 @@ StringRef ColumnDynamic::serializeValueIntoArena(size_t n, Arena & arena, const
{
const auto & variant_type = assert_cast<const DataTypeVariant &>(*variant_info.variant_type).getVariant(discr);
encodeDataType(variant_type, buf);
getVariantSerialization(variant_type, variant_info.variant_names[discr])
->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings());
variant_type->getDefaultSerialization()->serializeBinary(variant_col.getVariantByGlobalDiscriminator(discr), variant_col.offsetAt(n), buf, getFormatSettings());
type_and_value = buf.str();
}
@ -870,7 +869,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_
/// We have both values serialized in binary format, so we need to
/// create temporary column, insert both values into it and compare.
auto tmp_column = left_data_type->createColumn();
const auto & serialization = getVariantSerialization(left_data_type, left_data_type_name);
const auto & serialization = left_data_type->getDefaultSerialization();
serialization->deserializeBinary(*tmp_column, buf_left, getFormatSettings());
serialization->deserializeBinary(*tmp_column, buf_right, getFormatSettings());
return tmp_column->compareAt(0, 1, *tmp_column, nan_direction_hint);
@ -892,7 +891,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_
/// We have left value serialized in binary format, we need to
/// create temporary column, insert the value into it and compare.
auto tmp_column = left_data_type->createColumn();
getVariantSerialization(left_data_type, left_data_type_name)->deserializeBinary(*tmp_column, buf_left, getFormatSettings());
left_data_type->getDefaultSerialization()->deserializeBinary(*tmp_column, buf_left, getFormatSettings());
return tmp_column->compareAt(0, right_variant.offsetAt(m), right_variant.getVariantByGlobalDiscriminator(right_discr), nan_direction_hint);
}
/// Check if only right value is in shared data.
@ -912,7 +911,7 @@ int ColumnDynamic::doCompareAt(size_t n, size_t m, const IColumn & rhs, int nan_
/// We have right value serialized in binary format, we need to
/// create temporary column, insert the value into it and compare.
auto tmp_column = right_data_type->createColumn();
getVariantSerialization(right_data_type, right_data_type_name)->deserializeBinary(*tmp_column, buf_right, getFormatSettings());
right_data_type->getDefaultSerialization()->deserializeBinary(*tmp_column, buf_right, getFormatSettings());
return left_variant.getVariantByGlobalDiscriminator(left_discr).compareAt(left_variant.offsetAt(n), 0, *tmp_column, nan_direction_hint);
}
/// Otherwise both values are regular variants.

View File

@ -414,7 +414,7 @@ public:
/// Insert value into shared variant. Also updates Variant discriminators and offsets.
void insertValueIntoSharedVariant(const IColumn & src, const DataTypePtr & type, const String & type_name, size_t n);
const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name) const
const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type, const String & variant_name)
{
/// Get serialization for provided data type.
/// To avoid calling type->getDefaultSerialization() every time we use simple cache with max size.
@ -428,7 +428,7 @@ public:
return serialization_cache.emplace(variant_name, variant_type->getDefaultSerialization()).first->second;
}
const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) const { return getVariantSerialization(variant_type, variant_type->getName()); }
const SerializationPtr & getVariantSerialization(const DataTypePtr & variant_type) { return getVariantSerialization(variant_type, variant_type->getName()); }
private:
void createVariantInfo(const DataTypePtr & variant_type);
@ -473,7 +473,7 @@ private:
/// We can use serializations of different data types to serialize values into shared variant.
/// To avoid creating the same serialization multiple times, use simple cache.
static const size_t SERIALIZATION_CACHE_MAX_SIZE = 256;
mutable std::unordered_map<String, SerializationPtr> serialization_cache;
std::unordered_map<String, SerializationPtr> serialization_cache;
};
void extendVariantColumn(

View File

@ -13,6 +13,7 @@
#include <IO/ReadHelpers.h>
#include <Interpreters/Context.h>
#include <Core/Settings.h>
#include <Poco/Environment.h>
#pragma clang diagnostic ignored "-Wreserved-identifier"
@ -371,8 +372,8 @@ try
/// in case of double fault.
LOG_FATAL(log, "########## Short fault info ############");
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}) (from thread {}) Received signal {}",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH,
LOG_FATAL(log, "(version {}{}, build id: {}, git hash: {}, architecture: {}) (from thread {}) Received signal {}",
VERSION_STRING, VERSION_OFFICIAL, daemon ? daemon->build_id : "", GIT_HASH, Poco::Environment::osArchitecture(),
thread_num, sig);
std::string signal_description = "Unknown signal";

View File

@ -185,7 +185,7 @@ std::unique_ptr<IDataType::SubstreamData> DataTypeDynamic::getDynamicSubcolumnDa
auto type = decodeDataType(buf);
if (type->getName() == subcolumn_type_name)
{
dynamic_column.getVariantSerialization(subcolumn_type, subcolumn_type_name)->deserializeBinary(*subcolumn, buf, format_settings);
subcolumn_type->getDefaultSerialization()->deserializeBinary(*subcolumn, buf, format_settings);
null_map.push_back(0);
}
else

View File

@ -1,10 +1,12 @@
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeObject.h>
#include <DataTypes/DataTypeObjectDeprecated.h>
#include <DataTypes/Serializations/SerializationJSON.h>
#include <DataTypes/Serializations/SerializationObjectTypedPath.h>
#include <DataTypes/Serializations/SerializationObjectDynamicPath.h>
#include <DataTypes/Serializations/SerializationSubObject.h>
#include <Columns/ColumnObject.h>
#include <Common/CurrentThread.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
@ -513,13 +515,24 @@ static DataTypePtr createObject(const ASTPtr & arguments, const DataTypeObject::
static DataTypePtr createJSON(const ASTPtr & arguments)
{
auto context = CurrentThread::getQueryContext();
if (!context)
context = Context::getGlobalContextInstance();
if (context->getSettingsRef().use_json_alias_for_old_object_type)
{
if (arguments && !arguments->children.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Experimental Object type doesn't support any arguments. If you want to use new JSON type, set setting allow_experimental_json_type = 1");
return std::make_shared<DataTypeObjectDeprecated>("JSON", false);
}
return createObject(arguments, DataTypeObject::SchemaFormat::JSON);
}
void registerDataTypeJSON(DataTypeFactory & factory)
{
if (!Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type)
factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive);
factory.registerDataType("JSON", createJSON, DataTypeFactory::Case::Insensitive);
}
}

View File

@ -78,10 +78,6 @@ static DataTypePtr create(const ASTPtr & arguments)
void registerDataTypeObjectDeprecated(DataTypeFactory & factory)
{
factory.registerDataType("Object", create);
if (Context::getGlobalContextInstance()->getSettingsRef().use_json_alias_for_old_object_type)
factory.registerSimpleDataType("JSON",
[] { return std::make_shared<DataTypeObjectDeprecated>("JSON", false); },
DataTypeFactory::Case::Insensitive);
}
}

View File

@ -489,9 +489,8 @@ void SerializationDynamic::serializeBinary(const IColumn & column, size_t row_nu
}
const auto & variant_type = assert_cast<const DataTypeVariant &>(*variant_info.variant_type).getVariant(global_discr);
const auto & variant_type_name = variant_info.variant_names[global_discr];
encodeDataType(variant_type, ostr);
dynamic_column.getVariantSerialization(variant_type, variant_type_name)->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings);
variant_type->getDefaultSerialization()->serializeBinary(variant_column.getVariantByGlobalDiscriminator(global_discr), variant_column.offsetAt(row_num), ostr, settings);
}
template <typename ReturnType = void, typename DeserializeFunc>
@ -629,7 +628,7 @@ static void serializeTextImpl(
ReadBufferFromMemory buf(value.data, value.size);
auto variant_type = decodeDataType(buf);
auto tmp_variant_column = variant_type->createColumn();
auto variant_serialization = dynamic_column.getVariantSerialization(variant_type);
auto variant_serialization = variant_type->getDefaultSerialization();
variant_serialization->deserializeBinary(*tmp_variant_column, buf, settings);
nested_serialize(*variant_serialization, *tmp_variant_column, 0, ostr);
}

View File

@ -1179,6 +1179,12 @@ public:
const FormatSettings & format_settings,
String & error) const override
{
if (element.isNull() && format_settings.null_as_default)
{
column.insertDefault();
return true;
}
auto & tuple = assert_cast<ColumnTuple &>(column);
size_t old_size = column.size();
bool were_valid_elements = false;
@ -1298,6 +1304,12 @@ public:
const FormatSettings & format_settings,
String & error) const override
{
if (element.isNull() && format_settings.null_as_default)
{
column.insertDefault();
return true;
}
if (!element.isObject())
{
error = fmt::format("cannot read Map value from JSON element: {}", jsonElementToString<JSONParser>(element, format_settings));
@ -1362,6 +1374,14 @@ public:
String & error) const override
{
auto & column_variant = assert_cast<ColumnVariant &>(column);
/// Check if element is NULL.
if (element.isNull())
{
column_variant.insertDefault();
return true;
}
for (size_t i : order)
{
auto & variant = column_variant.getVariantByGlobalDiscriminator(i);

View File

@ -4,17 +4,21 @@
#if USE_ICU
#include <Columns/ColumnString.h>
#include <Functions/LowerUpperImpl.h>
#include <unicode/unistr.h>
#include <Common/StringUtils.h>
# include <Columns/ColumnString.h>
# include <Functions/LowerUpperImpl.h>
# include <unicode/ucasemap.h>
# include <unicode/unistr.h>
# include <unicode/urename.h>
# include <unicode/utypes.h>
# include <Common/StringUtils.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
template <char not_case_lower_bound, char not_case_upper_bound, bool upper>
@ -27,7 +31,7 @@ struct LowerUpperUTF8Impl
ColumnString::Offsets & res_offsets,
size_t input_rows_count)
{
if (data.empty())
if (input_rows_count == 0)
return;
bool all_ascii = isAllASCII(data.data(), data.size());
@ -38,39 +42,56 @@ struct LowerUpperUTF8Impl
}
res_data.resize(data.size());
res_offsets.resize_exact(offsets.size());
res_offsets.resize_exact(input_rows_count);
UErrorCode error_code = U_ZERO_ERROR;
UCaseMap * case_map = ucasemap_open("", U_FOLD_CASE_DEFAULT, &error_code);
if (U_FAILURE(error_code))
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Error calling ucasemap_open: {}", u_errorName(error_code));
String output;
size_t curr_offset = 0;
for (size_t i = 0; i < input_rows_count; ++i)
for (size_t row_i = 0; row_i < input_rows_count; ++row_i)
{
const auto * data_start = reinterpret_cast<const char *>(&data[offsets[i - 1]]);
size_t size = offsets[i] - offsets[i - 1];
const auto * src = reinterpret_cast<const char *>(&data[offsets[row_i - 1]]);
size_t src_size = offsets[row_i] - offsets[row_i - 1] - 1;
icu::UnicodeString input(data_start, static_cast<int32_t>(size), "UTF-8");
int32_t dst_size;
if constexpr (upper)
input.toUpper();
dst_size = ucasemap_utf8ToUpper(
case_map, reinterpret_cast<char *>(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code);
else
input.toLower();
dst_size = ucasemap_utf8ToLower(
case_map, reinterpret_cast<char *>(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code);
output.clear();
input.toUTF8String(output);
if (error_code == U_BUFFER_OVERFLOW_ERROR || error_code == U_STRING_NOT_TERMINATED_WARNING)
{
size_t new_size = curr_offset + dst_size + 1;
res_data.resize(new_size);
/// For valid UTF-8 input strings, ICU sometimes produces output with an extra '\0 at the end. Only the data before that
/// '\0' is valid. If the input is not valid UTF-8, then the behavior of lower/upperUTF8 is undefined by definition. In this
/// case, the behavior is also reasonable.
size_t valid_size = output.size();
if (!output.empty() && output.back() == '\0')
--valid_size;
error_code = U_ZERO_ERROR;
if constexpr (upper)
dst_size = ucasemap_utf8ToUpper(
case_map, reinterpret_cast<char *>(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code);
else
dst_size = ucasemap_utf8ToLower(
case_map, reinterpret_cast<char *>(&res_data[curr_offset]), res_data.size() - curr_offset, src, src_size, &error_code);
}
res_data.resize(curr_offset + valid_size + 1);
if (error_code != U_ZERO_ERROR)
throw DB::Exception(
ErrorCodes::LOGICAL_ERROR,
"Error calling {}: {} input: {} input_size: {}",
upper ? "ucasemap_utf8ToUpper" : "ucasemap_utf8ToLower",
u_errorName(error_code),
std::string_view(src, src_size),
src_size);
memcpy(&res_data[curr_offset], output.data(), valid_size);
res_data[curr_offset + valid_size] = 0;
curr_offset += valid_size + 1;
res_offsets[i] = curr_offset;
res_data[curr_offset + dst_size] = 0;
curr_offset += dst_size + 1;
res_offsets[row_i] = curr_offset;
}
res_data.resize(curr_offset);
}
static void vectorFixed(const ColumnString::Chars &, size_t, ColumnString::Chars &, size_t)

View File

@ -59,6 +59,18 @@ class CompiledAggregateFunctionsHolder;
class NativeWriter;
struct OutputBlockColumns;
struct GroupingSetsParams
{
GroupingSetsParams() = default;
GroupingSetsParams(Names used_keys_, Names missing_keys_) : used_keys(std::move(used_keys_)), missing_keys(std::move(missing_keys_)) { }
Names used_keys;
Names missing_keys;
};
using GroupingSetsParamsList = std::vector<GroupingSetsParams>;
/** How are "total" values calculated with WITH TOTALS?
* (For more details, see TotalsHavingTransform.)
*

View File

@ -389,6 +389,10 @@ AsynchronousInsertQueue::pushDataChunk(ASTPtr query, DataChunk chunk, ContextPtr
if (data_kind == DataKind::Preprocessed)
insert_query.format = "Native";
/// Query parameters make sense only for format Values.
if (insert_query.format == "Values")
entry->query_parameters = query_context->getQueryParameters();
InsertQuery key{query, query_context->getUserID(), query_context->getCurrentRoles(), settings, data_kind};
InsertDataPtr data_to_process;
std::future<void> insert_future;
@ -999,6 +1003,7 @@ Chunk AsynchronousInsertQueue::processEntriesWithParsing(
"Expected entry with data kind Parsed. Got: {}", entry->chunk.getDataKind());
auto buffer = std::make_unique<ReadBufferFromString>(*bytes);
executor.setQueryParameters(entry->query_parameters);
size_t num_bytes = bytes->size();
size_t num_rows = executor.execute(*buffer);

View File

@ -147,6 +147,7 @@ private:
const String format;
MemoryTracker * const user_memory_tracker;
const std::chrono::time_point<std::chrono::system_clock> create_time;
NameToNameMap query_parameters;
Entry(
DataChunk && chunk_,

View File

@ -347,6 +347,27 @@ bool shouldIgnoreQuotaAndLimits(const StorageID & table_id)
return false;
}
GroupingSetsParamsList getAggregatorGroupingSetsParams(const NamesAndTypesLists & aggregation_keys_list, const Names & all_keys)
{
GroupingSetsParamsList result;
for (const auto & aggregation_keys : aggregation_keys_list)
{
NameSet keys;
for (const auto & key : aggregation_keys)
keys.insert(key.name);
Names missing_keys;
for (const auto & key : all_keys)
if (!keys.contains(key))
missing_keys.push_back(key);
result.emplace_back(aggregation_keys.getNames(), std::move(missing_keys));
}
return result;
}
}
InterpreterSelectQuery::InterpreterSelectQuery(
@ -2005,13 +2026,12 @@ static void executeMergeAggregatedImpl(
bool has_grouping_sets,
const Settings & settings,
const NamesAndTypesList & aggregation_keys,
const NamesAndTypesLists & aggregation_keys_list,
const AggregateDescriptions & aggregates,
bool should_produce_results_in_order_of_bucket_number,
SortDescription group_by_sort_description)
{
auto keys = aggregation_keys.getNames();
if (has_grouping_sets)
keys.insert(keys.begin(), "__grouping_set");
/** There are two modes of distributed aggregation.
*
@ -2029,10 +2049,12 @@ static void executeMergeAggregatedImpl(
*/
Aggregator::Params params(keys, aggregates, overflow_row, settings.max_threads, settings.max_block_size, settings.min_hit_rate_to_use_consecutive_keys_optimization);
auto grouping_sets_params = getAggregatorGroupingSetsParams(aggregation_keys_list, keys);
auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
query_plan.getCurrentDataStream(),
params,
grouping_sets_params,
final,
/// Grouping sets don't work with distributed_aggregation_memory_efficient enabled (#43989)
settings.distributed_aggregation_memory_efficient && is_remote_storage && !has_grouping_sets,
@ -2653,30 +2675,6 @@ static Aggregator::Params getAggregatorParams(
};
}
static GroupingSetsParamsList getAggregatorGroupingSetsParams(const SelectQueryExpressionAnalyzer & query_analyzer, const Names & all_keys)
{
GroupingSetsParamsList result;
if (query_analyzer.useGroupingSetKey())
{
auto const & aggregation_keys_list = query_analyzer.aggregationKeysList();
for (const auto & aggregation_keys : aggregation_keys_list)
{
NameSet keys;
for (const auto & key : aggregation_keys)
keys.insert(key.name);
Names missing_keys;
for (const auto & key : all_keys)
if (!keys.contains(key))
missing_keys.push_back(key);
result.emplace_back(aggregation_keys.getNames(), std::move(missing_keys));
}
}
return result;
}
void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const ActionsAndProjectInputsFlagPtr & expression, bool overflow_row, bool final, InputOrderInfoPtr group_by_info)
{
executeExpression(query_plan, expression, "Before GROUP BY");
@ -2696,7 +2694,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPlan & query_plan, const Ac
settings.group_by_two_level_threshold,
settings.group_by_two_level_threshold_bytes);
auto grouping_sets_params = getAggregatorGroupingSetsParams(*query_analyzer, keys);
auto grouping_sets_params = getAggregatorGroupingSetsParams(query_analyzer->aggregationKeysList(), keys);
SortDescription group_by_sort_description;
SortDescription sort_description_for_merging;
@ -2764,6 +2762,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPlan & query_plan, bool
has_grouping_sets,
context->getSettingsRef(),
query_analyzer->aggregationKeys(),
query_analyzer->aggregationKeysList(),
query_analyzer->aggregates(),
should_produce_results_in_order_of_bucket_number,
std::move(group_by_sort_description));

View File

@ -504,8 +504,6 @@ void addMergingAggregatedStep(QueryPlan & query_plan,
*/
auto keys = aggregation_analysis_result.aggregation_keys;
if (!aggregation_analysis_result.grouping_sets_parameters_list.empty())
keys.insert(keys.begin(), "__grouping_set");
Aggregator::Params params(keys,
aggregation_analysis_result.aggregate_descriptions,
@ -530,6 +528,7 @@ void addMergingAggregatedStep(QueryPlan & query_plan,
auto merging_aggregated = std::make_unique<MergingAggregatedStep>(
query_plan.getCurrentDataStream(),
params,
aggregation_analysis_result.grouping_sets_parameters_list,
query_analysis_result.aggregate_final,
/// Grouping sets don't work with distributed_aggregation_memory_efficient enabled (#43989)
settings.distributed_aggregation_memory_efficient && (is_remote_storage || parallel_replicas_from_merge_tree) && !query_analysis_result.aggregation_with_rollup_or_cube_or_grouping_sets,

View File

@ -1,5 +1,6 @@
#include <Processors/Executors/StreamingFormatExecutor.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
namespace DB
{
@ -32,6 +33,13 @@ MutableColumns StreamingFormatExecutor::getResultColumns()
return ret_columns;
}
void StreamingFormatExecutor::setQueryParameters(const NameToNameMap & parameters)
{
/// Query parameters make sense only for format Values.
if (auto * values_format = typeid_cast<ValuesBlockInputFormat *>(format.get()))
values_format->setQueryParameters(parameters);
}
size_t StreamingFormatExecutor::execute(ReadBuffer & buffer)
{
format->setReadBuffer(buffer);

View File

@ -39,6 +39,9 @@ public:
/// Releases currently accumulated columns.
MutableColumns getResultColumns();
/// Sets query parameters for input format if applicable.
void setQueryParameters(const NameToNameMap & parameters);
private:
const Block header;
const InputFormatPtr format;

View File

@ -663,6 +663,16 @@ void ValuesBlockInputFormat::resetReadBuffer()
IInputFormat::resetReadBuffer();
}
void ValuesBlockInputFormat::setQueryParameters(const NameToNameMap & parameters)
{
if (parameters == context->getQueryParameters())
return;
auto context_copy = Context::createCopy(context);
context_copy->setQueryParameters(parameters);
context = std::move(context_copy);
}
ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowSchemaReader(buf, format_settings_), buf(in_)
{

View File

@ -38,6 +38,7 @@ public:
/// TODO: remove context somehow.
void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); }
void setQueryParameters(const NameToNameMap & parameters);
const BlockMissingValues & getMissingValues() const override { return block_missing_values; }

View File

@ -151,6 +151,61 @@ void AggregatingStep::applyOrder(SortDescription sort_description_for_merging_,
explicit_sorting_required_for_aggregation_in_order = false;
}
ActionsDAG AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG(
const Block & in_header,
const Block & out_header,
const GroupingSetsParamsList & grouping_sets_params,
UInt64 group,
bool group_by_use_nulls)
{
/// Here we create a DAG which fills missing keys and adds `__grouping_set` column
ActionsDAG dag(in_header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs outputs;
outputs.reserve(out_header.columns() + 1);
auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, group), 0);
const auto * grouping_node = &dag.addColumn(
{ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"});
grouping_node = &dag.materializeNode(*grouping_node);
outputs.push_back(grouping_node);
const auto & missing_columns = grouping_sets_params[group].missing_keys;
const auto & used_keys = grouping_sets_params[group].used_keys;
auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr);
for (size_t i = 0; i < out_header.columns(); ++i)
{
const auto & col = out_header.getByPosition(i);
const auto missing_it = std::find_if(
missing_columns.begin(), missing_columns.end(), [&](const auto & missing_col) { return missing_col == col.name; });
const auto used_it = std::find_if(
used_keys.begin(), used_keys.end(), [&](const auto & used_col) { return used_col == col.name; });
if (missing_it != missing_columns.end())
{
auto column_with_default = col.column->cloneEmpty();
col.type->insertDefaultInto(*column_with_default);
column_with_default->finalize();
auto column = ColumnConst::create(std::move(column_with_default), 0);
const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name});
node = &dag.materializeNode(*node);
outputs.push_back(node);
}
else
{
const auto * column_node = dag.getOutputs()[in_header.getPositionByName(col.name)];
if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable())
outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name));
else
outputs.push_back(column_node);
}
}
dag.getOutputs().swap(outputs);
return dag;
}
void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings & settings)
{
QueryPipelineProcessorsCollector collector(pipeline, this);
@ -300,51 +355,7 @@ void AggregatingStep::transformPipeline(QueryPipelineBuilder & pipeline, const B
{
const auto & header = ports[set_counter]->getHeader();
/// Here we create a DAG which fills missing keys and adds `__grouping_set` column
ActionsDAG dag(header.getColumnsWithTypeAndName());
ActionsDAG::NodeRawConstPtrs outputs;
outputs.reserve(output_header.columns() + 1);
auto grouping_col = ColumnConst::create(ColumnUInt64::create(1, set_counter), 0);
const auto * grouping_node = &dag.addColumn(
{ColumnPtr(std::move(grouping_col)), std::make_shared<DataTypeUInt64>(), "__grouping_set"});
grouping_node = &dag.materializeNode(*grouping_node);
outputs.push_back(grouping_node);
const auto & missing_columns = grouping_sets_params[set_counter].missing_keys;
const auto & used_keys = grouping_sets_params[set_counter].used_keys;
auto to_nullable_function = FunctionFactory::instance().get("toNullable", nullptr);
for (size_t i = 0; i < output_header.columns(); ++i)
{
auto & col = output_header.getByPosition(i);
const auto missing_it = std::find_if(
missing_columns.begin(), missing_columns.end(), [&](const auto & missing_col) { return missing_col == col.name; });
const auto used_it = std::find_if(
used_keys.begin(), used_keys.end(), [&](const auto & used_col) { return used_col == col.name; });
if (missing_it != missing_columns.end())
{
auto column_with_default = col.column->cloneEmpty();
col.type->insertDefaultInto(*column_with_default);
column_with_default->finalize();
auto column = ColumnConst::create(std::move(column_with_default), 0);
const auto * node = &dag.addColumn({ColumnPtr(std::move(column)), col.type, col.name});
node = &dag.materializeNode(*node);
outputs.push_back(node);
}
else
{
const auto * column_node = dag.getOutputs()[header.getPositionByName(col.name)];
if (used_it != used_keys.end() && group_by_use_nulls && column_node->result_type->canBeInsideNullable())
outputs.push_back(&dag.addFunction(to_nullable_function, { column_node }, col.name));
else
outputs.push_back(column_node);
}
}
dag.getOutputs().swap(outputs);
auto dag = makeCreatingMissingKeysForGroupingSetDAG(header, output_header, grouping_sets_params, set_counter, group_by_use_nulls);
auto expression = std::make_shared<ExpressionActions>(std::move(dag), settings.getActionsSettings());
auto transform = std::make_shared<ExpressionTransform>(header, expression);

View File

@ -7,18 +7,6 @@
namespace DB
{
struct GroupingSetsParams
{
GroupingSetsParams() = default;
GroupingSetsParams(Names used_keys_, Names missing_keys_) : used_keys(std::move(used_keys_)), missing_keys(std::move(missing_keys_)) { }
Names used_keys;
Names missing_keys;
};
using GroupingSetsParamsList = std::vector<GroupingSetsParams>;
Block appendGroupingSetColumn(Block header);
Block generateOutputHeader(const Block & input_header, const Names & keys, bool use_nulls);
@ -77,6 +65,13 @@ public:
/// Argument input_stream would be the second input (from projection).
std::unique_ptr<AggregatingProjectionStep> convertToAggregatingProjection(const DataStream & input_stream) const;
static ActionsDAG makeCreatingMissingKeysForGroupingSetDAG(
const Block & in_header,
const Block & out_header,
const GroupingSetsParamsList & grouping_sets_params,
UInt64 group,
bool group_by_use_nulls);
private:
void updateOutputStream() override;

View File

@ -10,6 +10,11 @@
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
static bool memoryBoundMergingWillBeUsed(
const DataStream & input_stream,
bool memory_bound_merging_of_aggregation_results_enabled,
@ -37,6 +42,7 @@ static ITransformingStep::Traits getTraits(bool should_produce_results_in_order_
MergingAggregatedStep::MergingAggregatedStep(
const DataStream & input_stream_,
Aggregator::Params params_,
GroupingSetsParamsList grouping_sets_params_,
bool final_,
bool memory_efficient_aggregation_,
size_t max_threads_,
@ -48,9 +54,10 @@ MergingAggregatedStep::MergingAggregatedStep(
bool memory_bound_merging_of_aggregation_results_enabled_)
: ITransformingStep(
input_stream_,
params_.getHeader(input_stream_.header, final_),
MergingAggregatedTransform::appendGroupingIfNeeded(input_stream_.header, params_.getHeader(input_stream_.header, final_)),
getTraits(should_produce_results_in_order_of_bucket_number_))
, params(std::move(params_))
, grouping_sets_params(std::move(grouping_sets_params_))
, final(final_)
, memory_efficient_aggregation(memory_efficient_aggregation_)
, max_threads(max_threads_)
@ -89,10 +96,13 @@ void MergingAggregatedStep::applyOrder(SortDescription sort_description, DataStr
void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
auto transform_params = std::make_shared<AggregatingTransformParams>(pipeline.getHeader(), std::move(params), final);
if (memoryBoundMergingWillBeUsed())
{
if (input_streams.front().header.has("__grouping_set") || !grouping_sets_params.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Memory bound merging of aggregated results is not supported for grouping sets.");
auto transform_params = std::make_shared<AggregatingTransformParams>(pipeline.getHeader(), std::move(params), final);
auto transform = std::make_shared<FinishAggregatingInOrderTransform>(
pipeline.getHeader(),
pipeline.getNumStreams(),
@ -127,15 +137,19 @@ void MergingAggregatedStep::transformPipeline(QueryPipelineBuilder & pipeline, c
pipeline.resize(1);
/// Now merge the aggregated blocks
pipeline.addSimpleTransform([&](const Block & header)
{ return std::make_shared<MergingAggregatedTransform>(header, transform_params, max_threads); });
auto transform = std::make_shared<MergingAggregatedTransform>(pipeline.getHeader(), params, final, grouping_sets_params, max_threads);
pipeline.addTransform(std::move(transform));
}
else
{
if (input_streams.front().header.has("__grouping_set") || !grouping_sets_params.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Memory efficient merging of aggregated results is not supported for grouping sets.");
auto num_merge_threads = memory_efficient_merge_threads
? memory_efficient_merge_threads
: max_threads;
auto transform_params = std::make_shared<AggregatingTransformParams>(pipeline.getHeader(), std::move(params), final);
pipeline.addMergingAggregatedMemoryEfficientTransform(transform_params, num_merge_threads);
}
@ -154,7 +168,9 @@ void MergingAggregatedStep::describeActions(JSONBuilder::JSONMap & map) const
void MergingAggregatedStep::updateOutputStream()
{
output_stream = createOutputStream(input_streams.front(), params.getHeader(input_streams.front().header, final), getDataStreamTraits());
const auto & in_header = input_streams.front().header;
output_stream = createOutputStream(input_streams.front(),
MergingAggregatedTransform::appendGroupingIfNeeded(in_header, params.getHeader(in_header, final)), getDataStreamTraits());
if (is_order_overwritten) /// overwrite order again
applyOrder(group_by_sort_description, overwritten_sort_scope);
}

View File

@ -16,6 +16,7 @@ public:
MergingAggregatedStep(
const DataStream & input_stream_,
Aggregator::Params params_,
GroupingSetsParamsList grouping_sets_params_,
bool final_,
bool memory_efficient_aggregation_,
size_t max_threads_,
@ -43,6 +44,7 @@ private:
Aggregator::Params params;
GroupingSetsParamsList grouping_sets_params;
bool final;
bool memory_efficient_aggregation;
size_t max_threads;

View File

@ -1,7 +1,10 @@
#include <Processors/Transforms/MergingAggregatedTransform.h>
#include <Processors/Transforms/AggregatingTransform.h>
#include <Processors/Transforms/AggregatingInOrderTransform.h>
#include <Processors/QueryPlan/AggregatingStep.h>
#include <Common/logger_useful.h>
#include <Interpreters/ExpressionActions.h>
#include <DataTypes/DataTypesNumber.h>
namespace DB
{
@ -10,11 +13,192 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
MergingAggregatedTransform::MergingAggregatedTransform(
Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_)
: IAccumulatingTransform(std::move(header_), params_->getHeader())
, params(std::move(params_)), max_threads(max_threads_)
Block MergingAggregatedTransform::appendGroupingIfNeeded(const Block & in_header, Block out_header)
{
/// __grouping_set is neither GROUP BY key nor an aggregate function.
/// It behaves like a GROUP BY key, but we cannot append it to keys
/// because it changes hashing method and buckets for two level aggregation.
/// Now, this column is processed "manually" by merging each group separately.
if (in_header.has("__grouping_set"))
out_header.insert(0, in_header.getByName("__grouping_set"));
return out_header;
}
/// We should keep the order for GROUPING SET keys.
/// Initiator creates a separate Aggregator for every group, so should we do here.
/// Otherwise, two-level aggregation will split the data into different buckets,
/// and the result may have duplicating rows.
static ActionsDAG makeReorderingActions(const Block & in_header, const GroupingSetsParams & params)
{
ActionsDAG reordering(in_header.getColumnsWithTypeAndName());
auto & outputs = reordering.getOutputs();
ActionsDAG::NodeRawConstPtrs new_outputs;
new_outputs.reserve(in_header.columns() + params.used_keys.size() - params.used_keys.size());
std::unordered_map<std::string_view, size_t> index;
for (size_t pos = 0; pos < outputs.size(); ++pos)
index.emplace(outputs[pos]->result_name, pos);
for (const auto & used_name : params.used_keys)
{
auto & idx = index[used_name];
new_outputs.push_back(outputs[idx]);
}
for (const auto & used_name : params.used_keys)
index[used_name] = outputs.size();
for (const auto & missing_name : params.missing_keys)
index[missing_name] = outputs.size();
for (const auto * output : outputs)
{
if (index[output->result_name] != outputs.size())
new_outputs.push_back(output);
}
outputs.swap(new_outputs);
return reordering;
}
MergingAggregatedTransform::~MergingAggregatedTransform() = default;
MergingAggregatedTransform::MergingAggregatedTransform(
Block header_,
Aggregator::Params params,
bool final,
GroupingSetsParamsList grouping_sets_params,
size_t max_threads_)
: IAccumulatingTransform(header_, appendGroupingIfNeeded(header_, params.getHeader(header_, final)))
, max_threads(max_threads_)
{
if (!grouping_sets_params.empty())
{
if (!header_.has("__grouping_set"))
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Cannot find __grouping_set column in header of MergingAggregatedTransform with grouping sets."
"Header {}", header_.dumpStructure());
auto in_header = header_;
in_header.erase(header_.getPositionByName("__grouping_set"));
auto out_header = params.getHeader(header_, final);
grouping_sets.reserve(grouping_sets_params.size());
for (const auto & grouping_set_params : grouping_sets_params)
{
size_t group = grouping_sets.size();
auto reordering = makeReorderingActions(in_header, grouping_set_params);
Aggregator::Params set_params(grouping_set_params.used_keys,
params.aggregates,
params.overflow_row,
params.max_threads,
params.max_block_size,
params.min_hit_rate_to_use_consecutive_keys_optimization);
auto transform_params = std::make_shared<AggregatingTransformParams>(reordering.updateHeader(in_header), std::move(set_params), final);
auto creating = AggregatingStep::makeCreatingMissingKeysForGroupingSetDAG(
transform_params->getHeader(),
out_header,
grouping_sets_params, group, false);
auto & groupiung_set = grouping_sets.emplace_back();
groupiung_set.reordering_key_columns_actions = std::make_shared<ExpressionActions>(std::move(reordering));
groupiung_set.creating_missing_keys_actions = std::make_shared<ExpressionActions>(std::move(creating));
groupiung_set.params = std::move(transform_params);
}
}
else
{
auto & groupiung_set = grouping_sets.emplace_back();
groupiung_set.params = std::make_shared<AggregatingTransformParams>(header_, std::move(params), final);
}
}
void MergingAggregatedTransform::addBlock(Block block)
{
if (grouping_sets.size() == 1)
{
auto bucket = block.info.bucket_num;
if (grouping_sets[0].reordering_key_columns_actions)
grouping_sets[0].reordering_key_columns_actions->execute(block);
grouping_sets[0].bucket_to_blocks[bucket].emplace_back(std::move(block));
return;
}
auto grouping_position = block.getPositionByName("__grouping_set");
auto grouping_column = block.getByPosition(grouping_position).column;
block.erase(grouping_position);
/// Split a block by __grouping_set values.
const auto * grouping_column_typed = typeid_cast<const ColumnUInt64 *>(grouping_column.get());
if (!grouping_column_typed)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected UInt64 column for __grouping_set, got {}", grouping_column->getName());
IColumn::Selector selector;
const auto & grouping_data = grouping_column_typed->getData();
size_t num_rows = grouping_data.size();
UInt64 last_group = grouping_data[0];
UInt64 max_group = last_group;
for (size_t row = 1; row < num_rows; ++row)
{
auto group = grouping_data[row];
/// Optimization for equal ranges.
if (last_group == group)
continue;
/// Optimization for single group.
if (selector.empty())
selector.reserve(num_rows);
/// Fill the last equal range.
selector.resize_fill(row, last_group);
last_group = group;
max_group = std::max(last_group, max_group);
}
if (max_group >= grouping_sets.size())
throw Exception(ErrorCodes::LOGICAL_ERROR,
"Invalid group number {}. Number of groups {}.", last_group, grouping_sets.size());
/// Optimization for single group.
if (selector.empty())
{
auto bucket = block.info.bucket_num;
grouping_sets[last_group].reordering_key_columns_actions->execute(block);
grouping_sets[last_group].bucket_to_blocks[bucket].emplace_back(std::move(block));
return;
}
/// Fill the last equal range.
selector.resize_fill(num_rows, last_group);
const size_t num_groups = max_group + 1;
Blocks splitted_blocks(num_groups);
for (size_t group_id = 0; group_id < num_groups; ++group_id)
splitted_blocks[group_id] = block.cloneEmpty();
size_t columns_in_block = block.columns();
for (size_t col_idx_in_block = 0; col_idx_in_block < columns_in_block; ++col_idx_in_block)
{
MutableColumns splitted_columns = block.getByPosition(col_idx_in_block).column->scatter(num_groups, selector);
for (size_t group_id = 0; group_id < num_groups; ++group_id)
splitted_blocks[group_id].getByPosition(col_idx_in_block).column = std::move(splitted_columns[group_id]);
}
for (size_t group = 0; group < num_groups; ++group)
{
auto & splitted_block = splitted_blocks[group];
splitted_block.info = block.info;
grouping_sets[group].reordering_key_columns_actions->execute(splitted_block);
grouping_sets[group].bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(splitted_block));
}
}
void MergingAggregatedTransform::consume(Chunk chunk)
@ -46,7 +230,7 @@ void MergingAggregatedTransform::consume(Chunk chunk)
block.info.is_overflows = agg_info->is_overflows;
block.info.bucket_num = agg_info->bucket_num;
bucket_to_blocks[agg_info->bucket_num].emplace_back(std::move(block));
addBlock(std::move(block));
}
else if (chunk.getChunkInfos().get<ChunkInfoWithAllocatedBytes>())
{
@ -54,7 +238,7 @@ void MergingAggregatedTransform::consume(Chunk chunk)
block.info.is_overflows = false;
block.info.bucket_num = -1;
bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block));
addBlock(std::move(block));
}
else
throw Exception(ErrorCodes::LOGICAL_ERROR, "Chunk should have AggregatedChunkInfo in MergingAggregatedTransform.");
@ -70,9 +254,23 @@ Chunk MergingAggregatedTransform::generate()
/// Exception safety. Make iterator valid in case any method below throws.
next_block = blocks.begin();
/// TODO: this operation can be made async. Add async for IAccumulatingTransform.
params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled);
blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads);
for (auto & grouping_set : grouping_sets)
{
auto & params = grouping_set.params;
auto & bucket_to_blocks = grouping_set.bucket_to_blocks;
AggregatedDataVariants data_variants;
/// TODO: this operation can be made async. Add async for IAccumulatingTransform.
params->aggregator.mergeBlocks(std::move(bucket_to_blocks), data_variants, max_threads, is_cancelled);
auto merged_blocks = params->aggregator.convertToBlocks(data_variants, params->final, max_threads);
if (grouping_set.creating_missing_keys_actions)
for (auto & block : merged_blocks)
grouping_set.creating_missing_keys_actions->execute(block);
blocks.splice(blocks.end(), std::move(merged_blocks));
}
next_block = blocks.begin();
}

View File

@ -6,26 +6,46 @@
namespace DB
{
class ExpressionActions;
using ExpressionActionsPtr = std::shared_ptr<ExpressionActions>;
/** A pre-aggregate stream of blocks in which each block is already aggregated.
* Aggregate functions in blocks should not be finalized so that their states can be merged.
*/
class MergingAggregatedTransform : public IAccumulatingTransform
{
public:
MergingAggregatedTransform(Block header_, AggregatingTransformParamsPtr params_, size_t max_threads_);
MergingAggregatedTransform(
Block header_,
Aggregator::Params params_,
bool final_,
GroupingSetsParamsList grouping_sets_params,
size_t max_threads_);
~MergingAggregatedTransform() override;
String getName() const override { return "MergingAggregatedTransform"; }
static Block appendGroupingIfNeeded(const Block & in_header, Block out_header);
protected:
void consume(Chunk chunk) override;
Chunk generate() override;
private:
AggregatingTransformParamsPtr params;
LoggerPtr log = getLogger("MergingAggregatedTransform");
size_t max_threads;
AggregatedDataVariants data_variants;
Aggregator::BucketToBlocks bucket_to_blocks;
struct GroupingSet
{
Aggregator::BucketToBlocks bucket_to_blocks;
ExpressionActionsPtr reordering_key_columns_actions;
ExpressionActionsPtr creating_missing_keys_actions;
AggregatingTransformParamsPtr params;
};
using GroupingSets = std::vector<GroupingSet>;
GroupingSets grouping_sets;
UInt64 total_input_rows = 0;
UInt64 total_input_blocks = 0;
@ -35,6 +55,8 @@ private:
bool consume_started = false;
bool generate_started = false;
void addBlock(Block block);
};
}

View File

@ -128,14 +128,14 @@ void MergeTreeIndexAggregatorFullText::update(const Block & block, size_t * pos,
"Position: {}, Block rows: {}.", *pos, block.rows());
size_t rows_read = std::min(limit, block.rows() - *pos);
auto row_id = store->getNextRowIDRange(rows_read);
auto start_row_id = row_id;
auto start_row_id = store->getNextRowIDRange(rows_read);
for (size_t col = 0; col < index_columns.size(); ++col)
{
const auto & column_with_type = block.getByName(index_columns[col]);
const auto & column = column_with_type.column;
size_t current_position = *pos;
auto row_id = start_row_id;
bool need_to_write = false;
if (isArray(column_with_type.type))

View File

@ -52,8 +52,10 @@ const std::unordered_map<String, unum::usearch::metric_kind_t> distanceFunctionT
/// Maps from user-facing name to internal name
const std::unordered_map<String, unum::usearch::scalar_kind_t> quantizationToScalarKind = {
{"f64", unum::usearch::scalar_kind_t::f64_k},
{"f32", unum::usearch::scalar_kind_t::f32_k},
{"f16", unum::usearch::scalar_kind_t::f16_k},
{"bf16", unum::usearch::scalar_kind_t::bf16_k},
{"i8", unum::usearch::scalar_kind_t::i8_k}};
/// Usearch provides more quantizations but ^^ above ones seem the only ones comprehensively supported across all distance functions.
@ -461,7 +463,7 @@ MergeTreeIndexPtr vectorSimilarityIndexCreator(const IndexDescription & index)
{
/// Default parameters:
unum::usearch::metric_kind_t metric_kind = distanceFunctionToMetricKind.at(index.arguments[1].safeGet<String>());
unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::f32_k;
unum::usearch::scalar_kind_t scalar_kind = unum::usearch::scalar_kind_t::bf16_k;
UsearchHnswParams usearch_hnsw_params;
/// Optional parameters:

View File

@ -0,0 +1,275 @@
#include <Storages/System/StorageSystemProjections.h>
#include <Access/ContextAccess.h>
#include <Columns/ColumnString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Databases/IDatabase.h>
#include <Storages/VirtualColumnUtils.h>
#include <Storages/System/getQueriedColumnsMaskAndHeader.h>
#include <Interpreters/Context.h>
#include <Interpreters/DatabaseCatalog.h>
#include <Parsers/ASTIndexDeclaration.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/formatAST.h>
#include <Parsers/queryToString.h>
#include <Processors/ISource.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/QueryPlan/SourceStepWithFilter.h>
#include <QueryPipeline/Pipe.h>
#include <QueryPipeline/QueryPipelineBuilder.h>
namespace DB
{
StorageSystemProjections::StorageSystemProjections(const StorageID & table_id_)
: IStorage(table_id_)
{
auto projection_type_datatype = std::make_shared<DataTypeEnum8>(
DataTypeEnum8::Values
{
{"Normal", static_cast<UInt8>(ProjectionDescription::Type::Normal)},
{"Aggregate", static_cast<UInt8>(ProjectionDescription::Type::Aggregate)}
}
);
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(ColumnsDescription(
{
{ "database", std::make_shared<DataTypeString>(), "Database name."},
{ "table", std::make_shared<DataTypeString>(), "Table name."},
{ "name", std::make_shared<DataTypeString>(), "Projection name."},
{ "type", std::move(projection_type_datatype), "Projection type."},
{ "sorting_key", std::make_shared<DataTypeArray>(std::make_shared<DataTypeString>()), "Projection sorting key."},
{ "query", std::make_shared<DataTypeString>(), "Projection query."},
}));
setInMemoryMetadata(storage_metadata);
}
class ProjectionsSource : public ISource
{
public:
ProjectionsSource(
std::vector<UInt8> columns_mask_,
Block header,
UInt64 max_block_size_,
ColumnPtr databases_,
ContextPtr context_)
: ISource(header)
, column_mask(std::move(columns_mask_))
, max_block_size(max_block_size_)
, databases(std::move(databases_))
, context(Context::createCopy(context_))
, database_idx(0)
{}
String getName() const override { return "Projections"; }
protected:
Chunk generate() override
{
if (database_idx >= databases->size())
return {};
MutableColumns res_columns = getPort().getHeader().cloneEmptyColumns();
const auto access = context->getAccess();
const bool check_access_for_databases = !access->isGranted(AccessType::SHOW_TABLES);
size_t rows_count = 0;
while (rows_count < max_block_size)
{
if (tables_it && !tables_it->isValid())
++database_idx;
while (database_idx < databases->size() && (!tables_it || !tables_it->isValid()))
{
database_name = databases->getDataAt(database_idx).toString();
database = DatabaseCatalog::instance().tryGetDatabase(database_name);
if (database)
break;
++database_idx;
}
if (database_idx >= databases->size())
break;
if (!tables_it || !tables_it->isValid())
tables_it = database->getTablesIterator(context);
const bool check_access_for_tables = check_access_for_databases && !access->isGranted(AccessType::SHOW_TABLES, database_name);
for (; rows_count < max_block_size && tables_it->isValid(); tables_it->next())
{
auto table_name = tables_it->name();
if (check_access_for_tables && !access->isGranted(AccessType::SHOW_TABLES, database_name, table_name))
continue;
const auto table = tables_it->table();
if (!table)
continue;
StorageMetadataPtr metadata_snapshot = table->getInMemoryMetadataPtr();
if (!metadata_snapshot)
continue;
const auto & projections = metadata_snapshot->getProjections();
for (const auto & projection : projections)
{
++rows_count;
size_t src_index = 0;
size_t res_index = 0;
// 'database' column
if (column_mask[src_index++])
res_columns[res_index++]->insert(database_name);
// 'table' column
if (column_mask[src_index++])
res_columns[res_index++]->insert(table_name);
// 'name' column
if (column_mask[src_index++])
res_columns[res_index++]->insert(projection.name);
// 'type' column
if (column_mask[src_index++])
res_columns[res_index++]->insert(projection.type);
// 'sorting_key' column
if (column_mask[src_index++])
{
auto columns = projection.metadata->getSortingKeyColumns();
Array sorting_key;
sorting_key.reserve(columns.size());
for (const auto & column : columns)
{
sorting_key.push_back(column);
}
res_columns[res_index++]->insert(sorting_key);
}
// 'query' column
if (column_mask[src_index++])
{
res_columns[res_index++]->insert(serializeAST(*projection.definition_ast->children.at(0)));
}
}
}
}
return Chunk(std::move(res_columns), rows_count);
}
private:
std::vector<UInt8> column_mask;
UInt64 max_block_size;
ColumnPtr databases;
ContextPtr context;
size_t database_idx;
DatabasePtr database;
std::string database_name;
DatabaseTablesIteratorPtr tables_it;
};
class ReadFromSystemProjections : public SourceStepWithFilter
{
public:
std::string getName() const override { return "ReadFromSystemProjections"; }
void initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &) override;
ReadFromSystemProjections(
const Names & column_names_,
const SelectQueryInfo & query_info_,
const StorageSnapshotPtr & storage_snapshot_,
const ContextPtr & context_,
Block sample_block,
std::shared_ptr<StorageSystemProjections> storage_,
std::vector<UInt8> columns_mask_,
size_t max_block_size_)
: SourceStepWithFilter(
DataStream{.header = std::move(sample_block)},
column_names_,
query_info_,
storage_snapshot_,
context_)
, storage(std::move(storage_))
, columns_mask(std::move(columns_mask_))
, max_block_size(max_block_size_)
{
}
void applyFilters(ActionDAGNodes added_filter_nodes) override;
private:
std::shared_ptr<StorageSystemProjections> storage;
std::vector<UInt8> columns_mask;
const size_t max_block_size;
ExpressionActionsPtr virtual_columns_filter;
};
void ReadFromSystemProjections::applyFilters(ActionDAGNodes added_filter_nodes)
{
SourceStepWithFilter::applyFilters(std::move(added_filter_nodes));
if (filter_actions_dag)
{
Block block_to_filter
{
{ ColumnString::create(), std::make_shared<DataTypeString>(), "database" },
};
auto dag = VirtualColumnUtils::splitFilterDagForAllowedInputs(filter_actions_dag->getOutputs().at(0), &block_to_filter);
if (dag)
virtual_columns_filter = VirtualColumnUtils::buildFilterExpression(std::move(*dag), context);
}
}
void StorageSystemProjections::read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum /* processed_stage */,
size_t max_block_size,
size_t /* num_streams */)
{
storage_snapshot->check(column_names);
Block sample_block = storage_snapshot->metadata->getSampleBlock();
auto [columns_mask, header] = getQueriedColumnsMaskAndHeader(sample_block, column_names);
auto this_ptr = std::static_pointer_cast<StorageSystemProjections>(shared_from_this());
auto reading = std::make_unique<ReadFromSystemProjections>(
column_names, query_info, storage_snapshot,
std::move(context), std::move(header), std::move(this_ptr), std::move(columns_mask), max_block_size);
query_plan.addStep(std::move(reading));
}
void ReadFromSystemProjections::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
{
MutableColumnPtr column = ColumnString::create();
const auto databases = DatabaseCatalog::instance().getDatabases();
for (const auto & [database_name, database] : databases)
{
if (database_name == DatabaseCatalog::TEMPORARY_DATABASE)
continue;
/// Lazy database can contain only very primitive tables, it cannot contain tables with projections.
/// Skip it to avoid unnecessary tables loading in the Lazy database.
if (database->getEngineName() != "Lazy")
column->insert(database_name);
}
/// Condition on "database" in a query acts like an index.
Block block { ColumnWithTypeAndName(std::move(column), std::make_shared<DataTypeString>(), "database") };
if (virtual_columns_filter)
VirtualColumnUtils::filterBlockWithExpression(virtual_columns_filter, block);
ColumnPtr & filtered_databases = block.getByPosition(0).column;
pipeline.init(Pipe(std::make_shared<ProjectionsSource>(
std::move(columns_mask), getOutputStream().header, max_block_size, std::move(filtered_databases), context)));
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Storages/IStorage.h>
namespace DB
{
/// For system.projections table - describes the projections in tables, similar to system.data_skipping_indices.
class StorageSystemProjections : public IStorage
{
public:
explicit StorageSystemProjections(const StorageID & table_id_);
std::string getName() const override { return "StorageSystemProjections"; }
void read(
QueryPlan & query_plan,
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams) override;
bool isSystemStorage() const override { return true; }
};
}

View File

@ -51,6 +51,7 @@
#include <Storages/System/StorageSystemTableEngines.h>
#include <Storages/System/StorageSystemTableFunctions.h>
#include <Storages/System/StorageSystemTables.h>
#include <Storages/System/StorageSystemProjections.h>
#include <Storages/System/StorageSystemZooKeeper.h>
#include <Storages/System/StorageSystemContributors.h>
#include <Storages/System/StorageSystemErrors.h>
@ -166,6 +167,7 @@ void attachSystemTablesServer(ContextPtr context, IDatabase & system_database, b
attach<StorageSystemErrors>(context, system_database, "errors", "Contains a list of all errors which have ever happened including the error code, last time and message with unsymbolized stacktrace.");
attach<StorageSystemWarnings>(context, system_database, "warnings", "Contains warnings about server configuration to be displayed by clickhouse-client right after it connects to the server.");
attachNoDescription<StorageSystemDataSkippingIndices>(context, system_database, "data_skipping_indices", "Contains all the information about all the data skipping indices in tables, similar to system.columns.");
attachNoDescription<StorageSystemProjections>(context, system_database, "projections", "Contains all the information about all the projections in tables, similar to system.data_skipping_indices.");
attach<StorageSystemLicenses>(context, system_database, "licenses", "Contains licenses of third-party libraries that are located in the contrib directory of ClickHouse sources.");
attach<StorageSystemTimeZones>(context, system_database, "time_zones", "Contains a list of time zones that are supported by the ClickHouse server. This list of timezones might vary depending on the version of ClickHouse.");
attach<StorageSystemBackups>(context, system_database, "backups", "Contains a list of all BACKUP or RESTORE operations with their current states and other propertis. Note, that table is not persistent and it shows only operations executed after the last server restart.");

View File

@ -30,6 +30,7 @@ import subprocess
import sys
import traceback
import urllib.parse
import io
# for crc32
import zlib
@ -39,8 +40,10 @@ from errno import ESRCH
from subprocess import PIPE, Popen
from time import sleep, time
from typing import Dict, List, Optional, Set, Tuple, Union
from contextlib import redirect_stdout
from ast import literal_eval as make_tuple
try:
import termcolor # type: ignore
except ImportError:
@ -1342,9 +1345,13 @@ class TestCase:
return None
def process_result_impl(self, proc, total_time: float):
kill_output = ""
if proc:
if proc.returncode is None:
kill_process_group(os.getpgid(proc.pid))
f = io.StringIO()
with redirect_stdout(f):
kill_process_group(os.getpgid(proc.pid))
kill_output = f.getvalue()
description = ""
@ -1360,7 +1367,7 @@ class TestCase:
with open(self.stdout_file, "rb") as stdfd:
stdout = str(stdfd.read(), errors="replace", encoding="utf-8")
stderr = ""
stderr = kill_output
if os.path.exists(self.stderr_file):
with open(self.stderr_file, "rb") as stdfd:
stderr += str(stdfd.read(), errors="replace", encoding="utf-8")

View File

@ -272,8 +272,10 @@ function check_logs_for_critical_errors()
# Remove file no_such_key_errors.txt if it's empty
[ -s /test_output/no_such_key_errors.txt ] || rm /test_output/no_such_key_errors.txt
# Crash
rg -Fa "###################""#####################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \
# Crash. This must have fewer '#'s than the command below, otherwise the command below will match
# the echo of this command (if set -x is enabled, and this script's stdout is sent
# to /test_output/run.log).
rg -Fa "#######################################" /var/log/clickhouse-server/clickhouse-server*.log > /dev/null \
&& echo -e "Killed by signal (in clickhouse-server.log)$FAIL" >> /test_output/test_results.tsv \
|| echo -e "Not crashed$OK" >> /test_output/test_results.tsv
@ -285,7 +287,7 @@ function check_logs_for_critical_errors()
# Remove file fatal_messages.txt if it's empty
[ -s /test_output/fatal_messages.txt ] || rm /test_output/fatal_messages.txt
rg -Faz "####################""####################" /test_output/* > /dev/null \
rg -Faz "########################################" /test_output/* > /dev/null \
&& echo -e "Killed by signal (output files)$FAIL" >> /test_output/test_results.tsv
function get_gdb_log_context()

View File

@ -6,6 +6,7 @@ import logging
import os
import random
import shlex
import shutil
import signal
import string
import subprocess
@ -135,6 +136,53 @@ def check_args_and_update_paths(args):
)
def check_iptables_legacy():
iptables_path = shutil.which("iptables")
ip6tables_path = shutil.which("ip6tables")
if iptables_path is None:
print("Error: 'iptables' not found in PATH")
sys.exit(1)
if ip6tables_path is None:
print("Error: 'ip6tables' not found in PATH, ignoring")
try:
file_info = os.stat(iptables_path)
file_info_str = str(file_info)
if "legacy" in file_info_str:
print(
"""
iptables on your host machine is in 'legacy' mode. This is not supported.
Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual.
Or, use --ignore-iptables-legacy-check.
"""
)
sys.exit(1)
if not ip6tables_path:
return
file_info = os.stat(ip6tables_path)
file_info_str = str(file_info)
if "legacy" in file_info_str:
print(
"""
ip6tables on your host machine is in 'legacy' mode. This is not supported.
Please switch to 'nftables' mode, usualy by installing `iptables-nft` or `nftables`, consult your distribution manual.
Or, use --ignore-iptables-legacy-check.
"""
)
sys.exit(1)
except FileNotFoundError:
print(f"Error: '{iptables_path}' not found")
sys.exit(1)
def docker_kill_handler_handler(signum, frame):
_, _ = signum, frame
subprocess.check_call(
@ -163,6 +211,7 @@ if __name__ == "__main__":
level=logging.INFO,
format="%(asctime)s [ %(process)d ] %(levelname)s : %(message)s (%(filename)s:%(lineno)s, %(funcName)s)",
)
parser = argparse.ArgumentParser(description="ClickHouse integration tests runner")
parser.add_argument(
@ -311,12 +360,24 @@ if __name__ == "__main__":
help="Bind volume to this dir to use for dockerd files",
)
parser.add_argument(
"--ignore-iptables-legacy-check",
action="store_true",
default=False,
help="Ignore iptables-legacy usage check",
)
parser.add_argument("pytest_args", nargs="*", help="args for pytest command")
args = parser.parse_args()
check_args_and_update_paths(args)
if not args.ignore_iptables_legacy_check:
check_iptables_legacy()
else:
logging.warning("Skipping iptables-legacy check")
parallel_args = ""
if args.parallel:
parallel_args += "--dist=loadfile"

View File

@ -1,40 +1,36 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance(
"node1", main_configs=["configs/distributed_servers.xml"], with_zookeeper=True
"node1", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"]
)
node2 = cluster.add_instance(
"node2", with_zookeeper=True, main_configs=["configs/distributed_servers.xml"]
)
def fill_nodes(nodes):
for node in nodes:
node.query(
"""
DROP DATABASE IF EXISTS test;
CREATE DATABASE test;
CREATE TABLE test.local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id;
CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table);
INSERT INTO test.local_table VALUES ({pos}, '{replica}');
""".format(
pos=node.name[4:], replica=node.name
)
)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster.start()
for node in (node1, node2):
node.query(
"""
CREATE DATABASE test;
CREATE TABLE test.local_table(id UInt32, val String) ENGINE = MergeTree ORDER BY id
"""
)
node1.query("INSERT INTO test.local_table VALUES (1, 'node1')")
node2.query("INSERT INTO test.local_table VALUES (2, 'node2')")
node1.query(
"CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)"
)
node2.query(
"CREATE TABLE test.distributed(id UInt32, val String) ENGINE = Distributed(test_cluster, test, local_table)"
)
yield cluster
finally:
@ -42,6 +38,8 @@ def started_cluster():
def test_truncate_database_distributed(started_cluster):
fill_nodes([node1, node2])
query1 = "SELECT count() FROM test.distributed WHERE (id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))"
query2 = "SELECT sum((id, val) IN ((1, 'node1'), (2, 'a'), (3, 'b'))) FROM test.distributed"
assert node1.query(query1) == "1\n"

View File

@ -1,5 +1,3 @@
import time
import pytest
from helpers.cluster import ClickHouseCluster
@ -8,8 +6,9 @@ def fill_nodes(nodes, shard):
for node in nodes:
node.query(
"""
DROP DATABASE IF EXISTS test;
CREATE DATABASE test;
CREATE TABLE test.test_table(date Date, id UInt32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/test/{shard}/replicated/test_table', '{replica}') ORDER BY id PARTITION BY toYYYYMM(date);
""".format(
@ -35,8 +34,6 @@ def start_cluster():
try:
cluster.start()
fill_nodes([node1, node2, node3], 1)
yield cluster
except Exception as ex:
@ -47,10 +44,13 @@ def start_cluster():
def test_truncate_database_replicated(start_cluster):
fill_nodes([node1, node2, node3], 1)
node1.query(
"INSERT INTO test.test_table SELECT number, toString(number) FROM numbers(100)"
)
assert node2.query("SELECT id FROM test.test_table LIMIT 1") == "0\n"
assert node2.query("SELECT min(id) FROM test.test_table") == "0\n"
assert node2.query("SELECT id FROM test.test_table ORDER BY id LIMIT 1") == "0\n"
assert node3.query("SHOW DATABASES LIKE 'test'") == "test\n"
node3.query("TRUNCATE DATABASE test ON CLUSTER test_cluster SYNC")
assert node2.query("SHOW TABLES FROM test") == ""

View File

@ -1,26 +1,26 @@
no monotonic int case: String -> UInt64
no monotonic int case: FixedString -> UInt64
monotonic int case: Int32 -> Int64
monotonic int case: Int32 -> UInt64
monotonic int case: Int32 -> Int32
monotonic int case: Int32 -> UInt32
monotonic int case: Int32 -> Int16
monotonic int case: Int32 -> UInt16
monotonic int case: UInt32 -> Int64
monotonic int case: UInt32 -> UInt64
monotonic int case: UInt32 -> Int32
monotonic int case: UInt32 -> UInt32
monotonic int case: UInt32 -> Int16
monotonic int case: UInt32 -> UInt16
monotonic int case: Enum16 -> Int32
monotonic int case: Enum16 -> UInt32
monotonic int case: Enum16 -> Int16
monotonic int case: Enum16 -> UInt16
monotonic int case: Enum16 -> Int8
monotonic int case: Enum16 -> UInt8
monotonic int case: Date -> Int32
monotonic int case: Date -> UInt32
monotonic int case: Date -> Int16
monotonic int case: Date -> UInt16
monotonic int case: Date -> Int8
monotonic int case: Date -> UInt8
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK
OK

View File

@ -47,38 +47,42 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO date_test_table VALUES (1), (2), (2),
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=debug/g')
CLICKHOUSE_CLIENT="${CLICKHOUSE_CLIENT} --optimize_use_implicit_projections 0"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: String -> UInt64"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "no monotonic int case: FixedString -> UInt64"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM fixed_string_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "3 marks to read from 1 ranges" && echo "OK" &
wait
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int64"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt64"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> Int32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Int32 -> UInt32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> Int16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Int32 -> UInt16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM signed_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "5 marks to read from" && echo "OK" &
wait
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> Int64"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt64"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: UInt32 -> Int32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: UInt32 -> UInt32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> Int16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "monotonic int case: UInt32 -> UInt16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt64(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt32(val) == 0;" 2>&1 |grep -q "2 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt32(val) == 0;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM unsigned_integer_test_table WHERE toUInt16(val) == 0;" 2>&1 |grep -q "4 marks to read from" && echo "OK" &
wait
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> Int16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Enum16 -> UInt16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> Int8"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Enum16 -> UInt8"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM enum_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" &
wait
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> Int32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt32"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "monotonic int case: Date -> Int16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "monotonic int case: Date -> UInt16"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> Int8"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "monotonic int case: Date -> UInt8"
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt32(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt16(val) == 1;" 2>&1 |grep -q "2 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt16(val) == 1;" 2>&1 |grep -q "1 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" &
${CLICKHOUSE_CLIENT} --query="SELECT count() FROM date_test_table WHERE toUInt8(val) == 1;" 2>&1 |grep -q "5 marks to read from" && echo "OK" &
wait
CLICKHOUSE_CLIENT=$(echo ${CLICKHOUSE_CLIENT} | sed 's/--send_logs_level=debug/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/g')

View File

@ -8,58 +8,64 @@ CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS contributors"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE contributors (name String) ENGINE = Memory"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS contributors;
CREATE TABLE contributors (name String) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.contributors ORDER BY name DESC FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO contributors FORMAT Parquet"
# random results
${CLICKHOUSE_CLIENT} --query="SELECT * FROM contributors LIMIT 10" > /dev/null
${CLICKHOUSE_CLIENT} --query="DROP TABLE contributors"
${CLICKHOUSE_CLIENT} -n --query="
-- random results
SELECT * FROM contributors LIMIT 10 FORMAT Null;
DROP TABLE contributors;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_numbers"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory"
DROP TABLE IF EXISTS parquet_numbers;
CREATE TABLE parquet_numbers (number UInt64) ENGINE = Memory;"
# less than default block size (65k)
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 10000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
TRUNCATE TABLE parquet_numbers;"
# More than default block size
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.numbers LIMIT 100000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
TRUNCATE TABLE parquet_numbers;"
${CLICKHOUSE_CLIENT} --max_block_size=2 --query="SELECT * FROM system.numbers LIMIT 3 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_numbers"
TRUNCATE TABLE parquet_numbers;"
${CLICKHOUSE_CLIENT} --max_block_size=1 --query="SELECT * FROM system.numbers LIMIT 1000 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_numbers FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_numbers ORDER BY number DESC LIMIT 10;
DROP TABLE parquet_numbers;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_numbers"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_events"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory"
DROP TABLE IF EXISTS parquet_events;
CREATE TABLE parquet_events (event String, value UInt64, description String) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM system.events FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_events FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_events"
${CLICKHOUSE_CLIENT} -n --query="
SELECT event, description FROM parquet_events WHERE event IN ('ContextLock', 'Query') ORDER BY event;
DROP TABLE parquet_events;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types1"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types2"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types3"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types4"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
# convert min type
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory"
# convert max type
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory"
DROP TABLE IF EXISTS parquet_types1;
DROP TABLE IF EXISTS parquet_types2;
DROP TABLE IF EXISTS parquet_types3;
DROP TABLE IF EXISTS parquet_types4;
CREATE TABLE parquet_types1 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory;
CREATE TABLE parquet_types2 (int8 Int8, uint8 UInt8, int16 Int16, uint16 UInt16, int32 Int32, uint32 UInt32, int64 Int64, uint64 UInt64, float32 Float32, float64 Float64, string String, fixedstring FixedString(15), date Date, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory;
-- convert min type
CREATE TABLE parquet_types3 (int8 Int8, uint8 Int8, int16 Int8, uint16 Int8, int32 Int8, uint32 Int8, int64 Int8, uint64 Int8, float32 Int8, float64 Int8, string FixedString(15), fixedstring FixedString(15), date Date, datetime Date, datetime64 DateTime64(9)) ENGINE = Memory;
-- convert max type
CREATE TABLE parquet_types4 (int8 Int64, uint8 Int64, int16 Int64, uint16 Int64, int32 Int64, uint32 Int64, int64 Int64, uint64 Int64, float32 Int64, float64 Int64, string String, fixedstring String, date DateTime, datetime DateTime, datetime64 DateTime64(9)) ENGINE = Memory;
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9))"
INSERT INTO parquet_types1 values ( -108, 108, -1016, 1116, -1032, 1132, -1064, 1164, -1.032, -1.064, 'string-0', 'fixedstring', '2001-02-03', '2002-02-03 04:05:06', toDateTime64('2002-02-03 04:05:06.789', 9));
# min
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9))"
-- min
INSERT INTO parquet_types1 values ( -128, 0, -32768, 0, -2147483648, 0, -9223372036854775808, 0, -1.032, -1.064, 'string-1', 'fixedstring-1', '2003-04-05', '2003-02-03 04:05:06', toDateTime64('2003-02-03 04:05:06.789', 9));
# max
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9))"
-- max
INSERT INTO parquet_types1 values ( 127, 255, 32767, 65535, 2147483647, 4294967295, 9223372036854775807, 9223372036854775807, -1.032, -1.064, 'string-2', 'fixedstring-2', '2004-06-07', '2004-02-03 04:05:06', toDateTime64('2004-02-03 04:05:06.789', 9));"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
@ -72,8 +78,9 @@ ${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8 FORMAT
echo diff:
diff "${CLICKHOUSE_TMP}"/parquet_all_types_1.dump "${CLICKHOUSE_TMP}"/parquet_all_types_2.dump
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9))"
${CLICKHOUSE_CLIENT} -n --query="
TRUNCATE TABLE parquet_types2;
INSERT INTO parquet_types3 values ( 79, 81, 82, 83, 84, 85, 86, 87, 88, 89, 'str01', 'fstr1', '2003-03-04', '2004-05-06', toDateTime64('2004-05-06 07:08:09.012', 9));"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types3 FORMAT Parquet"
@ -81,70 +88,69 @@ ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 values ( 80,
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types4 FORMAT Parquet"
echo dest:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types2 ORDER BY int8"
echo min:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types3 ORDER BY int8"
echo max:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types4 ORDER BY int8"
${CLICKHOUSE_CLIENT} -n --query="
SELECT 'dest:';
SELECT * FROM parquet_types2 ORDER BY int8;
SELECT 'min:';
SELECT * FROM parquet_types3 ORDER BY int8;
SELECT 'max:';
SELECT * FROM parquet_types4 ORDER BY int8;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types5"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_types6"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE parquet_types2"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL)"
DROP TABLE IF EXISTS parquet_types5;
DROP TABLE IF EXISTS parquet_types6;
TRUNCATE TABLE parquet_types2;
CREATE TABLE parquet_types5 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory;
CREATE TABLE parquet_types6 (int8 Nullable(Int8), uint8 Nullable(UInt8), int16 Nullable(Int16), uint16 Nullable(UInt16), int32 Nullable(Int32), uint32 Nullable(UInt32), int64 Nullable(Int64), uint64 Nullable(UInt64), float32 Nullable(Float32), float64 Nullable(Float64), string Nullable(String), fixedstring Nullable(FixedString(15)), date Nullable(Date), datetime Nullable(DateTime), datetime64 Nullable(DateTime64(9))) ENGINE = Memory;
INSERT INTO parquet_types5 values ( NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL, NULL);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" > "${CLICKHOUSE_TMP}"/parquet_all_types_5.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types5 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types1 ORDER BY int8 FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_types6 FORMAT Parquet"
echo dest from null:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_types6 ORDER BY int8"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_types6 ORDER BY int8;
DROP TABLE parquet_types5;
DROP TABLE parquet_types6;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types5"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types6"
DROP TABLE parquet_types1;
DROP TABLE parquet_types2;
DROP TABLE parquet_types3;
DROP TABLE parquet_types4;
DROP TABLE IF EXISTS parquet_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types1"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types2"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types3"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_types4"
CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory();
INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001]);
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_arrays (id UInt32, a1 Array(Int8), a2 Array(UInt8), a3 Array(Int16), a4 Array(UInt16), a5 Array(Int32), a6 Array(UInt32), a7 Array(Int64), a8 Array(UInt64), a9 Array(String), a10 Array(FixedString(4)), a11 Array(Float32), a12 Array(Float64), a13 Array(Date), a14 Array(DateTime), a15 Array(Decimal(4, 2)), a16 Array(Decimal(10, 2)), a17 Array(Decimal(25, 2))) engine=Memory()"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (1, [1,-2,3], [1,2,3], [100, -200, 300], [100, 200, 300], [10000000, -20000000, 30000000], [10000000, 2000000, 3000000], [100000000000000, -200000000000, 3000000000000], [100000000000000, 20000000000000, 3000000000000], ['Some string', 'Some string', 'Some string'], ['0000', '1111', '2222'], [42.42, 424.2, 0.4242], [424242.424242, 4242042420.242424, 42], ['2000-01-01', '2001-01-01', '2002-01-01'], ['2000-01-01', '2001-01-01', '2002-01-01'], [0.2, 10.003, 4.002], [4.000000001, 10000.10000, 10000.100001], [1000000000.000000001123, 90.0000000010010101, 0101001.0112341001])"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [])"
INSERT INTO parquet_arrays VALUES (2, [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], [], []);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_arrays FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_arrays ORDER BY id"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_arrays ORDER BY id;
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_arrays"
DROP TABLE parquet_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nullable_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory()"
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], [])"
DROP TABLE IF EXISTS parquet_nullable_arrays;
CREATE TABLE parquet_nullable_arrays (id UInt32, a1 Array(Nullable(UInt32)), a2 Array(Nullable(String)), a3 Array(Nullable(Decimal(4, 2)))) engine=Memory();
INSERT INTO parquet_nullable_arrays VALUES (1, [1, Null, 2], [Null, 'Some string', Null], [0.001, Null, 42.42]), (2, [Null], [Null], [Null]), (3, [], [], []);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nullable_arrays FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nullable_arrays ORDER BY id"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nullable_arrays"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_nullable_arrays ORDER BY id;
DROP TABLE parquet_nullable_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_nested_arrays"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory() "
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []])"
DROP TABLE IF EXISTS parquet_nested_arrays;
CREATE TABLE parquet_nested_arrays (a1 Array(Array(Array(UInt32))), a2 Array(Array(Array(String))), a3 Array(Array(Nullable(UInt32))), a4 Array(Array(Nullable(String)))) engine=Memory();
INSERT INTO parquet_nested_arrays VALUES ([[[1,2,3], [1,2,3]], [[1,2,3]], [[], [1,2,3]]], [[['Some string', 'Some string'], []], [['Some string']], [[]]], [[Null, 1, 2], [Null], [1, 2], []], [['Some string', Null, 'Some string'], [Null], []]);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays FORMAT Parquet" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_nested_arrays FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_nested_arrays"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_nested_arrays"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_nested_arrays;
DROP TABLE parquet_nested_arrays;
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_decimal"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123)"
DROP TABLE IF EXISTS parquet_decimal;
CREATE TABLE parquet_decimal (d1 Decimal32(4), d2 Decimal64(8), d3 Decimal128(16), d4 Decimal256(32)) ENGINE = Memory;
INSERT INTO TABLE parquet_decimal VALUES (0.123, 0.123123123, 0.123123123123, 0.123123123123123123);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal FORMAT Arrow" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_decimal FORMAT Arrow"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_decimal"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_decimal"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_decimal;
DROP TABLE parquet_decimal;"

View File

@ -6,108 +6,115 @@
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
# Simple small values
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);"
#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1);"
#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10);"
#${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (1, 1, 1);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
-- Simple small values
CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
-- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (1, 1, 1, 0.1, 0.1, 1, 1, 1, 1, 1);
-- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (10, 10, 10, 0.1, 0.1, 0.1, 10, 10, 10, 10);
-- INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-100, -100, -100, -0.1, -0.1, -0.1, -100, -100, -100, -100);
INSERT INTO decimal (a, b, c) VALUES (1, 1, 1);
INSERT INTO decimal (a, b, c) VALUES (10, 10, 10);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal0.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump
echo diff0:
diff "${CLICKHOUSE_TMP}"/parquet_decimal0_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal0_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (0.000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (i) VALUES (-0.000000000000000001);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');"
CREATE TABLE IF NOT EXISTS decimal ( a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e DECIMAL(18, 18), f DECIMAL(38, 38), g Decimal(9, 5), h decimal(18, 9), i deciMAL(38, 18), j DECIMAL(1,0)) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;
INSERT INTO decimal (a, b, d, g) VALUES (999999999, 999999999999999999, 0.999999999, 9999.99999);
INSERT INTO decimal (a, b, d, g) VALUES (-999999999, -999999999999999999, -0.999999999, -9999.99999);
INSERT INTO decimal (c) VALUES (99999999999999999999999999999999999999);
INSERT INTO decimal (c) VALUES (-99999999999999999999999999999999999999);
INSERT INTO decimal (f) VALUES (0.99999999999999999999999999999999999999);
INSERT INTO decimal (f) VALUES (-0.99999999999999999999999999999999999999);
INSERT INTO decimal (e, h) VALUES (0.999999999999999999, 999999999.999999999);
INSERT INTO decimal (e, h) VALUES (-0.999999999999999999, -999999999.999999999);
INSERT INTO decimal (i) VALUES (99999999999999999999.999999999999999999);
INSERT INTO decimal (i) VALUES (-99999999999999999999.999999999999999999);
INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (1, 1, 1, 0.000000001, 0.00001, 1, 0.000000001);
INSERT INTO decimal (a, b, c, d, g, j, h) VALUES (-1, -1, -1, -0.000000001, -0.00001, -1, -0.000000001);
INSERT INTO decimal (e, f) VALUES (0.000000000000000001, 0.00000000000000000000000000000000000001);
INSERT INTO decimal (e, f) VALUES (-0.000000000000000001, -0.00000000000000000000000000000000000001);
INSERT INTO decimal (i) VALUES (0.000000000000000001);
INSERT INTO decimal (i) VALUES (-0.000000000000000001);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0, 0, 0, 0, 0, 0, 0, 0, 0, 0);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0, -0, -0, -0, -0, -0, -0, -0, -0, -0);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0, 0.0);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0, -0.0);
INSERT INTO decimal (a, b, g) VALUES ('42.00000', 42.0000000000000000000000000000000, '0.999990');
INSERT INTO decimal (a, b, c, d, e, f) VALUES ('0.9e9', '0.9e18', '0.9e38', '9e-9', '9e-18', '9e-38');
INSERT INTO decimal (a, b, c, d, e, f) VALUES ('-0.9e9', '-0.9e18', '-0.9e38', '-9e-9', '-9e-18', '-9e-38');"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal1.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump
echo diff1:
diff "${CLICKHOUSE_TMP}"/parquet_decimal1_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal1_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42);"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);"
CREATE TABLE IF NOT EXISTS decimal (a DECIMAL(9,0), b DECIMAL(18,0), c DECIMAL(38,0), d DECIMAL(9, 9), e Decimal64(18), f Decimal128(38), g Decimal32(5), h Decimal64(9), i Decimal128(18), j dec(4,2)) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (42, 42, 42, 0.42, 0.42, 0.42, 42.42, 42.42, 42.42, 42.42);
INSERT INTO decimal (a, b, c, d, e, f, g, h, i, j) VALUES (-42, -42, -42, -0.42, -0.42, -0.42, -42.42, -42.42, -42.42, -42.42);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal2.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d, e, f, g, h, i, j FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d, e, f, g, h, i, j;" > "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump
echo diff2:
diff "${CLICKHOUSE_TMP}"/parquet_decimal2_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal2_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory;"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
CREATE TABLE IF NOT EXISTS decimal (a Nullable(DECIMAL(9,0)), b Nullable(DECIMAL(18,0)), c Nullable(DECIMAL(38,0)), d Nullable(DECIMAL(9,0))) ENGINE = Memory;
CREATE TABLE IF NOT EXISTS decimal2 AS decimal ENGINE = Memory;"
# Empty table test
# throws No data to insert
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet" 2> /dev/null
echo nothing:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM decimal2 ORDER BY a, b, c, d;
TRUNCATE TABLE decimal2;
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, Null, Null)"
INSERT INTO decimal VALUES (Null, Null, Null, Null);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
echo nulls:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} --query="TRUNCATE TABLE decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM decimal2 ORDER BY a, b, c, d;
TRUNCATE TABLE decimal2;
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (1, Null, Null, Null)"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, 1, Null, Null)"
${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal VALUES (Null, Null, 1, Null)"
INSERT INTO decimal VALUES (1, Null, Null, Null);
INSERT INTO decimal VALUES (Null, 1, Null, Null);
INSERT INTO decimal VALUES (Null, Null, 1, Null);"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_3.parquet
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d FORMAT Parquet;" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO decimal2 FORMAT Parquet"
echo full orig:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal ORDER BY a, b, c, d;"
echo full inserted:
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} -n --query="
SELECT 'full orig:';
SELECT * FROM decimal ORDER BY a, b, c, d;
SELECT 'full inserted:';
SELECT * FROM decimal2 ORDER BY a, b, c, d;"
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump
${CLICKHOUSE_CLIENT} --query="SELECT * FROM decimal2 ORDER BY a, b, c, d;" > "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump
echo diff3:
diff "${CLICKHOUSE_TMP}"/parquet_decimal3_1.dump "${CLICKHOUSE_TMP}"/parquet_decimal3_2.dump
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal;"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS decimal2;"
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS decimal;
DROP TABLE IF EXISTS decimal2;"

View File

@ -57,14 +57,14 @@ for NAME in $(find "$DATA_DIR"/*.parquet -print0 | xargs -0 -n 1 basename | LC_A
# COLUMNS=`$CUR_DIR/00900_parquet_create_table_columns.py $JSON` 2>&1 || continue
COLUMNS=$(cat "$COLUMNS_FILE") || continue
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS parquet_load"
$CLICKHOUSE_CLIENT <<EOF
CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory;
EOF
${CLICKHOUSE_CLIENT} -n --query="
DROP TABLE IF EXISTS parquet_load;
CREATE TABLE parquet_load ($COLUMNS) ENGINE = Memory;"
# Some files contain unsupported data structures, exception is ok.
${CLICKHOUSE_CLIENT} --query="INSERT INTO parquet_load FORMAT Parquet" < "$DATA_DIR"/"$NAME" 2>&1 | sed 's/Exception/Ex---tion/'
${CLICKHOUSE_CLIENT} --query="SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100"
${CLICKHOUSE_CLIENT} --query="DROP TABLE parquet_load"
${CLICKHOUSE_CLIENT} -n --query="
SELECT * FROM parquet_load ORDER BY tuple(*) LIMIT 100;
DROP TABLE parquet_load;"
done

View File

@ -1,17 +1,17 @@
1
CREATE DATABASE test_01114_1\nENGINE = Atomic
CREATE DATABASE test_01114_2\nENGINE = Atomic
CREATE DATABASE test_01114_3\nENGINE = Ordinary
test_01114_1 Atomic store 00001114-1000-4000-8000-000000000001 1
test_01114_2 Atomic store 00001114-1000-4000-8000-000000000002 1
test_01114_3 Ordinary test_01114_3 test_01114_3 1
CREATE DATABASE default_1\nENGINE = Atomic
CREATE DATABASE default_2\nENGINE = Atomic
CREATE DATABASE default_3\nENGINE = Ordinary
default_1 Atomic store 00001114-1000-4000-8000-000000000001 1
default_2 Atomic store 00001114-1000-4000-8000-000000000002 1
default_3 Ordinary default_3 default_3 1
110
100
CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
mt 00001114-0000-4000-8000-000000000002 CREATE TABLE test_01114_2.mt (`n` UInt64) ENGINE = MergeTree PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192
CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
mt 00001114-0000-4000-8000-000000000002 CREATE TABLE default_2.mt (`n` UInt64) ENGINE = MergeTree PARTITION BY n % 5 ORDER BY tuple() SETTINGS index_granularity = 8192
110
CREATE TABLE test_01114_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE test_01114_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default_1.mt UUID \'00001114-0000-4000-8000-000000000001\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
CREATE TABLE default_2.mt UUID \'00001114-0000-4000-8000-000000000002\'\n(\n `n` UInt64\n)\nENGINE = MergeTree\nPARTITION BY n % 5\nORDER BY tuple()\nSETTINGS index_granularity = 8192
5
dropped
110 5995

View File

@ -9,81 +9,98 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
DATABASE_1="${CLICKHOUSE_DATABASE}_1"
DATABASE_2="${CLICKHOUSE_DATABASE}_2"
DATABASE_3="${CLICKHOUSE_DATABASE}_3"
$CLICKHOUSE_CLIENT -nm -q "
DROP DATABASE IF EXISTS test_01114_1;
DROP DATABASE IF EXISTS test_01114_2;
DROP DATABASE IF EXISTS test_01114_3;
"
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=0 -q "CREATE DATABASE ${DATABASE_1} ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE"
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=0 -q "CREATE DATABASE test_01114_1 ENGINE=Ordinary" 2>&1| grep -Fac "UNKNOWN_DATABASE_ENGINE"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${DATABASE_1} ENGINE=Atomic"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE ${DATABASE_2}"
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE ${DATABASE_3} ENGINE=Ordinary"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_1 ENGINE=Atomic"
$CLICKHOUSE_CLIENT -q "CREATE DATABASE test_01114_2"
$CLICKHOUSE_CLIENT --allow_deprecated_database_ordinary=1 -q "CREATE DATABASE test_01114_3 ENGINE=Ordinary"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE ${DATABASE_1}"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE ${DATABASE_2}"
$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE ${DATABASE_3}"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_1"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=0 -q "SHOW CREATE DATABASE test_01114_2"
$CLICKHOUSE_CLIENT -q "SHOW CREATE DATABASE test_01114_3"
uuid_db_1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='test_01114_1'"`
uuid_db_2=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='test_01114_2'"`
uuid_db_1=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='${DATABASE_1}'"`
uuid_db_2=`$CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.databases WHERE name='${DATABASE_2}'"`
$CLICKHOUSE_CLIENT -q "SELECT name,
engine,
splitByChar('/', data_path)[-2],
splitByChar('/', metadata_path)[-2] as uuid_path, ((splitByChar('/', metadata_path)[-3] as metadata) = substr(uuid_path, 1, 3)) OR metadata='metadata'
FROM system.databases WHERE name LIKE 'test_01114_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g"
FROM system.databases WHERE name LIKE '${CLICKHOUSE_DATABASE}_%'" | sed "s/$uuid_db_1/00001114-1000-4000-8000-000000000001/g" | sed "s/$uuid_db_2/00001114-1000-4000-8000-000000000002/g"
$CLICKHOUSE_CLIENT -nm -q "
CREATE TABLE test_01114_1.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple();
INSERT INTO test_01114_1.mt_tmp SELECT * FROM numbers(100);
CREATE TABLE test_01114_3.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5);
INSERT INTO test_01114_3.mt SELECT * FROM numbers(110);
CREATE TABLE ${DATABASE_1}.mt_tmp (n UInt64) ENGINE=MergeTree() ORDER BY tuple();
INSERT INTO ${DATABASE_1}.mt_tmp SELECT * FROM numbers(100);
CREATE TABLE ${DATABASE_3}.mt (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5);
INSERT INTO ${DATABASE_3}.mt SELECT * FROM numbers(110);
RENAME TABLE test_01114_1.mt_tmp TO test_01114_3.mt_tmp; /* move from Atomic to Ordinary */
RENAME TABLE test_01114_3.mt TO test_01114_1.mt; /* move from Ordinary to Atomic */
SELECT count() FROM test_01114_1.mt;
SELECT count() FROM test_01114_3.mt_tmp;
RENAME TABLE ${DATABASE_1}.mt_tmp TO ${DATABASE_3}.mt_tmp; /* move from Atomic to Ordinary */
RENAME TABLE ${DATABASE_3}.mt TO ${DATABASE_1}.mt; /* move from Ordinary to Atomic */
SELECT count() FROM ${DATABASE_1}.mt;
SELECT count() FROM ${DATABASE_3}.mt_tmp;
DROP DATABASE test_01114_3;
DROP DATABASE ${DATABASE_3};
"
explicit_uuid=$($CLICKHOUSE_CLIENT -q "SELECT generateUUIDv4()")
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_01114_2.mt UUID '$explicit_uuid' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g"
$CLICKHOUSE_CLIENT -q "SELECT name, uuid, create_table_query FROM system.tables WHERE database='test_01114_2'" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g"
$CLICKHOUSE_CLIENT -q "CREATE TABLE ${DATABASE_2}.mt UUID '$explicit_uuid' (n UInt64) ENGINE=MergeTree() ORDER BY tuple() PARTITION BY (n % 5)"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_2}.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g"
$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 --function_sleep_max_microseconds_per_block 60000000 -q "SELECT count(col), sum(col) FROM (SELECT n + sleepEachRow(1.5) AS col FROM test_01114_1.mt)" & # 33s (1.5s * 22 rows per partition), result: 110, 5995
$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "INSERT INTO test_01114_2.mt SELECT number + sleepEachRow(1.5) FROM numbers(30)" & # 45s (1.5s * 30 rows)
sleep 1 # SELECT and INSERT should start before the following RENAMEs
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
it=$((it+1))
if [ $it -ge 50 ];
then
echo "Failed to wait for first batch of queries"
$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_COMMENT}%'"
fi
sleep 0.1
done
$CLICKHOUSE_CLIENT -nm -q "
RENAME TABLE test_01114_1.mt TO test_01114_1.mt_tmp;
RENAME TABLE test_01114_1.mt_tmp TO test_01114_2.mt_tmp;
EXCHANGE TABLES test_01114_2.mt AND test_01114_2.mt_tmp;
RENAME TABLE test_01114_2.mt_tmp TO test_01114_1.mt;
EXCHANGE TABLES test_01114_1.mt AND test_01114_2.mt;
RENAME TABLE ${DATABASE_1}.mt TO ${DATABASE_1}.mt_tmp;
RENAME TABLE ${DATABASE_1}.mt_tmp TO ${DATABASE_2}.mt_tmp;
EXCHANGE TABLES ${DATABASE_2}.mt AND ${DATABASE_2}.mt_tmp;
RENAME TABLE ${DATABASE_2}.mt_tmp TO ${DATABASE_1}.mt;
EXCHANGE TABLES ${DATABASE_1}.mt AND ${DATABASE_2}.mt;
"
# Check that nothing changed
$CLICKHOUSE_CLIENT -q "SELECT count() FROM test_01114_1.mt"
uuid_mt1=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='test_01114_1' AND name='mt'")
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_1.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE test_01114_2.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g"
$CLICKHOUSE_CLIENT -q "SELECT count() FROM ${DATABASE_1}.mt"
uuid_mt1=$($CLICKHOUSE_CLIENT -q "SELECT uuid FROM system.tables WHERE database='${DATABASE_1}' AND name='mt'")
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_1}.mt" | sed "s/$uuid_mt1/00001114-0000-4000-8000-000000000001/g"
$CLICKHOUSE_CLIENT --show_table_uuid_in_table_create_query_if_not_nil=1 -q "SHOW CREATE TABLE ${DATABASE_2}.mt" | sed "s/$explicit_uuid/00001114-0000-4000-8000-000000000002/g"
$CLICKHOUSE_CLIENT -nm -q "
DROP TABLE test_01114_1.mt SETTINGS database_atomic_wait_for_drop_and_detach_synchronously=0;
CREATE TABLE test_01114_1.mt (s String) ENGINE=Log();
INSERT INTO test_01114_1.mt SELECT 's' || toString(number) FROM numbers(5);
SELECT count() FROM test_01114_1.mt
DROP TABLE ${DATABASE_1}.mt SETTINGS database_atomic_wait_for_drop_and_detach_synchronously=0;
CREATE TABLE ${DATABASE_1}.mt (s String) ENGINE=Log();
INSERT INTO ${DATABASE_1}.mt SELECT 's' || toString(number) FROM numbers(5);
SELECT count() FROM ${DATABASE_1}.mt
" # result: 5
$CLICKHOUSE_CLIENT --function_sleep_max_microseconds_per_block 60000000 -q "SELECT tuple(s, sleepEachRow(3)) FROM test_01114_1.mt" > /dev/null & # 15s (3s * 5 rows)
sleep 1
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_1" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped"
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)
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))
if [ $it -ge 50 ];
then
echo "Failed to wait for second batch of queries"
$CLICKHOUSE_CLIENT -q "SELECT count() FROM system.processes WHERE query_id != queryID() AND current_database = currentDatabase() AND query LIKE '%-- ${RANDOM_TUPLE}%'"
fi
sleep 0.1
done
$CLICKHOUSE_CLIENT -q "DROP DATABASE ${DATABASE_1}" --database_atomic_wait_for_drop_and_detach_synchronously=0 && echo "dropped"
wait # for INSERT and SELECT
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM test_01114_2.mt" # result: 30, 435
$CLICKHOUSE_CLIENT -q "DROP DATABASE test_01114_2" --database_atomic_wait_for_drop_and_detach_synchronously=0
$CLICKHOUSE_CLIENT -q "SELECT count(n), sum(n) FROM ${DATABASE_2}.mt" # result: 30, 435
$CLICKHOUSE_CLIENT -q "DROP DATABASE ${DATABASE_2}" --database_atomic_wait_for_drop_and_detach_synchronously=0

View File

@ -1,2 +1,2 @@
EFBFBD
EFBFBD
FF
FF

View File

@ -5,9 +5,9 @@ insert into utf8_overlap values ('\xe2'), ('Foo⚊BarBazBam'), ('\xe2'), ('Foo
-- MONOGRAM FOR YANG
with lowerUTF8(str) as l_, upperUTF8(str) as u_, '0x' || hex(str) as h_
select length(str), if(l_ == '\xe2', h_, l_), if(u_ == '\xe2', h_, u_) from utf8_overlap format CSV;
1,"<EFBFBD>","<22>"
1,"0xE2","0xE2"
15,"foo⚊barbazbam","FOO⚊BARBAZBAM"
1,"<EFBFBD>","<22>"
1,"0xE2","0xE2"
15,"foo⚊barbazbam","FOO⚊BARBAZBAM"
-- NOTE: regression test for introduced bug
-- https://github.com/ClickHouse/ClickHouse/issues/42756

View File

@ -11,13 +11,21 @@ $CLICKHOUSE_CLIENT -q "CREATE TABLE t_index_hypothesis (a UInt32, b UInt32, INDE
$CLICKHOUSE_CLIENT -q "INSERT INTO t_index_hypothesis SELECT number, number + 1 FROM numbers(10000000)"
for _ in {0..30}; do
run_query() {
output=`$CLICKHOUSE_CLIENT -q "SELECT count() FROM t_index_hypothesis WHERE a = b"`
if [[ $output != "0" ]]; then
echo "output: $output, expected: 0"
exit 1
fi
done
}
export -f run_query
parallel -j 8 run_query ::: {0..30}
if [ $? -ne 0 ]; then
echo FAILED
exit 1
fi
echo OK

View File

@ -11,3 +11,215 @@
0 6 4
1 10 4
2 14 4
-- { echo On }
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
1 ['.']
2 ['.','.']
2 ['.','.','.']
2 ['.','.','.','.']
2 ['.','.','.','.','.']
2 ['.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.','.']
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
1 ['.']
1 ['.']
2 ['.','.']
2 ['.','.']
2 ['.','.','.']
2 ['.','.','.']
2 ['.','.','.','.']
2 ['.','.','.','.']
2 ['.','.','.','.','.']
2 ['.','.','.','.','.']
2 ['.','.','.','.','.','.']
2 ['.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.','.']
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
1 1
2 2
2 3
2 4
2 5
2 6
2 7
2 8
2 9
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
1 1
1 1
2 2
2 2
2 3
2 3
2 4
2 4
2 5
2 5
2 6
2 6
2 7
2 7
2 8
2 8
2 9
2 9
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
1 1
1 1
2 2
2 2
2 3
2 3
2 4
2 4
2 5
2 5
2 6
2 6
2 7
2 7
2 8
2 8
2 9
2 9
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
1 1
1 1
1 1
2 2
2 2
2 2
2 3
2 3
2 3
2 4
2 4
2 4
2 5
2 5
2 5
2 6
2 6
2 6
2 7
2 7
2 7
2 8
2 8
2 8
2 9
2 9
2 9
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
2 ['.']
2 ['.','.']
2 ['.','.','.']
2 ['.','.','.','.']
2 ['.','.','.','.','.']
2 ['.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.','.']
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
2 ['.']
2 ['.']
2 ['.','.']
2 ['.','.']
2 ['.','.','.']
2 ['.','.','.']
2 ['.','.','.','.']
2 ['.','.','.','.']
2 ['.','.','.','.','.']
2 ['.','.','.','.','.']
2 ['.','.','.','.','.','.']
2 ['.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.','.']
2 ['.','.','.','.','.','.','.','.','.']
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
2 1
2 2
2 3
2 4
2 5
2 6
2 7
2 8
2 9
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
2 1
2 1
2 2
2 2
2 3
2 3
2 4
2 4
2 5
2 5
2 6
2 6
2 7
2 7
2 8
2 8
2 9
2 9
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
2 1
2 1
2 2
2 2
2 3
2 3
2 4
2 4
2 5
2 5
2 6
2 6
2 7
2 7
2 8
2 8
2 9
2 9
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
2 1
2 1
2 1
2 2
2 2
2 2
2 3
2 3
2 3
2 4
2 4
2 4
2 5
2 5
2 5
2 6
2 6
2 6
2 7
2 7
2 7
2 8
2 8
2 8
2 9
2 9
2 9

View File

@ -43,3 +43,23 @@ GROUP BY
ORDER BY
sum_value ASC,
count_value ASC;
set prefer_localhost_replica = 1;
-- { echo On }
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{1,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), arrayMap(x -> '.', range(number % 10)) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (k, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;
SELECT count(), toString(number) AS k FROM remote('127.0.0.{3,2}', numbers(10)) where number > ( queryID() = initialQueryID()) GROUP BY GROUPING SETS ((k), (number + 1, k), (k, number + 2)) ORDER BY k settings group_by_two_level_threshold=9, max_bytes_before_external_group_by=10000000000;

View File

@ -56,6 +56,24 @@ Expression (Projection)
1 [2,3.2] 2.3323807824711897
2 [4.2,3.4] 4.427188573446585
0 [4.6,2.3] 4.609772130377966
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_f64)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 2/4
1 [2,3.2] 2.3323807824711897
2 [4.2,3.4] 4.427188573446585
0 [4.6,2.3] 4.609772130377966
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
@ -92,6 +110,24 @@ Expression (Projection)
1 [2,3.2] 2.3323807824711897
2 [4.2,3.4] 4.427188573446585
0 [4.6,2.3] 4.609772130377966
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)
Expression (Before ORDER BY)
ReadFromMergeTree (default.tab_bf16)
Indexes:
PrimaryKey
Condition: true
Parts: 1/1
Granules: 4/4
Skip
Name: idx
Description: vector_similarity GRANULARITY 2
Parts: 1/1
Granules: 2/4
1 [2,3.2] 2.3323807824711897
2 [4.2,3.4] 4.427188573446585
0 [4.6,2.3] 4.609772130377966
Expression (Projection)
Limit (preliminary LIMIT (without OFFSET))
Sorting (Sorting for ORDER BY)

View File

@ -75,13 +75,30 @@ SETTINGS max_limit_for_ann_queries = 2; -- LIMIT 3 > 2 --> don't use the ann ind
DROP TABLE tab;
SELECT '-- Non-default quantization';
CREATE TABLE tab_f64(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f64', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f32(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f32', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_f16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'f16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_bf16(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'bf16', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
CREATE TABLE tab_i8(id Int32, vec Array(Float32), INDEX idx vec TYPE vector_similarity('hnsw', 'L2Distance', 'i8', 0, 0, 0) GRANULARITY 2) ENGINE = MergeTree ORDER BY id SETTINGS index_granularity = 3;
INSERT INTO tab_f64 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]);
INSERT INTO tab_f32 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]);
INSERT INTO tab_f16 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]);
INSERT INTO tab_bf16 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]);
INSERT INTO tab_i8 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]);
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f64
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f64
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_f32
@ -108,6 +125,19 @@ FROM tab_f16
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_bf16
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
EXPLAIN indexes = 1
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_bf16
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
WITH [0.0, 2.0] AS reference_vec
SELECT id, vec, L2Distance(vec, reference_vec)
FROM tab_i8
@ -121,8 +151,10 @@ FROM tab_i8
ORDER BY L2Distance(vec, reference_vec)
LIMIT 3;
DROP TABLE tab_f64;
DROP TABLE tab_f32;
DROP TABLE tab_f16;
DROP TABLE tab_bf16;
DROP TABLE tab_i8;
SELECT '-- Index on Array(Float64) column';

View File

@ -0,0 +1,13 @@
11
12
13
14
15
16
17
18
19
20
21
22
23

View File

@ -0,0 +1,36 @@
#!/usr/bin/env bash
# Tags: no-parallel
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_CLIENT -q "
DROP TABLE IF EXISTS t_async_insert_params;
CREATE TABLE t_async_insert_params (id UInt64) ENGINE = Memory;
"
cmd_params="--async_insert 1 --async_insert_busy_timeout_max_ms 300000 --async_insert_busy_timeout_min_ms 300000 --wait_for_async_insert 0 --async_insert_use_adaptive_busy_timeout 0"
$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p1 = 11; INSERT INTO t_async_insert_params VALUES ({p1:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 12; INSERT INTO t_async_insert_params VALUES ({p2:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params -q "SET param_p2 = 1000; INSERT INTO t_async_insert_params VALUES (13);"
$CLICKHOUSE_CLIENT $cmd_params -q 'SET param_p2 = 1000; INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 14};'
$CLICKHOUSE_CLIENT $cmd_params --param_p1 15 -q "INSERT INTO t_async_insert_params VALUES ({p1:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params --param_p2 16 -q "INSERT INTO t_async_insert_params VALUES ({p2:UInt64});"
$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q "INSERT INTO t_async_insert_params VALUES (17);"
$CLICKHOUSE_CLIENT $cmd_params --param_p2 1000 -q 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 18};'
url="${CLICKHOUSE_URL}&async_insert=1&async_insert_busy_timeout_max_ms=300000&async_insert_busy_timeout_min_ms=300000&wait_for_async_insert=0&async_insert_use_adaptive_busy_timeout=0"
${CLICKHOUSE_CURL} -sS "$url&param_p1=19" -d "INSERT INTO t_async_insert_params VALUES ({p1:UInt64})"
${CLICKHOUSE_CURL} -sS "$url&param_p2=20" -d "INSERT INTO t_async_insert_params VALUES ({p2:UInt64})"
${CLICKHOUSE_CURL} -sS "$url&param_p3=21" -d "INSERT INTO t_async_insert_params VALUES ({p3:UInt64})"
${CLICKHOUSE_CURL} -sS "$url&param_p2=1000" -d "INSERT INTO t_async_insert_params VALUES (22)"
${CLICKHOUSE_CURL} -sS "$url&param_p2=1000" -d 'INSERT INTO t_async_insert_params FORMAT JSONEachRow {"id": 23}'
$CLICKHOUSE_CLIENT -q "
SYSTEM FLUSH ASYNC INSERT QUEUE;
SELECT id FROM t_async_insert_params ORDER BY id;
DROP TABLE IF EXISTS t_async_insert_params;
"

View File

@ -0,0 +1,20 @@
DROP TABLE IF EXISTS t_async_insert_params;
CREATE TABLE t_async_insert_params (id UInt64) ENGINE = MergeTree ORDER BY tuple();
SET param_p1 = 'Hello';
SET async_insert = 1;
SET wait_for_async_insert = 1;
INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER }
INSERT INTO t_async_insert_params VALUES ({p1:String}); -- { serverError TYPE_MISMATCH }
ALTER TABLE t_async_insert_params MODIFY COLUMN id String;
INSERT INTO t_async_insert_params VALUES ({p1:UInt64}); -- { serverError BAD_QUERY_PARAMETER }
INSERT INTO t_async_insert_params VALUES ({p1:String});
SELECT * FROM t_async_insert_params ORDER BY id;
DROP TABLE t_async_insert_params;

View File

@ -0,0 +1,4 @@
str 3 \N
str 3 \N
str 3 \N
str 3 \N

View File

@ -0,0 +1,4 @@
set allow_experimental_dynamic_type=1;
set cast_keep_nullable=1;
SELECT toFixedString('str', 3), 3, CAST(if(1 = 0, toInt8(3), NULL), 'Int32') AS x from numbers(10) GROUP BY GROUPING SETS ((CAST(toInt32(1), 'Int32')), ('str', 3), (CAST(toFixedString('str', 3), 'Dynamic')), (CAST(toFixedString(toFixedString('str', 3), 3), 'Dynamic')));

View File

@ -0,0 +1,2 @@
Query column at granularity boundary
0,0 0,1

View File

@ -0,0 +1,18 @@
SET allow_experimental_full_text_index=1;
DROP TABLE IF EXISTS multi_col_ivt;
CREATE TABLE tab (
v0 String,
v1 String,
INDEX idx (v0, v1) TYPE full_text GRANULARITY 1)
ENGINE = MergeTree
ORDER BY tuple()
SETTINGS index_granularity = 1;
INSERT INTO tab VALUES('0,0', '0,1')('2,2','2,3');
SELECT 'Query column at granularity boundary';
SELECT * FROM tab WHERE hasToken(v1, '1');
DROP TABLE tab;

View File

@ -0,0 +1 @@
{"a":[["1",{}],[null,{}]]} [(1,'{}'),(NULL,'{}')] Array(Tuple(Nullable(Int64), JSON(max_dynamic_types=16, max_dynamic_paths=256)))

View File

@ -0,0 +1,4 @@
set allow_experimental_json_type=1;
select materialize('{"a" : [[1, {}], null]}')::JSON as json, getSubcolumn(json, 'a'), dynamicType(getSubcolumn(json, 'a'));

View File

@ -0,0 +1,2 @@
{"a":"42"} JSON
{"a":42} Object(\'json\')

View File

@ -0,0 +1,8 @@
set allow_experimental_object_type=1;
set allow_experimental_json_type=1;
set use_json_alias_for_old_object_type=0;
select materialize('{"a" : 42}')::JSON as json, toTypeName(json);
set use_json_alias_for_old_object_type=1;
select '{"a" : 42}'::JSON as json, toTypeName(json);
select '{"a" : 42}'::JSON(max_dynamic_paths=100) as json, toTypeName(json); -- {serverError BAD_ARGUMENTS}

View File

@ -0,0 +1 @@
['key1','key2'] ['value1','value2']

View File

@ -0,0 +1,37 @@
DROP TABLE IF EXISTS rawtable;
DROP TABLE IF EXISTS raw_to_attributes_mv;
DROP TABLE IF EXISTS attributes;
SET optimize_functions_to_subcolumns = 1;
CREATE TABLE rawtable
(
`Attributes` Map(String, String),
)
ENGINE = MergeTree
ORDER BY tuple();
CREATE MATERIALIZED VIEW raw_to_attributes_mv TO attributes
(
`AttributeKeys` Array(String),
`AttributeValues` Array(String)
)
AS SELECT
mapKeys(Attributes) AS AttributeKeys,
mapValues(Attributes) AS AttributeValues
FROM rawtable;
CREATE TABLE attributes
(
`AttributeKeys` Array(String),
`AttributeValues` Array(String)
)
ENGINE = ReplacingMergeTree
ORDER BY tuple();
INSERT INTO rawtable VALUES ({'key1': 'value1', 'key2': 'value2'});
SELECT * FROM raw_to_attributes_mv ORDER BY AttributeKeys;
DROP TABLE IF EXISTS rawtable;
DROP TABLE IF EXISTS raw_to_attributes_mv;
DROP TABLE IF EXISTS attributes;

View File

@ -0,0 +1,6 @@
default projections improved_sorting_key Normal ['d1','key'] SELECT * ORDER BY d1, key
default projections_2 agg Aggregate ['name'] SELECT name, max(frequency) AS max_frequency GROUP BY name
default projections_2 agg_no_key Aggregate [] SELECT max(frequency) AS max_frequency
1
2
improved_sorting_key

View File

@ -0,0 +1,39 @@
DROP TABLE IF EXISTS projections;
DROP TABLE IF EXISTS projections_2;
CREATE TABLE projections
(
key String,
d1 Int,
PROJECTION improved_sorting_key (
SELECT *
ORDER BY d1, key
)
)
Engine=MergeTree()
ORDER BY key;
CREATE TABLE projections_2
(
name String,
frequency UInt64,
PROJECTION agg (
SELECT name, max(frequency) max_frequency
GROUP BY name
),
PROJECTION agg_no_key (
SELECT max(frequency) max_frequency
)
)
Engine=MergeTree()
ORDER BY name;
SELECT * FROM system.projections WHERE database = currentDatabase();
SELECT count(*) FROM system.projections WHERE table = 'projections' AND database = currentDatabase();
SELECT count(*) FROM system.projections WHERE table = 'projections_2' AND database = currentDatabase();
SELECT name FROM system.projections WHERE type = 'Normal' AND database = currentDatabase();
DROP TABLE projections;
DROP TABLE projections_2;