mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge branch 'master' into keeper-destroy-dispatcher-first
This commit is contained in:
commit
b57e57fcdd
@ -4,7 +4,11 @@ sidebar_position: 50
|
||||
sidebar_label: MySQL
|
||||
---
|
||||
|
||||
# MySQL
|
||||
import CloudNotSupportedBadge from '@theme/badges/CloudNotSupportedBadge';
|
||||
|
||||
# MySQL Database Engine
|
||||
|
||||
<CloudNotSupportedBadge />
|
||||
|
||||
Allows to connect to databases on a remote MySQL server and perform `INSERT` and `SELECT` queries to exchange data between ClickHouse and MySQL.
|
||||
|
||||
|
@ -4,7 +4,11 @@ sidebar_position: 138
|
||||
sidebar_label: MySQL
|
||||
---
|
||||
|
||||
# MySQL
|
||||
import CloudAvailableBadge from '@theme/badges/CloudAvailableBadge';
|
||||
|
||||
# MySQL Table Engine
|
||||
|
||||
<CloudAvailableBadge />
|
||||
|
||||
The MySQL engine allows you to perform `SELECT` and `INSERT` queries on data that is stored on a remote MySQL server.
|
||||
|
||||
|
@ -21,6 +21,35 @@ The queries to terminate are selected from the system.processes table using the
|
||||
|
||||
Examples:
|
||||
|
||||
First, you'll need to get the list of incomplete queries. This SQL query provides them according to those running the longest:
|
||||
|
||||
List from a single ClickHouse node:
|
||||
``` sql
|
||||
SELECT
|
||||
initial_query_id,
|
||||
query_id,
|
||||
formatReadableTimeDelta(elapsed) AS time_delta,
|
||||
query,
|
||||
*
|
||||
FROM system.processes
|
||||
WHERE query ILIKE 'SELECT%'
|
||||
ORDER BY time_delta DESC;
|
||||
```
|
||||
|
||||
List from a ClickHouse cluster:
|
||||
``` sql
|
||||
SELECT
|
||||
initial_query_id,
|
||||
query_id,
|
||||
formatReadableTimeDelta(elapsed) AS time_delta,
|
||||
query,
|
||||
*
|
||||
FROM clusterAllReplicas(default, system.processes)
|
||||
WHERE query ILIKE 'SELECT%'
|
||||
ORDER BY time_delta DESC;
|
||||
```
|
||||
|
||||
Kill the query:
|
||||
``` sql
|
||||
-- Forcibly terminates all queries with the specified query_id:
|
||||
KILL QUERY WHERE query_id='2-857d-4a57-9ee0-327da5d60a90'
|
||||
@ -44,6 +73,11 @@ A test query (`TEST`) only checks the user’s rights and displays a list of que
|
||||
|
||||
## KILL MUTATION
|
||||
|
||||
The presence of long-running or incomplete mutations often indicates that a ClickHouse service is running poorly. The asynchronous nature of mutations can cause them to consume all available resources on a system. You may need to either:
|
||||
|
||||
- Pause all new mutations, `INSERT`s , and `SELECT`s and allow the queue of mutations to complete.
|
||||
- Or manually kill some of these mutations by sending a `KILL` command.
|
||||
|
||||
``` sql
|
||||
KILL MUTATION [ON CLUSTER cluster]
|
||||
WHERE <where expression to SELECT FROM system.mutations query>
|
||||
@ -57,6 +91,39 @@ A test query (`TEST`) only checks the user’s rights and displays a list of mut
|
||||
|
||||
Examples:
|
||||
|
||||
Get a `count()` of the number of incomplete mutations:
|
||||
|
||||
Count of mutations from a single ClickHouse node:
|
||||
``` sql
|
||||
SELECT count(*)
|
||||
FROM system.mutations
|
||||
WHERE is_done = 0;
|
||||
```
|
||||
|
||||
Count of mutations from a ClickHouse cluster of replicas:
|
||||
``` sql
|
||||
SELECT count(*)
|
||||
FROM clusterAllReplicas('default', system.mutations)
|
||||
WHERE is_done = 0;
|
||||
```
|
||||
|
||||
Query the list of incomplete mutations:
|
||||
|
||||
List of mutations from a single ClickHouse node:
|
||||
``` sql
|
||||
SELECT mutation_id, *
|
||||
FROM system.mutations
|
||||
WHERE is_done = 0;
|
||||
```
|
||||
|
||||
List of mutations from a ClickHouse cluster:
|
||||
``` sql
|
||||
SELECT mutation_id, *
|
||||
FROM clusterAllReplicas('default', system.mutations)
|
||||
WHERE is_done = 0;
|
||||
```
|
||||
|
||||
Kill the mutations as needed:
|
||||
``` sql
|
||||
-- Cancel and remove all mutations of the single table:
|
||||
KILL MUTATION WHERE database = 'default' AND table = 'table'
|
||||
|
@ -287,6 +287,7 @@ class IColumn;
|
||||
M(UInt64, read_backoff_min_concurrency, 1, "Settings to try keeping the minimal number of threads in case of slow reads.", 0) \
|
||||
\
|
||||
M(Float, memory_tracker_fault_probability, 0., "For testing of `exception safety` - throw an exception every time you allocate memory with the specified probability.", 0) \
|
||||
M(Float, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability.", 0) \
|
||||
\
|
||||
M(Bool, enable_http_compression, false, "Compress the result if the client over HTTP said that it understands data compressed by gzip, deflate, zstd, br, lz4, bz2, xz.", 0) \
|
||||
M(Int64, http_zlib_compression_level, 3, "Compression level - used if the client on HTTP said that it understands data compressed by gzip or deflate.", 0) \
|
||||
|
@ -103,6 +103,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
|
||||
{"keeper_retry_initial_backoff_ms", 100, 100, "Initial backoff timeout for general keeper operations"},
|
||||
{"keeper_retry_max_backoff_ms", 5000, 5000, "Max backoff timeout for general keeper operations"},
|
||||
{"s3queue_allow_experimental_sharded_mode", false, false, "Enable experimental sharded mode of S3Queue table engine. It is experimental because it will be rewritten"},
|
||||
{"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability", 0.0, 0.0, "For testing of `PartsSplitter` - split read ranges into intersecting and non intersecting every time you read from MergeTree with the specified probability."},
|
||||
}},
|
||||
{"24.2", {{"allow_suspicious_variant_types", true, false, "Don't allow creating Variant type with suspicious variants by default"},
|
||||
{"validate_experimental_and_suspicious_types_inside_nested_types", false, true, "Validate usage of experimental and suspicious types inside nested types"},
|
||||
|
@ -1127,23 +1127,24 @@ struct AccurateOrNullConvertStrategyAdditions
|
||||
UInt32 scale { 0 };
|
||||
};
|
||||
|
||||
|
||||
struct ConvertDefaultBehaviorTag {};
|
||||
struct ConvertReturnNullOnErrorTag {};
|
||||
struct ConvertReturnZeroOnErrorTag {};
|
||||
enum class BehaviourOnErrorFromString
|
||||
{
|
||||
ConvertDefaultBehaviorTag,
|
||||
ConvertReturnNullOnErrorTag,
|
||||
ConvertReturnZeroOnErrorTag
|
||||
};
|
||||
|
||||
/** Conversion of number types to each other, enums to numbers, dates and datetimes to numbers and back: done by straight assignment.
|
||||
* (Date is represented internally as number of days from some day; DateTime - as unix timestamp)
|
||||
*/
|
||||
template <typename FromDataType, typename ToDataType, typename Name,
|
||||
typename SpecialTag = ConvertDefaultBehaviorTag,
|
||||
FormatSettings::DateTimeOverflowBehavior date_time_overflow_behavior = default_date_time_overflow_behavior>
|
||||
struct ConvertImpl
|
||||
{
|
||||
template <typename Additions = void *>
|
||||
static ColumnPtr NO_SANITIZE_UNDEFINED execute(
|
||||
const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type [[maybe_unused]], size_t input_rows_count,
|
||||
Additions additions = Additions())
|
||||
BehaviourOnErrorFromString from_string_tag [[maybe_unused]], Additions additions = Additions())
|
||||
{
|
||||
const ColumnWithTypeAndName & named_from = arguments[0];
|
||||
|
||||
@ -1176,8 +1177,7 @@ struct ConvertImpl
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTimeImpl<date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
}
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64> && std::is_same_v<ToDataType, DataTypeDate32>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64> && std::is_same_v<ToDataType, DataTypeDate32>)
|
||||
{
|
||||
return DateTimeTransformImpl<DataTypeDateTime64, DataTypeDate32, TransformDateTime64<ToDate32Impl>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
@ -1194,8 +1194,7 @@ struct ConvertImpl
|
||||
else if constexpr ((
|
||||
std::is_same_v<FromDataType, DataTypeUInt32>
|
||||
|| std::is_same_v<FromDataType, DataTypeUInt64>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTransform32Or64<typename FromDataType::FieldType, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1203,8 +1202,7 @@ struct ConvertImpl
|
||||
else if constexpr ((
|
||||
std::is_same_v<FromDataType, DataTypeInt8>
|
||||
|| std::is_same_v<FromDataType, DataTypeInt16>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTransform8Or16Signed<typename FromDataType::FieldType, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1214,8 +1212,7 @@ struct ConvertImpl
|
||||
|| std::is_same_v<FromDataType, DataTypeInt64>
|
||||
|| std::is_same_v<FromDataType, DataTypeFloat32>
|
||||
|| std::is_same_v<FromDataType, DataTypeFloat64>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTransform32Or64Signed<typename FromDataType::FieldType, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1223,8 +1220,7 @@ struct ConvertImpl
|
||||
else if constexpr ((
|
||||
std::is_same_v<FromDataType, DataTypeUInt32>
|
||||
|| std::is_same_v<FromDataType, DataTypeUInt64>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate32>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate32>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDate32Transform32Or64<typename FromDataType::FieldType, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1232,8 +1228,7 @@ struct ConvertImpl
|
||||
else if constexpr ((
|
||||
std::is_same_v<FromDataType, DataTypeInt8>
|
||||
|| std::is_same_v<FromDataType, DataTypeInt16>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate32>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate32>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDate32Transform8Or16Signed<typename FromDataType::FieldType>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1243,8 +1238,7 @@ struct ConvertImpl
|
||||
|| std::is_same_v<FromDataType, DataTypeInt64>
|
||||
|| std::is_same_v<FromDataType, DataTypeFloat32>
|
||||
|| std::is_same_v<FromDataType, DataTypeFloat64>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate32>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate32>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDate32Transform32Or64Signed<typename FromDataType::FieldType, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1254,15 +1248,13 @@ struct ConvertImpl
|
||||
std::is_same_v<FromDataType, DataTypeInt8>
|
||||
|| std::is_same_v<FromDataType, DataTypeInt16>
|
||||
|| std::is_same_v<FromDataType, DataTypeInt32>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTimeTransformSigned<typename FromDataType::FieldType, UInt32, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
}
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeUInt64>
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTimeTransform64<typename FromDataType::FieldType, UInt32, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1271,8 +1263,7 @@ struct ConvertImpl
|
||||
std::is_same_v<FromDataType, DataTypeInt64>
|
||||
|| std::is_same_v<FromDataType, DataTypeFloat32>
|
||||
|| std::is_same_v<FromDataType, DataTypeFloat64>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTimeTransform64Signed<typename FromDataType::FieldType, UInt32, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
@ -1282,15 +1273,13 @@ struct ConvertImpl
|
||||
|| std::is_same_v<FromDataType, DataTypeInt16>
|
||||
|| std::is_same_v<FromDataType, DataTypeInt32>
|
||||
|| std::is_same_v<FromDataType, DataTypeInt64>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTime64TransformSigned<typename FromDataType::FieldType, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
}
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeUInt64>
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTime64TransformUnsigned<UInt64, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
@ -1298,23 +1287,20 @@ struct ConvertImpl
|
||||
else if constexpr ((
|
||||
std::is_same_v<FromDataType, DataTypeFloat32>
|
||||
|| std::is_same_v<FromDataType, DataTypeFloat64>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTime64TransformFloat<FromDataType, typename FromDataType::FieldType, default_date_time_overflow_behavior>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
}
|
||||
/// Conversion of DateTime64 to Date or DateTime: discards fractional part.
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDate>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, TransformDateTime64<ToDateImpl<date_time_overflow_behavior>>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
}
|
||||
else if constexpr (std::is_same_v<FromDataType, DataTypeDateTime64>
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, TransformDateTime64<ToDateTimeImpl<date_time_overflow_behavior>>, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
@ -1324,8 +1310,7 @@ struct ConvertImpl
|
||||
std::is_same_v<FromDataType, DataTypeDate>
|
||||
|| std::is_same_v<FromDataType, DataTypeDate32>
|
||||
|| std::is_same_v<FromDataType, DataTypeDateTime>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
{
|
||||
return DateTimeTransformImpl<FromDataType, ToDataType, ToDateTime64Transform, false>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
@ -1443,8 +1428,7 @@ struct ConvertImpl
|
||||
/// Conversion from FixedString to String.
|
||||
/// Cutting sequences of zero bytes from end of strings.
|
||||
else if constexpr (std::is_same_v<ToDataType, DataTypeString>
|
||||
&& std::is_same_v<FromDataType, DataTypeFixedString>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
&& std::is_same_v<FromDataType, DataTypeFixedString>)
|
||||
{
|
||||
ColumnUInt8::MutablePtr null_map = copyNullMap(arguments[0].column);
|
||||
const auto & nested = columnGetNested(arguments[0]);
|
||||
@ -1488,8 +1472,7 @@ struct ConvertImpl
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}",
|
||||
arguments[0].column->getName(), Name::name);
|
||||
}
|
||||
else if constexpr (std::is_same_v<ToDataType, DataTypeString>
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
else if constexpr (std::is_same_v<ToDataType, DataTypeString>)
|
||||
{
|
||||
/// Anything else to String.
|
||||
|
||||
@ -1550,27 +1533,35 @@ struct ConvertImpl
|
||||
&& std::is_same_v<FromDataType, DataTypeString>
|
||||
&& std::is_same_v<ToDataType, DataTypeUInt32>)
|
||||
{
|
||||
return ConvertImpl<FromDataType, DataTypeDateTime, Name, SpecialTag, date_time_overflow_behavior>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count);
|
||||
return ConvertImpl<FromDataType, DataTypeDateTime, Name, date_time_overflow_behavior>::template execute<Additions>(
|
||||
arguments, result_type, input_rows_count, from_string_tag);
|
||||
}
|
||||
else if constexpr ((std::is_same_v<FromDataType, DataTypeString> || std::is_same_v<FromDataType, DataTypeFixedString>)
|
||||
&& std::is_same_v<SpecialTag, ConvertDefaultBehaviorTag>)
|
||||
else if constexpr ((std::is_same_v<FromDataType, DataTypeString> || std::is_same_v<FromDataType, DataTypeFixedString>))
|
||||
{
|
||||
return ConvertThroughParsing<FromDataType, ToDataType, Name, ConvertFromStringExceptionMode::Throw, ConvertFromStringParsingMode::Normal>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
}
|
||||
else if constexpr ((std::is_same_v<FromDataType, DataTypeString> || std::is_same_v<FromDataType, DataTypeFixedString>)
|
||||
&& std::is_same_v<SpecialTag, ConvertReturnNullOnErrorTag>)
|
||||
{
|
||||
return ConvertThroughParsing<FromDataType, ToDataType, Name, ConvertFromStringExceptionMode::Null, ConvertFromStringParsingMode::Normal>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
}
|
||||
else if constexpr ((std::is_same_v<FromDataType, DataTypeString> || std::is_same_v<FromDataType, DataTypeFixedString>)
|
||||
&& is_any_of<ToDataType, DataTypeIPv4, DataTypeIPv6>
|
||||
&& std::is_same_v<SpecialTag, ConvertReturnZeroOnErrorTag>)
|
||||
{
|
||||
return ConvertThroughParsing<FromDataType, ToDataType, Name, ConvertFromStringExceptionMode::Zero, ConvertFromStringParsingMode::Normal>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
switch (from_string_tag)
|
||||
{
|
||||
case BehaviourOnErrorFromString::ConvertDefaultBehaviorTag:
|
||||
return ConvertThroughParsing<FromDataType,
|
||||
ToDataType,
|
||||
Name,
|
||||
ConvertFromStringExceptionMode::Throw,
|
||||
ConvertFromStringParsingMode::Normal>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
case BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag:
|
||||
return ConvertThroughParsing<FromDataType,
|
||||
ToDataType,
|
||||
Name,
|
||||
ConvertFromStringExceptionMode::Null,
|
||||
ConvertFromStringParsingMode::Normal>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
case BehaviourOnErrorFromString::ConvertReturnZeroOnErrorTag:
|
||||
return ConvertThroughParsing<FromDataType,
|
||||
ToDataType,
|
||||
Name,
|
||||
ConvertFromStringExceptionMode::Zero,
|
||||
ConvertFromStringParsingMode::Normal>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
@ -2165,12 +2156,11 @@ private:
|
||||
if (context)
|
||||
date_time_overflow_behavior = context->getSettingsRef().date_time_overflow_behavior.value;
|
||||
|
||||
auto call = [&](const auto & types, const auto & tag) -> bool
|
||||
auto call = [&](const auto & types, BehaviourOnErrorFromString from_string_tag) -> bool
|
||||
{
|
||||
using Types = std::decay_t<decltype(types)>;
|
||||
using LeftDataType = typename Types::LeftType;
|
||||
using RightDataType = typename Types::RightType;
|
||||
using SpecialTag = std::decay_t<decltype(tag)>;
|
||||
|
||||
if constexpr (IsDataTypeDecimal<RightDataType>)
|
||||
{
|
||||
@ -2191,13 +2181,13 @@ private:
|
||||
switch (date_time_overflow_behavior)
|
||||
{
|
||||
case FormatSettings::DateTimeOverflowBehavior::Throw:
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag, FormatSettings::DateTimeOverflowBehavior::Throw>::execute(arguments, result_type, input_rows_count, scale);
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, FormatSettings::DateTimeOverflowBehavior::Throw>::execute(arguments, result_type, input_rows_count, from_string_tag, scale);
|
||||
break;
|
||||
case FormatSettings::DateTimeOverflowBehavior::Ignore:
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag, FormatSettings::DateTimeOverflowBehavior::Ignore>::execute(arguments, result_type, input_rows_count, scale);
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, FormatSettings::DateTimeOverflowBehavior::Ignore>::execute(arguments, result_type, input_rows_count, from_string_tag, scale);
|
||||
break;
|
||||
case FormatSettings::DateTimeOverflowBehavior::Saturate:
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag, FormatSettings::DateTimeOverflowBehavior::Saturate>::execute(arguments, result_type, input_rows_count, scale);
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, FormatSettings::DateTimeOverflowBehavior::Saturate>::execute(arguments, result_type, input_rows_count, from_string_tag, scale);
|
||||
break;
|
||||
}
|
||||
|
||||
@ -2208,20 +2198,20 @@ private:
|
||||
switch (date_time_overflow_behavior)
|
||||
{
|
||||
case FormatSettings::DateTimeOverflowBehavior::Throw:
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag, FormatSettings::DateTimeOverflowBehavior::Throw>::execute(arguments, result_type, input_rows_count, dt64->getScale());
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, FormatSettings::DateTimeOverflowBehavior::Throw>::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale());
|
||||
break;
|
||||
case FormatSettings::DateTimeOverflowBehavior::Ignore:
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag, FormatSettings::DateTimeOverflowBehavior::Ignore>::execute(arguments, result_type, input_rows_count, dt64->getScale());
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, FormatSettings::DateTimeOverflowBehavior::Ignore>::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale());
|
||||
break;
|
||||
case FormatSettings::DateTimeOverflowBehavior::Saturate:
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag, FormatSettings::DateTimeOverflowBehavior::Saturate>::execute(arguments, result_type, input_rows_count, dt64->getScale());
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, FormatSettings::DateTimeOverflowBehavior::Saturate>::execute(arguments, result_type, input_rows_count, from_string_tag, dt64->getScale());
|
||||
break;
|
||||
}
|
||||
}
|
||||
#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE) \
|
||||
case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag, FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE>::execute( \
|
||||
arguments, result_type, input_rows_count); \
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE>::execute( \
|
||||
arguments, result_type, input_rows_count, from_string_tag); \
|
||||
break;
|
||||
|
||||
else if constexpr (IsDataTypeDecimalOrNumber<LeftDataType> && IsDataTypeDecimalOrNumber<RightDataType>)
|
||||
@ -2262,7 +2252,7 @@ private:
|
||||
}
|
||||
#undef GENERATE_OVERFLOW_MODE_CASE
|
||||
else
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name, SpecialTag>::execute(arguments, result_type, input_rows_count);
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, Name>::execute(arguments, result_type, input_rows_count, from_string_tag);
|
||||
|
||||
return true;
|
||||
};
|
||||
@ -2275,7 +2265,7 @@ private:
|
||||
|
||||
if (to_datetime64 || scale != 0) /// When scale = 0, the data type is DateTime otherwise the data type is DateTime64
|
||||
{
|
||||
if (!callOnIndexAndDataType<DataTypeDateTime64>(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{}))
|
||||
if (!callOnIndexAndDataType<DataTypeDateTime64>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal type {} of argument of function {}",
|
||||
arguments[0].type->getName(), getName());
|
||||
|
||||
@ -2292,23 +2282,25 @@ private:
|
||||
bool done = false;
|
||||
if constexpr (is_any_of<ToDataType, DataTypeString, DataTypeFixedString>)
|
||||
{
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{});
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag);
|
||||
}
|
||||
else
|
||||
{
|
||||
bool cast_ipv4_ipv6_default_on_conversion_error = false;
|
||||
if constexpr (is_any_of<ToDataType, DataTypeIPv4, DataTypeIPv6>)
|
||||
{
|
||||
if (context && (cast_ipv4_ipv6_default_on_conversion_error = context->getSettingsRef().cast_ipv4_ipv6_default_on_conversion_error))
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, ConvertReturnZeroOnErrorTag{});
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertReturnZeroOnErrorTag);
|
||||
}
|
||||
|
||||
if (!cast_ipv4_ipv6_default_on_conversion_error)
|
||||
{
|
||||
/// We should use ConvertFromStringExceptionMode::Null mode when converting from String (or FixedString)
|
||||
/// to Nullable type, to avoid 'value is too short' error on attempt to parse empty string from NULL values.
|
||||
if (to_nullable && WhichDataType(from_type).isStringOrFixedString())
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, ConvertReturnNullOnErrorTag{});
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag);
|
||||
else
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, ConvertDefaultBehaviorTag{});
|
||||
done = callOnIndexAndDataType<ToDataType>(from_type->getTypeId(), call, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag);
|
||||
}
|
||||
}
|
||||
|
||||
@ -3185,8 +3177,8 @@ private:
|
||||
{
|
||||
#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \
|
||||
case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName, ConvertDefaultBehaviorTag, FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE>::execute( \
|
||||
arguments, result_type, input_rows_count, ADDITIONS()); \
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName, FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE>::execute( \
|
||||
arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, ADDITIONS()); \
|
||||
break;
|
||||
if (wrapper_cast_type == CastType::accurate)
|
||||
{
|
||||
@ -3215,8 +3207,8 @@ private:
|
||||
{
|
||||
#define GENERATE_OVERFLOW_MODE_CASE(OVERFLOW_MODE, ADDITIONS) \
|
||||
case FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE: \
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName, ConvertDefaultBehaviorTag, FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE>::template execute<ADDITIONS>( \
|
||||
arguments, result_type, input_rows_count); \
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName, FormatSettings::DateTimeOverflowBehavior::OVERFLOW_MODE>::template execute<ADDITIONS>( \
|
||||
arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag); \
|
||||
break;
|
||||
if (wrapper_cast_type == CastType::accurate)
|
||||
{
|
||||
@ -3376,7 +3368,7 @@ arguments, result_type, input_rows_count); \
|
||||
AccurateConvertStrategyAdditions additions;
|
||||
additions.scale = scale;
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, additions);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -3385,7 +3377,7 @@ arguments, result_type, input_rows_count); \
|
||||
AccurateOrNullConvertStrategyAdditions additions;
|
||||
additions.scale = scale;
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName>::execute(
|
||||
arguments, result_type, input_rows_count, additions);
|
||||
arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, additions);
|
||||
|
||||
return true;
|
||||
}
|
||||
@ -3397,14 +3389,14 @@ arguments, result_type, input_rows_count); \
|
||||
/// Consistent with CAST(Nullable(String) AS Nullable(Numbers))
|
||||
/// In case when converting to Nullable type, we apply different parsing rule,
|
||||
/// that will not throw an exception but return NULL in case of malformed input.
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName, ConvertReturnNullOnErrorTag>::execute(
|
||||
arguments, result_type, input_rows_count, scale);
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName>::execute(
|
||||
arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertReturnNullOnErrorTag, scale);
|
||||
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName>::execute(arguments, result_type, input_rows_count, scale);
|
||||
result_column = ConvertImpl<LeftDataType, RightDataType, FunctionCastName>::execute(arguments, result_type, input_rows_count, BehaviourOnErrorFromString::ConvertDefaultBehaviorTag, scale);
|
||||
|
||||
return true;
|
||||
});
|
||||
|
@ -48,11 +48,12 @@ dispatchForIntervalColumns(const ColumnWithTypeAndName & interval_column, const
|
||||
return {interval_type->getKind(), num_units};
|
||||
}
|
||||
|
||||
ColumnPtr executeWindowBound(const ColumnPtr & column, int index, const String & function_name)
|
||||
ColumnPtr executeWindowBound(const ColumnPtr & column, size_t index, const String & function_name)
|
||||
{
|
||||
chassert(index == 0 || index == 1);
|
||||
if (const ColumnTuple * col_tuple = checkAndGetColumn<ColumnTuple>(column.get()); col_tuple)
|
||||
{
|
||||
if (!checkColumn<ColumnVector<UInt32>>(*col_tuple->getColumnPtr(index)))
|
||||
if (index >= col_tuple->tupleSize() || !checkColumn<ColumnVector<UInt32>>(*col_tuple->getColumnPtr(index)))
|
||||
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Illegal column for first argument of function {}. "
|
||||
"Must be a Tuple(DataTime, DataTime)", function_name);
|
||||
return col_tuple->getColumnPtr(index);
|
||||
@ -109,6 +110,17 @@ bool checkIntervalOrTimeZoneArgument(const ColumnWithTypeAndName & argument, con
|
||||
return true;
|
||||
}
|
||||
|
||||
enum TimeWindowFunctionName
|
||||
{
|
||||
TUMBLE,
|
||||
TUMBLE_START,
|
||||
TUMBLE_END,
|
||||
HOP,
|
||||
HOP_START,
|
||||
HOP_END,
|
||||
WINDOW_ID
|
||||
};
|
||||
|
||||
template <TimeWindowFunctionName type>
|
||||
struct TimeWindowImpl
|
||||
{
|
||||
|
@ -21,16 +21,6 @@ namespace DB
|
||||
* hopEnd(window_id)
|
||||
* hopEnd(time_attr, hop_interval, window_interval [, timezone])
|
||||
*/
|
||||
enum TimeWindowFunctionName
|
||||
{
|
||||
TUMBLE,
|
||||
TUMBLE_START,
|
||||
TUMBLE_END,
|
||||
HOP,
|
||||
HOP_START,
|
||||
HOP_END,
|
||||
WINDOW_ID
|
||||
};
|
||||
|
||||
template <IntervalKind::Kind unit>
|
||||
struct ToStartOfTransform;
|
||||
@ -73,17 +63,15 @@ struct ToStartOfTransform;
|
||||
TRANSFORM_TIME(Second)
|
||||
#undef TRANSFORM_TIME
|
||||
|
||||
/// NOLINTBEGIN(bugprone-macro-parentheses)
|
||||
|
||||
#define TRANSFORM_SUBSECONDS(INTERVAL_KIND, DEF_SCALE) \
|
||||
template<> \
|
||||
struct ToStartOfTransform<IntervalKind::Kind::INTERVAL_KIND> \
|
||||
{ \
|
||||
static Int64 execute(Int64 t, UInt64 delta, const UInt32 scale) \
|
||||
{ \
|
||||
if (scale <= DEF_SCALE) \
|
||||
if (scale <= (DEF_SCALE)) \
|
||||
{ \
|
||||
auto val = t * DecimalUtils::scaleMultiplier<DateTime64>(DEF_SCALE - scale); \
|
||||
auto val = t * DecimalUtils::scaleMultiplier<DateTime64>((DEF_SCALE) - scale); \
|
||||
if (delta == 1) \
|
||||
return val; \
|
||||
else \
|
||||
@ -91,7 +79,7 @@ template<> \
|
||||
} \
|
||||
else \
|
||||
{ \
|
||||
return t - (t % (delta * DecimalUtils::scaleMultiplier<DateTime64>(scale - DEF_SCALE))) ; \
|
||||
return t - (t % (delta * DecimalUtils::scaleMultiplier<DateTime64>(scale - (DEF_SCALE)))) ; \
|
||||
} \
|
||||
} \
|
||||
};
|
||||
@ -131,7 +119,7 @@ template<> \
|
||||
struct AddTime<IntervalKind::Kind::INTERVAL_KIND> \
|
||||
{ \
|
||||
static inline NO_SANITIZE_UNDEFINED UInt32 execute(UInt32 t, Int64 delta, const DateLUTImpl &) \
|
||||
{ return static_cast<UInt32>(t + delta * INTERVAL); } \
|
||||
{ return static_cast<UInt32>(t + delta * (INTERVAL)); } \
|
||||
};
|
||||
ADD_TIME(Day, 86400)
|
||||
ADD_TIME(Hour, 3600)
|
||||
@ -145,12 +133,12 @@ template <> \
|
||||
{ \
|
||||
static inline NO_SANITIZE_UNDEFINED Int64 execute(Int64 t, UInt64 delta, const UInt32 scale) \
|
||||
{ \
|
||||
if (scale < DEF_SCALE) \
|
||||
if (scale < (DEF_SCALE)) \
|
||||
{ \
|
||||
return t + delta * DecimalUtils::scaleMultiplier<DateTime64>(DEF_SCALE - scale); \
|
||||
return t + delta * DecimalUtils::scaleMultiplier<DateTime64>((DEF_SCALE) - scale); \
|
||||
} \
|
||||
else \
|
||||
return t + delta * DecimalUtils::scaleMultiplier<DateTime64>(scale - DEF_SCALE); \
|
||||
return t + delta * DecimalUtils::scaleMultiplier<DateTime64>(scale - (DEF_SCALE)); \
|
||||
} \
|
||||
};
|
||||
ADD_SUBSECONDS(Millisecond, 3)
|
||||
@ -158,6 +146,4 @@ template <> \
|
||||
ADD_SUBSECONDS(Nanosecond, 9)
|
||||
#undef ADD_SUBSECONDS
|
||||
|
||||
/// NOLINTEND(bugprone-macro-parentheses)
|
||||
|
||||
}
|
||||
|
@ -1,5 +1,7 @@
|
||||
#include <Planner/PlannerExpressionAnalysis.h>
|
||||
|
||||
#include <Columns/ColumnNullable.h>
|
||||
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
|
||||
@ -111,7 +113,8 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
|
||||
continue;
|
||||
|
||||
auto expression_type_after_aggregation = group_by_use_nulls ? makeNullableSafe(expression_dag_node->result_type) : expression_dag_node->result_type;
|
||||
available_columns_after_aggregation.emplace_back(expression_dag_node->column, expression_type_after_aggregation, expression_dag_node->result_name);
|
||||
auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column;
|
||||
available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name);
|
||||
aggregation_keys.push_back(expression_dag_node->result_name);
|
||||
before_aggregation_actions->getOutputs().push_back(expression_dag_node);
|
||||
before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name);
|
||||
@ -161,7 +164,8 @@ std::optional<AggregationAnalysisResult> analyzeAggregation(const QueryTreeNodeP
|
||||
continue;
|
||||
|
||||
auto expression_type_after_aggregation = group_by_use_nulls ? makeNullableSafe(expression_dag_node->result_type) : expression_dag_node->result_type;
|
||||
available_columns_after_aggregation.emplace_back(expression_dag_node->column, expression_type_after_aggregation, expression_dag_node->result_name);
|
||||
auto column_after_aggregation = group_by_use_nulls && expression_dag_node->column != nullptr ? makeNullableSafe(expression_dag_node->column) : expression_dag_node->column;
|
||||
available_columns_after_aggregation.emplace_back(std::move(column_after_aggregation), expression_type_after_aggregation, expression_dag_node->result_name);
|
||||
aggregation_keys.push_back(expression_dag_node->result_name);
|
||||
before_aggregation_actions->getOutputs().push_back(expression_dag_node);
|
||||
before_aggregation_actions_output_node_names.insert(expression_dag_node->result_name);
|
||||
|
@ -34,6 +34,12 @@ std::string toString(const Values & value)
|
||||
return fmt::format("({})", fmt::join(value, ", "));
|
||||
}
|
||||
|
||||
/** We rely that FieldVisitorAccurateLess will have strict weak ordering for any Field values including
|
||||
* NaN, Null and containers (Array, Tuple, Map) that contain NaN or Null. But right now it does not properly
|
||||
* support NaN and Nulls inside containers, because it uses Field operator< or accurate::lessOp for comparison
|
||||
* that compares Nulls and NaNs differently than FieldVisitorAccurateLess.
|
||||
* TODO: Update Field operator< to compare NaNs and Nulls the same way as FieldVisitorAccurateLess.
|
||||
*/
|
||||
bool isSafePrimaryDataKeyType(const IDataType & data_type)
|
||||
{
|
||||
auto type_id = data_type.getTypeId();
|
||||
@ -316,12 +322,12 @@ struct PartRangeIndex
|
||||
|
||||
bool operator==(const PartRangeIndex & other) const
|
||||
{
|
||||
return part_index == other.part_index && range.begin == other.range.begin && range.end == other.range.end;
|
||||
return std::tie(part_index, range.begin, range.end) == std::tie(other.part_index, other.range.begin, other.range.end);
|
||||
}
|
||||
|
||||
bool operator<(const PartRangeIndex & other) const
|
||||
{
|
||||
return part_index < other.part_index && range.begin < other.range.begin && range.end < other.range.end;
|
||||
return std::tie(part_index, range.begin, range.end) < std::tie(other.part_index, other.range.begin, other.range.end);
|
||||
}
|
||||
|
||||
size_t part_index;
|
||||
@ -786,7 +792,7 @@ ASTs buildFilters(const KeyDescription & primary_key, const std::vector<Values>
|
||||
const auto & type = primary_key.data_types.at(i);
|
||||
|
||||
// PK may contain functions of the table columns, so we need the actual PK AST with all expressions it contains.
|
||||
auto pk_ast = primary_key.expression_list_ast->children.at(i);
|
||||
auto pk_ast = primary_key.expression_list_ast->children.at(i)->clone();
|
||||
|
||||
// If PK is nullable, prepend a null mask column for > comparison.
|
||||
// Also transform the AST into assumeNotNull(pk) so that the result type is not-nullable.
|
||||
|
@ -766,6 +766,82 @@ Pipe ReadFromMergeTree::spreadMarkRangesAmongStreams(RangesInDataParts && parts_
|
||||
|
||||
auto read_type = is_parallel_reading_from_replicas ? ReadType::ParallelReplicas : ReadType::Default;
|
||||
|
||||
double read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = settings.merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability;
|
||||
std::bernoulli_distribution fault(read_split_ranges_into_intersecting_and_non_intersecting_injection_probability);
|
||||
|
||||
if (read_type != ReadType::ParallelReplicas &&
|
||||
num_streams > 1 &&
|
||||
read_split_ranges_into_intersecting_and_non_intersecting_injection_probability > 0.0 &&
|
||||
fault(thread_local_rng) &&
|
||||
!isQueryWithFinal() &&
|
||||
data.merging_params.is_deleted_column.empty() &&
|
||||
!prewhere_info)
|
||||
{
|
||||
NameSet column_names_set(column_names.begin(), column_names.end());
|
||||
Names in_order_column_names_to_read(column_names);
|
||||
|
||||
/// Add columns needed to calculate the sorting expression
|
||||
for (const auto & column_name : metadata_for_reading->getColumnsRequiredForSortingKey())
|
||||
{
|
||||
if (column_names_set.contains(column_name))
|
||||
continue;
|
||||
|
||||
in_order_column_names_to_read.push_back(column_name);
|
||||
column_names_set.insert(column_name);
|
||||
}
|
||||
|
||||
auto in_order_reading_step_getter = [this, &in_order_column_names_to_read, &info](auto parts)
|
||||
{
|
||||
return this->read(
|
||||
std::move(parts),
|
||||
in_order_column_names_to_read,
|
||||
ReadType::InOrder,
|
||||
1 /* num_streams */,
|
||||
0 /* min_marks_for_concurrent_read */,
|
||||
info.use_uncompressed_cache);
|
||||
};
|
||||
|
||||
auto sorting_expr = std::make_shared<ExpressionActions>(metadata_for_reading->getSortingKey().expression->getActionsDAG().clone());
|
||||
|
||||
SplitPartsWithRangesByPrimaryKeyResult split_ranges_result = splitPartsWithRangesByPrimaryKey(
|
||||
metadata_for_reading->getPrimaryKey(),
|
||||
std::move(sorting_expr),
|
||||
std::move(parts_with_ranges),
|
||||
num_streams,
|
||||
context,
|
||||
std::move(in_order_reading_step_getter),
|
||||
true /*split_parts_ranges_into_intersecting_and_non_intersecting_final*/,
|
||||
true /*split_intersecting_parts_ranges_into_layers*/);
|
||||
|
||||
auto merging_pipes = std::move(split_ranges_result.merging_pipes);
|
||||
auto non_intersecting_parts_ranges_read_pipe = read(std::move(split_ranges_result.non_intersecting_parts_ranges),
|
||||
column_names,
|
||||
read_type,
|
||||
num_streams,
|
||||
info.min_marks_for_concurrent_read,
|
||||
info.use_uncompressed_cache);
|
||||
|
||||
if (merging_pipes.empty())
|
||||
return non_intersecting_parts_ranges_read_pipe;
|
||||
|
||||
Pipes pipes;
|
||||
pipes.resize(2);
|
||||
pipes[0] = Pipe::unitePipes(std::move(merging_pipes));
|
||||
pipes[1] = std::move(non_intersecting_parts_ranges_read_pipe);
|
||||
|
||||
auto conversion_action = ActionsDAG::makeConvertingActions(
|
||||
pipes[0].getHeader().getColumnsWithTypeAndName(),
|
||||
pipes[1].getHeader().getColumnsWithTypeAndName(),
|
||||
ActionsDAG::MatchColumnsMode::Name);
|
||||
pipes[0].addSimpleTransform(
|
||||
[conversion_action](const Block & header)
|
||||
{
|
||||
auto converting_expr = std::make_shared<ExpressionActions>(conversion_action);
|
||||
return std::make_shared<ExpressionTransform>(header, converting_expr);
|
||||
});
|
||||
return Pipe::unitePipes(std::move(pipes));
|
||||
}
|
||||
|
||||
return read(std::move(parts_with_ranges),
|
||||
column_names,
|
||||
read_type,
|
||||
|
@ -1,10 +1,8 @@
|
||||
#include "MergeTreeDataPartCompact.h"
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartWriterCompact.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Storages/MergeTree/LoadedMergeTreeDataPartInfoForReader.h>
|
||||
#include <Compression/CompressedReadBufferFromFile.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -41,21 +39,12 @@ IMergeTreeDataPart::MergeTreeReaderPtr MergeTreeDataPartCompact::getReader(
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback) const
|
||||
{
|
||||
auto read_info = std::make_shared<LoadedMergeTreeDataPartInfoForReader>(shared_from_this(), alter_conversions);
|
||||
auto * load_marks_threadpool
|
||||
= reader_settings.read_settings.load_marks_asynchronously ? &read_info->getContext()->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReaderCompact>(
|
||||
read_info,
|
||||
columns_to_read,
|
||||
virtual_fields,
|
||||
storage_snapshot,
|
||||
uncompressed_cache,
|
||||
mark_cache,
|
||||
mark_ranges,
|
||||
reader_settings,
|
||||
load_marks_threadpool,
|
||||
avg_value_size_hints,
|
||||
profile_callback);
|
||||
return std::make_unique<MergeTreeReaderCompactSingleBuffer>(
|
||||
read_info, columns_to_read, virtual_fields,
|
||||
storage_snapshot, uncompressed_cache,
|
||||
mark_cache, mark_ranges, reader_settings,
|
||||
avg_value_size_hints, profile_callback, CLOCK_MONOTONIC_COARSE);
|
||||
}
|
||||
|
||||
IMergeTreeDataPart::MergeTreeWriterPtr MergeTreeDataPartCompact::getWriter(
|
||||
|
@ -13,6 +13,11 @@ namespace DB
|
||||
class MMappedFileCache;
|
||||
using MMappedFileCachePtr = std::shared_ptr<MMappedFileCache>;
|
||||
|
||||
enum class CompactPartsReadMethod
|
||||
{
|
||||
SingleBuffer,
|
||||
MultiBuffer,
|
||||
};
|
||||
|
||||
struct MergeTreeReaderSettings
|
||||
{
|
||||
@ -25,12 +30,20 @@ struct MergeTreeReaderSettings
|
||||
bool checksum_on_read = true;
|
||||
/// True if we read in order of sorting key.
|
||||
bool read_in_order = false;
|
||||
/// Use one buffer for each column or for all columns while reading from compact.
|
||||
CompactPartsReadMethod compact_parts_read_method = CompactPartsReadMethod::SingleBuffer;
|
||||
/// True if we read stream for dictionary of LowCardinality type.
|
||||
bool is_low_cardinality_dictionary = false;
|
||||
/// True if data may be compressed by different codecs in one stream.
|
||||
bool allow_different_codecs = false;
|
||||
/// Deleted mask is applied to all reads except internal select from mutate some part columns.
|
||||
bool apply_deleted_mask = true;
|
||||
/// Put reading task in a common I/O pool, return Async state on prepare()
|
||||
bool use_asynchronous_read_from_pool = false;
|
||||
/// If PREWHERE has multiple conditions combined with AND, execute them in separate read/filtering steps.
|
||||
bool enable_multiple_prewhere_read_steps = false;
|
||||
/// If true, try to lower size of read buffer according to granule size and compressed block size.
|
||||
bool adjust_read_buffer_size = true;
|
||||
};
|
||||
|
||||
struct MergeTreeWriterSettings
|
||||
|
@ -20,14 +20,23 @@ std::unique_ptr<MergeTreeReaderStream> makeIndexReader(
|
||||
auto context = part->storage.getContext();
|
||||
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
return std::make_unique<MergeTreeReaderStream>(
|
||||
auto marks_loader = std::make_shared<MergeTreeMarksLoader>(
|
||||
std::make_shared<LoadedMergeTreeDataPartInfoForReader>(part, std::make_shared<AlterConversions>()),
|
||||
mark_cache,
|
||||
part->index_granularity_info.getMarksFilePath(index->getFileName()),
|
||||
marks_count,
|
||||
part->index_granularity_info,
|
||||
settings.save_marks_in_cache,
|
||||
settings.read_settings,
|
||||
load_marks_threadpool,
|
||||
/*num_columns_in_mark=*/ 1);
|
||||
|
||||
return std::make_unique<MergeTreeReaderStreamSingleColumn>(
|
||||
part->getDataPartStoragePtr(),
|
||||
index->getFileName(), extension, marks_count,
|
||||
all_mark_ranges,
|
||||
std::move(settings), mark_cache, uncompressed_cache,
|
||||
part->getFileSizeOrZero(index->getFileName() + extension),
|
||||
&part->index_granularity_info,
|
||||
ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE, false, load_marks_threadpool);
|
||||
all_mark_ranges, std::move(settings), uncompressed_cache,
|
||||
part->getFileSizeOrZero(index->getFileName() + extension), std::move(marks_loader),
|
||||
ReadBufferFromFileBase::ProfileCallback{}, CLOCK_MONOTONIC_COARSE);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -28,6 +28,23 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
MergeTreeMarksGetter::MergeTreeMarksGetter(MarkCache::MappedPtr marks_, size_t num_columns_in_mark_)
|
||||
: marks(std::move(marks_)), num_columns_in_mark(num_columns_in_mark_)
|
||||
{
|
||||
assert(marks);
|
||||
}
|
||||
|
||||
MarkInCompressedFile MergeTreeMarksGetter::getMark(size_t row_index, size_t column_index) const
|
||||
{
|
||||
#ifndef NDEBUG
|
||||
if (column_index >= num_columns_in_mark)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Column index: {} is out of range [0, {})", column_index, num_columns_in_mark);
|
||||
#endif
|
||||
|
||||
return marks->get(row_index * num_columns_in_mark + column_index);
|
||||
}
|
||||
|
||||
MergeTreeMarksLoader::MergeTreeMarksLoader(
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
MarkCache * mark_cache_,
|
||||
@ -37,58 +54,49 @@ MergeTreeMarksLoader::MergeTreeMarksLoader(
|
||||
bool save_marks_in_cache_,
|
||||
const ReadSettings & read_settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
size_t columns_in_mark_)
|
||||
size_t num_columns_in_mark_)
|
||||
: data_part_reader(data_part_reader_)
|
||||
, mark_cache(mark_cache_)
|
||||
, mrk_path(mrk_path_)
|
||||
, marks_count(marks_count_)
|
||||
, index_granularity_info(index_granularity_info_)
|
||||
, save_marks_in_cache(save_marks_in_cache_)
|
||||
, columns_in_mark(columns_in_mark_)
|
||||
, read_settings(read_settings_)
|
||||
, num_columns_in_mark(num_columns_in_mark_)
|
||||
, load_marks_threadpool(load_marks_threadpool_)
|
||||
{
|
||||
if (load_marks_threadpool)
|
||||
{
|
||||
future = loadMarksAsync();
|
||||
}
|
||||
}
|
||||
|
||||
MergeTreeMarksLoader::~MergeTreeMarksLoader()
|
||||
{
|
||||
if (future.valid())
|
||||
{
|
||||
future.wait();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
MarkInCompressedFile MergeTreeMarksLoader::getMark(size_t row_index, size_t column_index)
|
||||
MergeTreeMarksGetterPtr MergeTreeMarksLoader::loadMarks()
|
||||
{
|
||||
if (!marks)
|
||||
std::lock_guard lock(load_mutex);
|
||||
|
||||
if (marks)
|
||||
return std::make_unique<MergeTreeMarksGetter>(marks, num_columns_in_mark);
|
||||
|
||||
Stopwatch watch(CLOCK_MONOTONIC);
|
||||
|
||||
if (future.valid())
|
||||
{
|
||||
Stopwatch watch(CLOCK_MONOTONIC);
|
||||
|
||||
if (future.valid())
|
||||
{
|
||||
marks = future.get();
|
||||
future = {};
|
||||
}
|
||||
else
|
||||
{
|
||||
marks = loadMarks();
|
||||
}
|
||||
|
||||
watch.stop();
|
||||
ProfileEvents::increment(ProfileEvents::WaitMarksLoadMicroseconds, watch.elapsedMicroseconds());
|
||||
marks = future.get();
|
||||
future = {};
|
||||
}
|
||||
else
|
||||
{
|
||||
marks = loadMarksSync();
|
||||
}
|
||||
|
||||
#ifndef NDEBUG
|
||||
if (column_index >= columns_in_mark)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Column index: {} is out of range [0, {})", column_index, columns_in_mark);
|
||||
#endif
|
||||
|
||||
return marks->get(row_index * columns_in_mark + column_index);
|
||||
watch.stop();
|
||||
ProfileEvents::increment(ProfileEvents::WaitMarksLoadMicroseconds, watch.elapsedMicroseconds());
|
||||
return std::make_unique<MergeTreeMarksGetter>(marks, num_columns_in_mark);
|
||||
}
|
||||
|
||||
|
||||
@ -100,12 +108,12 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
|
||||
auto data_part_storage = data_part_reader->getDataPartStorage();
|
||||
|
||||
size_t file_size = data_part_storage->getFileSize(mrk_path);
|
||||
size_t mark_size = index_granularity_info.getMarkSizeInBytes(columns_in_mark);
|
||||
size_t mark_size = index_granularity_info.getMarkSizeInBytes(num_columns_in_mark);
|
||||
size_t expected_uncompressed_size = mark_size * marks_count;
|
||||
|
||||
// We first read the marks into a temporary simple array, then compress them into a more compact
|
||||
// representation.
|
||||
PODArray<MarkInCompressedFile> plain_marks(marks_count * columns_in_mark); // temporary
|
||||
PODArray<MarkInCompressedFile> plain_marks(marks_count * num_columns_in_mark); // temporary
|
||||
auto full_mark_path = std::string(fs::path(data_part_storage->getFullPath()) / mrk_path);
|
||||
|
||||
if (file_size == 0 && marks_count != 0)
|
||||
@ -159,7 +167,7 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
|
||||
|
||||
size_t granularity;
|
||||
reader->readStrict(
|
||||
reinterpret_cast<char *>(plain_marks.data() + i * columns_in_mark), columns_in_mark * sizeof(MarkInCompressedFile));
|
||||
reinterpret_cast<char *>(plain_marks.data() + i * num_columns_in_mark), num_columns_in_mark * sizeof(MarkInCompressedFile));
|
||||
readBinaryLittleEndian(granularity, *reader);
|
||||
}
|
||||
|
||||
@ -182,13 +190,13 @@ MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksImpl()
|
||||
|
||||
auto res = std::make_shared<MarksInCompressedFile>(plain_marks);
|
||||
|
||||
ProfileEvents::increment(ProfileEvents::LoadedMarksCount, marks_count * columns_in_mark);
|
||||
ProfileEvents::increment(ProfileEvents::LoadedMarksCount, marks_count * num_columns_in_mark);
|
||||
ProfileEvents::increment(ProfileEvents::LoadedMarksMemoryBytes, res->approximateMemoryUsage());
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
MarkCache::MappedPtr MergeTreeMarksLoader::loadMarks()
|
||||
MarkCache::MappedPtr MergeTreeMarksLoader::loadMarksSync()
|
||||
{
|
||||
MarkCache::MappedPtr loaded_marks;
|
||||
|
||||
@ -227,7 +235,7 @@ std::future<MarkCache::MappedPtr> MergeTreeMarksLoader::loadMarksAsync()
|
||||
[this]() -> MarkCache::MappedPtr
|
||||
{
|
||||
ProfileEvents::increment(ProfileEvents::BackgroundLoadingMarksTasks);
|
||||
return loadMarks();
|
||||
return loadMarksSync();
|
||||
},
|
||||
*load_marks_threadpool,
|
||||
"LoadMarksThread");
|
||||
|
@ -10,13 +10,33 @@ namespace DB
|
||||
{
|
||||
|
||||
struct MergeTreeIndexGranularityInfo;
|
||||
using MarksPtr = MarkCache::MappedPtr;
|
||||
class Threadpool;
|
||||
|
||||
/// Class that helps to get marks by indexes.
|
||||
/// Always immutable and thread safe.
|
||||
/// Marks can be shared between several threads
|
||||
/// that read columns from the same file.
|
||||
class MergeTreeMarksGetter
|
||||
{
|
||||
public:
|
||||
MergeTreeMarksGetter(MarkCache::MappedPtr marks_, size_t num_columns_in_mark_);
|
||||
|
||||
MarkInCompressedFile getMark(size_t row_index, size_t column_index) const;
|
||||
size_t getNumColumns() const { return num_columns_in_mark; }
|
||||
|
||||
private:
|
||||
const MarkCache::MappedPtr marks;
|
||||
const size_t num_columns_in_mark;
|
||||
};
|
||||
|
||||
using MergeTreeMarksGetterPtr = std::unique_ptr<const MergeTreeMarksGetter>;
|
||||
|
||||
/// Class that helps to load marks on demand.
|
||||
/// Thread safe, but locks while loading marks.
|
||||
class MergeTreeMarksLoader
|
||||
{
|
||||
public:
|
||||
using MarksPtr = MarkCache::MappedPtr;
|
||||
|
||||
MergeTreeMarksLoader(
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
MarkCache * mark_cache_,
|
||||
@ -26,24 +46,27 @@ public:
|
||||
bool save_marks_in_cache_,
|
||||
const ReadSettings & read_settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
size_t columns_in_mark_ = 1);
|
||||
size_t num_columns_in_mark_);
|
||||
|
||||
~MergeTreeMarksLoader();
|
||||
|
||||
MarkInCompressedFile getMark(size_t row_index, size_t column_index = 0);
|
||||
MergeTreeMarksGetterPtr loadMarks();
|
||||
size_t getNumColumns() const { return num_columns_in_mark; }
|
||||
|
||||
private:
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader;
|
||||
MarkCache * mark_cache = nullptr;
|
||||
String mrk_path;
|
||||
size_t marks_count;
|
||||
const MergeTreeDataPartInfoForReaderPtr data_part_reader;
|
||||
MarkCache * const mark_cache;
|
||||
const String mrk_path;
|
||||
const size_t marks_count;
|
||||
const MergeTreeIndexGranularityInfo & index_granularity_info;
|
||||
bool save_marks_in_cache = false;
|
||||
size_t columns_in_mark;
|
||||
MarkCache::MappedPtr marks;
|
||||
ReadSettings read_settings;
|
||||
const bool save_marks_in_cache;
|
||||
const ReadSettings read_settings;
|
||||
const size_t num_columns_in_mark;
|
||||
|
||||
MarkCache::MappedPtr loadMarks();
|
||||
std::mutex load_mutex;
|
||||
MarkCache::MappedPtr marks;
|
||||
|
||||
MarkCache::MappedPtr loadMarksSync();
|
||||
std::future<MarkCache::MappedPtr> loadMarksAsync();
|
||||
MarkCache::MappedPtr loadMarksImpl();
|
||||
|
||||
@ -51,4 +74,6 @@ private:
|
||||
ThreadPool * load_marks_threadpool;
|
||||
};
|
||||
|
||||
using MergeTreeMarksLoaderPtr = std::shared_ptr<MergeTreeMarksLoader>;
|
||||
|
||||
}
|
||||
|
@ -10,10 +10,8 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
|
||||
MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_info_for_read_,
|
||||
NamesAndTypesList columns_,
|
||||
@ -23,7 +21,6 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
MergeTreeReaderSettings settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
ValueSizeMap avg_value_size_hints_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_)
|
||||
@ -37,91 +34,22 @@ MergeTreeReaderCompact::MergeTreeReaderCompact(
|
||||
mark_ranges_,
|
||||
settings_,
|
||||
avg_value_size_hints_)
|
||||
, marks_loader(
|
||||
data_part_info_for_read_,
|
||||
mark_cache,
|
||||
data_part_info_for_read_->getIndexGranularityInfo().getMarksFilePath(MergeTreeDataPartCompact::DATA_FILE_NAME),
|
||||
data_part_info_for_read_->getMarksCount(),
|
||||
data_part_info_for_read_->getIndexGranularityInfo(),
|
||||
settings.save_marks_in_cache,
|
||||
settings.read_settings,
|
||||
load_marks_threadpool_,
|
||||
data_part_info_for_read_->getColumns().size())
|
||||
, marks_loader(std::make_shared<MergeTreeMarksLoader>(
|
||||
data_part_info_for_read_,
|
||||
mark_cache,
|
||||
data_part_info_for_read_->getIndexGranularityInfo().getMarksFilePath(MergeTreeDataPartCompact::DATA_FILE_NAME),
|
||||
data_part_info_for_read_->getMarksCount(),
|
||||
data_part_info_for_read_->getIndexGranularityInfo(),
|
||||
settings.save_marks_in_cache,
|
||||
settings.read_settings,
|
||||
settings_.read_settings.load_marks_asynchronously
|
||||
? &data_part_info_for_read_->getContext()->getLoadMarksThreadpool() : nullptr,
|
||||
data_part_info_for_read_->getColumns().size()))
|
||||
, profile_callback(profile_callback_)
|
||||
, clock_type(clock_type_)
|
||||
{
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompact::initialize()
|
||||
{
|
||||
try
|
||||
{
|
||||
fillColumnPositions();
|
||||
|
||||
/// Do not use max_read_buffer_size, but try to lower buffer size with maximal size of granule to avoid reading much data.
|
||||
auto buffer_size = getReadBufferSize(*data_part_info_for_read, marks_loader, column_positions, all_mark_ranges);
|
||||
if (buffer_size)
|
||||
settings.read_settings = settings.read_settings.adjustBufferSize(buffer_size);
|
||||
|
||||
if (!settings.read_settings.local_fs_buffer_size || !settings.read_settings.remote_fs_buffer_size)
|
||||
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA, "Cannot read to empty buffer.");
|
||||
|
||||
const String path = MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION;
|
||||
auto data_part_storage = data_part_info_for_read->getDataPartStorage();
|
||||
|
||||
if (uncompressed_cache)
|
||||
{
|
||||
auto buffer = std::make_unique<CachedCompressedReadBuffer>(
|
||||
std::string(fs::path(data_part_storage->getFullPath()) / path),
|
||||
[this, path, data_part_storage]()
|
||||
{
|
||||
return data_part_storage->readFile(
|
||||
path,
|
||||
settings.read_settings,
|
||||
std::nullopt, std::nullopt);
|
||||
},
|
||||
uncompressed_cache,
|
||||
/* allow_different_codecs = */ true);
|
||||
|
||||
if (profile_callback)
|
||||
buffer->setProfileCallback(profile_callback, clock_type);
|
||||
|
||||
if (!settings.checksum_on_read)
|
||||
buffer->disableChecksumming();
|
||||
|
||||
cached_buffer = std::move(buffer);
|
||||
data_buffer = cached_buffer.get();
|
||||
compressed_data_buffer = cached_buffer.get();
|
||||
}
|
||||
else
|
||||
{
|
||||
auto buffer =
|
||||
std::make_unique<CompressedReadBufferFromFile>(
|
||||
data_part_storage->readFile(
|
||||
path,
|
||||
settings.read_settings,
|
||||
std::nullopt, std::nullopt),
|
||||
/* allow_different_codecs = */ true);
|
||||
|
||||
if (profile_callback)
|
||||
buffer->setProfileCallback(profile_callback, clock_type);
|
||||
|
||||
if (!settings.checksum_on_read)
|
||||
buffer->disableChecksumming();
|
||||
|
||||
non_cached_buffer = std::move(buffer);
|
||||
data_buffer = non_cached_buffer.get();
|
||||
compressed_data_buffer = non_cached_buffer.get();
|
||||
}
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompact::fillColumnPositions()
|
||||
{
|
||||
size_t columns_num = columns_to_read.size();
|
||||
@ -150,31 +78,7 @@ void MergeTreeReaderCompact::fillColumnPositions()
|
||||
/// we have to read its offsets if they exist.
|
||||
if (!position && is_array)
|
||||
{
|
||||
NameAndTypePair column_to_read_with_subcolumns = column_to_read;
|
||||
auto [name_in_storage, subcolumn_name] = Nested::splitName(column_to_read.name);
|
||||
|
||||
/// If it is a part of Nested, we need to get the column from
|
||||
/// storage metadata which is converted to Nested type with subcolumns.
|
||||
/// It is required for proper counting of shared streams.
|
||||
if (!subcolumn_name.empty())
|
||||
{
|
||||
/// If column is renamed get the new name from storage metadata.
|
||||
if (alter_conversions->columnHasNewName(name_in_storage))
|
||||
name_in_storage = alter_conversions->getColumnNewName(name_in_storage);
|
||||
|
||||
if (!storage_columns_with_collected_nested)
|
||||
{
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects();
|
||||
auto storage_columns_list = Nested::collect(storage_snapshot->getColumns(options));
|
||||
storage_columns_with_collected_nested = ColumnsDescription(std::move(storage_columns_list));
|
||||
}
|
||||
|
||||
column_to_read_with_subcolumns = storage_columns_with_collected_nested
|
||||
->getColumnOrSubcolumn(
|
||||
GetColumnsOptions::All,
|
||||
Nested::concatenateName(name_in_storage, subcolumn_name));
|
||||
}
|
||||
|
||||
auto column_to_read_with_subcolumns = getColumnConvertedToSubcolumnOfNested(column_to_read);
|
||||
auto name_level_for_offsets = findColumnForOffsets(column_to_read_with_subcolumns);
|
||||
|
||||
if (name_level_for_offsets.has_value())
|
||||
@ -191,189 +95,60 @@ void MergeTreeReaderCompact::fillColumnPositions()
|
||||
}
|
||||
}
|
||||
|
||||
size_t MergeTreeReaderCompact::readRows(
|
||||
size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns)
|
||||
NameAndTypePair MergeTreeReaderCompact::getColumnConvertedToSubcolumnOfNested(const NameAndTypePair & column)
|
||||
{
|
||||
if (!initialized)
|
||||
if (!isArray(column.type))
|
||||
return column;
|
||||
|
||||
/// If it is a part of Nested, we need to get the column from
|
||||
/// storage metadata which is converted to Nested type with subcolumns.
|
||||
/// It is required for proper counting of shared streams.
|
||||
auto [name_in_storage, subcolumn_name] = Nested::splitName(column.name);
|
||||
|
||||
if (subcolumn_name.empty())
|
||||
return column;
|
||||
|
||||
/// If column is renamed get the new name from storage metadata.
|
||||
if (alter_conversions->columnHasNewName(name_in_storage))
|
||||
name_in_storage = alter_conversions->getColumnNewName(name_in_storage);
|
||||
|
||||
if (!storage_columns_with_collected_nested)
|
||||
{
|
||||
initialize();
|
||||
initialized = true;
|
||||
auto options = GetColumnsOptions(GetColumnsOptions::AllPhysical).withExtendedObjects();
|
||||
auto storage_columns_list = Nested::collect(storage_snapshot->getColumns(options));
|
||||
storage_columns_with_collected_nested = ColumnsDescription(std::move(storage_columns_list));
|
||||
}
|
||||
|
||||
if (continue_reading)
|
||||
from_mark = next_mark;
|
||||
|
||||
size_t read_rows = 0;
|
||||
size_t num_columns = columns_to_read.size();
|
||||
checkNumberOfColumns(num_columns);
|
||||
|
||||
MutableColumns mutable_columns(num_columns);
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
{
|
||||
if (column_positions[i] && res_columns[i] == nullptr)
|
||||
res_columns[i] = columns_to_read[i].type->createColumn(*serializations[i]);
|
||||
}
|
||||
|
||||
while (read_rows < max_rows_to_read)
|
||||
{
|
||||
size_t rows_to_read = data_part_info_for_read->getIndexGranularity().getMarkRows(from_mark);
|
||||
|
||||
/// If we need to read multiple subcolumns from a single column in storage,
|
||||
/// we will read it this column only once and then reuse to extract all subcolumns.
|
||||
std::unordered_map<String, ColumnPtr> columns_cache_for_subcolumns;
|
||||
|
||||
for (size_t pos = 0; pos < num_columns; ++pos)
|
||||
{
|
||||
if (!res_columns[pos])
|
||||
continue;
|
||||
|
||||
try
|
||||
{
|
||||
auto & column = res_columns[pos];
|
||||
size_t column_size_before_reading = column->size();
|
||||
|
||||
readData(columns_to_read[pos], column, from_mark, current_task_last_mark, *column_positions[pos], rows_to_read, columns_for_offsets[pos], columns_cache_for_subcolumns);
|
||||
|
||||
size_t read_rows_in_column = column->size() - column_size_before_reading;
|
||||
if (read_rows_in_column != rows_to_read)
|
||||
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA,
|
||||
"Cannot read all data in MergeTreeReaderCompact. Rows read: {}. Rows expected: {}.",
|
||||
read_rows_in_column, rows_to_read);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
|
||||
/// Better diagnostics.
|
||||
try
|
||||
{
|
||||
rethrow_exception(std::current_exception());
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
|
||||
}
|
||||
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
++from_mark;
|
||||
read_rows += rows_to_read;
|
||||
}
|
||||
|
||||
next_mark = from_mark;
|
||||
|
||||
return read_rows;
|
||||
return storage_columns_with_collected_nested->getColumnOrSubcolumn(
|
||||
GetColumnsOptions::All,
|
||||
Nested::concatenateName(name_in_storage, subcolumn_name));
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompact::readData(
|
||||
const NameAndTypePair & name_and_type, ColumnPtr & column,
|
||||
size_t from_mark, size_t current_task_last_mark, size_t column_position, size_t rows_to_read,
|
||||
ColumnNameLevel name_level_for_offsets, std::unordered_map<String, ColumnPtr> & columns_cache_for_subcolumns)
|
||||
const NameAndTypePair & name_and_type,
|
||||
ColumnPtr & column,
|
||||
size_t rows_to_read,
|
||||
const InputStreamGetter & getter)
|
||||
{
|
||||
const auto & [name, type] = name_and_type;
|
||||
std::optional<NameAndTypePair> column_for_offsets;
|
||||
|
||||
if (name_level_for_offsets.has_value())
|
||||
try
|
||||
{
|
||||
const auto & part_columns = data_part_info_for_read->getColumnsDescription();
|
||||
column_for_offsets = part_columns.getPhysical(name_level_for_offsets->first);
|
||||
}
|
||||
const auto [name, type] = name_and_type;
|
||||
size_t column_size_before_reading = column->size();
|
||||
|
||||
adjustUpperBound(current_task_last_mark); /// Must go before seek.
|
||||
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
|
||||
deserialize_settings.getter = getter;
|
||||
deserialize_settings.avg_value_size_hint = avg_value_size_hints[name];
|
||||
|
||||
if (!isContinuousReading(from_mark, column_position))
|
||||
seekToMark(from_mark, column_position);
|
||||
|
||||
/// If we read only offsets we have to read prefix anyway
|
||||
/// to preserve correctness of serialization.
|
||||
auto buffer_getter_for_prefix = [&](const auto &) -> ReadBuffer *
|
||||
{
|
||||
return data_buffer;
|
||||
};
|
||||
|
||||
auto buffer_getter = [&](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer *
|
||||
{
|
||||
/// Offset stream from another column could be read, in case of current
|
||||
/// column does not exists (see findColumnForOffsets() in
|
||||
/// MergeTreeReaderCompact::fillColumnPositions())
|
||||
if (name_level_for_offsets.has_value())
|
||||
if (name_and_type.isSubcolumn())
|
||||
{
|
||||
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
|
||||
if (!is_offsets)
|
||||
return nullptr;
|
||||
const auto & type_in_storage = name_and_type.getTypeInStorage();
|
||||
const auto & name_in_storage = name_and_type.getNameInStorage();
|
||||
|
||||
/// Offset stream can be read only from columns of current level or
|
||||
/// below (since it is OK to read all parent streams from the
|
||||
/// alternative).
|
||||
///
|
||||
/// Consider the following columns in nested "root":
|
||||
/// - root.array Array(UInt8) - exists
|
||||
/// - root.nested_array Array(Array(UInt8)) - does not exists (only_offsets_level=1)
|
||||
///
|
||||
/// For root.nested_array it will try to read multiple streams:
|
||||
/// - offsets (substream_path = {ArraySizes})
|
||||
/// OK
|
||||
/// - root.nested_array elements (substream_path = {ArrayElements, ArraySizes})
|
||||
/// NOT OK - cannot use root.array offsets stream for this
|
||||
///
|
||||
/// Here only_offsets_level is the level of the alternative stream,
|
||||
/// and substream_path.size() is the level of the current stream.
|
||||
if (name_level_for_offsets->second < ISerialization::getArrayLevel(substream_path))
|
||||
return nullptr;
|
||||
}
|
||||
auto serialization = getSerializationInPart({name_in_storage, type_in_storage});
|
||||
ColumnPtr temp_column = type_in_storage->createColumn(*serialization);
|
||||
|
||||
return data_buffer;
|
||||
};
|
||||
|
||||
ISerialization::DeserializeBinaryBulkStatePtr state;
|
||||
ISerialization::DeserializeBinaryBulkStatePtr state_for_prefix;
|
||||
|
||||
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
|
||||
deserialize_settings.avg_value_size_hint = avg_value_size_hints[name];
|
||||
bool columns_cache_was_used = false;
|
||||
|
||||
if (name_and_type.isSubcolumn())
|
||||
{
|
||||
NameAndTypePair name_type_in_storage{name_and_type.getNameInStorage(), name_and_type.getTypeInStorage()};
|
||||
ColumnPtr temp_column;
|
||||
|
||||
auto it = columns_cache_for_subcolumns.find(name_type_in_storage.name);
|
||||
if (!column_for_offsets && it != columns_cache_for_subcolumns.end())
|
||||
{
|
||||
temp_column = it->second;
|
||||
auto subcolumn = name_type_in_storage.type->getSubcolumn(name_and_type.getSubcolumnName(), temp_column);
|
||||
if (column->empty())
|
||||
column = IColumn::mutate(subcolumn);
|
||||
else
|
||||
column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size());
|
||||
|
||||
columns_cache_was_used = true;
|
||||
}
|
||||
else
|
||||
{
|
||||
/// In case of reading only offset use the correct serialization for reading of the prefix
|
||||
auto serialization = getSerializationInPart(name_type_in_storage);
|
||||
temp_column = name_type_in_storage.type->createColumn(*serialization);
|
||||
|
||||
if (column_for_offsets)
|
||||
{
|
||||
auto serialization_for_prefix = getSerializationInPart(*column_for_offsets);
|
||||
|
||||
deserialize_settings.getter = buffer_getter_for_prefix;
|
||||
serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix);
|
||||
}
|
||||
|
||||
deserialize_settings.getter = buffer_getter;
|
||||
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
|
||||
serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, state, nullptr);
|
||||
|
||||
if (!column_for_offsets)
|
||||
columns_cache_for_subcolumns[name_type_in_storage.name] = temp_column;
|
||||
|
||||
auto subcolumn = name_type_in_storage.type->getSubcolumn(name_and_type.getSubcolumnName(), temp_column);
|
||||
serialization->deserializeBinaryBulkWithMultipleStreams(temp_column, rows_to_read, deserialize_settings, deserialize_binary_bulk_state_map[name], nullptr);
|
||||
auto subcolumn = type_in_storage->getSubcolumn(name_and_type.getSubcolumnName(), temp_column);
|
||||
|
||||
/// TODO: Avoid extra copying.
|
||||
if (column->empty())
|
||||
@ -381,185 +156,98 @@ void MergeTreeReaderCompact::readData(
|
||||
else
|
||||
column->assumeMutable()->insertRangeFrom(*subcolumn, 0, subcolumn->size());
|
||||
}
|
||||
}
|
||||
else
|
||||
{
|
||||
/// In case of reading only offsets use the correct serialization for reading the prefix
|
||||
auto serialization = getSerializationInPart(name_and_type);
|
||||
|
||||
if (column_for_offsets)
|
||||
else
|
||||
{
|
||||
auto serialization_for_prefix = getSerializationInPart(*column_for_offsets);
|
||||
|
||||
deserialize_settings.getter = buffer_getter_for_prefix;
|
||||
serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix);
|
||||
auto serialization = getSerializationInPart(name_and_type);
|
||||
serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, deserialize_binary_bulk_state_map[name], nullptr);
|
||||
}
|
||||
|
||||
deserialize_settings.getter = buffer_getter;
|
||||
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, state);
|
||||
serialization->deserializeBinaryBulkWithMultipleStreams(column, rows_to_read, deserialize_settings, state, nullptr);
|
||||
}
|
||||
|
||||
/// The buffer is left in inconsistent state after reading single offsets or using columns cache during subcolumns reading.
|
||||
if (name_level_for_offsets.has_value() || columns_cache_was_used)
|
||||
last_read_granule.reset();
|
||||
else
|
||||
last_read_granule.emplace(from_mark, column_position);
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompact::prefetchBeginOfRange(Priority priority)
|
||||
try
|
||||
{
|
||||
if (!initialized)
|
||||
{
|
||||
initialize();
|
||||
initialized = true;
|
||||
}
|
||||
|
||||
adjustUpperBound(all_mark_ranges.back().end);
|
||||
seekToMark(all_mark_ranges.front().begin, 0);
|
||||
data_buffer->prefetch(priority);
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompact::seekToMark(size_t row_index, size_t column_index)
|
||||
{
|
||||
MarkInCompressedFile mark = marks_loader.getMark(row_index, column_index);
|
||||
try
|
||||
{
|
||||
compressed_data_buffer->seek(mark.offset_in_compressed_file, mark.offset_in_decompressed_block);
|
||||
size_t read_rows_in_column = column->size() - column_size_before_reading;
|
||||
if (read_rows_in_column != rows_to_read)
|
||||
throw Exception(ErrorCodes::CANNOT_READ_ALL_DATA,
|
||||
"Cannot read all data in MergeTreeReaderCompact. Rows read: {}. Rows expected: {}.",
|
||||
read_rows_in_column, rows_to_read);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
/// Better diagnostics.
|
||||
if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND)
|
||||
e.addMessage("(while seeking to mark (" + toString(row_index) + ", " + toString(column_index) + ")");
|
||||
|
||||
e.addMessage("(while reading column " + name_and_type.name + ")");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompact::adjustUpperBound(size_t last_mark)
|
||||
|
||||
void MergeTreeReaderCompact::readPrefix(
|
||||
const NameAndTypePair & name_and_type,
|
||||
const InputStreamGetter & buffer_getter,
|
||||
const InputStreamGetter & buffer_getter_for_prefix,
|
||||
const ColumnNameLevel & name_level_for_offsets)
|
||||
{
|
||||
size_t right_offset = 0;
|
||||
if (last_mark < data_part_info_for_read->getMarksCount()) /// Otherwise read until the end of file
|
||||
right_offset = marks_loader.getMark(last_mark).offset_in_compressed_file;
|
||||
|
||||
if (right_offset == 0)
|
||||
try
|
||||
{
|
||||
/// If already reading till the end of file.
|
||||
if (last_right_offset && *last_right_offset == 0)
|
||||
return;
|
||||
ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
|
||||
|
||||
last_right_offset = 0; // Zero value means the end of file.
|
||||
data_buffer->setReadUntilEnd();
|
||||
if (name_level_for_offsets.has_value())
|
||||
{
|
||||
const auto & part_columns = data_part_info_for_read->getColumnsDescription();
|
||||
auto column_for_offsets = part_columns.getPhysical(name_level_for_offsets->first);
|
||||
|
||||
auto serialization_for_prefix = getSerializationInPart(column_for_offsets);
|
||||
deserialize_settings.getter = buffer_getter_for_prefix;
|
||||
ISerialization::DeserializeBinaryBulkStatePtr state_for_prefix;
|
||||
|
||||
serialization_for_prefix->deserializeBinaryBulkStatePrefix(deserialize_settings, state_for_prefix);
|
||||
}
|
||||
|
||||
SerializationPtr serialization;
|
||||
if (name_and_type.isSubcolumn())
|
||||
serialization = getSerializationInPart({name_and_type.getNameInStorage(), name_and_type.getTypeInStorage()});
|
||||
else
|
||||
serialization = getSerializationInPart(name_and_type);
|
||||
|
||||
|
||||
deserialize_settings.getter = buffer_getter;
|
||||
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name_and_type.name]);
|
||||
}
|
||||
else
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (last_right_offset && right_offset <= last_right_offset.value())
|
||||
return;
|
||||
|
||||
last_right_offset = right_offset;
|
||||
data_buffer->setReadUntilPosition(right_offset);
|
||||
e.addMessage("(while reading column " + name_and_type.name + ")");
|
||||
throw;
|
||||
}
|
||||
}
|
||||
|
||||
bool MergeTreeReaderCompact::isContinuousReading(size_t mark, size_t column_position)
|
||||
void MergeTreeReaderCompact::createColumnsForReading(Columns & res_columns) const
|
||||
{
|
||||
if (!last_read_granule)
|
||||
for (size_t i = 0; i < columns_to_read.size(); ++i)
|
||||
{
|
||||
if (column_positions[i] && res_columns[i] == nullptr)
|
||||
res_columns[i] = columns_to_read[i].type->createColumn(*serializations[i]);
|
||||
}
|
||||
}
|
||||
|
||||
bool MergeTreeReaderCompact::needSkipStream(size_t column_pos, const ISerialization::SubstreamPath & substream) const
|
||||
{
|
||||
/// Offset stream can be read only from columns of current level or
|
||||
/// below (since it is OK to read all parent streams from the
|
||||
/// alternative).
|
||||
///
|
||||
/// Consider the following columns in nested "root":
|
||||
/// - root.array Array(UInt8) - exists
|
||||
/// - root.nested_array Array(Array(UInt8)) - does not exists (only_offsets_level=1)
|
||||
///
|
||||
/// For root.nested_array it will try to read multiple streams:
|
||||
/// - offsets (substream_path = {ArraySizes})
|
||||
/// OK
|
||||
/// - root.nested_array elements (substream_path = {ArrayElements, ArraySizes})
|
||||
/// NOT OK - cannot use root.array offsets stream for this
|
||||
///
|
||||
/// Here only_offsets_level is the level of the alternative stream,
|
||||
/// and substream_path.size() is the level of the current stream.
|
||||
|
||||
if (!columns_for_offsets[column_pos])
|
||||
return false;
|
||||
const auto & [last_mark, last_column] = *last_read_granule;
|
||||
return (mark == last_mark && column_position == last_column + 1)
|
||||
|| (mark == last_mark + 1 && column_position == 0 && last_column == data_part_info_for_read->getColumns().size() - 1);
|
||||
}
|
||||
|
||||
namespace
|
||||
{
|
||||
|
||||
/// A simple class that helps to iterate over 2-dim marks of compact parts.
|
||||
class MarksCounter
|
||||
{
|
||||
public:
|
||||
MarksCounter(size_t rows_num_, size_t columns_num_)
|
||||
: rows_num(rows_num_), columns_num(columns_num_) {}
|
||||
|
||||
struct Iterator
|
||||
{
|
||||
size_t row;
|
||||
size_t column;
|
||||
MarksCounter * counter;
|
||||
|
||||
Iterator(size_t row_, size_t column_, MarksCounter * counter_)
|
||||
: row(row_), column(column_), counter(counter_) {}
|
||||
|
||||
Iterator operator++()
|
||||
{
|
||||
if (column + 1 == counter->columns_num)
|
||||
{
|
||||
++row;
|
||||
column = 0;
|
||||
}
|
||||
else
|
||||
{
|
||||
++column;
|
||||
}
|
||||
|
||||
return *this;
|
||||
}
|
||||
|
||||
bool operator==(const Iterator & other) const { return row == other.row && column == other.column; }
|
||||
bool operator!=(const Iterator & other) const { return !(*this == other); }
|
||||
};
|
||||
|
||||
Iterator get(size_t row, size_t column) { return Iterator(row, column, this); }
|
||||
Iterator end() { return get(rows_num, 0); }
|
||||
|
||||
private:
|
||||
size_t rows_num;
|
||||
size_t columns_num;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
size_t MergeTreeReaderCompact::getReadBufferSize(
|
||||
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
|
||||
MergeTreeMarksLoader & marks_loader,
|
||||
const ColumnPositions & column_positions,
|
||||
const MarkRanges & mark_ranges)
|
||||
{
|
||||
size_t buffer_size = 0;
|
||||
size_t columns_num = column_positions.size();
|
||||
size_t file_size = data_part_info_for_reader.getFileSizeOrZero(MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION);
|
||||
|
||||
MarksCounter counter(data_part_info_for_reader.getMarksCount(), data_part_info_for_reader.getColumns().size());
|
||||
|
||||
for (const auto & mark_range : mark_ranges)
|
||||
{
|
||||
for (size_t mark = mark_range.begin; mark < mark_range.end; ++mark)
|
||||
{
|
||||
for (size_t i = 0; i < columns_num; ++i)
|
||||
{
|
||||
if (!column_positions[i])
|
||||
continue;
|
||||
|
||||
auto it = counter.get(mark, *column_positions[i]);
|
||||
size_t cur_offset = marks_loader.getMark(it.row, it.column).offset_in_compressed_file;
|
||||
|
||||
while (it != counter.end() && cur_offset == marks_loader.getMark(it.row, it.column).offset_in_compressed_file)
|
||||
++it;
|
||||
|
||||
size_t next_offset = (it == counter.end() ? file_size : marks_loader.getMark(it.row, it.column).offset_in_compressed_file);
|
||||
buffer_size = std::max(buffer_size, next_offset - cur_offset);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return buffer_size;
|
||||
bool is_offsets = !substream.empty() && substream.back().type == ISerialization::Substream::ArraySizes;
|
||||
return !is_offsets || columns_for_offsets[column_pos]->second < ISerialization::getArrayLevel(substream);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <Core/NamesAndTypes.h>
|
||||
#include <Storages/MergeTree/IMergeTreeReader.h>
|
||||
#include <IO/ReadBufferFromFileBase.h>
|
||||
|
||||
#include <DataTypes/Serializations/ISerialization.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -14,7 +14,7 @@ using DataPartCompactPtr = std::shared_ptr<const MergeTreeDataPartCompact>;
|
||||
class IMergeTreeDataPart;
|
||||
using DataPartPtr = std::shared_ptr<const IMergeTreeDataPart>;
|
||||
|
||||
/// Reader for compact parts
|
||||
/// Base class of readers for compact parts.
|
||||
class MergeTreeReaderCompact : public IMergeTreeReader
|
||||
{
|
||||
public:
|
||||
@ -27,31 +27,38 @@ public:
|
||||
MarkCache * mark_cache_,
|
||||
MarkRanges mark_ranges_,
|
||||
MergeTreeReaderSettings settings_,
|
||||
ThreadPool * load_marks_threadpool_,
|
||||
ValueSizeMap avg_value_size_hints_ = {},
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_ = {},
|
||||
clockid_t clock_type_ = CLOCK_MONOTONIC_COARSE);
|
||||
ValueSizeMap avg_value_size_hints_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_);
|
||||
|
||||
/// Return the number of rows has been read or zero if there is no columns to read.
|
||||
/// If continue_reading is true, continue reading from last state, otherwise seek to from_mark
|
||||
size_t readRows(size_t from_mark, size_t current_task_last_mark,
|
||||
bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override;
|
||||
bool canReadIncompleteGranules() const final { return false; }
|
||||
|
||||
bool canReadIncompleteGranules() const override { return false; }
|
||||
|
||||
void prefetchBeginOfRange(Priority priority) override;
|
||||
|
||||
private:
|
||||
bool isContinuousReading(size_t mark, size_t column_position);
|
||||
protected:
|
||||
void fillColumnPositions();
|
||||
void initialize();
|
||||
NameAndTypePair getColumnConvertedToSubcolumnOfNested(const NameAndTypePair & column);
|
||||
|
||||
ReadBuffer * data_buffer;
|
||||
CompressedReadBufferBase * compressed_data_buffer;
|
||||
std::unique_ptr<CachedCompressedReadBuffer> cached_buffer;
|
||||
std::unique_ptr<CompressedReadBufferFromFile> non_cached_buffer;
|
||||
using InputStreamGetter = ISerialization::InputStreamGetter;
|
||||
|
||||
MergeTreeMarksLoader marks_loader;
|
||||
void readData(
|
||||
const NameAndTypePair & name_and_type,
|
||||
ColumnPtr & column,
|
||||
size_t rows_to_read,
|
||||
const InputStreamGetter & getter);
|
||||
|
||||
void readPrefix(
|
||||
const NameAndTypePair & name_and_type,
|
||||
const InputStreamGetter & buffer_getter,
|
||||
const InputStreamGetter & buffer_getter_for_prefix,
|
||||
const ColumnNameLevel & name_level_for_offsets);
|
||||
|
||||
void createColumnsForReading(Columns & res_columns) const;
|
||||
bool needSkipStream(size_t column_pos, const ISerialization::SubstreamPath & substream) const;
|
||||
|
||||
const MergeTreeMarksLoaderPtr marks_loader;
|
||||
MergeTreeMarksGetterPtr marks_getter;
|
||||
|
||||
ReadBufferFromFileBase::ProfileCallback profile_callback;
|
||||
clockid_t clock_type;
|
||||
|
||||
/// Storage columns with collected separate arrays of Nested to columns of Nested type.
|
||||
/// They maybe be needed for finding offsets of missed Nested columns in parts.
|
||||
@ -67,32 +74,9 @@ private:
|
||||
/// Element of the vector is the level of the alternative stream.
|
||||
std::vector<ColumnNameLevel> columns_for_offsets;
|
||||
|
||||
/// For asynchronous reading from remote fs. Same meaning as in MergeTreeReaderStream.
|
||||
std::optional<size_t> last_right_offset;
|
||||
|
||||
/// Mark to read in next 'readRows' call in case,
|
||||
/// when 'continue_reading' is true.
|
||||
size_t next_mark = 0;
|
||||
std::optional<std::pair<size_t, size_t>> last_read_granule;
|
||||
|
||||
void seekToMark(size_t row_index, size_t column_index);
|
||||
|
||||
void readData(const NameAndTypePair & name_and_type, ColumnPtr & column, size_t from_mark,
|
||||
size_t current_task_last_mark, size_t column_position,
|
||||
size_t rows_to_read, ColumnNameLevel name_level_for_offsets, std::unordered_map<String, ColumnPtr> & columns_cache_for_subcolumns);
|
||||
|
||||
/// Returns maximal value of granule size in compressed file from @mark_ranges.
|
||||
/// This value is used as size of read buffer.
|
||||
static size_t getReadBufferSize(
|
||||
const IMergeTreeDataPartInfoForReader & data_part_info_for_reader,
|
||||
MergeTreeMarksLoader & marks_loader,
|
||||
const ColumnPositions & column_positions,
|
||||
const MarkRanges & mark_ranges);
|
||||
|
||||
/// For asynchronous reading from remote fs.
|
||||
void adjustUpperBound(size_t last_mark);
|
||||
|
||||
ReadBufferFromFileBase::ProfileCallback profile_callback;
|
||||
clockid_t clock_type;
|
||||
bool initialized = false;
|
||||
};
|
||||
|
||||
}
|
||||
|
108
src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp
Normal file
108
src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.cpp
Normal file
@ -0,0 +1,108 @@
|
||||
#include <Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.h>
|
||||
#include <Storages/MergeTree/MergeTreeDataPartCompact.h>
|
||||
#include <Storages/MergeTree/checkDataPart.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/NestedUtils.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
size_t MergeTreeReaderCompactSingleBuffer::readRows(
|
||||
size_t from_mark, size_t current_task_last_mark, bool continue_reading, size_t max_rows_to_read, Columns & res_columns)
|
||||
try
|
||||
{
|
||||
init();
|
||||
|
||||
if (continue_reading)
|
||||
from_mark = next_mark;
|
||||
|
||||
size_t read_rows = 0;
|
||||
size_t num_columns = columns_to_read.size();
|
||||
|
||||
checkNumberOfColumns(num_columns);
|
||||
createColumnsForReading(res_columns);
|
||||
|
||||
while (read_rows < max_rows_to_read)
|
||||
{
|
||||
size_t rows_to_read = data_part_info_for_read->getIndexGranularity().getMarkRows(from_mark);
|
||||
|
||||
for (size_t pos = 0; pos < num_columns; ++pos)
|
||||
{
|
||||
if (!res_columns[pos])
|
||||
continue;
|
||||
|
||||
auto & column = res_columns[pos];
|
||||
|
||||
stream->adjustRightMark(current_task_last_mark); /// Must go before seek.
|
||||
stream->seekToMarkAndColumn(from_mark, *column_positions[pos]);
|
||||
|
||||
auto buffer_getter = [&](const ISerialization::SubstreamPath & substream_path) -> ReadBuffer *
|
||||
{
|
||||
if (needSkipStream(pos, substream_path))
|
||||
return nullptr;
|
||||
|
||||
return stream->getDataBuffer();
|
||||
};
|
||||
|
||||
/// If we read only offsets we have to read prefix anyway
|
||||
/// to preserve correctness of serialization.
|
||||
auto buffer_getter_for_prefix = [&](const auto &) -> ReadBuffer *
|
||||
{
|
||||
return stream->getDataBuffer();
|
||||
};
|
||||
|
||||
readPrefix(columns_to_read[pos], buffer_getter, buffer_getter_for_prefix, columns_for_offsets[pos]);
|
||||
readData(columns_to_read[pos], column, rows_to_read, buffer_getter);
|
||||
}
|
||||
|
||||
++from_mark;
|
||||
read_rows += rows_to_read;
|
||||
}
|
||||
|
||||
next_mark = from_mark;
|
||||
return read_rows;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
|
||||
/// Better diagnostics.
|
||||
try
|
||||
{
|
||||
rethrow_exception(std::current_exception());
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(getMessageForDiagnosticOfBrokenPart(from_mark, max_rows_to_read));
|
||||
}
|
||||
|
||||
throw;
|
||||
}
|
||||
|
||||
void MergeTreeReaderCompactSingleBuffer::init()
|
||||
try
|
||||
{
|
||||
if (initialized)
|
||||
return;
|
||||
|
||||
auto stream_settings = settings;
|
||||
stream_settings.allow_different_codecs = true;
|
||||
|
||||
stream = std::make_unique<MergeTreeReaderStreamAllOfMultipleColumns>(
|
||||
data_part_info_for_read->getDataPartStorage(), MergeTreeDataPartCompact::DATA_FILE_NAME,
|
||||
MergeTreeDataPartCompact::DATA_FILE_EXTENSION, data_part_info_for_read->getMarksCount(),
|
||||
all_mark_ranges, stream_settings,uncompressed_cache,
|
||||
data_part_info_for_read->getFileSizeOrZero(MergeTreeDataPartCompact::DATA_FILE_NAME_WITH_EXTENSION),
|
||||
marks_loader, profile_callback, clock_type);
|
||||
|
||||
initialized = true;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
if (!isRetryableException(std::current_exception()))
|
||||
data_part_info_for_read->reportBroken();
|
||||
throw;
|
||||
}
|
||||
|
||||
}
|
33
src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.h
Normal file
33
src/Storages/MergeTree/MergeTreeReaderCompactSingleBuffer.h
Normal file
@ -0,0 +1,33 @@
|
||||
#pragma once
|
||||
#include <Storages/MergeTree/MergeTreeReaderCompact.h>
|
||||
#include <Storages/MergeTree/MergeTreeReaderStream.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Reader for compact parts, that uses one buffer for
|
||||
/// all column and doesn't support parallel prefetch of columns.
|
||||
/// It's suitable for compact parts with small size of stripe.
|
||||
class MergeTreeReaderCompactSingleBuffer : public MergeTreeReaderCompact
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeReaderCompactSingleBuffer(Args &&... args)
|
||||
: MergeTreeReaderCompact{std::forward<Args>(args)...}
|
||||
{
|
||||
fillColumnPositions();
|
||||
}
|
||||
|
||||
/// Returns the number of rows has been read or zero if there is no columns to read.
|
||||
/// If continue_reading is true, continue reading from last state, otherwise seek to from_mark
|
||||
size_t readRows(size_t from_mark, size_t current_task_last_mark,
|
||||
bool continue_reading, size_t max_rows_to_read, Columns & res_columns) override;
|
||||
|
||||
private:
|
||||
void init();
|
||||
|
||||
bool initialized = false;
|
||||
std::unique_ptr<MergeTreeReaderStream> stream;
|
||||
};
|
||||
|
||||
}
|
@ -16,43 +16,28 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
MergeTreeReaderStream::MergeTreeReaderStream(
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
const String & path_prefix_,
|
||||
const String & data_file_extension_,
|
||||
size_t marks_count_,
|
||||
const MarkRanges & all_mark_ranges_,
|
||||
const MergeTreeReaderSettings & settings_,
|
||||
MarkCache * mark_cache_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
size_t file_size_,
|
||||
const MergeTreeIndexGranularityInfo * index_granularity_info_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_,
|
||||
bool is_low_cardinality_dictionary_,
|
||||
ThreadPool * load_marks_cache_threadpool_)
|
||||
: settings(settings_)
|
||||
, profile_callback(profile_callback_)
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
const String & path_prefix_,
|
||||
const String & data_file_extension_,
|
||||
size_t marks_count_,
|
||||
const MarkRanges & all_mark_ranges_,
|
||||
const MergeTreeReaderSettings & settings_,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
size_t file_size_,
|
||||
MergeTreeMarksLoaderPtr marks_loader_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_)
|
||||
: profile_callback(profile_callback_)
|
||||
, clock_type(clock_type_)
|
||||
, all_mark_ranges(all_mark_ranges_)
|
||||
, file_size(file_size_)
|
||||
, uncompressed_cache(uncompressed_cache_)
|
||||
, data_part_storage(data_part_reader_->getDataPartStorage())
|
||||
, data_part_storage(std::move(data_part_storage_))
|
||||
, path_prefix(path_prefix_)
|
||||
, data_file_extension(data_file_extension_)
|
||||
, is_low_cardinality_dictionary(is_low_cardinality_dictionary_)
|
||||
, uncompressed_cache(uncompressed_cache_)
|
||||
, settings(settings_)
|
||||
, marks_count(marks_count_)
|
||||
, mark_cache(mark_cache_)
|
||||
, save_marks_in_cache(settings.save_marks_in_cache)
|
||||
, index_granularity_info(index_granularity_info_)
|
||||
, marks_loader(
|
||||
data_part_reader_,
|
||||
mark_cache,
|
||||
index_granularity_info->getMarksFilePath(path_prefix),
|
||||
marks_count,
|
||||
*index_granularity_info,
|
||||
save_marks_in_cache,
|
||||
settings.read_settings,
|
||||
load_marks_cache_threadpool_)
|
||||
, file_size(file_size_)
|
||||
, marks_loader(std::move(marks_loader_))
|
||||
{
|
||||
}
|
||||
|
||||
@ -60,21 +45,12 @@ void MergeTreeReaderStream::init()
|
||||
{
|
||||
if (initialized)
|
||||
return;
|
||||
|
||||
initialized = true;
|
||||
marks_getter = marks_loader->loadMarks();
|
||||
|
||||
/// Compute the size of the buffer.
|
||||
size_t max_mark_range_bytes = 0;
|
||||
size_t sum_mark_range_bytes = 0;
|
||||
|
||||
for (const auto & mark_range : all_mark_ranges)
|
||||
{
|
||||
size_t left_mark = mark_range.begin;
|
||||
size_t right_mark = mark_range.end;
|
||||
size_t left_offset = left_mark < marks_count ? marks_loader.getMark(left_mark).offset_in_compressed_file : 0;
|
||||
auto mark_range_bytes = getRightOffset(right_mark) - left_offset;
|
||||
|
||||
max_mark_range_bytes = std::max(max_mark_range_bytes, mark_range_bytes);
|
||||
sum_mark_range_bytes += mark_range_bytes;
|
||||
}
|
||||
auto [max_mark_range_bytes, sum_mark_range_bytes] = estimateMarkRangeBytes(all_mark_ranges);
|
||||
|
||||
std::optional<size_t> estimated_sum_mark_range_bytes;
|
||||
if (sum_mark_range_bytes)
|
||||
@ -83,7 +59,7 @@ void MergeTreeReaderStream::init()
|
||||
/// Avoid empty buffer. May happen while reading dictionary for DataTypeLowCardinality.
|
||||
/// For example: part has single dictionary and all marks point to the same position.
|
||||
ReadSettings read_settings = settings.read_settings;
|
||||
if (max_mark_range_bytes != 0)
|
||||
if (settings.adjust_read_buffer_size && max_mark_range_bytes != 0)
|
||||
read_settings = read_settings.adjustBufferSize(max_mark_range_bytes);
|
||||
|
||||
//// Empty buffer does not makes progress.
|
||||
@ -102,7 +78,8 @@ void MergeTreeReaderStream::init()
|
||||
read_settings,
|
||||
estimated_sum_mark_range_bytes, std::nullopt);
|
||||
},
|
||||
uncompressed_cache);
|
||||
uncompressed_cache,
|
||||
settings.allow_different_codecs);
|
||||
|
||||
if (profile_callback)
|
||||
buffer->setProfileCallback(profile_callback, clock_type);
|
||||
@ -121,7 +98,7 @@ void MergeTreeReaderStream::init()
|
||||
path_prefix + data_file_extension,
|
||||
read_settings,
|
||||
estimated_sum_mark_range_bytes,
|
||||
std::nullopt));
|
||||
std::nullopt), settings.allow_different_codecs);
|
||||
|
||||
if (profile_callback)
|
||||
buffer->setProfileCallback(profile_callback, clock_type);
|
||||
@ -135,99 +112,10 @@ void MergeTreeReaderStream::init()
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
size_t MergeTreeReaderStream::getRightOffset(size_t right_mark)
|
||||
{
|
||||
/// NOTE: if we are reading the whole file, then right_mark == marks_count
|
||||
/// and we will use max_read_buffer_size for buffer size, thus avoiding the need to load marks.
|
||||
|
||||
/// Special case, can happen in Collapsing/Replacing engines
|
||||
if (marks_count == 0)
|
||||
return 0;
|
||||
|
||||
assert(right_mark <= marks_count);
|
||||
|
||||
if (0 < right_mark && right_mark < marks_count)
|
||||
{
|
||||
/// Find the right border of the last mark we need to read.
|
||||
/// To do that let's find the upper bound of the offset of the last
|
||||
/// included mark.
|
||||
|
||||
if (is_low_cardinality_dictionary)
|
||||
{
|
||||
|
||||
/// In LowCardinality dictionary several consecutive marks can point to the same offset.
|
||||
///
|
||||
/// Also, in some cases, when one granule is not-atomically written (which is possible at merges)
|
||||
/// one granule may require reading of two dictionaries which starts from different marks.
|
||||
/// The only correct way is to take offset from at least next different granule from the right one.
|
||||
/// So, that's why we have to read one extra granule to the right,
|
||||
/// while reading dictionary of LowCardinality.
|
||||
///
|
||||
/// Example:
|
||||
/// Mark 0, points to [0, 8]
|
||||
/// Mark 1, points to [0, 8]
|
||||
/// Mark 2, points to [0, 8]
|
||||
/// Mark 3, points to [0, 8]
|
||||
/// Mark 4, points to [42336, 2255]
|
||||
/// Mark 5, points to [42336, 2255] <--- for example need to read until 5
|
||||
/// Mark 6, points to [42336, 2255] <--- not suitable, because have same offset
|
||||
/// Mark 7, points to [84995, 7738] <--- next different mark
|
||||
/// Mark 8, points to [84995, 7738]
|
||||
/// Mark 9, points to [126531, 8637] <--- what we are looking for
|
||||
|
||||
auto indices = collections::range(right_mark, marks_count);
|
||||
auto next_different_mark = [&](auto lhs, auto rhs)
|
||||
{
|
||||
return marks_loader.getMark(lhs).asTuple() < marks_loader.getMark(rhs).asTuple();
|
||||
};
|
||||
auto it = std::upper_bound(indices.begin(), indices.end(), right_mark, std::move(next_different_mark));
|
||||
|
||||
if (it == indices.end())
|
||||
return file_size;
|
||||
|
||||
right_mark = *it;
|
||||
}
|
||||
|
||||
/// This is a good scenario. The compressed block is finished within the right mark,
|
||||
/// and previous mark was different.
|
||||
if (marks_loader.getMark(right_mark).offset_in_decompressed_block == 0
|
||||
&& marks_loader.getMark(right_mark) != marks_loader.getMark(right_mark - 1))
|
||||
return marks_loader.getMark(right_mark).offset_in_compressed_file;
|
||||
|
||||
/// If right_mark has non-zero offset in decompressed block, we have to
|
||||
/// read its compressed block in a whole, because it may consist of data from previous granule.
|
||||
///
|
||||
/// For example:
|
||||
/// Mark 6, points to [42336, 2255]
|
||||
/// Mark 7, points to [84995, 7738] <--- right_mark
|
||||
/// Mark 8, points to [84995, 7738]
|
||||
/// Mark 9, points to [126531, 8637] <--- what we are looking for
|
||||
///
|
||||
/// Since mark 7 starts from offset in decompressed block 7738,
|
||||
/// it has some data from mark 6 and we have to read
|
||||
/// compressed block [84995; 126531 in a whole.
|
||||
|
||||
auto indices = collections::range(right_mark, marks_count);
|
||||
auto next_different_compressed_offset = [&](auto lhs, auto rhs)
|
||||
{
|
||||
return marks_loader.getMark(lhs).offset_in_compressed_file < marks_loader.getMark(rhs).offset_in_compressed_file;
|
||||
};
|
||||
auto it = std::upper_bound(indices.begin(), indices.end(), right_mark, std::move(next_different_compressed_offset));
|
||||
|
||||
if (it != indices.end())
|
||||
return marks_loader.getMark(*it).offset_in_compressed_file;
|
||||
}
|
||||
else if (right_mark == 0)
|
||||
return marks_loader.getMark(right_mark).offset_in_compressed_file;
|
||||
|
||||
return file_size;
|
||||
}
|
||||
|
||||
void MergeTreeReaderStream::seekToMark(size_t index)
|
||||
void MergeTreeReaderStream::seekToMarkAndColumn(size_t row_index, size_t column_position)
|
||||
{
|
||||
init();
|
||||
MarkInCompressedFile mark = marks_loader.getMark(index);
|
||||
const auto & mark = marks_getter->getMark(row_index, column_position);
|
||||
|
||||
try
|
||||
{
|
||||
@ -237,7 +125,7 @@ void MergeTreeReaderStream::seekToMark(size_t index)
|
||||
{
|
||||
/// Better diagnostics.
|
||||
if (e.code() == ErrorCodes::ARGUMENT_OUT_OF_BOUND)
|
||||
e.addMessage("(while seeking to mark " + toString(index)
|
||||
e.addMessage("(while seeking to mark " + toString(row_index)
|
||||
+ " of column " + path_prefix + "; offsets are: "
|
||||
+ toString(mark.offset_in_compressed_file) + " "
|
||||
+ toString(mark.offset_in_decompressed_block) + ")");
|
||||
@ -274,6 +162,7 @@ void MergeTreeReaderStream::adjustRightMark(size_t right_mark)
|
||||
*/
|
||||
init();
|
||||
auto right_offset = getRightOffset(right_mark);
|
||||
|
||||
if (!right_offset)
|
||||
{
|
||||
if (last_right_offset && *last_right_offset == 0)
|
||||
@ -304,4 +193,276 @@ CompressedReadBufferBase * MergeTreeReaderStream::getCompressedDataBuffer()
|
||||
return compressed_data_buffer;
|
||||
}
|
||||
|
||||
size_t MergeTreeReaderStreamSingleColumn::getRightOffset(size_t right_mark) const
|
||||
{
|
||||
/// NOTE: if we are reading the whole file, then right_mark == marks_count
|
||||
/// and we will use max_read_buffer_size for buffer size, thus avoiding the need to load marks.
|
||||
|
||||
/// Special case, can happen in Collapsing/Replacing engines
|
||||
if (marks_count == 0)
|
||||
return 0;
|
||||
|
||||
assert(right_mark <= marks_count);
|
||||
|
||||
if (right_mark == 0)
|
||||
return marks_getter->getMark(right_mark, 0).offset_in_compressed_file;
|
||||
|
||||
if (right_mark == marks_count)
|
||||
return file_size;
|
||||
|
||||
/// Find the right border of the last mark we need to read.
|
||||
/// To do that let's find the upper bound of the offset of the last
|
||||
/// included mark.
|
||||
|
||||
if (settings.is_low_cardinality_dictionary)
|
||||
{
|
||||
/// In LowCardinality dictionary several consecutive marks can point to the same offset.
|
||||
///
|
||||
/// Also, in some cases, when one granule is not-atomically written (which is possible at merges)
|
||||
/// one granule may require reading of two dictionaries which starts from different marks.
|
||||
/// The only correct way is to take offset from at least next different granule from the right one.
|
||||
/// So, that's why we have to read one extra granule to the right,
|
||||
/// while reading dictionary of LowCardinality.
|
||||
///
|
||||
/// Example:
|
||||
/// Mark 0, points to [0, 8]
|
||||
/// Mark 1, points to [0, 8]
|
||||
/// Mark 2, points to [0, 8]
|
||||
/// Mark 3, points to [0, 8]
|
||||
/// Mark 4, points to [42336, 2255]
|
||||
/// Mark 5, points to [42336, 2255] <--- for example need to read until 5
|
||||
/// Mark 6, points to [42336, 2255] <--- not suitable, because have same offset
|
||||
/// Mark 7, points to [84995, 7738] <--- next different mark
|
||||
/// Mark 8, points to [84995, 7738]
|
||||
/// Mark 9, points to [126531, 8637] <--- what we are looking for
|
||||
|
||||
auto indices = collections::range(right_mark, marks_count);
|
||||
auto next_different_mark = [&](auto lhs, auto rhs)
|
||||
{
|
||||
return marks_getter->getMark(lhs, 0).asTuple() < marks_getter->getMark(rhs, 0).asTuple();
|
||||
};
|
||||
|
||||
auto it = std::upper_bound(indices.begin(), indices.end(), right_mark, std::move(next_different_mark));
|
||||
if (it == indices.end())
|
||||
return file_size;
|
||||
|
||||
right_mark = *it;
|
||||
}
|
||||
|
||||
/// This is a good scenario. The compressed block is finished within the right mark,
|
||||
/// and previous mark was different.
|
||||
if (marks_getter->getMark(right_mark, 0).offset_in_decompressed_block == 0
|
||||
&& marks_getter->getMark(right_mark, 0) != marks_getter->getMark(right_mark - 1, 0))
|
||||
return marks_getter->getMark(right_mark, 0).offset_in_compressed_file;
|
||||
|
||||
/// If right_mark has non-zero offset in decompressed block, we have to
|
||||
/// read its compressed block in a whole, because it may consist of data from previous granule.
|
||||
///
|
||||
/// For example:
|
||||
/// Mark 6, points to [42336, 2255]
|
||||
/// Mark 7, points to [84995, 7738] <--- right_mark
|
||||
/// Mark 8, points to [84995, 7738]
|
||||
/// Mark 9, points to [126531, 8637] <--- what we are looking for
|
||||
///
|
||||
/// Since mark 7 starts from offset in decompressed block 7738,
|
||||
/// it has some data from mark 6 and we have to read
|
||||
/// compressed block [84995; 126531 in a whole.
|
||||
|
||||
auto indices = collections::range(right_mark, marks_count);
|
||||
auto next_different_compressed_offset = [&](auto lhs, auto rhs)
|
||||
{
|
||||
return marks_getter->getMark(lhs, 0).offset_in_compressed_file < marks_getter->getMark(rhs, 0).offset_in_compressed_file;
|
||||
};
|
||||
|
||||
auto it = std::upper_bound(indices.begin(), indices.end(), right_mark, std::move(next_different_compressed_offset));
|
||||
if (it != indices.end())
|
||||
return marks_getter->getMark(*it, 0).offset_in_compressed_file;
|
||||
|
||||
return file_size;
|
||||
}
|
||||
|
||||
std::pair<size_t, size_t> MergeTreeReaderStreamSingleColumn::estimateMarkRangeBytes(const MarkRanges & mark_ranges) const
|
||||
{
|
||||
assert(marks_getter != nullptr);
|
||||
|
||||
size_t max_range_bytes = 0;
|
||||
size_t sum_range_bytes = 0;
|
||||
|
||||
for (const auto & mark_range : mark_ranges)
|
||||
{
|
||||
size_t left_mark = mark_range.begin;
|
||||
size_t right_mark = mark_range.end;
|
||||
size_t left_offset = left_mark < marks_count ? marks_getter->getMark(left_mark, 0).offset_in_compressed_file : 0;
|
||||
auto mark_range_bytes = getRightOffset(right_mark) - left_offset;
|
||||
|
||||
max_range_bytes = std::max(max_range_bytes, mark_range_bytes);
|
||||
sum_range_bytes += mark_range_bytes;
|
||||
}
|
||||
|
||||
return {max_range_bytes, sum_range_bytes};
|
||||
}
|
||||
|
||||
size_t MergeTreeReaderStreamMultipleColumns::getRightOffsetOneColumn(size_t right_mark_non_included, size_t column_position) const
|
||||
{
|
||||
/// NOTE: if we are reading the whole file, then right_mark == marks_count
|
||||
/// and we will use max_read_buffer_size for buffer size, thus avoiding the need to load marks.
|
||||
|
||||
/// Special case, can happen in Collapsing/Replacing engines
|
||||
if (marks_count == 0)
|
||||
return 0;
|
||||
|
||||
assert(right_mark_non_included <= marks_count);
|
||||
|
||||
if (right_mark_non_included == 0)
|
||||
return marks_getter->getMark(right_mark_non_included, column_position).offset_in_compressed_file;
|
||||
|
||||
size_t right_mark_included = right_mark_non_included - 1;
|
||||
if (right_mark_non_included != marks_count
|
||||
&& marks_getter->getMark(right_mark_non_included, column_position).offset_in_decompressed_block != 0)
|
||||
++right_mark_included;
|
||||
|
||||
/// The right bound for case, where there is no smaller suitable mark
|
||||
/// is the start of the next stripe (in which the next column is written)
|
||||
/// because each stripe always start from a new compressed block.
|
||||
const auto & right_mark_in_file = marks_getter->getMark(right_mark_included, column_position);
|
||||
auto next_stripe_right_mark_in_file = getStartOfNextStripeMark(right_mark_included, column_position);
|
||||
|
||||
/// Try to find suitable right mark in current stripe.
|
||||
for (size_t mark = right_mark_included + 1; mark < marks_count; ++mark)
|
||||
{
|
||||
const auto & current_mark = marks_getter->getMark(mark, column_position);
|
||||
/// We found first mark that starts from the new compressed block.
|
||||
if (current_mark.offset_in_compressed_file > right_mark_in_file.offset_in_compressed_file)
|
||||
{
|
||||
/// If it is in current stripe return it to reduce amount of read data.
|
||||
if (current_mark < next_stripe_right_mark_in_file)
|
||||
return current_mark.offset_in_compressed_file;
|
||||
|
||||
/// Otherwise return start of new stripe as an upper bound.
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
return next_stripe_right_mark_in_file.offset_in_compressed_file;
|
||||
}
|
||||
|
||||
std::pair<size_t, size_t>
|
||||
MergeTreeReaderStreamMultipleColumns::estimateMarkRangeBytesOneColumn(const MarkRanges & mark_ranges, size_t column_position) const
|
||||
{
|
||||
assert(marks_getter != nullptr);
|
||||
|
||||
/// As a maximal range we return the maximal size of a whole stripe.
|
||||
size_t max_range_bytes = 0;
|
||||
size_t sum_range_bytes = 0;
|
||||
|
||||
for (const auto & mark_range : mark_ranges)
|
||||
{
|
||||
auto start_of_stripe_mark = marks_getter->getMark(mark_range.begin, column_position);
|
||||
auto start_of_next_stripe_mark = getStartOfNextStripeMark(mark_range.begin, column_position);
|
||||
|
||||
for (size_t mark = mark_range.begin; mark < mark_range.end; ++mark)
|
||||
{
|
||||
const auto & current_mark = marks_getter->getMark(mark, column_position);
|
||||
|
||||
/// We found a start of new stripe, now update values.
|
||||
if (current_mark > start_of_next_stripe_mark)
|
||||
{
|
||||
auto current_range_bytes = getRightOffsetOneColumn(mark, column_position) - start_of_stripe_mark.offset_in_compressed_file;
|
||||
|
||||
max_range_bytes = std::max(max_range_bytes, current_range_bytes);
|
||||
sum_range_bytes += current_range_bytes;
|
||||
|
||||
start_of_stripe_mark = current_mark;
|
||||
start_of_next_stripe_mark = getStartOfNextStripeMark(mark, column_position);
|
||||
}
|
||||
}
|
||||
|
||||
auto current_range_bytes = getRightOffsetOneColumn(mark_range.end, column_position) - start_of_stripe_mark.offset_in_compressed_file;
|
||||
|
||||
max_range_bytes = std::max(max_range_bytes, current_range_bytes);
|
||||
sum_range_bytes += current_range_bytes;
|
||||
}
|
||||
|
||||
return {max_range_bytes, sum_range_bytes};
|
||||
}
|
||||
|
||||
MarkInCompressedFile MergeTreeReaderStreamMultipleColumns::getStartOfNextStripeMark(size_t row_index, size_t column_position) const
|
||||
{
|
||||
const auto & current_mark = marks_getter->getMark(row_index, column_position);
|
||||
|
||||
if (marks_getter->getNumColumns() == 1)
|
||||
return MarkInCompressedFile{file_size, 0};
|
||||
|
||||
if (column_position + 1 == marks_getter->getNumColumns())
|
||||
{
|
||||
/**
|
||||
* In case of the last column (c3), we have the following picture:
|
||||
* c1 c2 c3
|
||||
* x x x
|
||||
* (row_index, 0) -> o x o <- (row_index, column_position)
|
||||
* x x x
|
||||
* ------- <- start of new stripe
|
||||
* what we are -> o x x
|
||||
* looking for x x x
|
||||
* x x x
|
||||
* -------
|
||||
* So, we need to iterate forward.
|
||||
*/
|
||||
size_t mark_index = row_index + 1;
|
||||
while (mark_index < marks_count && marks_getter->getMark(mark_index, 0) <= current_mark)
|
||||
++mark_index;
|
||||
|
||||
return mark_index == marks_count
|
||||
? MarkInCompressedFile{file_size, 0}
|
||||
: marks_getter->getMark(mark_index, 0);
|
||||
}
|
||||
|
||||
/**
|
||||
* Otherwise, we have the following picture:
|
||||
* c1 c2 c3
|
||||
* x x o <- what we are looking for
|
||||
* (row, column) --> o o <- (row, column + 1)
|
||||
* x x x
|
||||
* ------- <- start of new stripe
|
||||
* So, we need to iterate backward.
|
||||
*/
|
||||
|
||||
ssize_t mark_index = row_index;
|
||||
while (mark_index >= 0 && marks_getter->getMark(mark_index, column_position + 1) >= current_mark)
|
||||
--mark_index;
|
||||
|
||||
return marks_getter->getMark(mark_index + 1, column_position + 1);
|
||||
}
|
||||
|
||||
size_t MergeTreeReaderStreamOneOfMultipleColumns::getRightOffset(size_t right_mark_non_included) const
|
||||
{
|
||||
return getRightOffsetOneColumn(right_mark_non_included, column_position);
|
||||
}
|
||||
|
||||
std::pair<size_t, size_t> MergeTreeReaderStreamOneOfMultipleColumns::estimateMarkRangeBytes(const MarkRanges & mark_ranges) const
|
||||
{
|
||||
return estimateMarkRangeBytesOneColumn(mark_ranges, column_position);
|
||||
}
|
||||
|
||||
size_t MergeTreeReaderStreamAllOfMultipleColumns::getRightOffset(size_t right_mark_non_included) const
|
||||
{
|
||||
return getRightOffsetOneColumn(right_mark_non_included, marks_loader->getNumColumns() - 1);
|
||||
}
|
||||
|
||||
std::pair<size_t, size_t> MergeTreeReaderStreamAllOfMultipleColumns::estimateMarkRangeBytes(const MarkRanges & mark_ranges) const
|
||||
{
|
||||
size_t max_range_bytes = 0;
|
||||
size_t sum_range_bytes = 0;
|
||||
|
||||
for (size_t i = 0; i < marks_getter->getNumColumns(); ++i)
|
||||
{
|
||||
auto [current_max, current_sum] = estimateMarkRangeBytesOneColumn(mark_ranges, i);
|
||||
|
||||
max_range_bytes = std::max(max_range_bytes, current_max);
|
||||
sum_range_bytes += current_sum;
|
||||
}
|
||||
|
||||
return {max_range_bytes, sum_range_bytes};
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -14,27 +14,31 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Class for reading a single column (or index).
|
||||
/// Basic and the most low-level class
|
||||
/// for reading single columns or indexes.
|
||||
class MergeTreeReaderStream
|
||||
{
|
||||
public:
|
||||
MergeTreeReaderStream(
|
||||
MergeTreeDataPartInfoForReaderPtr data_part_reader_,
|
||||
DataPartStoragePtr data_part_storage_,
|
||||
const String & path_prefix_,
|
||||
const String & data_file_extension_,
|
||||
size_t marks_count_,
|
||||
const MarkRanges & all_mark_ranges,
|
||||
const MarkRanges & all_mark_ranges_,
|
||||
const MergeTreeReaderSettings & settings_,
|
||||
MarkCache * mark_cache,
|
||||
UncompressedCache * uncompressed_cache,
|
||||
UncompressedCache * uncompressed_cache_,
|
||||
size_t file_size_,
|
||||
const MergeTreeIndexGranularityInfo * index_granularity_info_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback,
|
||||
clockid_t clock_type,
|
||||
bool is_low_cardinality_dictionary_,
|
||||
ThreadPool * load_marks_cache_threadpool_);
|
||||
MergeTreeMarksLoaderPtr marks_loader_,
|
||||
const ReadBufferFromFileBase::ProfileCallback & profile_callback_,
|
||||
clockid_t clock_type_);
|
||||
|
||||
void seekToMark(size_t index);
|
||||
virtual ~MergeTreeReaderStream() = default;
|
||||
|
||||
/// Seeks to start of @row_index mark. Column position is implementation defined.
|
||||
virtual void seekToMark(size_t row_index) = 0;
|
||||
|
||||
/// Seeks to exact mark in file.
|
||||
void seekToMarkAndColumn(size_t row_index, size_t column_position);
|
||||
|
||||
void seekToStart();
|
||||
|
||||
@ -48,39 +52,111 @@ public:
|
||||
CompressedReadBufferBase * getCompressedDataBuffer();
|
||||
|
||||
private:
|
||||
void init();
|
||||
size_t getRightOffset(size_t right_mark);
|
||||
/// Returns offset in file up to which it's needed to read file to read all rows up to @right_mark mark.
|
||||
virtual size_t getRightOffset(size_t right_mark) const = 0;
|
||||
|
||||
/// Returns estimated max amount of bytes to read among mark ranges (which is used as size for read buffer)
|
||||
/// and total amount of bytes to read in all mark ranges.
|
||||
virtual std::pair<size_t, size_t> estimateMarkRangeBytes(const MarkRanges & mark_ranges) const = 0;
|
||||
|
||||
const MergeTreeReaderSettings settings;
|
||||
const ReadBufferFromFileBase::ProfileCallback profile_callback;
|
||||
clockid_t clock_type;
|
||||
const clockid_t clock_type;
|
||||
const MarkRanges all_mark_ranges;
|
||||
size_t file_size;
|
||||
UncompressedCache * uncompressed_cache;
|
||||
|
||||
DataPartStoragePtr data_part_storage;
|
||||
std::string path_prefix;
|
||||
std::string data_file_extension;
|
||||
|
||||
bool is_low_cardinality_dictionary = false;
|
||||
|
||||
size_t marks_count;
|
||||
const DataPartStoragePtr data_part_storage;
|
||||
const std::string path_prefix;
|
||||
const std::string data_file_extension;
|
||||
|
||||
UncompressedCache * const uncompressed_cache;
|
||||
|
||||
ReadBuffer * data_buffer;
|
||||
CompressedReadBufferBase * compressed_data_buffer;
|
||||
MarkCache * mark_cache;
|
||||
bool save_marks_in_cache;
|
||||
|
||||
bool initialized = false;
|
||||
|
||||
std::optional<size_t> last_right_offset;
|
||||
|
||||
const MergeTreeIndexGranularityInfo * index_granularity_info;
|
||||
|
||||
std::unique_ptr<CachedCompressedReadBuffer> cached_buffer;
|
||||
std::unique_ptr<CompressedReadBufferFromFile> non_cached_buffer;
|
||||
|
||||
MergeTreeMarksLoader marks_loader;
|
||||
protected:
|
||||
void init();
|
||||
|
||||
const MergeTreeReaderSettings settings;
|
||||
const size_t marks_count;
|
||||
const size_t file_size;
|
||||
|
||||
const MergeTreeMarksLoaderPtr marks_loader;
|
||||
MergeTreeMarksGetterPtr marks_getter;
|
||||
};
|
||||
|
||||
/// Class for reading a single column (or index) from file
|
||||
/// that contains a single column (for wide parts).
|
||||
class MergeTreeReaderStreamSingleColumn : public MergeTreeReaderStream
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeReaderStreamSingleColumn(Args &&... args)
|
||||
: MergeTreeReaderStream{std::forward<Args>(args)...}
|
||||
{
|
||||
}
|
||||
|
||||
size_t getRightOffset(size_t right_mark_non_included) const override;
|
||||
std::pair<size_t, size_t> estimateMarkRangeBytes(const MarkRanges & mark_ranges) const override;
|
||||
void seekToMark(size_t row_index) override { seekToMarkAndColumn(row_index, 0); }
|
||||
};
|
||||
|
||||
/// Base class for reading from file that contains multiple columns.
|
||||
/// It is used to read from compact parts.
|
||||
/// See more details about data layout in MergeTreeDataPartCompact.h.
|
||||
class MergeTreeReaderStreamMultipleColumns : public MergeTreeReaderStream
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeReaderStreamMultipleColumns(Args &&... args)
|
||||
: MergeTreeReaderStream{std::forward<Args>(args)...}
|
||||
{
|
||||
}
|
||||
|
||||
protected:
|
||||
size_t getRightOffsetOneColumn(size_t right_mark_non_included, size_t column_position) const;
|
||||
std::pair<size_t, size_t> estimateMarkRangeBytesOneColumn(const MarkRanges & mark_ranges, size_t column_position) const;
|
||||
MarkInCompressedFile getStartOfNextStripeMark(size_t row_index, size_t column_position) const;
|
||||
};
|
||||
|
||||
/// Class for reading a single column from file that contains multiple columns
|
||||
/// (for parallel reading from compact parts with large stripes).
|
||||
class MergeTreeReaderStreamOneOfMultipleColumns : public MergeTreeReaderStreamMultipleColumns
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeReaderStreamOneOfMultipleColumns(size_t column_position_, Args &&... args)
|
||||
: MergeTreeReaderStreamMultipleColumns{std::forward<Args>(args)...}
|
||||
, column_position(column_position_)
|
||||
{
|
||||
}
|
||||
|
||||
size_t getRightOffset(size_t right_mark_non_included) const override;
|
||||
std::pair<size_t, size_t> estimateMarkRangeBytes(const MarkRanges & mark_ranges) const override;
|
||||
void seekToMark(size_t row_index) override { seekToMarkAndColumn(row_index, column_position); }
|
||||
|
||||
private:
|
||||
const size_t column_position;
|
||||
};
|
||||
|
||||
/// Class for reading multiple columns from file that contains multiple columns
|
||||
/// (for reading from compact parts with small stripes).
|
||||
class MergeTreeReaderStreamAllOfMultipleColumns : public MergeTreeReaderStreamMultipleColumns
|
||||
{
|
||||
public:
|
||||
template <typename... Args>
|
||||
explicit MergeTreeReaderStreamAllOfMultipleColumns(Args &&... args)
|
||||
: MergeTreeReaderStreamMultipleColumns{std::forward<Args>(args)...}
|
||||
{
|
||||
}
|
||||
|
||||
size_t getRightOffset(size_t right_mark_non_included) const override;
|
||||
std::pair<size_t, size_t> estimateMarkRangeBytes(const MarkRanges & mark_ranges) const override;
|
||||
void seekToMark(size_t row_index) override { seekToMarkAndColumn(row_index, 0); }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -225,18 +225,29 @@ void MergeTreeReaderWide::addStreams(
|
||||
return;
|
||||
}
|
||||
|
||||
has_any_stream = true;
|
||||
bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys;
|
||||
|
||||
auto context = data_part_info_for_read->getContext();
|
||||
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
|
||||
|
||||
streams.emplace(*stream_name, std::make_unique<MergeTreeReaderStream>(
|
||||
data_part_info_for_read, *stream_name, DATA_FILE_EXTENSION,
|
||||
data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache,
|
||||
auto marks_loader = std::make_shared<MergeTreeMarksLoader>(
|
||||
data_part_info_for_read,
|
||||
mark_cache,
|
||||
data_part_info_for_read->getIndexGranularityInfo().getMarksFilePath(*stream_name),
|
||||
data_part_info_for_read->getMarksCount(),
|
||||
data_part_info_for_read->getIndexGranularityInfo(),
|
||||
settings.save_marks_in_cache,
|
||||
settings.read_settings,
|
||||
load_marks_threadpool,
|
||||
/*num_columns_in_mark=*/ 1);
|
||||
|
||||
has_any_stream = true;
|
||||
auto stream_settings = settings;
|
||||
stream_settings.is_low_cardinality_dictionary = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys;
|
||||
|
||||
streams.emplace(*stream_name, std::make_unique<MergeTreeReaderStreamSingleColumn>(
|
||||
data_part_info_for_read->getDataPartStorage(), *stream_name, DATA_FILE_EXTENSION,
|
||||
data_part_info_for_read->getMarksCount(), all_mark_ranges, stream_settings,
|
||||
uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(*stream_name + DATA_FILE_EXTENSION),
|
||||
&data_part_info_for_read->getIndexGranularityInfo(),
|
||||
profile_callback, clock_type, is_lc_dict, load_marks_threadpool));
|
||||
std::move(marks_loader), profile_callback, clock_type));
|
||||
};
|
||||
|
||||
serialization->enumerateStreams(callback);
|
||||
|
@ -184,9 +184,11 @@ private:
|
||||
auto & compressed_data = compressed->getData();
|
||||
auto & uncompressed_data = uncompressed->getData();
|
||||
|
||||
auto marks_getter = marks_loader->loadMarks();
|
||||
|
||||
for (size_t i = 0; i < num_rows; ++i)
|
||||
{
|
||||
auto mark = marks_loader->getMark(i, col_idx);
|
||||
auto mark = marks_getter->getMark(i, col_idx);
|
||||
|
||||
compressed_data[i] = mark.offset_in_compressed_file;
|
||||
uncompressed_data[i] = mark.offset_in_decompressed_block;
|
||||
|
@ -4,10 +4,12 @@
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeUUID.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
|
||||
#include <Storages/System/StorageSystemPartsBase.h>
|
||||
#include <Storages/System/getQueriedColumnsMaskAndHeader.h>
|
||||
#include <Storages/VirtualColumnUtils.h>
|
||||
#include <Processors/Sources/SourceFromSingleChunk.h>
|
||||
#include <QueryPipeline/Pipe.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
@ -320,7 +322,7 @@ protected:
|
||||
std::shared_ptr<StorageSystemDetachedParts> storage;
|
||||
std::vector<UInt8> columns_mask;
|
||||
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
ActionsDAGPtr filter;
|
||||
const size_t max_block_size;
|
||||
const size_t num_streams;
|
||||
};
|
||||
@ -329,7 +331,20 @@ void ReadFromSystemDetachedParts::applyFilters(ActionDAGNodes added_filter_nodes
|
||||
{
|
||||
filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes);
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
{
|
||||
const auto * predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
Block block;
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), "database"));
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), "table"));
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), "engine"));
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeUInt8>(), "active"));
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeUUID>(), "uuid"));
|
||||
|
||||
filter = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
||||
if (filter)
|
||||
VirtualColumnUtils::buildSetsForDAG(filter, context);
|
||||
}
|
||||
}
|
||||
|
||||
void StorageSystemDetachedParts::read(
|
||||
@ -358,7 +373,7 @@ void StorageSystemDetachedParts::read(
|
||||
|
||||
void ReadFromSystemDetachedParts::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
auto state = std::make_shared<SourceState>(StoragesInfoStream(predicate, context));
|
||||
auto state = std::make_shared<SourceState>(StoragesInfoStream(nullptr, filter, context));
|
||||
|
||||
Pipe pipe;
|
||||
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
{
|
||||
|
||||
|
||||
StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAG::Node * predicate, ContextPtr context)
|
||||
StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAGPtr & filter, ContextPtr context)
|
||||
: StoragesInfoStreamBase(context)
|
||||
{
|
||||
/// Will apply WHERE to subset of columns and then add more columns.
|
||||
@ -74,7 +74,8 @@ StoragesDroppedInfoStream::StoragesDroppedInfoStream(const ActionsDAG::Node * pr
|
||||
if (block_to_filter.rows())
|
||||
{
|
||||
/// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'.
|
||||
VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context);
|
||||
if (filter)
|
||||
VirtualColumnUtils::filterBlockWithDAG(filter, block_to_filter, context);
|
||||
rows = block_to_filter.rows();
|
||||
}
|
||||
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
class StoragesDroppedInfoStream : public StoragesInfoStreamBase
|
||||
{
|
||||
public:
|
||||
StoragesDroppedInfoStream(const ActionsDAG::Node * predicate, ContextPtr context);
|
||||
StoragesDroppedInfoStream(const ActionsDAGPtr & filter, ContextPtr context);
|
||||
protected:
|
||||
bool tryLockTable(StoragesInfo &) override
|
||||
{
|
||||
@ -30,9 +30,9 @@ public:
|
||||
|
||||
std::string getName() const override { return "SystemDroppedTablesParts"; }
|
||||
protected:
|
||||
std::unique_ptr<StoragesInfoStreamBase> getStoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context) override
|
||||
std::unique_ptr<StoragesInfoStreamBase> getStoragesInfoStream(const ActionsDAGPtr &, const ActionsDAGPtr & filter, ContextPtr context) override
|
||||
{
|
||||
return std::make_unique<StoragesDroppedInfoStream>(predicate, context);
|
||||
return std::make_unique<StoragesDroppedInfoStream>(filter, context);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -83,7 +83,7 @@ StoragesInfo::getProjectionParts(MergeTreeData::DataPartStateVector & state, boo
|
||||
return data->getProjectionPartsVectorForInternalUsage({State::Active}, &state);
|
||||
}
|
||||
|
||||
StoragesInfoStream::StoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context)
|
||||
StoragesInfoStream::StoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context)
|
||||
: StoragesInfoStreamBase(context)
|
||||
{
|
||||
/// Will apply WHERE to subset of columns and then add more columns.
|
||||
@ -115,7 +115,8 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAG::Node * predicate, Conte
|
||||
std::move(database_column_mut), std::make_shared<DataTypeString>(), "database"));
|
||||
|
||||
/// Filter block_to_filter with column 'database'.
|
||||
VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context);
|
||||
if (filter_by_database)
|
||||
VirtualColumnUtils::filterBlockWithDAG(filter_by_database, block_to_filter, context);
|
||||
rows = block_to_filter.rows();
|
||||
|
||||
/// Block contains new columns, update database_column.
|
||||
@ -194,7 +195,8 @@ StoragesInfoStream::StoragesInfoStream(const ActionsDAG::Node * predicate, Conte
|
||||
if (rows)
|
||||
{
|
||||
/// Filter block_to_filter with columns 'database', 'table', 'engine', 'active'.
|
||||
VirtualColumnUtils::filterBlockWithPredicate(predicate, block_to_filter, context);
|
||||
if (filter_by_other_columns)
|
||||
VirtualColumnUtils::filterBlockWithDAG(filter_by_other_columns, block_to_filter, context);
|
||||
rows = block_to_filter.rows();
|
||||
}
|
||||
|
||||
@ -226,7 +228,8 @@ protected:
|
||||
std::shared_ptr<StorageSystemPartsBase> storage;
|
||||
std::vector<UInt8> columns_mask;
|
||||
const bool has_state_column;
|
||||
const ActionsDAG::Node * predicate = nullptr;
|
||||
ActionsDAGPtr filter_by_database;
|
||||
ActionsDAGPtr filter_by_other_columns;
|
||||
};
|
||||
|
||||
ReadFromSystemPartsBase::ReadFromSystemPartsBase(
|
||||
@ -254,7 +257,25 @@ void ReadFromSystemPartsBase::applyFilters(ActionDAGNodes added_filter_nodes)
|
||||
{
|
||||
filter_actions_dag = ActionsDAG::buildFilterActionsDAG(added_filter_nodes.nodes);
|
||||
if (filter_actions_dag)
|
||||
predicate = filter_actions_dag->getOutputs().at(0);
|
||||
{
|
||||
const auto * predicate = filter_actions_dag->getOutputs().at(0);
|
||||
|
||||
Block block;
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), "database"));
|
||||
|
||||
filter_by_database = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
||||
if (filter_by_database)
|
||||
VirtualColumnUtils::buildSetsForDAG(filter_by_database, context);
|
||||
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), "table"));
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeString>(), "engine"));
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeUInt8>(), "active"));
|
||||
block.insert(ColumnWithTypeAndName({}, std::make_shared<DataTypeUUID>(), "uuid"));
|
||||
|
||||
filter_by_other_columns = VirtualColumnUtils::splitFilterDagForAllowedInputs(predicate, &block);
|
||||
if (filter_by_other_columns)
|
||||
VirtualColumnUtils::buildSetsForDAG(filter_by_other_columns, context);
|
||||
}
|
||||
}
|
||||
|
||||
void StorageSystemPartsBase::read(
|
||||
@ -288,7 +309,7 @@ void StorageSystemPartsBase::read(
|
||||
|
||||
void ReadFromSystemPartsBase::initializePipeline(QueryPipelineBuilder & pipeline, const BuildQueryPipelineSettings &)
|
||||
{
|
||||
auto stream = storage->getStoragesInfoStream(predicate, context);
|
||||
auto stream = storage->getStoragesInfoStream(filter_by_database, filter_by_other_columns, context);
|
||||
auto header = getOutputStream().header;
|
||||
|
||||
MutableColumns res_columns = header.cloneEmptyColumns();
|
||||
|
@ -115,7 +115,7 @@ protected:
|
||||
class StoragesInfoStream : public StoragesInfoStreamBase
|
||||
{
|
||||
public:
|
||||
StoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context);
|
||||
StoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context);
|
||||
};
|
||||
|
||||
/** Implements system table 'parts' which allows to get information about data parts for tables of MergeTree family.
|
||||
@ -145,9 +145,9 @@ protected:
|
||||
|
||||
StorageSystemPartsBase(const StorageID & table_id_, ColumnsDescription && columns);
|
||||
|
||||
virtual std::unique_ptr<StoragesInfoStreamBase> getStoragesInfoStream(const ActionsDAG::Node * predicate, ContextPtr context)
|
||||
virtual std::unique_ptr<StoragesInfoStreamBase> getStoragesInfoStream(const ActionsDAGPtr & filter_by_database, const ActionsDAGPtr & filter_by_other_columns, ContextPtr context)
|
||||
{
|
||||
return std::make_unique<StoragesInfoStream>(predicate, context);
|
||||
return std::make_unique<StoragesInfoStream>(filter_by_database, filter_by_other_columns, context);
|
||||
}
|
||||
|
||||
virtual void
|
||||
|
@ -53,9 +53,9 @@ namespace DB
|
||||
namespace VirtualColumnUtils
|
||||
{
|
||||
|
||||
static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & context)
|
||||
void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context)
|
||||
{
|
||||
for (const auto & node : actions->getNodes())
|
||||
for (const auto & node : dag->getNodes())
|
||||
{
|
||||
if (node.type == ActionsDAG::ActionType::COLUMN)
|
||||
{
|
||||
@ -78,8 +78,8 @@ static void makeSets(const ExpressionActionsPtr & actions, const ContextPtr & co
|
||||
|
||||
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context)
|
||||
{
|
||||
buildSetsForDAG(dag, context);
|
||||
auto actions = std::make_shared<ExpressionActions>(dag);
|
||||
makeSets(actions, context);
|
||||
Block block_with_filter = block;
|
||||
actions->execute(block_with_filter, /*dry_run=*/ false, /*allow_duplicates_in_input=*/ true);
|
||||
|
||||
|
@ -25,6 +25,9 @@ void filterBlockWithPredicate(const ActionsDAG::Node * predicate, Block & block,
|
||||
/// Just filters block. Block should contain all the required columns.
|
||||
void filterBlockWithDAG(ActionsDAGPtr dag, Block & block, ContextPtr context);
|
||||
|
||||
/// Builds sets used by ActionsDAG inplace.
|
||||
void buildSetsForDAG(const ActionsDAGPtr & dag, const ContextPtr & context);
|
||||
|
||||
/// Recursively checks if all functions used in DAG are deterministic in scope of query.
|
||||
bool isDeterministicInScopeOfQuery(const ActionsDAG::Node * node);
|
||||
|
||||
|
@ -52,6 +52,11 @@ def get_options(i: int, upgrade_check: bool) -> str:
|
||||
if i % 5 == 1:
|
||||
client_options.append("memory_tracker_fault_probability=0.001")
|
||||
|
||||
if i % 5 == 1:
|
||||
client_options.append(
|
||||
"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.05"
|
||||
)
|
||||
|
||||
if i % 2 == 1 and not upgrade_check:
|
||||
client_options.append("group_by_use_nulls=1")
|
||||
|
||||
|
@ -8,8 +8,9 @@ import subprocess
|
||||
import sys
|
||||
from concurrent.futures import ProcessPoolExecutor
|
||||
from pathlib import Path
|
||||
from typing import List, Tuple
|
||||
from typing import List, Tuple, Union
|
||||
|
||||
import magic
|
||||
from docker_images_helper import get_docker_image, pull_image
|
||||
from env_helper import CI, REPO_COPY, TEMP_PATH
|
||||
from git_helper import GIT_PREFIX, git_runner
|
||||
@ -95,34 +96,25 @@ def commit_push_staged(pr_info: PRInfo) -> None:
|
||||
git_runner(push_cmd)
|
||||
|
||||
|
||||
def is_python(file: str) -> bool:
|
||||
def _check_mime(file: Union[Path, str], mime: str) -> bool:
|
||||
# WARNING: python-magic v2:0.4.24-2 is used in ubuntu 22.04,
|
||||
# and `Support os.PathLike values in magic.from_file` is only from 0.4.25
|
||||
try:
|
||||
return bool(magic.from_file(os.path.join(REPO_COPY, file), mime=True) == mime)
|
||||
except (IsADirectoryError, FileNotFoundError) as e:
|
||||
# Process submodules and removed files w/o errors
|
||||
logging.warning("Captured error on file '%s': %s", file, e)
|
||||
return False
|
||||
|
||||
|
||||
def is_python(file: Union[Path, str]) -> bool:
|
||||
"""returns if the changed file in the repository is python script"""
|
||||
# WARNING: python-magic v2:0.4.24-2 is used in ubuntu 22.04,
|
||||
# and `Support os.PathLike values in magic.from_file` is only from 0.4.25
|
||||
# try:
|
||||
# return bool(
|
||||
# magic.from_file(os.path.join(REPO_COPY, file), mime=True)
|
||||
# == "text/x-script.python"
|
||||
# )
|
||||
# except IsADirectoryError:
|
||||
# # Process submodules w/o errors
|
||||
# return False
|
||||
return file.endswith(".py")
|
||||
return _check_mime(file, "text/x-script.python")
|
||||
|
||||
|
||||
def is_shell(file: str) -> bool:
|
||||
def is_shell(file: Union[Path, str]) -> bool:
|
||||
"""returns if the changed file in the repository is shell script"""
|
||||
# WARNING: python-magic v2:0.4.24-2 is used in ubuntu 22.04,
|
||||
# and `Support os.PathLike values in magic.from_file` is only from 0.4.25
|
||||
# try:
|
||||
# return bool(
|
||||
# magic.from_file(os.path.join(REPO_COPY, file), mime=True)
|
||||
# == "text/x-shellscript"
|
||||
# )
|
||||
# except IsADirectoryError:
|
||||
# # Process submodules w/o errors
|
||||
# return False
|
||||
return file.endswith(".sh")
|
||||
return _check_mime(file, "text/x-shellscript")
|
||||
|
||||
|
||||
def main():
|
||||
|
@ -843,6 +843,9 @@ class SettingsRandomizer:
|
||||
"prefer_warmed_unmerged_parts_seconds": lambda: random.randint(0, 10),
|
||||
"use_page_cache_for_disks_without_file_cache": lambda: random.random() < 0.7,
|
||||
"page_cache_inject_eviction": lambda: random.random() < 0.5,
|
||||
"merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability": lambda: round(
|
||||
random.random(), 2
|
||||
),
|
||||
}
|
||||
|
||||
@staticmethod
|
||||
|
@ -475,7 +475,7 @@ class ClickhouseIntegrationTestsRunner:
|
||||
return result
|
||||
|
||||
@staticmethod
|
||||
def _update_counters(main_counters, current_counters, broken_tests):
|
||||
def _update_counters(main_counters, current_counters):
|
||||
for test in current_counters["PASSED"]:
|
||||
if test not in main_counters["PASSED"]:
|
||||
if test in main_counters["FAILED"]:
|
||||
@ -483,21 +483,14 @@ class ClickhouseIntegrationTestsRunner:
|
||||
if test in main_counters["BROKEN"]:
|
||||
main_counters["BROKEN"].remove(test)
|
||||
|
||||
if test not in broken_tests:
|
||||
main_counters["PASSED"].append(test)
|
||||
else:
|
||||
main_counters["NOT_FAILED"].append(test)
|
||||
main_counters["PASSED"].append(test)
|
||||
|
||||
for state in ("ERROR", "FAILED"):
|
||||
for test in current_counters[state]:
|
||||
if test in main_counters["PASSED"]:
|
||||
main_counters["PASSED"].remove(test)
|
||||
if test not in broken_tests:
|
||||
if test not in main_counters[state]:
|
||||
main_counters[state].append(test)
|
||||
else:
|
||||
if test not in main_counters["BROKEN"]:
|
||||
main_counters["BROKEN"].append(test)
|
||||
if test not in main_counters[state]:
|
||||
main_counters[state].append(test)
|
||||
|
||||
for state in ("SKIPPED",):
|
||||
for test in current_counters[state]:
|
||||
@ -564,7 +557,6 @@ class ClickhouseIntegrationTestsRunner:
|
||||
tests_in_group,
|
||||
num_tries,
|
||||
num_workers,
|
||||
broken_tests,
|
||||
):
|
||||
try:
|
||||
return self.run_test_group(
|
||||
@ -573,7 +565,6 @@ class ClickhouseIntegrationTestsRunner:
|
||||
tests_in_group,
|
||||
num_tries,
|
||||
num_workers,
|
||||
broken_tests,
|
||||
)
|
||||
except Exception as e:
|
||||
logging.info("Failed to run %s:\n%s", test_group, e)
|
||||
@ -596,7 +587,6 @@ class ClickhouseIntegrationTestsRunner:
|
||||
tests_in_group,
|
||||
num_tries,
|
||||
num_workers,
|
||||
broken_tests,
|
||||
):
|
||||
counters = {
|
||||
"ERROR": [],
|
||||
@ -706,7 +696,7 @@ class ClickhouseIntegrationTestsRunner:
|
||||
)
|
||||
times_lines = parse_test_times(info_path)
|
||||
new_tests_times = get_test_times(times_lines)
|
||||
self._update_counters(counters, new_counters, broken_tests)
|
||||
self._update_counters(counters, new_counters)
|
||||
for test_name, test_time in new_tests_times.items():
|
||||
tests_times[test_name] = test_time
|
||||
|
||||
@ -783,7 +773,7 @@ class ClickhouseIntegrationTestsRunner:
|
||||
final_retry += 1
|
||||
logging.info("Running tests for the %s time", i)
|
||||
counters, tests_times, log_paths = self.try_run_test_group(
|
||||
repo_path, "bugfix" if should_fail else "flaky", tests_to_run, 1, 1, []
|
||||
repo_path, "bugfix" if should_fail else "flaky", tests_to_run, 1, 1
|
||||
)
|
||||
logs += log_paths
|
||||
if counters["FAILED"]:
|
||||
@ -915,20 +905,10 @@ class ClickhouseIntegrationTestsRunner:
|
||||
logging.info("Shuffling test groups")
|
||||
random.shuffle(items_to_run)
|
||||
|
||||
broken_tests = []
|
||||
if self.use_analyzer:
|
||||
with open(
|
||||
f"{repo_path}/tests/analyzer_integration_broken_tests.txt",
|
||||
"r",
|
||||
encoding="utf-8",
|
||||
) as f:
|
||||
broken_tests = f.read().splitlines()
|
||||
logging.info("Broken tests in the list: %s", len(broken_tests))
|
||||
|
||||
for group, tests in items_to_run:
|
||||
logging.info("Running test group %s containing %s tests", group, len(tests))
|
||||
group_counters, group_test_times, log_paths = self.try_run_test_group(
|
||||
repo_path, group, tests, MAX_RETRY, NUM_WORKERS, broken_tests
|
||||
repo_path, group, tests, MAX_RETRY, NUM_WORKERS
|
||||
)
|
||||
total_tests = 0
|
||||
for counter, value in group_counters.items():
|
||||
|
@ -7,11 +7,11 @@ INSERT INTO sorted (x) SELECT intDiv(number, 100000) AS x FROM system.numbers LI
|
||||
SET max_threads = 1;
|
||||
|
||||
SELECT count() FROM sorted;
|
||||
SELECT DISTINCT x FROM sorted;
|
||||
SELECT x FROM (SELECT DISTINCT x FROM sorted) ORDER BY x;
|
||||
|
||||
INSERT INTO sorted (x) SELECT (intHash64(number) % 1000 = 0 ? 999 : intDiv(number, 100000)) AS x FROM system.numbers LIMIT 1000000;
|
||||
|
||||
SELECT count() FROM sorted;
|
||||
SELECT DISTINCT x FROM sorted;
|
||||
SELECT x FROM (SELECT DISTINCT x FROM sorted) ORDER BY x;
|
||||
|
||||
DROP TABLE sorted;
|
||||
|
@ -1,8 +1,10 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS index_for_like;
|
||||
set allow_deprecated_syntax_for_merge_tree=1;
|
||||
CREATE TABLE index_for_like (s String, d Date DEFAULT today()) ENGINE = MergeTree(d, (s, d), 1);
|
||||
|
||||
INSERT INTO index_for_like (s) VALUES ('Hello'), ('Hello, World'), ('Hello, World 1'), ('Hello 1'), ('Goodbye'), ('Goodbye, World'), ('Goodbye 1'), ('Goodbye, World 1');
|
||||
INSERT INTO index_for_like (s) VALUES ('Hello'), ('Hello, World'), ('Hello, World 1'), ('Hello 1'), ('Goodbye'), ('Goodbye, World'), ('Goodbye 1'), ('Goodbye, World 1');
|
||||
|
||||
SET max_rows_to_read = 3;
|
||||
SELECT s FROM index_for_like WHERE s LIKE 'Hello, World%';
|
||||
|
@ -20,7 +20,7 @@ SETTINGS index_granularity = 1;
|
||||
INSERT INTO pk_in_tuple_perf SELECT number, number * 10 FROM numbers(100);
|
||||
EOF
|
||||
|
||||
query="SELECT count() FROM pk_in_tuple_perf WHERE (v, u) IN ((2, 10), (2, 20))"
|
||||
query="SELECT count() FROM pk_in_tuple_perf WHERE (v, u) IN ((2, 10), (2, 20)) SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "$query"
|
||||
$CLICKHOUSE_CLIENT --query "$query FORMAT JSON" | grep "rows_read"
|
||||
@ -40,7 +40,7 @@ SETTINGS index_granularity = 1;
|
||||
INSERT INTO pk_in_tuple_perf_non_const SELECT today() - number, number FROM numbers(100);
|
||||
EOF
|
||||
|
||||
query="SELECT count() FROM pk_in_tuple_perf_non_const WHERE (u, d) IN ((0, today()), (1, today()))"
|
||||
query="SELECT count() FROM pk_in_tuple_perf_non_const WHERE (u, d) IN ((0, today()), (1, today())) SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query "$query"
|
||||
$CLICKHOUSE_CLIENT --query "$query FORMAT JSON" | grep "rows_read"
|
||||
|
@ -31,31 +31,31 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
(8, 1, 2),
|
||||
(9, 1, 2)"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
ALTER TABLE minmax_idx ADD INDEX idx (i64, u64 * i64) TYPE minmax GRANULARITY 1 SETTINGS mutations_sync = 2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1 SETTINGS mutations_sync = 2;"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 2 SETTINGS mutations_sync = 2"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1 SETTINGS mutations_sync = 2"
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 2 SETTINGS mutations_sync = 2"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx SETTINGS mutations_sync = 2"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
|
@ -32,17 +32,17 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO minmax_idx VALUES
|
||||
(8, 1, 2),
|
||||
(9, 1, 2)"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" # Returns 4
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read" # Returns 4
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx CLEAR INDEX idx IN PARTITION 1;" --mutations_sync=1
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" # Returns 6
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read" # Returns 6
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE minmax_idx MATERIALIZE INDEX idx IN PARTITION 1;" --mutations_sync=1
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 FORMAT JSON" | grep "rows_read" # Returns 4
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM minmax_idx WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read" # Returns 4
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE minmax_idx"
|
||||
|
@ -1,4 +1,5 @@
|
||||
SET allow_suspicious_low_cardinality_types=1;
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS single_column_bloom_filter;
|
||||
|
||||
|
@ -32,9 +32,9 @@ $CLICKHOUSE_CLIENT --query="insert into lowString (a, b) select top 100000 toStr
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="insert into string (a, b) select top 100000 toString(number), today() from system.numbers"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="select count() from lowString where a in ('1', '2') FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="select count() from lowString where a in ('1', '2') SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="select count() from string where a in ('1', '2') FORMAT JSON" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="select count() from string where a in ('1', '2') SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE lowString;"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE string;"
|
||||
|
@ -49,18 +49,18 @@ $CLICKHOUSE_CLIENT --query="INSERT INTO indices_mutaions1 VALUES
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SYSTEM SYNC REPLICA indices_mutaions2"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 CLEAR INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="ALTER TABLE indices_mutaions1 MATERIALIZE INDEX idx IN PARTITION 1;" --replication_alter_partitions_sync=2 --mutations_sync=2
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 FORMAT JSON;" | grep "rows_read"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
$CLICKHOUSE_CLIENT --query="SELECT count() FROM indices_mutaions2 WHERE i64 = 2 SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0 FORMAT JSON;" | grep "rows_read"
|
||||
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions1"
|
||||
$CLICKHOUSE_CLIENT --query="DROP TABLE indices_mutaions2"
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS test1;
|
||||
DROP TABLE IF EXISTS test2;
|
||||
DROP TABLE IF EXISTS test3;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS minmax_compact;
|
||||
|
||||
CREATE TABLE minmax_compact
|
||||
|
@ -1,4 +1,4 @@
|
||||
-- Tags: no-debug, no-parallel, long, no-s3-storage, no-random-merge-tree-settings
|
||||
-- Tags: no-debug, no-parallel, long, no-s3-storage, no-random-settings, no-random-merge-tree-settings
|
||||
|
||||
DROP TABLE IF EXISTS table_with_single_pk;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS test1;
|
||||
DROP TABLE IF EXISTS test2;
|
||||
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
CREATE TABLE test (x UInt64) ENGINE = MergeTree ORDER BY x SETTINGS index_granularity = 1000, index_granularity_bytes = '10Mi';
|
||||
INSERT INTO test SELECT * FROM numbers(1000000);
|
||||
|
@ -1,5 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: long, no-polymorphic-parts, no-random-merge-tree-settings, no-debug
|
||||
# Tags: long, no-polymorphic-parts, no-random-settings, no-random-merge-tree-settings, no-debug
|
||||
|
||||
# Description of test result:
|
||||
# Test the correctness of the partition pruning
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: global
|
||||
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
drop table if exists xp;
|
||||
drop table if exists xp_d;
|
||||
|
||||
|
@ -6,7 +6,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; SET optimize_substitute_columns = 1; SET optimize_append_index = 1"
|
||||
SETTINGS="SET convert_query_to_cnf = 1; SET optimize_using_constraints = 1; SET optimize_move_to_prewhere = 1; SET optimize_substitute_columns = 1; SET optimize_append_index = 1; SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;"
|
||||
|
||||
$CLICKHOUSE_CLIENT -n --query="
|
||||
$SETTINGS;
|
||||
|
@ -5,7 +5,7 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
. "$CURDIR"/../shell_config.sh
|
||||
|
||||
# Number of read rows depends on max_bytes_before_external_group_by.
|
||||
CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --max_bytes_before_external_group_by 0"
|
||||
CLICKHOUSE_CLIENT="$CLICKHOUSE_CLIENT --max_bytes_before_external_group_by 0 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.0"
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_agg_proj (x Int32, y Int32, PROJECTION x_plus_y (SELECT sum(x - y), argMax(x, y) group by x + y)) ENGINE = MergeTree ORDER BY tuple() settings index_granularity = 1"
|
||||
$CLICKHOUSE_CLIENT -q "insert into test_agg_proj select intDiv(number, 2), -intDiv(number,3) - 1 from numbers(100)"
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
drop table if exists t;
|
||||
|
||||
create table t (i int, j int, k int, projection p (select * order by j)) engine MergeTree order by i settings index_granularity = 1;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
drop table if exists x;
|
||||
|
||||
create table x (i int, j int) engine MergeTree partition by i order by j settings index_granularity = 1;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS t_max_rows_to_read;
|
||||
|
||||
CREATE TABLE t_max_rows_to_read (a UInt64)
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-s3-storage
|
||||
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
drop table if exists data_02233;
|
||||
create table data_02233 (parent_key Int, child_key Int, value Int) engine=MergeTree() order by parent_key;
|
||||
|
||||
|
@ -1,6 +1,8 @@
|
||||
-- Tags: no-parallel, no-fasttest
|
||||
-- Tag no-fasttest: Depends on S3
|
||||
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
-- { echo }
|
||||
drop table if exists test_02302;
|
||||
create table test_02302 (a UInt64) engine = S3(s3_conn, filename='test_02302_{_partition_id}', format=Parquet) partition by a;
|
||||
|
@ -15,11 +15,13 @@ function check_refcnt_for_table()
|
||||
local query_id
|
||||
query_id="$table-$(random_str 10)"
|
||||
|
||||
SETTINGS="--format Null --max_threads 1 --max_block_size 1 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability 0.0"
|
||||
|
||||
# Notes:
|
||||
# - query may sleep 1*(200/4)=50 seconds maximum, it is enough to check system.parts
|
||||
# - "part = 1" condition should prune all parts except first
|
||||
# - max_block_size=1 with index_granularity=1 will allow to cancel the query earlier
|
||||
$CLICKHOUSE_CLIENT --format Null --max_threads 1 --max_block_size 1 --query_id "$query_id" -q "select sleepEachRow(1) from $table where part = 1" &
|
||||
$CLICKHOUSE_CLIENT $SETTINGS --query_id "$query_id" -q "select sleepEachRow(1) from $table where part = 1" &
|
||||
PID=$!
|
||||
|
||||
# wait for query to be started
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
-- produces different pipeline if enabled
|
||||
set enable_memory_bound_merging_of_aggregation_results = 0;
|
||||
set merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
set max_threads = 16;
|
||||
set prefer_localhost_replica = 1;
|
||||
|
@ -1,5 +1,6 @@
|
||||
SET allow_experimental_inverted_index = 1;
|
||||
SET log_queries = 1;
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
----------------------------------------------------
|
||||
SELECT 'Test inverted(2)';
|
||||
@ -23,7 +24,7 @@ SELECT * FROM tab WHERE s == 'Alick a01';
|
||||
|
||||
-- check the query only read 1 granules (2 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==2 from system.query_log
|
||||
SELECT read_rows==2 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE s == \'Alick a01\';')
|
||||
@ -36,7 +37,7 @@ SELECT * FROM tab WHERE s LIKE '%01%' ORDER BY k;
|
||||
|
||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==4 from system.query_log
|
||||
SELECT read_rows==4 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE s LIKE \'%01%\' ORDER BY k;')
|
||||
@ -49,11 +50,11 @@ SELECT * FROM tab WHERE hasToken(s, 'Click') ORDER BY k;
|
||||
|
||||
-- check the query only read 4 granules (8 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==8 from system.query_log
|
||||
SELECT read_rows==8 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE hasToken(s, \'Click\') ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==4
|
||||
LIMIT 1;
|
||||
|
||||
@ -76,11 +77,11 @@ SELECT * FROM tab_x WHERE hasToken(s, 'Alick') ORDER BY k;
|
||||
|
||||
-- check the query only read 4 granules (8 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==8 from system.query_log
|
||||
SELECT read_rows==8 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE hasToken(s, \'Alick\');')
|
||||
AND type='QueryFinish'
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE hasToken(s, \'Alick\');')
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==4
|
||||
LIMIT 1;
|
||||
|
||||
@ -89,24 +90,24 @@ SELECT * FROM tab_x WHERE s IN ('Alick a01', 'Alick a06') ORDER BY k;
|
||||
|
||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==4 from system.query_log
|
||||
SELECT read_rows==4 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE s IN (\'Alick a01\', \'Alick a06\') ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE s IN (\'Alick a01\', \'Alick a06\') ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==2
|
||||
LIMIT 1;
|
||||
|
||||
-- search inverted index with multiSearch
|
||||
-- search inverted index with multiSearch
|
||||
SELECT * FROM tab_x WHERE multiSearchAny(s, ['a01', 'b01']) ORDER BY k;
|
||||
|
||||
-- check the query only read 2 granules (4 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==4 from system.query_log
|
||||
SELECT read_rows==4 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE multiSearchAny(s, [\'a01\', \'b01\']) ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab_x WHERE multiSearchAny(s, [\'a01\', \'b01\']) ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==2
|
||||
LIMIT 1;
|
||||
|
||||
@ -129,11 +130,11 @@ SELECT * FROM tab WHERE has(s, 'Click a03') ORDER BY k;
|
||||
|
||||
-- check the query must read all 10 granules (20 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==2 from system.query_log
|
||||
SELECT read_rows==2 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE has(s, \'Click a03\') ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE has(s, \'Click a03\') ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==1
|
||||
LIMIT 1;
|
||||
|
||||
@ -156,11 +157,11 @@ SELECT * FROM tab WHERE mapContains(s, 'Click') ORDER BY k;
|
||||
|
||||
-- check the query must read all 4 granules (8 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==8 from system.query_log
|
||||
SELECT read_rows==8 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE mapContains(s, \'Click\') ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE mapContains(s, \'Click\') ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==4
|
||||
LIMIT 1;
|
||||
|
||||
@ -169,11 +170,11 @@ SELECT * FROM tab WHERE s['Click'] = 'Click a03';
|
||||
|
||||
-- check the query must read all 4 granules (8 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==8 from system.query_log
|
||||
SELECT read_rows==8 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE s[\'Click\'] = \'Click a03\';')
|
||||
AND type='QueryFinish'
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE s[\'Click\'] = \'Click a03\';')
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==1
|
||||
LIMIT 1;
|
||||
|
||||
@ -199,10 +200,10 @@ SELECT * FROM tab WHERE s LIKE '%01%' ORDER BY k;
|
||||
-- check the query only read 3 granules (6 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==6 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE s LIKE \'%01%\' ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==3
|
||||
LIMIT 1;
|
||||
|
||||
@ -226,11 +227,11 @@ SELECT * FROM tab WHERE s LIKE '%你好%' ORDER BY k;
|
||||
|
||||
-- check the query only read 1 granule (2 rows total; each granule has 2 rows)
|
||||
SYSTEM FLUSH LOGS;
|
||||
SELECT read_rows==2 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE s LIKE \'%你好%\' ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
SELECT read_rows==2 from system.query_log
|
||||
WHERE query_kind ='Select'
|
||||
AND current_database = currentDatabase()
|
||||
AND endsWith(trimRight(query), 'SELECT * FROM tab WHERE s LIKE \'%你好%\' ORDER BY k;')
|
||||
AND type='QueryFinish'
|
||||
AND result_rows==1
|
||||
LIMIT 1;
|
||||
|
||||
|
@ -15,14 +15,14 @@ FIND_SORTMODE="$GREP_SORTMODE | $TRIM_LEADING_SPACES"
|
||||
|
||||
function explain_sorting {
|
||||
echo "-- QUERY: "$1
|
||||
$CLICKHOUSE_CLIENT -nq "$1" | eval $FIND_SORTING
|
||||
$CLICKHOUSE_CLIENT --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTING
|
||||
}
|
||||
|
||||
function explain_sortmode {
|
||||
echo "-- QUERY: "$1
|
||||
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 -nq "$1" | eval $FIND_SORTMODE
|
||||
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=0 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTMODE
|
||||
echo "-- QUERY (analyzer): "$1
|
||||
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 -nq "$1" | eval $FIND_SORTMODE
|
||||
$CLICKHOUSE_CLIENT --allow_experimental_analyzer=1 --merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0 -nq "$1" | eval $FIND_SORTMODE
|
||||
}
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "drop table if exists optimize_sorting sync"
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
-- #40014
|
||||
CREATE TABLE m0 (id UInt64) ENGINE=MergeTree ORDER BY id SETTINGS index_granularity = 1, ratio_of_defaults_for_sparse_serialization = 1.0;
|
||||
INSERT INTO m0 SELECT number FROM numbers(10);
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS constCondOptimization;
|
||||
|
||||
CREATE TABLE constCondOptimization
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-random-merge-tree-settings
|
||||
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
drop table if exists t;
|
||||
create table t (x UInt64) engine = MergeTree order by x;
|
||||
insert into t select number from numbers_mt(10000000) settings max_insert_threads=8;
|
||||
|
@ -15,9 +15,11 @@ ${CLICKHOUSE_CLIENT} --query "CREATE TABLE tab (a UInt64) ENGINE=MergeTree() ORD
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (1) (2) (3)"
|
||||
${CLICKHOUSE_CLIENT} --query "INSERT INTO tab VALUES (3) (4) (5)"
|
||||
|
||||
SETTINGS="SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0, merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability=0.0"
|
||||
|
||||
# Verify that the first query does two aggregations and the second query zero aggregations. Since query cache is currently not integrated
|
||||
# with EXPLAIN PLAN, we need need to check the logs.
|
||||
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0" 2>&1 | grep "Aggregated. " | wc -l
|
||||
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab SETTINGS use_query_cache=1, max_threads=1, allow_experimental_analyzer=0" 2>&1 | grep "Aggregated. " | wc -l
|
||||
# with EXPLAIN PLAN, we need to check the logs.
|
||||
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l
|
||||
${CLICKHOUSE_CLIENT} --send_logs_level=trace --query "SELECT count(a) / (SELECT sum(a) FROM tab) FROM tab $SETTINGS" 2>&1 | grep "Aggregated. " | wc -l
|
||||
|
||||
${CLICKHOUSE_CLIENT} --query "SYSTEM DROP QUERY CACHE"
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: long, no-s3-storage
|
||||
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
set max_threads = 16;
|
||||
set allow_aggregate_partitions_independently = 1;
|
||||
set force_aggregate_partitions_independently = 1;
|
||||
|
@ -17,6 +17,6 @@ $CLICKHOUSE_CLIENT -nm -q "
|
||||
# instead of "last" value, hence you cannot simply append another
|
||||
# --send_logs_level here.
|
||||
CLICKHOUSE_CLIENT_CLEAN=$(echo ${CLICKHOUSE_CLIENT} | sed 's/'"--send_logs_level=${CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL}"'/--send_logs_level=test/g')
|
||||
$CLICKHOUSE_CLIENT_CLEAN -q "select * from data" |& grep -o -e '<Unknown>.*' -e '<Test>.*'
|
||||
$CLICKHOUSE_CLIENT_CLEAN -q "select * from data SETTINGS merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;" |& grep -o -e '<Unknown>.*' -e '<Test>.*'
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "drop table data"
|
||||
|
@ -1,5 +1,7 @@
|
||||
-- Tags: no-s3-storage
|
||||
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
drop table if exists t;
|
||||
|
||||
create table t(a UInt64) engine=MergeTree order by tuple();
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
CREATE TABLE test (id UInt64, `amax` AggregateFunction(argMax, String, DateTime))
|
||||
ENGINE=MergeTree()
|
||||
ORDER BY id
|
||||
|
@ -10,3 +10,4 @@ $CLICKHOUSE_CLIENT -q "insert into t1 select number from numbers(10);"
|
||||
$CLICKHOUSE_CLIENT --max_threads=2 --max_result_rows=1 --result_overflow_mode=break -q "with tab as (select min(number) from t1 prewhere number in (select number from view(select number, row_number() OVER (partition by number % 2 ORDER BY number DESC) from numbers_mt(1e4)) where number != 2 order by number)) select number from t1 union all select * from tab;" > /dev/null
|
||||
|
||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM system.tables WHERE 1 in (SELECT number from numbers(2)) AND database = currentDatabase() format Null"
|
||||
$CLICKHOUSE_CLIENT -q "SELECT xor(1, 0) FROM system.parts WHERE 1 IN (SELECT 1) FORMAT Null"
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS t_sparse_distinct;
|
||||
|
||||
CREATE TABLE t_sparse_distinct (id UInt32, v String)
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS test;
|
||||
|
||||
CREATE TABLE test (x UInt32, y UInt32) ENGINE = MergeTree ORDER BY mortonEncode(x, y) SETTINGS index_granularity = 8192, index_granularity_bytes = '1Mi';
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS ttl_group_by_bug;
|
||||
|
||||
CREATE TABLE ttl_group_by_bug
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
drop table if exists a;
|
||||
|
||||
create table a (i int) engine MergeTree order by i settings index_granularity = 2;
|
||||
|
@ -1,3 +1,5 @@
|
||||
SET merge_tree_read_split_ranges_into_intersecting_and_non_intersecting_injection_probability = 0.0;
|
||||
|
||||
DROP TABLE IF EXISTS test_empty;
|
||||
CREATE TABLE test_empty (a Array(Int64)) engine=MergeTree ORDER BY a;
|
||||
INSERT INTO test_empty VALUES ([]);
|
||||
|
@ -0,0 +1,2 @@
|
||||
a b
|
||||
a b
|
@ -0,0 +1 @@
|
||||
SELECT 'a' AS key, 'b' as value GROUP BY key WITH CUBE SETTINGS group_by_use_nulls = 1;
|
2
tests/queries/0_stateless/03014_window_view_crash.sql
Normal file
2
tests/queries/0_stateless/03014_window_view_crash.sql
Normal file
@ -0,0 +1,2 @@
|
||||
SET allow_experimental_window_view = 1;
|
||||
SELECT hopEnd(tuple([[[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]], [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]], [[(20,20),(50,20),(50,50),(20,50)],[(30,30),(50,50),(50,30)]]])::Tuple(MultiPolygon)); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
|
@ -161,7 +161,9 @@ ClickHouseNIO
|
||||
ClickHouseVapor
|
||||
ClickVisual
|
||||
ClickableSquare
|
||||
CloudAvailableBadge
|
||||
CloudDetails
|
||||
CloudNotSupportedBadge
|
||||
CloudStorage
|
||||
CodeBlock
|
||||
CodeLLDB
|
||||
|
Loading…
Reference in New Issue
Block a user