Merge branch 'master' into json-object-as-tuple-inference

This commit is contained in:
Kruglov Pavel 2023-09-26 15:23:08 +02:00 committed by GitHub
commit bea80ab5b7
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
103 changed files with 1820 additions and 337 deletions

View File

@ -22,9 +22,11 @@ curl https://clickhouse.com/ | sh
## Upcoming Events ## Upcoming Events
* [**v23.8 Community Call**](https://clickhouse.com/company/events/v23-8-community-release-call?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-08) - Aug 31 - 23.8 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release. * [**v23.9 Community Call**]([https://clickhouse.com/company/events/v23-8-community-release-call](https://clickhouse.com/company/events/v23-9-community-release-call)?utm_source=github&utm_medium=social&utm_campaign=release-webinar-2023-08) - Sep 28 - 23.9 is rapidly approaching. Original creator, co-founder, and CTO of ClickHouse Alexey Milovidov will walk us through the highlights of the release.
* [**ClickHouse & AI - A Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/294472987) - Aug 8 * [**ClickHouse Meetup in Amsterdam**](https://www.meetup.com/clickhouse-netherlands-user-group/events/296334590/)) - Oct 31
* [**ClickHouse Meetup in Paris**](https://www.meetup.com/clickhouse-france-user-group/events/294283460) - Sep 12 * [**ClickHouse Meetup in Beijing**](https://www.meetup.com/clickhouse-beijing-user-group/events/296334856/) - Nov 4
* [**ClickHouse Meetup in San Francisco**](https://www.meetup.com/clickhouse-silicon-valley-meetup-group/events/296334923/) - Nov 14
* [**ClickHouse Meetup in Singapore**](https://www.meetup.com/clickhouse-singapore-meetup-group/events/296334976/) - Nov 15
Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler <at> clickhouse <dot> com. Also, keep an eye out for upcoming meetups around the world. Somewhere else you want us to be? Please feel free to reach out to tyler <at> clickhouse <dot> com.

View File

@ -209,6 +209,10 @@ constraint_name_1 CHECK length(vectors) = 256`. Also, empty `Arrays` and unspeci
values) are not supported. values) are not supported.
::: :::
The creation of Annoy indexes (whenever a new part is build, e.g. at the end of a merge) is a relatively slow process. You can increase
setting `max_threads_for_annoy_index_creation` (default: 4) which controls how many threads are used to create an Annoy index. Please be
careful with this setting, it is possible that multiple indexes are created in parallel in which case there can be overparallelization.
Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger Setting `annoy_index_search_k_nodes` (default: `NumTrees * LIMIT`) determines how many tree nodes are inspected during SELECTs. Larger
values mean more accurate results at the cost of longer query runtime: values mean more accurate results at the cost of longer query runtime:

View File

@ -720,3 +720,84 @@ $ curl -vv -H 'XXX:xxx' 'http://localhost:8123/get_relative_path_static_handler'
<html><body>Relative Path File</body></html> <html><body>Relative Path File</body></html>
* Connection #0 to host localhost left intact * Connection #0 to host localhost left intact
``` ```
## Valid JSON/XML response on exception during HTTP streaming {valid-output-on-exception-http-streaming}
While query execution over HTTP an exception can happen when part of the data has already been sent. Usually an exception is sent to the client in plain text
even if some specific data format was used to output data and the output may become invalid in terms of specified data format.
To prevent it, you can use setting `http_write_exception_in_output_format` (enabled by default) that will tell ClickHouse to write an exception in specified format (currently supported for XML and JSON* formats).
Examples:
```bash
$ curl 'http://localhost:8123/?query=SELECT+number,+throwIf(number>3)+from+system.numbers+format+JSON+settings+max_block_size=1&http_write_exception_in_output_format=1'
{
"meta":
[
{
"name": "number",
"type": "UInt64"
},
{
"name": "throwIf(greater(number, 2))",
"type": "UInt8"
}
],
"data":
[
{
"number": "0",
"throwIf(greater(number, 2))": 0
},
{
"number": "1",
"throwIf(greater(number, 2))": 0
},
{
"number": "2",
"throwIf(greater(number, 2))": 0
}
],
"rows": 3,
"exception": "Code: 395. DB::Exception: Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 2) :: 2) -> throwIf(greater(number, 2)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) (version 23.8.1.1)"
}
```
```bash
$ curl 'http://localhost:8123/?query=SELECT+number,+throwIf(number>2)+from+system.numbers+format+XML+settings+max_block_size=1&http_write_exception_in_output_format=1'
<?xml version='1.0' encoding='UTF-8' ?>
<result>
<meta>
<columns>
<column>
<name>number</name>
<type>UInt64</type>
</column>
<column>
<name>throwIf(greater(number, 2))</name>
<type>UInt8</type>
</column>
</columns>
</meta>
<data>
<row>
<number>0</number>
<field>0</field>
</row>
<row>
<number>1</number>
<field>0</field>
</row>
<row>
<number>2</number>
<field>0</field>
</row>
</data>
<rows>3</rows>
<exception>Code: 395. DB::Exception: Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 2) :: 2) -> throwIf(greater(number, 2)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) (version 23.8.1.1)</exception>
</result>
```

View File

@ -555,7 +555,7 @@ Merge reads rows from parts in blocks of `merge_max_block_size` rows, then merge
## number_of_free_entries_in_pool_to_lower_max_size_of_merge {#number-of-free-entries-in-pool-to-lower-max-size-of-merge} ## number_of_free_entries_in_pool_to_lower_max_size_of_merge {#number-of-free-entries-in-pool-to-lower-max-size-of-merge}
When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue). When there is less than specified number of free entries in pool (or replicated queue), start to lower maximum size of merge to process (or to put in queue).
This is to allow small merges to process - not filling the pool with long running merges. This is to allow small merges to process - not filling the pool with long running merges.
Possible values: Possible values:
@ -566,7 +566,7 @@ Default value: 8
## number_of_free_entries_in_pool_to_execute_mutation {#number-of-free-entries-in-pool-to-execute-mutation} ## number_of_free_entries_in_pool_to_execute_mutation {#number-of-free-entries-in-pool-to-execute-mutation}
When there is less than specified number of free entries in pool, do not execute part mutations. When there is less than specified number of free entries in pool, do not execute part mutations.
This is to leave free threads for regular merges and avoid "Too many parts". This is to leave free threads for regular merges and avoid "Too many parts".
Possible values: Possible values:
@ -845,6 +845,13 @@ You can see which parts of `s` were stored using the sparse serialization:
└────────┴────────────────────┘ └────────┴────────────────────┘
``` ```
## replace_long_file_name_to_hash {#replace_long_file_name_to_hash}
If the file name for column is too long (more than `max_file_name_length` bytes) replace it to SipHash128. Default value: `false`.
## max_file_name_length {#max_file_name_length}
The maximal length of the file name to keep it as is without hashing. Takes effect only if setting `replace_long_file_name_to_hash` is enabled. The value of this setting does not include the length of file extension. So, it is recommended to set it below the maximum filename length (usually 255 bytes) with some gap to avoid filesystem errors. Default value: 127.
## clean_deleted_rows ## clean_deleted_rows
Enable/disable automatic deletion of rows flagged as `is_deleted` when perform `OPTIMIZE ... FINAL` on a table using the ReplacingMergeTree engine. When disabled, the `CLEANUP` keyword has to be added to the `OPTIMIZE ... FINAL` to have the same behaviour. Enable/disable automatic deletion of rows flagged as `is_deleted` when perform `OPTIMIZE ... FINAL` on a table using the ReplacingMergeTree engine. When disabled, the `CLEANUP` keyword has to be added to the `OPTIMIZE ... FINAL` to have the same behaviour.

View File

@ -22,6 +22,7 @@
#include <base/extended_types.h> #include <base/extended_types.h>
#include <base/types.h> #include <base/types.h>
#include <base/unaligned.h> #include <base/unaligned.h>
#include <base/hex.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <city.h> #include <city.h>
@ -257,6 +258,16 @@ inline UInt128 sipHash128(const char * data, const size_t size)
return sipHash128Keyed(0, 0, data, size); return sipHash128Keyed(0, 0, data, size);
} }
inline String sipHash128String(const char * data, const size_t size)
{
return getHexUIntLowercase(sipHash128(data, size));
}
inline String sipHash128String(const String & str)
{
return sipHash128String(str.data(), str.size());
}
inline UInt128 sipHash128ReferenceKeyed(UInt64 key0, UInt64 key1, const char * data, const size_t size) inline UInt128 sipHash128ReferenceKeyed(UInt64 key0, UInt64 key1, const char * data, const size_t size)
{ {
SipHash hash(key0, key1, true); SipHash hash(key0, key1, true);

View File

@ -280,6 +280,7 @@ class IColumn;
\ \
M(UInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \ M(UInt64, http_headers_progress_interval_ms, 100, "Do not send HTTP headers X-ClickHouse-Progress more frequently than at each specified interval.", 0) \
M(Bool, http_wait_end_of_query, false, "Enable HTTP response buffering on the server-side.", 0) \ M(Bool, http_wait_end_of_query, false, "Enable HTTP response buffering on the server-side.", 0) \
M(Bool, http_write_exception_in_output_format, true, "Write exception in output format to produce valid output. Works with JSON and XML formats.", 0) \
M(UInt64, http_response_buffer_size, 0, "The number of bytes to buffer in the server memory before sending a HTTP response to the client or flushing to disk (when http_wait_end_of_query is enabled).", 0) \ M(UInt64, http_response_buffer_size, 0, "The number of bytes to buffer in the server memory before sending a HTTP response to the client or flushing to disk (when http_wait_end_of_query is enabled).", 0) \
\ \
M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \ M(Bool, fsync_metadata, true, "Do fsync after changing metadata for tables and databases (.sql files). Could be disabled in case of poor latency on server with high load of DDL queries and high load of disk subsystem.", 0) \
@ -791,6 +792,7 @@ class IColumn;
M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \ M(Bool, allow_experimental_usearch_index, false, "Allows to use USearch index. Disabled by default because this feature is experimental", 0) \
M(Bool, allow_experimental_s3queue, false, "Allows to use S3Queue engine. Disabled by default, because this feature is experimental", 0) \ M(Bool, allow_experimental_s3queue, false, "Allows to use S3Queue engine. Disabled by default, because this feature is experimental", 0) \
M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \ M(UInt64, max_limit_for_ann_queries, 1'000'000, "SELECT queries with LIMIT bigger than this setting cannot use ANN indexes. Helps to prevent memory overflows in ANN search indexes.", 0) \
M(UInt64, max_threads_for_annoy_index_creation, 4, "Number of threads used to build Annoy indexes (0 means all cores, not recommended)", 0) \
M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \ M(Int64, annoy_index_search_k_nodes, -1, "SELECT queries search up to this many nodes in Annoy indexes.", 0) \
M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \ M(Bool, throw_on_unsupported_query_inside_transaction, true, "Throw exception if unsupported query is used inside transaction", 0) \
M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \ M(TransactionsWaitCSNMode, wait_changes_become_visible_after_commit_mode, TransactionsWaitCSNMode::WAIT_UNKNOWN, "Wait for committed changes to become actually visible in the latest snapshot", 0) \

View File

@ -84,7 +84,8 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
{"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"}, {"input_format_json_try_infer_named_tuples_from_objects", false, true, "Try to infer named Tuples from JSON objects by default"},
{"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"}, {"input_format_json_read_numbers_as_strings", false, true, "Allow to read numbers as strings in JSON formats by default"},
{"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"}, {"input_format_json_read_arrays_as_strings", false, true, "Allow to read arrays as strings in JSON formats by default"},
{"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"}}}, {"input_format_json_infer_incomplete_types_as_strings", false, true, "Allow to infer incomplete types as Strings in JSON formats by default"},
{"http_write_exception_in_output_format", false, true, "Output valid JSON/XML on exception in HTTP streaming."}}},
{"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}}, {"23.8", {{"rewrite_count_distinct_if_with_count_distinct_implementation", false, true, "Rewrite countDistinctIf with count_distinct_implementation configuration"}}},
{"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}}, {"23.7", {{"function_sleep_max_microseconds_per_block", 0, 3000000, "In previous versions, the maximum sleep time of 3 seconds was applied only for `sleep`, but not for `sleepEachRow` function. In the new version, we introduce this setting. If you set compatibility with the previous versions, we will disable the limit altogether."}}},
{"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."}, {"23.6", {{"http_send_timeout", 180, 30, "3 minutes seems crazy long. Note that this is timeout for a single network write call, not for the whole upload operation."},

View File

@ -164,7 +164,7 @@ NameToDataType getSubcolumnsOfNested(const NamesAndTypesList & names_and_types)
std::unordered_map<String, NamesAndTypesList> nested; std::unordered_map<String, NamesAndTypesList> nested;
for (const auto & name_type : names_and_types) for (const auto & name_type : names_and_types)
{ {
const DataTypeArray * type_arr = typeid_cast<const DataTypeArray *>(name_type.type.get()); const auto * type_arr = typeid_cast<const DataTypeArray *>(name_type.type.get());
/// Ignore true Nested type, but try to unite flatten arrays to Nested type. /// Ignore true Nested type, but try to unite flatten arrays to Nested type.
if (!isNested(name_type.type) && type_arr) if (!isNested(name_type.type) && type_arr)
@ -191,8 +191,11 @@ NamesAndTypesList collect(const NamesAndTypesList & names_and_types)
auto nested_types = getSubcolumnsOfNested(names_and_types); auto nested_types = getSubcolumnsOfNested(names_and_types);
for (const auto & name_type : names_and_types) for (const auto & name_type : names_and_types)
if (!isArray(name_type.type) || !nested_types.contains(splitName(name_type.name).first)) {
auto split = splitName(name_type.name);
if (!isArray(name_type.type) || split.second.empty() || !nested_types.contains(split.first))
res.push_back(name_type); res.push_back(name_type);
}
for (const auto & name_type : nested_types) for (const auto & name_type : nested_types)
res.emplace_back(name_type.first, name_type.second); res.emplace_back(name_type.first, name_type.second);

View File

@ -370,6 +370,7 @@ public:
static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path); static String getFileNameForStream(const NameAndTypePair & column, const SubstreamPath & path);
static String getFileNameForStream(const String & name_in_storage, const SubstreamPath & path); static String getFileNameForStream(const String & name_in_storage, const SubstreamPath & path);
static String getSubcolumnNameForStream(const SubstreamPath & path); static String getSubcolumnNameForStream(const SubstreamPath & path);
static String getSubcolumnNameForStream(const SubstreamPath & path, size_t prefix_len); static String getSubcolumnNameForStream(const SubstreamPath & path, size_t prefix_len);

View File

@ -1,7 +1,6 @@
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <algorithm> #include <algorithm>
#include <Core/Settings.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/ProcessList.h> #include <Interpreters/ProcessList.h>
@ -233,6 +232,12 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
context->getRemoteHostFilter().checkURL(avro_schema_registry_url); context->getRemoteHostFilter().checkURL(avro_schema_registry_url);
} }
if (context->getClientInfo().interface == ClientInfo::Interface::HTTP && context->getSettingsRef().http_write_exception_in_output_format.value)
{
format_settings.json.valid_output_on_exception = true;
format_settings.xml.valid_output_on_exception = true;
}
return format_settings; return format_settings;
} }

View File

@ -199,6 +199,7 @@ struct FormatSettings
bool validate_types_from_metadata = true; bool validate_types_from_metadata = true;
bool validate_utf8 = false; bool validate_utf8 = false;
bool allow_object_type = false; bool allow_object_type = false;
bool valid_output_on_exception = false;
bool compact_allow_variable_number_of_columns = false; bool compact_allow_variable_number_of_columns = false;
bool try_infer_objects_as_tuples = false; bool try_infer_objects_as_tuples = false;
bool infer_incomplete_types_as_strings = true; bool infer_incomplete_types_as_strings = true;
@ -418,6 +419,11 @@ struct FormatSettings
bool allow_types_conversion = true; bool allow_types_conversion = true;
} native; } native;
struct
{
bool valid_output_on_exception = false;
} xml;
struct struct
{ {
bool escape_special_characters = false; bool escape_special_characters = false;

View File

@ -531,6 +531,12 @@ namespace JSONUtils
} }
} }
void writeException(const String & exception_message, WriteBuffer & out, const FormatSettings & settings, size_t indent)
{
writeTitle("exception", out, indent, " ");
writeJSONString(exception_message, out, settings);
}
Strings makeNamesValidJSONStrings(const Strings & names, const FormatSettings & settings, bool validate_utf8) Strings makeNamesValidJSONStrings(const Strings & names, const FormatSettings & settings, bool validate_utf8)
{ {
Strings result; Strings result;

View File

@ -108,6 +108,8 @@ namespace JSONUtils
bool write_statistics, bool write_statistics,
WriteBuffer & out); WriteBuffer & out);
void writeException(const String & exception_message, WriteBuffer & out, const FormatSettings & settings, size_t indent = 0);
void skipColon(ReadBuffer & in); void skipColon(ReadBuffer & in);
void skipComma(ReadBuffer & in); void skipComma(ReadBuffer & in);

View File

@ -973,9 +973,7 @@ public:
} }
for (auto & instruction : instructions) for (auto & instruction : instructions)
{
instruction.perform(pos, static_cast<Int64>(c.whole), c.fractional, scale, *time_zone); instruction.perform(pos, static_cast<Int64>(c.whole), c.fractional, scale, *time_zone);
}
} }
else else
{ {

View File

@ -0,0 +1,85 @@
#include <IO/PeekableWriteBuffer.h>
namespace DB
{
PeekableWriteBuffer::PeekableWriteBuffer(DB::WriteBuffer & sub_buf_) : BufferWithOwnMemory(0), sub_buf(sub_buf_)
{
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin() + sub_buf.offset(), sub_working.size() - sub_buf.offset(), 0);
}
void PeekableWriteBuffer::nextImpl()
{
if (checkpoint)
{
if (write_to_own_memory)
{
size_t prev_size = position() - memory.data();
size_t new_size = memory.size() * 2;
memory.resize(new_size);
BufferBase::set(memory.data(), memory.size(), prev_size);
return;
}
if (memory.size() == 0)
memory.resize(DBMS_DEFAULT_BUFFER_SIZE);
sub_buf.position() = position();
BufferBase::set(memory.data(), memory.size(), 0);
write_to_own_memory = true;
return;
}
sub_buf.position() = position();
sub_buf.next();
BufferBase::set(sub_buf.buffer().begin(), sub_buf.buffer().size(), sub_buf.offset());
}
void PeekableWriteBuffer::dropCheckpoint()
{
assert(checkpoint);
checkpoint = std::nullopt;
/// If we have saved data in own memory, write it to sub-buf.
if (write_to_own_memory)
{
try
{
sub_buf.next();
sub_buf.write(memory.data(), position() - memory.data());
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf.offset());
write_to_own_memory = false;
}
catch (...)
{
/// If exception happened during writing to sub buffer, we should
/// update buffer to not leave it in invalid state.
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf.offset());
write_to_own_memory = false;
}
}
}
void PeekableWriteBuffer::rollbackToCheckpoint(bool drop)
{
assert(checkpoint);
/// Just ignore all data written after checkpoint.
if (write_to_own_memory)
{
Buffer & sub_working = sub_buf.buffer();
BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf.offset());
write_to_own_memory = false;
}
position() = *checkpoint;
if (drop)
checkpoint = std::nullopt;
}
}

View File

@ -0,0 +1,59 @@
#pragma once
#include <IO/WriteBuffer.h>
#include <IO/BufferWithOwnMemory.h>
#include <stack>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/// Similar to PeekableReadBuffer.
/// Allows to set checkpoint at some position in stream and come back to this position later.
/// When next() is called, saves data between checkpoint and current position to own memory instead of writing it to sub-buffer.
/// So, all the data after checkpoint won't be written in sub-buffer until checkpoint is dropped.
/// Rollback to checkpoint means that all data after checkpoint will be ignored and not sent to sub-buffer.
/// Sub-buffer should not be accessed directly during the lifetime of peekable buffer (unless
/// you reset() the state of peekable buffer after each change of underlying buffer)
/// If position() of peekable buffer is explicitly set to some position before checkpoint
/// (e.g. by istr.position() = prev_pos), behavior is undefined.
class PeekableWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
{
friend class PeekableWriteBufferCheckpoint;
public:
explicit PeekableWriteBuffer(WriteBuffer & sub_buf_);
/// Sets checkpoint at current position
ALWAYS_INLINE inline void setCheckpoint()
{
if (checkpoint)
throw Exception(ErrorCodes::LOGICAL_ERROR, "PeekableWriteBuffer does not support recursive checkpoints.");
checkpoint.emplace(pos);
}
/// Forget checkpoint and send all data from checkpoint to position to sub-buffer.
void dropCheckpoint();
/// Sets position at checkpoint and forget all data written from checkpoint to position.
/// All pointers (such as this->buffer().end()) may be invalidated
void rollbackToCheckpoint(bool drop = false);
void finalizeImpl() override
{
assert(!checkpoint);
sub_buf.position() = position();
}
private:
void nextImpl() override;
WriteBuffer & sub_buf;
bool write_to_own_memory = false;
std::optional<Position> checkpoint = std::nullopt;
};
}

View File

@ -1059,6 +1059,9 @@ void FileCache::loadMetadataForKeys(const fs::path & keys_dir)
FileCache::~FileCache() FileCache::~FileCache()
{ {
deactivateBackgroundOperations(); deactivateBackgroundOperations();
#ifdef ABORT_ON_LOGICAL_ERROR
assertCacheCorrectness();
#endif
} }
void FileCache::deactivateBackgroundOperations() void FileCache::deactivateBackgroundOperations()
@ -1143,14 +1146,15 @@ size_t FileCache::getFileSegmentsNum() const
void FileCache::assertCacheCorrectness() void FileCache::assertCacheCorrectness()
{ {
auto lock = lockCache(); metadata.iterate([&](LockedKey & locked_key)
main_priority->iterate([&](LockedKey &, const FileSegmentMetadataPtr & segment_metadata)
{ {
const auto & file_segment = *segment_metadata->file_segment; for (const auto & [_, file_segment_metadata] : locked_key)
UNUSED(file_segment); {
chassert(file_segment.assertCorrectness()); const auto & file_segment = *file_segment_metadata->file_segment;
return PriorityIterationResult::CONTINUE; UNUSED(file_segment);
}, lock); chassert(file_segment.assertCorrectness());
}
});
} }
FileCache::QueryContextHolder::QueryContextHolder( FileCache::QueryContextHolder::QueryContextHolder(

View File

@ -362,12 +362,6 @@ public:
return; return;
inserted = keys.insert(key).second; inserted = keys.insert(key).second;
} }
/// There is an invariant that key cannot be submitted for removal if it is already in removal queue.
/// Because
/// 1) when submit key to removal it acquires state REMOVING and we submit key for removal only if it has ACTIVE state.
/// 2) if a key is added to cache and it was found in removal queue - it will be removed from the queue and get state ACTIVE.
/// and both these actions are synchronized by the same KeyGuard.
chassert(inserted);
if (inserted) if (inserted)
{ {
CurrentMetrics::add(CurrentMetrics::FilesystemCacheDelayedCleanupElements); CurrentMetrics::add(CurrentMetrics::FilesystemCacheDelayedCleanupElements);

View File

@ -358,7 +358,7 @@ void QueryCache::Writer::finalizeWrite()
cache.set(key, query_result); cache.set(key, query_result);
LOG_TRACE(logger, "Stored result of query: {}", key.query_string); LOG_TRACE(logger, "Stored query result, query: {}", key.query_string);
was_finalized = true; was_finalized = true;
} }

View File

@ -1266,7 +1266,8 @@ void executeQuery(
bool allow_into_outfile, bool allow_into_outfile,
ContextMutablePtr context, ContextMutablePtr context,
SetResultDetailsFunc set_result_details, SetResultDetailsFunc set_result_details,
const std::optional<FormatSettings> & output_format_settings) const std::optional<FormatSettings> & output_format_settings,
HandleExceptionInOutputFormatFunc handle_exception_in_output_format)
{ {
PODArray<char> parse_buf; PODArray<char> parse_buf;
const char * begin; const char * begin;
@ -1324,6 +1325,7 @@ void executeQuery(
ASTPtr ast; ASTPtr ast;
BlockIO streams; BlockIO streams;
OutputFormatPtr output_format;
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr); std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, &istr);
auto & pipeline = streams.pipeline; auto & pipeline = streams.pipeline;
@ -1366,30 +1368,30 @@ void executeQuery(
? getIdentifierName(ast_query_with_output->format) ? getIdentifierName(ast_query_with_output->format)
: context->getDefaultFormat(); : context->getDefaultFormat();
auto out = FormatFactory::instance().getOutputFormatParallelIfPossible( output_format = FormatFactory::instance().getOutputFormatParallelIfPossible(
format_name, format_name,
compressed_buffer ? *compressed_buffer : *out_buf, compressed_buffer ? *compressed_buffer : *out_buf,
materializeBlock(pipeline.getHeader()), materializeBlock(pipeline.getHeader()),
context, context,
output_format_settings); output_format_settings);
out->setAutoFlush(); output_format->setAutoFlush();
/// Save previous progress callback if any. TODO Do it more conveniently. /// Save previous progress callback if any. TODO Do it more conveniently.
auto previous_progress_callback = context->getProgressCallback(); auto previous_progress_callback = context->getProgressCallback();
/// NOTE Progress callback takes shared ownership of 'out'. /// NOTE Progress callback takes shared ownership of 'out'.
pipeline.setProgressCallback([out, previous_progress_callback] (const Progress & progress) pipeline.setProgressCallback([output_format, previous_progress_callback] (const Progress & progress)
{ {
if (previous_progress_callback) if (previous_progress_callback)
previous_progress_callback(progress); previous_progress_callback(progress);
out->onProgress(progress); output_format->onProgress(progress);
}); });
result_details.content_type = out->getContentType(); result_details.content_type = output_format->getContentType();
result_details.format = format_name; result_details.format = format_name;
pipeline.complete(std::move(out)); pipeline.complete(output_format);
} }
else else
{ {
@ -1419,6 +1421,8 @@ void executeQuery(
} }
catch (...) catch (...)
{ {
if (handle_exception_in_output_format && output_format)
handle_exception_in_output_format(*output_format);
streams.onException(); streams.onException();
throw; throw;
} }

View File

@ -15,6 +15,7 @@ namespace DB
class IInterpreter; class IInterpreter;
class ReadBuffer; class ReadBuffer;
class WriteBuffer; class WriteBuffer;
class IOutputFormat;
struct QueryStatusInfo; struct QueryStatusInfo;
struct QueryResultDetails struct QueryResultDetails
@ -26,6 +27,7 @@ struct QueryResultDetails
}; };
using SetResultDetailsFunc = std::function<void(const QueryResultDetails &)>; using SetResultDetailsFunc = std::function<void(const QueryResultDetails &)>;
using HandleExceptionInOutputFormatFunc = std::function<void(IOutputFormat & output_format)>;
/// Parse and execute a query. /// Parse and execute a query.
void executeQuery( void executeQuery(
@ -34,7 +36,8 @@ void executeQuery(
bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, redirect output to that file. bool allow_into_outfile, /// If true and the query contains INTO OUTFILE section, redirect output to that file.
ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions... ContextMutablePtr context, /// DB, tables, data types, storage engines, functions, aggregate functions...
SetResultDetailsFunc set_result_details, /// If a non-empty callback is passed, it will be called with the query id, the content-type, the format, and the timezone. SetResultDetailsFunc set_result_details, /// If a non-empty callback is passed, it will be called with the query id, the content-type, the format, and the timezone.
const std::optional<FormatSettings> & output_format_settings = std::nullopt /// Format settings for output format, will be calculated from the context if not set. const std::optional<FormatSettings> & output_format_settings = std::nullopt, /// Format settings for output format, will be calculated from the context if not set.
HandleExceptionInOutputFormatFunc handle_exception_in_output_format = {} /// If a non-empty callback is passed, it will be called on exception with created output format.
); );

View File

@ -72,6 +72,9 @@ public:
consumeExtremes(Chunk(extremes.getColumns(), extremes.rows())); consumeExtremes(Chunk(extremes.getColumns(), extremes.rows()));
} }
virtual bool supportsWritingException() const { return false; }
virtual void setException(const String & /*exception_message*/) {}
size_t getResultRows() const { return result_rows; } size_t getResultRows() const { return result_rows; }
size_t getResultBytes() const { return result_bytes; } size_t getResultBytes() const { return result_bytes; }
@ -164,6 +167,11 @@ protected:
/// outputs them in finalize() method. /// outputs them in finalize() method.
virtual bool areTotalsAndExtremesUsedInFinalize() const { return false; } virtual bool areTotalsAndExtremesUsedInFinalize() const { return false; }
/// Derived classes can use some wrappers around out WriteBuffer
/// and can override this method to return wrapper
/// that should be used in its derived classes.
virtual WriteBuffer * getWriteBufferPtr() { return &out; }
WriteBuffer & out; WriteBuffer & out;
Chunk current_chunk; Chunk current_chunk;

View File

@ -9,10 +9,11 @@ namespace DB
JSONColumnsBlockOutputFormatBase::JSONColumnsBlockOutputFormatBase( JSONColumnsBlockOutputFormatBase::JSONColumnsBlockOutputFormatBase(
WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_, bool validate_utf8) WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_, bool validate_utf8)
: OutputFormatWithUTF8ValidationAdaptor(validate_utf8, header_, out_) : OutputFormatWithUTF8ValidationAdaptor(header_, out_, validate_utf8)
, format_settings(format_settings_) , format_settings(format_settings_)
, serializations(header_.getSerializations()) , serializations(header_.getSerializations())
{ {
ostr = OutputFormatWithUTF8ValidationAdaptor::getWriteBufferPtr();
} }
void JSONColumnsBlockOutputFormatBase::consume(Chunk chunk) void JSONColumnsBlockOutputFormatBase::consume(Chunk chunk)

View File

@ -38,6 +38,7 @@ protected:
Chunk mono_chunk; Chunk mono_chunk;
size_t written_rows = 0; size_t written_rows = 0;
WriteBuffer * ostr;
}; };
} }

View File

@ -15,12 +15,13 @@ JSONCompactEachRowRowOutputFormat::JSONCompactEachRowRowOutputFormat(WriteBuffer
bool with_names_, bool with_names_,
bool with_types_, bool with_types_,
bool yield_strings_) bool yield_strings_)
: RowOutputFormatWithUTF8ValidationAdaptor(settings_.json.validate_utf8, header_, out_) : RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>(header_, out_, settings_.json.valid_output_on_exception, settings_.json.validate_utf8)
, settings(settings_) , settings(settings_)
, with_names(with_names_) , with_names(with_names_)
, with_types(with_types_) , with_types(with_types_)
, yield_strings(yield_strings_) , yield_strings(yield_strings_)
{ {
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
} }
@ -102,6 +103,25 @@ void JSONCompactEachRowRowOutputFormat::consumeTotals(DB::Chunk chunk)
IRowOutputFormat::consumeTotals(std::move(chunk)); IRowOutputFormat::consumeTotals(std::move(chunk));
} }
void JSONCompactEachRowRowOutputFormat::writeSuffix()
{
if (!exception_message.empty())
{
if (haveWrittenData())
writeRowBetweenDelimiter();
writeRowStartDelimiter();
writeJSONString(exception_message, *ostr, settings);
writeRowEndDelimiter();
}
}
void JSONCompactEachRowRowOutputFormat::resetFormatterImpl()
{
RowOutputFormatWithExceptionHandlerAdaptor::resetFormatterImpl();
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
}
void registerOutputFormatJSONCompactEachRow(FormatFactory & factory) void registerOutputFormatJSONCompactEachRow(FormatFactory & factory)
{ {
for (bool yield_strings : {false, true}) for (bool yield_strings : {false, true})

View File

@ -3,15 +3,16 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h> #include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h>
#include <Processors/Formats/RowOutputFormatWithExceptionHandlerAdaptor.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
namespace DB namespace DB
{ {
/** The stream for outputting data in JSON format, by object per line. /** The stream for outputting data in JSON format, by JSON array per line.
*/ */
class JSONCompactEachRowRowOutputFormat final : public RowOutputFormatWithUTF8ValidationAdaptor class JSONCompactEachRowRowOutputFormat final : public RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>
{ {
public: public:
JSONCompactEachRowRowOutputFormat( JSONCompactEachRowRowOutputFormat(
@ -33,6 +34,9 @@ private:
void writeFieldDelimiter() override; void writeFieldDelimiter() override;
void writeRowStartDelimiter() override; void writeRowStartDelimiter() override;
void writeRowEndDelimiter() override; void writeRowEndDelimiter() override;
void writeSuffix() override;
void resetFormatterImpl() override;
bool supportTotals() const override { return true; } bool supportTotals() const override { return true; }
void consumeTotals(Chunk) override; void consumeTotals(Chunk) override;
@ -43,5 +47,7 @@ private:
bool with_names; bool with_names;
bool with_types; bool with_types;
bool yield_strings; bool yield_strings;
WriteBuffer * ostr;
}; };
} }

View File

@ -14,10 +14,12 @@ JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(
const Block & header_, const Block & header_,
const FormatSettings & settings_, const FormatSettings & settings_,
bool pretty_json_) bool pretty_json_)
: RowOutputFormatWithUTF8ValidationAdaptor(settings_.json.validate_utf8, header_, out_), : RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>(
pretty_json(pretty_json_), header_, out_, settings_.json.valid_output_on_exception, settings_.json.validate_utf8)
settings(settings_) , pretty_json(pretty_json_)
, settings(settings_)
{ {
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
fields = JSONUtils::makeNamesValidJSONStrings(getPort(PortKind::Main).getHeader().getNames(), settings, settings.json.validate_utf8); fields = JSONUtils::makeNamesValidJSONStrings(getPort(PortKind::Main).getHeader().getNames(), settings, settings.json.validate_utf8);
} }
@ -76,10 +78,24 @@ void JSONEachRowRowOutputFormat::writePrefix()
void JSONEachRowRowOutputFormat::writeSuffix() void JSONEachRowRowOutputFormat::writeSuffix()
{ {
if (!exception_message.empty())
{
if (haveWrittenData())
writeRowBetweenDelimiter();
writeRowStartDelimiter();
JSONUtils::writeException(exception_message, *ostr, settings, pretty_json ? 1 : 0);
writeRowEndDelimiter();
}
if (settings.json.array_of_rows) if (settings.json.array_of_rows)
writeCString("\n]\n", *ostr); writeCString("\n]\n", *ostr);
} }
void JSONEachRowRowOutputFormat::resetFormatterImpl()
{
RowOutputFormatWithExceptionHandlerAdaptor::resetFormatterImpl();
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
}
void registerOutputFormatJSONEachRow(FormatFactory & factory) void registerOutputFormatJSONEachRow(FormatFactory & factory)
{ {

View File

@ -2,7 +2,9 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/PeekableWriteBuffer.h>
#include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h> #include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h>
#include <Processors/Formats/RowOutputFormatWithExceptionHandlerAdaptor.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
@ -11,7 +13,7 @@ namespace DB
/** The stream for outputting data in JSON format, by object per line. /** The stream for outputting data in JSON format, by object per line.
*/ */
class JSONEachRowRowOutputFormat : public RowOutputFormatWithUTF8ValidationAdaptor class JSONEachRowRowOutputFormat : public RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>
{ {
public: public:
JSONEachRowRowOutputFormat( JSONEachRowRowOutputFormat(
@ -37,13 +39,16 @@ protected:
void writePrefix() override; void writePrefix() override;
void writeSuffix() override; void writeSuffix() override;
void resetFormatterImpl() override;
size_t field_number = 0; size_t field_number = 0;
bool pretty_json; bool pretty_json;
FormatSettings settings;
WriteBuffer * ostr;
private: private:
Names fields; Names fields;
FormatSettings settings;
}; };
} }

View File

@ -26,9 +26,10 @@ void JSONObjectEachRowRowOutputFormat::write(const Columns & columns, size_t row
if (field_index_for_object_name) if (field_index_for_object_name)
object_name = columns[*field_index_for_object_name]->getDataAt(row).toString(); object_name = columns[*field_index_for_object_name]->getDataAt(row).toString();
else else
object_name = "row_" + std::to_string(row + 1); object_name = "row_" + std::to_string(getRowsReadBefore() + rows + 1);
IRowOutputFormat::write(columns, row); ++rows;
RowOutputFormatWithExceptionHandlerAdaptor::write(columns, row);
} }
void JSONObjectEachRowRowOutputFormat::writeFieldDelimiter() void JSONObjectEachRowRowOutputFormat::writeFieldDelimiter()
@ -62,6 +63,13 @@ void JSONObjectEachRowRowOutputFormat::writeRowBetweenDelimiter()
void JSONObjectEachRowRowOutputFormat::writeSuffix() void JSONObjectEachRowRowOutputFormat::writeSuffix()
{ {
if (!exception_message.empty())
{
if (haveWrittenData())
writeRowBetweenDelimiter();
JSONUtils::writeException(exception_message, *ostr, settings, 1);
}
JSONUtils::writeObjectEnd(*ostr); JSONUtils::writeObjectEnd(*ostr);
writeChar('\n', *ostr); writeChar('\n', *ostr);
} }

View File

@ -40,6 +40,7 @@ private:
std::optional<size_t> field_index_for_object_name; std::optional<size_t> field_index_for_object_name;
String object_name; String object_name;
size_t rows = 0;
}; };
} }

View File

@ -13,9 +13,10 @@ JSONRowOutputFormat::JSONRowOutputFormat(
const Block & header, const Block & header,
const FormatSettings & settings_, const FormatSettings & settings_,
bool yield_strings_) bool yield_strings_)
: RowOutputFormatWithUTF8ValidationAdaptor(true, header, out_), settings(settings_), yield_strings(yield_strings_) : RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>(header, out_, settings_.json.valid_output_on_exception, true), settings(settings_), yield_strings(yield_strings_)
{ {
names = JSONUtils::makeNamesValidJSONStrings(header.getNames(), settings, true); names = JSONUtils::makeNamesValidJSONStrings(header.getNames(), settings, true);
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
} }
@ -117,9 +118,15 @@ void JSONRowOutputFormat::finalizeImpl()
statistics.applied_limit, statistics.applied_limit,
statistics.watch, statistics.watch,
statistics.progress, statistics.progress,
settings.write_statistics, settings.write_statistics && exception_message.empty(),
*ostr); *ostr);
if (!exception_message.empty())
{
writeCString(",\n\n", *ostr);
JSONUtils::writeException(exception_message, *ostr, settings, 1);
}
JSONUtils::writeObjectEnd(*ostr); JSONUtils::writeObjectEnd(*ostr);
writeChar('\n', *ostr); writeChar('\n', *ostr);
ostr->next(); ostr->next();
@ -127,7 +134,8 @@ void JSONRowOutputFormat::finalizeImpl()
void JSONRowOutputFormat::resetFormatterImpl() void JSONRowOutputFormat::resetFormatterImpl()
{ {
RowOutputFormatWithUTF8ValidationAdaptor::resetFormatterImpl(); RowOutputFormatWithExceptionHandlerAdaptor::resetFormatterImpl();
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
row_count = 0; row_count = 0;
statistics = Statistics(); statistics = Statistics();
} }

View File

@ -3,8 +3,10 @@
#include <Core/Block.h> #include <Core/Block.h>
#include <IO/Progress.h> #include <IO/Progress.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/PeekableWriteBuffer.h>
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h> #include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h>
#include <Processors/Formats/RowOutputFormatWithExceptionHandlerAdaptor.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
@ -13,7 +15,7 @@ namespace DB
/** Stream for output data in JSON format. /** Stream for output data in JSON format.
*/ */
class JSONRowOutputFormat : public RowOutputFormatWithUTF8ValidationAdaptor class JSONRowOutputFormat : public RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>
{ {
public: public:
JSONRowOutputFormat( JSONRowOutputFormat(
@ -69,6 +71,7 @@ protected:
FormatSettings settings; FormatSettings settings;
bool yield_strings; bool yield_strings;
WriteBuffer * ostr;
}; };
} }

View File

@ -8,7 +8,6 @@ namespace DB
void ParallelFormattingOutputFormat::finalizeImpl() void ParallelFormattingOutputFormat::finalizeImpl()
{ {
need_flush = true; need_flush = true;
IOutputFormat::finalized = true;
/// Don't throw any background_exception here, because we want to finalize the execution. /// Don't throw any background_exception here, because we want to finalize the execution.
/// Exception will be checked after main thread is finished. /// Exception will be checked after main thread is finished.
addChunk(Chunk{}, ProcessingUnitType::FINALIZE, /*can_throw_exception*/ false); addChunk(Chunk{}, ProcessingUnitType::FINALIZE, /*can_throw_exception*/ false);
@ -24,8 +23,33 @@ namespace DB
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
if (background_exception) if (background_exception)
std::rethrow_exception(background_exception); {
collector_finished.set();
rethrowBackgroundException();
}
} }
/// The code below is required to write valid output in case of exception during parallel parsing,
/// because we finish formatting and collecting threads in case of exception.
/// So, in case of exception after finalize we could still not output prefix/suffix or finalize underlying format.
if (collected_prefix && collected_suffix && collected_finalize)
return;
auto formatter = internal_formatter_creator(out);
formatter->setRowsReadBefore(rows_collected);
formatter->setException(exception_message);
if (!collected_prefix && (need_write_prefix || started_prefix))
formatter->writePrefix();
if (!collected_suffix && (need_write_suffix || started_suffix))
formatter->writeSuffix();
if (!collected_finalize)
formatter->finalizeImpl();
formatter->finalizeBuffers();
} }
void ParallelFormattingOutputFormat::addChunk(Chunk chunk, ProcessingUnitType type, bool can_throw_exception) void ParallelFormattingOutputFormat::addChunk(Chunk chunk, ProcessingUnitType type, bool can_throw_exception)
@ -33,7 +57,7 @@ namespace DB
{ {
std::lock_guard lock(mutex); std::lock_guard lock(mutex);
if (background_exception && can_throw_exception) if (background_exception && can_throw_exception)
std::rethrow_exception(background_exception); rethrowBackgroundException();
} }
const auto current_unit_number = writer_unit_number % processing_units.size(); const auto current_unit_number = writer_unit_number % processing_units.size();
@ -62,7 +86,10 @@ namespace DB
size_t first_row_num = rows_consumed; size_t first_row_num = rows_consumed;
if (unit.type == ProcessingUnitType::PLAIN) if (unit.type == ProcessingUnitType::PLAIN)
{
rows_consumed += unit.chunk.getNumRows(); rows_consumed += unit.chunk.getNumRows();
unit.rows_num = unit.chunk.getNumRows();
}
scheduleFormatterThreadForUnitWithNumber(current_unit_number, first_row_num); scheduleFormatterThreadForUnitWithNumber(current_unit_number, first_row_num);
++writer_unit_number; ++writer_unit_number;
@ -125,7 +152,7 @@ namespace DB
assert(unit.status == READY_TO_READ); assert(unit.status == READY_TO_READ);
/// Use this copy to after notification to stop the execution. /// Use this copy to after notification to stop the execution.
auto copy_if_unit_type = unit.type; auto copy_of_unit_type = unit.type;
/// Do main work here. /// Do main work here.
out.write(unit.segment.data(), unit.actual_memory_size); out.write(unit.segment.data(), unit.actual_memory_size);
@ -134,6 +161,7 @@ namespace DB
IOutputFormat::flush(); IOutputFormat::flush();
++collector_unit_number; ++collector_unit_number;
rows_collected += unit.rows_num;
{ {
/// Notify other threads. /// Notify other threads.
@ -141,9 +169,19 @@ namespace DB
unit.status = READY_TO_INSERT; unit.status = READY_TO_INSERT;
writer_condvar.notify_all(); writer_condvar.notify_all();
} }
/// We can exit only after writing last piece of to out buffer.
if (copy_if_unit_type == ProcessingUnitType::FINALIZE) if (copy_of_unit_type == ProcessingUnitType::START)
{ {
collected_prefix = true;
}
else if (copy_of_unit_type == ProcessingUnitType::PLAIN_FINISH)
{
collected_suffix = true;
}
/// We can exit only after writing last piece of data to out buffer.
else if (copy_of_unit_type == ProcessingUnitType::FINALIZE)
{
collected_finalize = true;
break; break;
} }
} }
@ -156,7 +194,6 @@ namespace DB
} }
} }
void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group) void ParallelFormattingOutputFormat::formatterThreadFunction(size_t current_unit_number, size_t first_row_num, const ThreadGroupPtr & thread_group)
{ {
SCOPE_EXIT_SAFE( SCOPE_EXIT_SAFE(
@ -184,6 +221,7 @@ namespace DB
auto formatter = internal_formatter_creator(out_buffer); auto formatter = internal_formatter_creator(out_buffer);
formatter->setRowsReadBefore(first_row_num); formatter->setRowsReadBefore(first_row_num);
formatter->setException(exception_message);
switch (unit.type) switch (unit.type)
{ {

View File

@ -118,6 +118,7 @@ public:
void writePrefix() override void writePrefix() override
{ {
addChunk(Chunk{}, ProcessingUnitType::START, /*can_throw_exception*/ true); addChunk(Chunk{}, ProcessingUnitType::START, /*can_throw_exception*/ true);
started_prefix = true;
} }
void onCancel() override void onCancel() override
@ -134,6 +135,7 @@ public:
void writeSuffix() override void writeSuffix() override
{ {
addChunk(Chunk{}, ProcessingUnitType::PLAIN_FINISH, /*can_throw_exception*/ true); addChunk(Chunk{}, ProcessingUnitType::PLAIN_FINISH, /*can_throw_exception*/ true);
started_suffix = true;
} }
String getContentType() const override String getContentType() const override
@ -142,6 +144,14 @@ public:
return internal_formatter_creator(buffer)->getContentType(); return internal_formatter_creator(buffer)->getContentType();
} }
bool supportsWritingException() const override
{
WriteBufferFromOwnString buffer;
return internal_formatter_creator(buffer)->supportsWritingException();
}
void setException(const String & exception_message_) override { exception_message = exception_message_; }
private: private:
void consume(Chunk chunk) override final void consume(Chunk chunk) override final
{ {
@ -214,6 +224,7 @@ private:
Memory<> segment; Memory<> segment;
size_t actual_memory_size{0}; size_t actual_memory_size{0};
Statistics statistics; Statistics statistics;
size_t rows_num;
}; };
Poco::Event collector_finished{}; Poco::Event collector_finished{};
@ -241,12 +252,21 @@ private:
std::condition_variable writer_condvar; std::condition_variable writer_condvar;
size_t rows_consumed = 0; size_t rows_consumed = 0;
size_t rows_collected = 0;
std::atomic_bool are_totals_written = false; std::atomic_bool are_totals_written = false;
/// We change statistics in onProgress() which can be called from different threads. /// We change statistics in onProgress() which can be called from different threads.
std::mutex statistics_mutex; std::mutex statistics_mutex;
bool save_totals_and_extremes_in_statistics; bool save_totals_and_extremes_in_statistics;
String exception_message;
bool exception_is_rethrown = false;
bool started_prefix = false;
bool collected_prefix = false;
bool started_suffix = false;
bool collected_suffix = false;
bool collected_finalize = false;
void finishAndWait(); void finishAndWait();
void onBackgroundException() void onBackgroundException()
@ -261,6 +281,17 @@ private:
collector_condvar.notify_all(); collector_condvar.notify_all();
} }
void rethrowBackgroundException()
{
/// Rethrow background exception only once, because
/// OutputFormat can be used after it to write an exception.
if (!exception_is_rethrown)
{
exception_is_rethrown = true;
std::rethrow_exception(background_exception);
}
}
void scheduleFormatterThreadForUnitWithNumber(size_t ticket_number, size_t first_row_num) void scheduleFormatterThreadForUnitWithNumber(size_t ticket_number, size_t first_row_num)
{ {
pool.scheduleOrThrowOnError([this, thread_group = CurrentThread::getGroup(), ticket_number, first_row_num] pool.scheduleOrThrowOnError([this, thread_group = CurrentThread::getGroup(), ticket_number, first_row_num]

View File

@ -8,8 +8,9 @@ namespace DB
{ {
XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_) XMLRowOutputFormat::XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_)
: RowOutputFormatWithUTF8ValidationAdaptor(true, header_, out_), fields(header_.getNamesAndTypes()), format_settings(format_settings_) : RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>(header_, out_, format_settings_.xml.valid_output_on_exception, true), fields(header_.getNamesAndTypes()), format_settings(format_settings_)
{ {
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
const auto & sample = getPort(PortKind::Main).getHeader(); const auto & sample = getPort(PortKind::Main).getHeader();
field_tag_names.resize(sample.columns()); field_tag_names.resize(sample.columns());
@ -191,7 +192,9 @@ void XMLRowOutputFormat::finalizeImpl()
writeRowsBeforeLimitAtLeast(); writeRowsBeforeLimitAtLeast();
if (format_settings.write_statistics) if (!exception_message.empty())
writeException();
else if (format_settings.write_statistics)
writeStatistics(); writeStatistics();
writeCString("</result>\n", *ostr); writeCString("</result>\n", *ostr);
@ -200,7 +203,8 @@ void XMLRowOutputFormat::finalizeImpl()
void XMLRowOutputFormat::resetFormatterImpl() void XMLRowOutputFormat::resetFormatterImpl()
{ {
RowOutputFormatWithUTF8ValidationAdaptor::resetFormatterImpl(); RowOutputFormatWithExceptionHandlerAdaptor::resetFormatterImpl();
ostr = RowOutputFormatWithExceptionHandlerAdaptor::getWriteBufferPtr();
row_count = 0; row_count = 0;
statistics = Statistics(); statistics = Statistics();
} }
@ -230,6 +234,12 @@ void XMLRowOutputFormat::writeStatistics()
writeCString("\t</statistics>\n", *ostr); writeCString("\t</statistics>\n", *ostr);
} }
void XMLRowOutputFormat::writeException()
{
writeCString("\t<exception>", *ostr);
writeXMLStringForTextElement(exception_message, *ostr);
writeCString("</exception>\n", *ostr);
}
void registerOutputFormatXML(FormatFactory & factory) void registerOutputFormatXML(FormatFactory & factory)
{ {

View File

@ -6,6 +6,7 @@
#include <Common/Stopwatch.h> #include <Common/Stopwatch.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h> #include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h>
#include <Processors/Formats/RowOutputFormatWithExceptionHandlerAdaptor.h>
namespace DB namespace DB
@ -13,7 +14,7 @@ namespace DB
/** A stream for outputting data in XML format. /** A stream for outputting data in XML format.
*/ */
class XMLRowOutputFormat final : public RowOutputFormatWithUTF8ValidationAdaptor class XMLRowOutputFormat final : public RowOutputFormatWithExceptionHandlerAdaptor<RowOutputFormatWithUTF8ValidationAdaptor, bool>
{ {
public: public:
XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_); XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_);
@ -56,6 +57,7 @@ private:
void writeExtremesElement(const char * title, const Columns & columns, size_t row_num); void writeExtremesElement(const char * title, const Columns & columns, size_t row_num);
void writeRowsBeforeLimitAtLeast(); void writeRowsBeforeLimitAtLeast();
void writeStatistics(); void writeStatistics();
void writeException();
size_t field_number = 0; size_t field_number = 0;
size_t row_count = 0; size_t row_count = 0;
@ -63,6 +65,7 @@ private:
Names field_tag_names; Names field_tag_names;
const FormatSettings format_settings; const FormatSettings format_settings;
WriteBuffer * ostr;
}; };
} }

View File

@ -6,15 +6,17 @@
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteBufferValidUTF8.h> #include <IO/WriteBufferValidUTF8.h>
#include <Common/logger_useful.h>
namespace DB namespace DB
{ {
template <typename Base, typename... Args> template <typename Base>
class OutputFormatWithUTF8ValidationAdaptorBase : public Base class OutputFormatWithUTF8ValidationAdaptorBase : public Base
{ {
public: public:
OutputFormatWithUTF8ValidationAdaptorBase(bool validate_utf8, const Block & header, WriteBuffer & out_, Args... args) OutputFormatWithUTF8ValidationAdaptorBase(const Block & header, WriteBuffer & out_, bool validate_utf8)
: Base(header, out_, std::forward<Args>(args)...) : Base(header, out_)
{ {
bool values_can_contain_invalid_utf8 = false; bool values_can_contain_invalid_utf8 = false;
for (const auto & type : this->getPort(IOutputFormat::PortKind::Main).getHeader().getDataTypes()) for (const auto & type : this->getPort(IOutputFormat::PortKind::Main).getHeader().getDataTypes())
@ -24,37 +26,39 @@ public:
} }
if (validate_utf8 && values_can_contain_invalid_utf8) if (validate_utf8 && values_can_contain_invalid_utf8)
{ validating_ostr = std::make_unique<WriteBufferValidUTF8>(*Base::getWriteBufferPtr());
validating_ostr = std::make_unique<WriteBufferValidUTF8>(this->out);
ostr = validating_ostr.get();
}
else
ostr = &this->out;
} }
void flush() override void flush() override
{ {
ostr->next();
if (validating_ostr) if (validating_ostr)
this->out.next(); validating_ostr->next();
Base::flush();
} }
void finalizeBuffers() override void finalizeBuffers() override
{ {
if (validating_ostr) if (validating_ostr)
validating_ostr->finalize(); validating_ostr->finalize();
Base::finalizeBuffers();
} }
void resetFormatterImpl() override void resetFormatterImpl() override
{ {
validating_ostr = std::make_unique<WriteBufferValidUTF8>(this->out); LOG_DEBUG(&Poco::Logger::get("RowOutputFormatWithExceptionHandlerAdaptor"), "resetFormatterImpl");
ostr = validating_ostr.get(); Base::resetFormatterImpl();
if (validating_ostr)
validating_ostr = std::make_unique<WriteBufferValidUTF8>(*Base::getWriteBufferPtr());
} }
protected: protected:
/// Point to validating_ostr or out from IOutputFormat, should be used in derived classes instead of out. /// Returns buffer that should be used in derived classes instead of out.
WriteBuffer * ostr; WriteBuffer * getWriteBufferPtr() override
{
if (validating_ostr)
return validating_ostr.get();
return Base::getWriteBufferPtr();
}
private: private:
/// Validates UTF-8 sequences, replaces bad sequences with replacement character. /// Validates UTF-8 sequences, replaces bad sequences with replacement character.

View File

@ -0,0 +1,104 @@
#pragma once
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/Formats/IRowOutputFormat.h>
#include <Processors/Formats/OutputFormatWithUTF8ValidationAdaptor.h>
#include <IO/WriteBuffer.h>
#include <IO/PeekableWriteBuffer.h>
namespace DB
{
template <typename Base, typename... Args>
class RowOutputFormatWithExceptionHandlerAdaptor : public Base
{
public:
RowOutputFormatWithExceptionHandlerAdaptor(const Block & header, WriteBuffer & out_, bool handle_exceptions, Args... args)
: Base(header, out_, std::forward<Args>(args)...)
{
if (handle_exceptions)
peekable_out = std::make_unique<PeekableWriteBuffer>(*Base::getWriteBufferPtr());
}
void consume(DB::Chunk chunk) override
{
if (!peekable_out)
{
Base::consume(std::move(chunk));
return;
}
auto num_rows = chunk.getNumRows();
const auto & columns = chunk.getColumns();
for (size_t row = 0; row < num_rows; ++row)
{
/// It's important to set a checkpoint before writing row-between delimiter
peekable_out->setCheckpoint();
if (Base::haveWrittenData())
writeRowBetweenDelimiter();
try
{
write(columns, row);
}
catch (...)
{
peekable_out->rollbackToCheckpoint(/*drop=*/true);
throw;
}
peekable_out->dropCheckpoint();
Base::first_row = false;
}
}
void write(const Columns & columns, size_t row_num) override { Base::write(columns, row_num); }
void writeRowBetweenDelimiter() override { Base::writeRowBetweenDelimiter(); }
void flush() override
{
if (peekable_out)
peekable_out->next();
Base::flush();
}
void finalizeBuffers() override
{
if (peekable_out)
peekable_out->finalize();
Base::finalizeBuffers();
}
void resetFormatterImpl() override
{
Base::resetFormatterImpl();
if (peekable_out)
peekable_out = std::make_unique<PeekableWriteBuffer>(*Base::getWriteBufferPtr());
}
bool supportsWritingException() const override { return true; }
void setException(const String & exception_message_) override { exception_message = exception_message_; }
protected:
/// Returns buffer that should be used in derived classes instead of out.
WriteBuffer * getWriteBufferPtr() override
{
if (peekable_out)
return peekable_out.get();
return Base::getWriteBufferPtr();
}
String exception_message;
private:
std::unique_ptr<PeekableWriteBuffer> peekable_out;
};
}

View File

@ -28,6 +28,8 @@
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/typeid_cast.h> #include <Common/typeid_cast.h>
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <base/getFQDNOrHostName.h> #include <base/getFQDNOrHostName.h>
#include <base/scope_guard.h> #include <base/scope_guard.h>
@ -853,23 +855,40 @@ void HTTPHandler::processQuery(
customizeContext(request, context, *in_post_maybe_compressed); customizeContext(request, context, *in_post_maybe_compressed);
in = has_external_data ? std::move(in_param) : std::make_unique<ConcatReadBuffer>(*in_param, *in_post_maybe_compressed); in = has_external_data ? std::move(in_param) : std::make_unique<ConcatReadBuffer>(*in_param, *in_post_maybe_compressed);
executeQuery(*in, *used_output.out_maybe_delayed_and_compressed, /* allow_into_outfile = */ false, context, auto set_query_result = [&response, this] (const QueryResultDetails & details)
[&response, this] (const QueryResultDetails & details) {
response.add("X-ClickHouse-Query-Id", details.query_id);
if (content_type_override)
response.setContentType(*content_type_override);
else if (details.content_type)
response.setContentType(*details.content_type);
if (details.format)
response.add("X-ClickHouse-Format", *details.format);
if (details.timezone)
response.add("X-ClickHouse-Timezone", *details.timezone);
};
auto handle_exception_in_output_format = [&](IOutputFormat & output_format)
{
if (settings.http_write_exception_in_output_format && output_format.supportsWritingException())
{ {
response.add("X-ClickHouse-Query-Id", details.query_id); output_format.setException(getCurrentExceptionMessage(false));
output_format.finalize();
if (content_type_override) used_output.exception_is_written = true;
response.setContentType(*content_type_override);
else if (details.content_type)
response.setContentType(*details.content_type);
if (details.format)
response.add("X-ClickHouse-Format", *details.format);
if (details.timezone)
response.add("X-ClickHouse-Timezone", *details.timezone);
} }
); };
executeQuery(
*in,
*used_output.out_maybe_delayed_and_compressed,
/* allow_into_outfile = */ false,
context,
set_query_result,
{},
handle_exception_in_output_format);
if (used_output.hasDelayed()) if (used_output.hasDelayed())
{ {
@ -913,7 +932,7 @@ try
response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code)); response.setStatusAndReason(exceptionCodeToHTTPStatus(exception_code));
} }
if (!response.sent() && !used_output.out_maybe_compressed) if (!response.sent() && !used_output.out_maybe_compressed && !used_output.exception_is_written)
{ {
/// If nothing was sent yet and we don't even know if we must compress the response. /// If nothing was sent yet and we don't even know if we must compress the response.
*response.send() << s << std::endl; *response.send() << s << std::endl;
@ -929,21 +948,24 @@ try
used_output.out_maybe_delayed_and_compressed.reset(); used_output.out_maybe_delayed_and_compressed.reset();
} }
/// Send the error message into already used (and possibly compressed) stream. if (!used_output.exception_is_written)
/// Note that the error message will possibly be sent after some data.
/// Also HTTP code 200 could have already been sent.
/// If buffer has data, and that data wasn't sent yet, then no need to send that data
bool data_sent = used_output.out->count() != used_output.out->offset();
if (!data_sent)
{ {
used_output.out_maybe_compressed->position() = used_output.out_maybe_compressed->buffer().begin(); /// Send the error message into already used (and possibly compressed) stream.
used_output.out->position() = used_output.out->buffer().begin(); /// Note that the error message will possibly be sent after some data.
} /// Also HTTP code 200 could have already been sent.
writeString(s, *used_output.out_maybe_compressed); /// If buffer has data, and that data wasn't sent yet, then no need to send that data
writeChar('\n', *used_output.out_maybe_compressed); bool data_sent = used_output.out->count() != used_output.out->offset();
if (!data_sent)
{
used_output.out_maybe_compressed->position() = used_output.out_maybe_compressed->buffer().begin();
used_output.out->position() = used_output.out->buffer().begin();
}
writeString(s, *used_output.out_maybe_compressed);
writeChar('\n', *used_output.out_maybe_compressed);
}
used_output.out_maybe_compressed->next(); used_output.out_maybe_compressed->next();
} }

View File

@ -69,6 +69,8 @@ private:
bool finalized = false; bool finalized = false;
bool exception_is_written = false;
inline bool hasDelayed() const inline bool hasDelayed() const
{ {
return out_maybe_delayed_and_compressed != out_maybe_compressed; return out_maybe_delayed_and_compressed != out_maybe_compressed;

View File

@ -1032,11 +1032,14 @@ CompressionCodecPtr IMergeTreeDataPart::detectDefaultCompressionCodec() const
{ {
if (path_to_data_file.empty()) if (path_to_data_file.empty())
{ {
String candidate_path = /*fs::path(getRelativePath()) */ (ISerialization::getFileNameForStream(part_column, substream_path) + ".bin"); auto stream_name = getStreamNameForColumn(part_column, substream_path, ".bin", getDataPartStorage());
if (!stream_name)
return;
auto file_name = *stream_name + ".bin";
/// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns and column_name.dict.null.bin file. /// We can have existing, but empty .bin files. Example: LowCardinality(Nullable(...)) columns and column_name.dict.null.bin file.
if (getDataPartStorage().exists(candidate_path) && getDataPartStorage().getFileSize(candidate_path) != 0) if (getDataPartStorage().getFileSize(file_name) != 0)
path_to_data_file = candidate_path; path_to_data_file = file_name;
} }
}); });
@ -1321,8 +1324,8 @@ void IMergeTreeDataPart::loadColumns(bool require)
auto metadata_snapshot = storage.getInMemoryMetadataPtr(); auto metadata_snapshot = storage.getInMemoryMetadataPtr();
if (parent_part) if (parent_part)
metadata_snapshot = metadata_snapshot->projections.get(name).metadata; metadata_snapshot = metadata_snapshot->projections.get(name).metadata;
NamesAndTypesList loaded_columns;
NamesAndTypesList loaded_columns;
bool is_readonly_storage = getDataPartStorage().isReadonly(); bool is_readonly_storage = getDataPartStorage().isReadonly();
if (!metadata_manager->exists("columns.txt")) if (!metadata_manager->exists("columns.txt"))
@ -1334,7 +1337,7 @@ void IMergeTreeDataPart::loadColumns(bool require)
/// If there is no file with a list of columns, write it down. /// If there is no file with a list of columns, write it down.
for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAllPhysical()) for (const NameAndTypePair & column : metadata_snapshot->getColumns().getAllPhysical())
if (getDataPartStorage().exists(getFileNameForColumn(column) + ".bin")) if (getFileNameForColumn(column))
loaded_columns.push_back(column); loaded_columns.push_back(column);
if (columns.empty()) if (columns.empty())
@ -2064,6 +2067,73 @@ String IMergeTreeDataPart::getZeroLevelPartBlockID(std::string_view token) const
return info.partition_id + "_" + toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]); return info.partition_id + "_" + toString(hash_value.items[0]) + "_" + toString(hash_value.items[1]);
} }
std::optional<String> IMergeTreeDataPart::getStreamNameOrHash(
const String & stream_name,
const Checksums & checksums_)
{
if (checksums_.files.contains(stream_name + ".bin"))
return stream_name;
auto hash = sipHash128String(stream_name);
if (checksums_.files.contains(hash + ".bin"))
return hash;
return {};
}
std::optional<String> IMergeTreeDataPart::getStreamNameOrHash(
const String & stream_name,
const String & extension,
const IDataPartStorage & storage_)
{
if (storage_.exists(stream_name + extension))
return stream_name;
auto hash = sipHash128String(stream_name);
if (storage_.exists(hash + extension))
return hash;
return {};
}
std::optional<String> IMergeTreeDataPart::getStreamNameForColumn(
const String & column_name,
const ISerialization::SubstreamPath & substream_path,
const Checksums & checksums_)
{
auto stream_name = ISerialization::getFileNameForStream(column_name, substream_path);
return getStreamNameOrHash(stream_name, checksums_);
}
std::optional<String> IMergeTreeDataPart::getStreamNameForColumn(
const NameAndTypePair & column,
const ISerialization::SubstreamPath & substream_path,
const Checksums & checksums_)
{
auto stream_name = ISerialization::getFileNameForStream(column, substream_path);
return getStreamNameOrHash(stream_name, checksums_);
}
std::optional<String> IMergeTreeDataPart::getStreamNameForColumn(
const String & column_name,
const ISerialization::SubstreamPath & substream_path,
const String & extension,
const IDataPartStorage & storage_)
{
auto stream_name = ISerialization::getFileNameForStream(column_name, substream_path);
return getStreamNameOrHash(stream_name, extension, storage_);
}
std::optional<String> IMergeTreeDataPart::getStreamNameForColumn(
const NameAndTypePair & column,
const ISerialization::SubstreamPath & substream_path,
const String & extension,
const IDataPartStorage & storage_)
{
auto stream_name = ISerialization::getFileNameForStream(column, substream_path);
return getStreamNameOrHash(stream_name, extension, storage_);
}
bool isCompactPart(const MergeTreeDataPartPtr & data_part) bool isCompactPart(const MergeTreeDataPartPtr & data_part)
{ {
return (data_part && data_part->getType() == MergeTreeDataPartType::Compact); return (data_part && data_part->getType() == MergeTreeDataPartType::Compact);

View File

@ -131,7 +131,7 @@ public:
/// Return information about secondary indexes size on disk for all indexes in part /// Return information about secondary indexes size on disk for all indexes in part
IndexSize getTotalSeconaryIndicesSize() const { return total_secondary_indices_size; } IndexSize getTotalSeconaryIndicesSize() const { return total_secondary_indices_size; }
virtual String getFileNameForColumn(const NameAndTypePair & column) const = 0; virtual std::optional<String> getFileNameForColumn(const NameAndTypePair & column) const = 0;
virtual ~IMergeTreeDataPart(); virtual ~IMergeTreeDataPart();
@ -501,6 +501,37 @@ public:
/// This one is about removing file with version of part's metadata (columns, pk and so on) /// This one is about removing file with version of part's metadata (columns, pk and so on)
void removeMetadataVersion(); void removeMetadataVersion();
static std::optional<String> getStreamNameOrHash(
const String & name,
const IMergeTreeDataPart::Checksums & checksums);
static std::optional<String> getStreamNameOrHash(
const String & name,
const String & extension,
const IDataPartStorage & storage_);
static std::optional<String> getStreamNameForColumn(
const String & column_name,
const ISerialization::SubstreamPath & substream_path,
const Checksums & checksums_);
static std::optional<String> getStreamNameForColumn(
const NameAndTypePair & column,
const ISerialization::SubstreamPath & substream_path,
const Checksums & checksums_);
static std::optional<String> getStreamNameForColumn(
const String & column_name,
const ISerialization::SubstreamPath & substream_path,
const String & extension,
const IDataPartStorage & storage_);
static std::optional<String> getStreamNameForColumn(
const NameAndTypePair & column,
const ISerialization::SubstreamPath & substream_path,
const String & extension,
const IDataPartStorage & storage_);
mutable std::atomic<DataPartRemovalState> removal_state = DataPartRemovalState::NOT_ATTEMPTED; mutable std::atomic<DataPartRemovalState> removal_state = DataPartRemovalState::NOT_ATTEMPTED;
mutable std::atomic<time_t> last_removal_attempt_time = 0; mutable std::atomic<time_t> last_removal_attempt_time = 0;

View File

@ -32,7 +32,7 @@ public:
virtual void write(const Block & block, const IColumn::Permutation * permutation) = 0; virtual void write(const Block & block, const IColumn::Permutation * permutation) = 0;
virtual void fillChecksums(IMergeTreeDataPart::Checksums & checksums) = 0; virtual void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) = 0;
virtual void finish(bool sync) = 0; virtual void finish(bool sync) = 0;

View File

@ -51,7 +51,9 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart(
data_part->getSerialization(column.name)->enumerateStreams( data_part->getSerialization(column.name)->enumerateStreams(
[&](const ISerialization::SubstreamPath & substream_path) [&](const ISerialization::SubstreamPath & substream_path)
{ {
++stream_counts[ISerialization::getFileNameForStream(column.name, substream_path)]; auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column, substream_path, checksums);
if (stream_name)
++stream_counts[*stream_name];
}); });
} }
@ -65,12 +67,13 @@ NameSet IMergedBlockOutputStream::removeEmptyColumnsFromPart(
ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path)
{ {
String stream_name = ISerialization::getFileNameForStream(column_name, substream_path); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column_name, substream_path, checksums);
/// Delete files if they are no longer shared with another column. /// Delete files if they are no longer shared with another column.
if (--stream_counts[stream_name] == 0) if (stream_name && --stream_counts[*stream_name] == 0)
{ {
remove_files.emplace(stream_name + ".bin"); remove_files.emplace(*stream_name + ".bin");
remove_files.emplace(stream_name + mrk_extension); remove_files.emplace(*stream_name + mrk_extension);
} }
}; };

View File

@ -399,6 +399,7 @@ MergeTreeData::MergeTreeData(
settings->check_sample_column_is_correct && !attach); settings->check_sample_column_is_correct && !attach);
} }
checkColumnFilenamesForCollision(metadata_.getColumns(), *settings, !attach);
checkTTLExpressions(metadata_, metadata_); checkTTLExpressions(metadata_, metadata_);
String reason; String reason;
@ -3351,6 +3352,7 @@ void MergeTreeData::checkAlterIsPossible(const AlterCommands & commands, Context
} }
} }
checkColumnFilenamesForCollision(new_metadata, /*throw_on_error=*/ true);
checkProperties(new_metadata, old_metadata, false, false, allow_nullable_key, local_context); checkProperties(new_metadata, old_metadata, false, false, allow_nullable_key, local_context);
checkTTLExpressions(new_metadata, old_metadata); checkTTLExpressions(new_metadata, old_metadata);
@ -7445,6 +7447,73 @@ bool MergeTreeData::canUseParallelReplicasBasedOnPKAnalysis(
return decision; return decision;
} }
void MergeTreeData::checkColumnFilenamesForCollision(const StorageInMemoryMetadata & metadata, bool throw_on_error) const
{
auto settings = getDefaultSettings();
if (metadata.settings_changes)
{
const auto & changes = metadata.settings_changes->as<const ASTSetQuery &>().changes;
settings->applyChanges(changes);
}
checkColumnFilenamesForCollision(metadata.getColumns(), *settings, throw_on_error);
}
void MergeTreeData::checkColumnFilenamesForCollision(const ColumnsDescription & columns, const MergeTreeSettings & settings, bool throw_on_error) const
{
std::unordered_map<String, std::pair<String, String>> stream_name_to_full_name;
auto columns_list = Nested::collect(columns.getAllPhysical());
for (const auto & column : columns_list)
{
std::unordered_map<String, String> column_streams;
auto callback = [&](const auto & substream_path)
{
String stream_name;
auto full_stream_name = ISerialization::getFileNameForStream(column, substream_path);
if (settings.replace_long_file_name_to_hash && full_stream_name.size() > settings.max_file_name_length)
stream_name = sipHash128String(full_stream_name);
else
stream_name = full_stream_name;
column_streams.emplace(stream_name, full_stream_name);
};
auto serialization = column.type->getDefaultSerialization();
serialization->enumerateStreams(callback);
if (column.type->supportsSparseSerialization() && settings.ratio_of_defaults_for_sparse_serialization < 1.0)
{
auto sparse_serialization = column.type->getSparseSerialization();
sparse_serialization->enumerateStreams(callback);
}
for (const auto & [stream_name, full_stream_name] : column_streams)
{
auto [it, inserted] = stream_name_to_full_name.emplace(stream_name, std::pair{full_stream_name, column.name});
if (!inserted)
{
const auto & [other_full_name, other_column_name] = it->second;
auto other_type = columns.getPhysical(other_column_name).type;
auto message = fmt::format(
"Columns '{} {}' and '{} {}' have streams ({} and {}) with collision in file name {}",
column.name, column.type->getName(), other_column_name, other_type->getName(), full_stream_name, other_full_name, stream_name);
if (settings.replace_long_file_name_to_hash)
message += ". It may be a collision between a filename for one column and a hash of filename for another column (see setting 'replace_long_file_name_to_hash')";
if (throw_on_error)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{}", message);
LOG_ERROR(log, "Table definition is incorrect. {}. It may lead to corruption of data or crashes. You need to resolve it manually", message);
return;
}
}
}
}
MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const MergeTreeData & MergeTreeData::checkStructureAndGetMergeTreeData(IStorage & source_table, const StorageMetadataPtr & src_snapshot, const StorageMetadataPtr & my_snapshot) const
{ {

View File

@ -1585,6 +1585,9 @@ private:
ContextPtr query_context, ContextPtr query_context,
const StorageSnapshotPtr & storage_snapshot, const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info) const; SelectQueryInfo & query_info) const;
void checkColumnFilenamesForCollision(const StorageInMemoryMetadata & metadata, bool throw_on_error) const;
void checkColumnFilenamesForCollision(const ColumnsDescription & columns, const MergeTreeSettings & settings, bool throw_on_error) const;
}; };
/// RAII struct to record big parts that are submerging or emerging. /// RAII struct to record big parts that are submerging or emerging.

View File

@ -8,6 +8,7 @@
#include <Compression/CompressedReadBuffer.h> #include <Compression/CompressedReadBuffer.h>
#include <Compression/CompressedWriteBuffer.h> #include <Compression/CompressedWriteBuffer.h>
#include <Storages/MergeTree/IDataPartStorage.h> #include <Storages/MergeTree/IDataPartStorage.h>
#include <optional>
namespace DB namespace DB
@ -67,44 +68,35 @@ void MergeTreeDataPartChecksum::checkSize(const IDataPartStorage & storage, cons
void MergeTreeDataPartChecksums::checkEqual(const MergeTreeDataPartChecksums & rhs, bool have_uncompressed) const void MergeTreeDataPartChecksums::checkEqual(const MergeTreeDataPartChecksums & rhs, bool have_uncompressed) const
{ {
for (const auto & it : rhs.files) for (const auto & [name, _] : rhs.files)
{
const String & name = it.first;
if (!files.contains(name)) if (!files.contains(name))
throw Exception(ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART, "Unexpected file {} in data part", name); throw Exception(ErrorCodes::UNEXPECTED_FILE_IN_DATA_PART, "Unexpected file {} in data part", name);
}
for (const auto & it : files) for (const auto & [name, checksum] : files)
{ {
const String & name = it.first;
/// Exclude files written by inverted index from check. No correct checksums are available for them currently. /// Exclude files written by inverted index from check. No correct checksums are available for them currently.
if (name.ends_with(".gin_dict") || name.ends_with(".gin_post") || name.ends_with(".gin_seg") || name.ends_with(".gin_sid")) if (name.ends_with(".gin_dict") || name.ends_with(".gin_post") || name.ends_with(".gin_seg") || name.ends_with(".gin_sid"))
continue; continue;
auto jt = rhs.files.find(name); auto it = rhs.files.find(name);
if (jt == rhs.files.end()) if (it == rhs.files.end())
throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No file {} in data part", name); throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No file {} in data part", name);
it.second.checkEqual(jt->second, have_uncompressed, name); checksum.checkEqual(it->second, have_uncompressed, name);
} }
} }
void MergeTreeDataPartChecksums::checkSizes(const IDataPartStorage & storage) const void MergeTreeDataPartChecksums::checkSizes(const IDataPartStorage & storage) const
{ {
for (const auto & it : files) for (const auto & [name, checksum] : files)
{ checksum.checkSize(storage, name);
const String & name = it.first;
it.second.checkSize(storage, name);
}
} }
UInt64 MergeTreeDataPartChecksums::getTotalSizeOnDisk() const UInt64 MergeTreeDataPartChecksums::getTotalSizeOnDisk() const
{ {
UInt64 res = 0; UInt64 res = 0;
for (const auto & it : files) for (const auto & [_, checksum] : files)
res += it.second.file_size; res += checksum.file_size;
return res; return res;
} }
@ -218,11 +210,8 @@ void MergeTreeDataPartChecksums::write(WriteBuffer & to) const
writeVarUInt(files.size(), out); writeVarUInt(files.size(), out);
for (const auto & it : files) for (const auto & [name, sum] : files)
{ {
const String & name = it.first;
const Checksum & sum = it.second;
writeStringBinary(name, out); writeStringBinary(name, out);
writeVarUInt(sum.file_size, out); writeVarUInt(sum.file_size, out);
writeBinaryLittleEndian(sum.file_hash, out); writeBinaryLittleEndian(sum.file_hash, out);
@ -255,11 +244,8 @@ void MergeTreeDataPartChecksums::add(MergeTreeDataPartChecksums && rhs_checksums
void MergeTreeDataPartChecksums::computeTotalChecksumDataOnly(SipHash & hash) const void MergeTreeDataPartChecksums::computeTotalChecksumDataOnly(SipHash & hash) const
{ {
/// We use fact that iteration is in deterministic (lexicographical) order. /// We use fact that iteration is in deterministic (lexicographical) order.
for (const auto & it : files) for (const auto & [name, sum] : files)
{ {
const String & name = it.first;
const Checksum & sum = it.second;
if (!endsWith(name, ".bin")) if (!endsWith(name, ".bin"))
continue; continue;

View File

@ -90,7 +90,6 @@ struct MergeTreeDataPartChecksums
UInt64 getTotalSizeOnDisk() const; UInt64 getTotalSizeOnDisk() const;
}; };
/// A kind of MergeTreeDataPartChecksums intended to be stored in ZooKeeper (to save its RAM) /// A kind of MergeTreeDataPartChecksums intended to be stored in ZooKeeper (to save its RAM)
/// MinimalisticDataPartChecksums and MergeTreeDataPartChecksums have the same serialization format /// MinimalisticDataPartChecksums and MergeTreeDataPartChecksums have the same serialization format
/// for versions less than MINIMAL_VERSION_WITH_MINIMALISTIC_CHECKSUMS. /// for versions less than MINIMAL_VERSION_WITH_MINIMALISTIC_CHECKSUMS.

View File

@ -57,7 +57,7 @@ public:
std::optional<time_t> getColumnModificationTime(const String & column_name) const override; std::optional<time_t> getColumnModificationTime(const String & column_name) const override;
String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; } std::optional<String> getFileNameForColumn(const NameAndTypePair & /* column */) const override { return DATA_FILE_NAME; }
~MergeTreeDataPartCompact() override; ~MergeTreeDataPartCompact() override;

View File

@ -40,7 +40,7 @@ public:
bool isStoredOnRemoteDisk() const override { return false; } bool isStoredOnRemoteDisk() const override { return false; }
bool isStoredOnRemoteDiskWithZeroCopySupport() const override { return false; } bool isStoredOnRemoteDiskWithZeroCopySupport() const override { return false; }
bool hasColumnFiles(const NameAndTypePair & column) const override { return !!getColumnPosition(column.getNameInStorage()); } bool hasColumnFiles(const NameAndTypePair & column) const override { return !!getColumnPosition(column.getNameInStorage()); }
String getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; } std::optional<String> getFileNameForColumn(const NameAndTypePair & /* column */) const override { return ""; }
void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) override; void renameTo(const String & new_relative_path, bool remove_new_dir_if_exists) override;
DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot, DataPartStoragePtr makeCloneInDetached(const String & prefix, const StorageMetadataPtr & metadata_snapshot,
const DiskTransactionPtr & disk_transaction) const override; const DiskTransactionPtr & disk_transaction) const override;

View File

@ -73,19 +73,22 @@ ColumnSize MergeTreeDataPartWide::getColumnSizeImpl(
getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{ {
String file_name = ISerialization::getFileNameForStream(column, substream_path); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column, substream_path, checksums);
if (processed_substreams && !processed_substreams->insert(file_name).second) if (!stream_name)
return; return;
auto bin_checksum = checksums.files.find(file_name + ".bin"); if (processed_substreams && !processed_substreams->insert(*stream_name).second)
return;
auto bin_checksum = checksums.files.find(*stream_name + ".bin");
if (bin_checksum != checksums.files.end()) if (bin_checksum != checksums.files.end())
{ {
size.data_compressed += bin_checksum->second.file_size; size.data_compressed += bin_checksum->second.file_size;
size.data_uncompressed += bin_checksum->second.uncompressed_size; size.data_uncompressed += bin_checksum->second.uncompressed_size;
} }
auto mrk_checksum = checksums.files.find(file_name + getMarksFileExtension()); auto mrk_checksum = checksums.files.find(*stream_name + getMarksFileExtension());
if (mrk_checksum != checksums.files.end()) if (mrk_checksum != checksums.files.end())
size.marks += mrk_checksum->second.file_size; size.marks += mrk_checksum->second.file_size;
}); });
@ -153,7 +156,13 @@ void MergeTreeDataPartWide::loadIndexGranularity()
if (columns.empty()) if (columns.empty())
throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No columns in part {}", name); throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART, "No columns in part {}", name);
loadIndexGranularityImpl(index_granularity, index_granularity_info, getDataPartStorage(), getFileNameForColumn(columns.front())); auto any_column_filename = getFileNameForColumn(columns.front());
if (!any_column_filename)
throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART,
"There are no files for column {} in part {}",
columns.front().name, getDataPartStorage().getFullPath());
loadIndexGranularityImpl(index_granularity, index_granularity_info, getDataPartStorage(), *any_column_filename);
} }
@ -185,21 +194,19 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const
{ {
getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{ {
String file_name = ISerialization::getFileNameForStream(name_type, substream_path); auto stream_name = getStreamNameForColumn(name_type, substream_path, checksums);
String mrk_file_name = file_name + marks_file_extension; if (!stream_name)
String bin_file_name = file_name + DATA_FILE_EXTENSION; throw Exception(
ErrorCodes::NO_FILE_IN_DATA_PART,
"No {}.{} file checksum for column {} in part {}",
*stream_name, DATA_FILE_EXTENSION, name_type.name, getDataPartStorage().getFullPath());
auto mrk_file_name = *stream_name + marks_file_extension;
if (!checksums.files.contains(mrk_file_name)) if (!checksums.files.contains(mrk_file_name))
throw Exception( throw Exception(
ErrorCodes::NO_FILE_IN_DATA_PART, ErrorCodes::NO_FILE_IN_DATA_PART,
"No {} file checksum for column {} in part {} ", "No {} file checksum for column {} in part {} ",
mrk_file_name, name_type.name, getDataPartStorage().getFullPath()); mrk_file_name, name_type.name, getDataPartStorage().getFullPath());
if (!checksums.files.contains(bin_file_name))
throw Exception(
ErrorCodes::NO_FILE_IN_DATA_PART,
"No {} file checksum for column {} in part {}",
bin_file_name, name_type.name, getDataPartStorage().getFullPath());
}); });
} }
} }
@ -212,27 +219,28 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const
{ {
getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) getSerialization(name_type.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{ {
auto file_path = ISerialization::getFileNameForStream(name_type, substream_path) + marks_file_extension; auto stream_name = getStreamNameForColumn(name_type, substream_path, marks_file_extension, getDataPartStorage());
/// Missing file is Ok for case when new column was added. /// Missing file is Ok for case when new column was added.
if (getDataPartStorage().exists(file_path)) if (!stream_name)
{ return;
UInt64 file_size = getDataPartStorage().getFileSize(file_path);
if (!file_size) auto file_path = *stream_name + marks_file_extension;
throw Exception( UInt64 file_size = getDataPartStorage().getFileSize(file_path);
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART,
"Part {} is broken: {} is empty.",
getDataPartStorage().getFullPath(),
std::string(fs::path(getDataPartStorage().getFullPath()) / file_path));
if (!marks_size) if (!file_size)
marks_size = file_size; throw Exception(
else if (file_size != *marks_size) ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART,
throw Exception( "Part {} is broken: {} is empty.",
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART, getDataPartStorage().getFullPath(),
"Part {} is broken: marks have different sizes.", getDataPartStorage().getFullPath()); std::string(fs::path(getDataPartStorage().getFullPath()) / file_path));
}
if (!marks_size)
marks_size = file_size;
else if (file_size != *marks_size)
throw Exception(
ErrorCodes::BAD_SIZE_OF_FILE_IN_DATA_PART,
"Part {} is broken: marks have different sizes.", getDataPartStorage().getFullPath());
}); });
} }
} }
@ -240,20 +248,13 @@ void MergeTreeDataPartWide::checkConsistency(bool require_part_metadata) const
bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const bool MergeTreeDataPartWide::hasColumnFiles(const NameAndTypePair & column) const
{ {
std::string marks_file_extension = index_granularity_info.mark_type.getFileExtension(); auto marks_file_extension = index_granularity_info.mark_type.getFileExtension();
auto check_stream_exists = [this, &marks_file_extension](const String & stream_name)
{
auto bin_checksum = checksums.files.find(stream_name + DATA_FILE_EXTENSION);
auto mrk_checksum = checksums.files.find(stream_name + marks_file_extension);
return bin_checksum != checksums.files.end() && mrk_checksum != checksums.files.end();
};
bool res = true; bool res = true;
getSerialization(column.name)->enumerateStreams([&](const auto & substream_path) getSerialization(column.name)->enumerateStreams([&](const auto & substream_path)
{ {
String file_name = ISerialization::getFileNameForStream(column, substream_path); auto stream_name = getStreamNameForColumn(column, substream_path, checksums);
if (!check_stream_exists(file_name)) if (!stream_name || !checksums.files.contains(*stream_name + marks_file_extension))
res = false; res = false;
}); });
@ -264,7 +265,11 @@ std::optional<time_t> MergeTreeDataPartWide::getColumnModificationTime(const Str
{ {
try try
{ {
return getDataPartStorage().getFileLastModified(column_name + DATA_FILE_EXTENSION).epochTime(); auto stream_name = getStreamNameOrHash(column_name, checksums);
if (!stream_name)
return {};
return getDataPartStorage().getFileLastModified(*stream_name + DATA_FILE_EXTENSION).epochTime();
} }
catch (const fs::filesystem_error &) catch (const fs::filesystem_error &)
{ {
@ -272,13 +277,19 @@ std::optional<time_t> MergeTreeDataPartWide::getColumnModificationTime(const Str
} }
} }
String MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const std::optional<String> MergeTreeDataPartWide::getFileNameForColumn(const NameAndTypePair & column) const
{ {
String filename; std::optional<String> filename;
getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) getSerialization(column.name)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{ {
if (filename.empty()) if (!filename.has_value())
filename = ISerialization::getFileNameForStream(column, substream_path); {
/// This method may be called when checksums are not initialized yet.
if (!checksums.empty())
filename = getStreamNameForColumn(column, substream_path, checksums);
else
filename = getStreamNameForColumn(column, substream_path, DATA_FILE_EXTENSION, getDataPartStorage());
}
}); });
return filename; return filename;
} }

View File

@ -48,7 +48,7 @@ public:
bool isStoredOnRemoteDiskWithZeroCopySupport() const override; bool isStoredOnRemoteDiskWithZeroCopySupport() const override;
String getFileNameForColumn(const NameAndTypePair & column) const override; std::optional<String> getFileNameForColumn(const NameAndTypePair & column) const override;
~MergeTreeDataPartWide() override; ~MergeTreeDataPartWide() override;

View File

@ -423,7 +423,7 @@ size_t MergeTreeDataPartWriterCompact::ColumnsBuffer::size() const
return accumulated_columns.at(0)->size(); return accumulated_columns.at(0)->size();
} }
void MergeTreeDataPartWriterCompact::fillChecksums(IMergeTreeDataPart::Checksums & checksums) void MergeTreeDataPartWriterCompact::fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & /*checksums_to_remove*/)
{ {
// If we don't have anything to write, skip finalization. // If we don't have anything to write, skip finalization.
if (!columns_list.empty()) if (!columns_list.empty())

View File

@ -22,7 +22,7 @@ public:
void write(const Block & block, const IColumn::Permutation * permutation) override; void write(const Block & block, const IColumn::Permutation * permutation) override;
void fillChecksums(IMergeTreeDataPart::Checksums & checksums) override; void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) override;
void finish(bool sync) override; void finish(bool sync) override;
private: private:

View File

@ -76,7 +76,7 @@ void MergeTreeDataPartWriterInMemory::calculateAndSerializePrimaryIndex(const Bl
} }
} }
void MergeTreeDataPartWriterInMemory::fillChecksums(IMergeTreeDataPart::Checksums & checksums) void MergeTreeDataPartWriterInMemory::fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & /*checksums_to_remove*/)
{ {
/// If part is empty we still need to initialize block by empty columns. /// If part is empty we still need to initialize block by empty columns.
if (!part_in_memory->block) if (!part_in_memory->block)

View File

@ -18,7 +18,7 @@ public:
/// You can write only one block. In-memory part can be written only at INSERT. /// You can write only one block. In-memory part can be written only at INSERT.
void write(const Block & block, const IColumn::Permutation * permutation) override; void write(const Block & block, const IColumn::Permutation * permutation) override;
void fillChecksums(IMergeTreeDataPart::Checksums & checksums) override; void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) override;
void finish(bool /*sync*/) override {} void finish(bool /*sync*/) override {}
private: private:

View File

@ -232,7 +232,7 @@ void MergeTreeDataPartWriterOnDisk::initSkipIndices()
store = std::make_shared<GinIndexStore>(stream_name, data_part->getDataPartStoragePtr(), data_part->getDataPartStoragePtr(), storage.getSettings()->max_digestion_size_per_segment); store = std::make_shared<GinIndexStore>(stream_name, data_part->getDataPartStoragePtr(), data_part->getDataPartStoragePtr(), storage.getSettings()->max_digestion_size_per_segment);
gin_index_stores[stream_name] = store; gin_index_stores[stream_name] = store;
} }
skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store)); skip_indices_aggregators.push_back(skip_index->createIndexAggregatorForPart(store, settings));
skip_index_accumulated_marks.push_back(0); skip_index_accumulated_marks.push_back(0);
} }
} }
@ -308,7 +308,7 @@ void MergeTreeDataPartWriterOnDisk::calculateAndSerializeSkipIndices(const Block
if (skip_indices_aggregators[i]->empty() && granule.mark_on_start) if (skip_indices_aggregators[i]->empty() && granule.mark_on_start)
{ {
skip_indices_aggregators[i] = index_helper->createIndexAggregatorForPart(store); skip_indices_aggregators[i] = index_helper->createIndexAggregatorForPart(store, settings);
if (stream.compressed_hashing.offset() >= settings.min_compress_block_size) if (stream.compressed_hashing.offset() >= settings.min_compress_block_size)
stream.compressed_hashing.next(); stream.compressed_hashing.next();

View File

@ -15,6 +15,7 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR; extern const int LOGICAL_ERROR;
extern const int INCORRECT_FILE_NAME;
} }
namespace namespace
@ -107,7 +108,22 @@ void MergeTreeDataPartWriterWide::addStreams(
ISerialization::StreamCallback callback = [&](const auto & substream_path) ISerialization::StreamCallback callback = [&](const auto & substream_path)
{ {
assert(!substream_path.empty()); assert(!substream_path.empty());
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
auto storage_settings = storage.getSettings();
auto full_stream_name = ISerialization::getFileNameForStream(column, substream_path);
String stream_name;
if (storage_settings->replace_long_file_name_to_hash && full_stream_name.size() > storage_settings->max_file_name_length)
stream_name = sipHash128String(full_stream_name);
else
stream_name = full_stream_name;
auto it = stream_name_to_full_name.find(stream_name);
if (it != stream_name_to_full_name.end() && it->second != full_stream_name)
throw Exception(ErrorCodes::INCORRECT_FILE_NAME,
"Stream with name {} already created (full stream name: {}). Current full stream name: {}."
" It is a collision between a filename for one column and a hash of filename for another column or a bug",
stream_name, it->second, full_stream_name);
/// Shared offsets for Nested type. /// Shared offsets for Nested type.
if (column_streams.contains(stream_name)) if (column_streams.contains(stream_name))
@ -136,12 +152,22 @@ void MergeTreeDataPartWriterWide::addStreams(
marks_compression_codec, marks_compression_codec,
settings.marks_compress_block_size, settings.marks_compress_block_size,
settings.query_write_settings); settings.query_write_settings);
full_name_to_stream_name.emplace(full_stream_name, stream_name);
stream_name_to_full_name.emplace(stream_name, full_stream_name);
}; };
ISerialization::SubstreamPath path; ISerialization::SubstreamPath path;
data_part->getSerialization(column.name)->enumerateStreams(callback, column.type); data_part->getSerialization(column.name)->enumerateStreams(callback, column.type);
} }
const String & MergeTreeDataPartWriterWide::getStreamName(
const NameAndTypePair & column,
const ISerialization::SubstreamPath & substream_path) const
{
auto full_stream_name = ISerialization::getFileNameForStream(column, substream_path);
return full_name_to_stream_name.at(full_stream_name);
}
ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGetter( ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGetter(
const NameAndTypePair & column, WrittenOffsetColumns & offset_columns) const const NameAndTypePair & column, WrittenOffsetColumns & offset_columns) const
@ -149,8 +175,7 @@ ISerialization::OutputStreamGetter MergeTreeDataPartWriterWide::createStreamGett
return [&, this] (const ISerialization::SubstreamPath & substream_path) -> WriteBuffer * return [&, this] (const ISerialization::SubstreamPath & substream_path) -> WriteBuffer *
{ {
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
auto stream_name = getStreamName(column, substream_path);
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
/// Don't write offsets more than one time for Nested type. /// Don't write offsets more than one time for Nested type.
if (is_offsets && offset_columns.contains(stream_name)) if (is_offsets && offset_columns.contains(stream_name))
@ -299,8 +324,7 @@ StreamsWithMarks MergeTreeDataPartWriterWide::getCurrentMarksForColumn(
data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) data_part->getSerialization(column.name)->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path)
{ {
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
auto stream_name = getStreamName(column, substream_path);
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
/// Don't write offsets more than one time for Nested type. /// Don't write offsets more than one time for Nested type.
if (is_offsets && offset_columns.contains(stream_name)) if (is_offsets && offset_columns.contains(stream_name))
@ -338,14 +362,13 @@ void MergeTreeDataPartWriterWide::writeSingleGranule(
serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path) serialization->enumerateStreams([&] (const ISerialization::SubstreamPath & substream_path)
{ {
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
auto stream_name = getStreamName(name_and_type, substream_path);
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path);
/// Don't write offsets more than one time for Nested type. /// Don't write offsets more than one time for Nested type.
if (is_offsets && offset_columns.contains(stream_name)) if (is_offsets && offset_columns.contains(stream_name))
return; return;
column_streams[stream_name]->compressed_hashing.nextIfAtEnd(); column_streams.at(stream_name)->compressed_hashing.nextIfAtEnd();
}); });
} }
@ -416,10 +439,7 @@ void MergeTreeDataPartWriterWide::writeColumn(
{ {
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
if (is_offsets) if (is_offsets)
{ offset_columns.insert(getStreamName(name_and_type, substream_path));
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path);
offset_columns.insert(stream_name);
}
}); });
} }
@ -561,7 +581,7 @@ void MergeTreeDataPartWriterWide::validateColumnOfFixedSize(const NameAndTypePai
} }
void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums) void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove)
{ {
const auto & global_settings = storage.getContext()->getSettingsRef(); const auto & global_settings = storage.getContext()->getSettingsRef();
ISerialization::SerializeBinaryBulkSettings serialize_settings; ISerialization::SerializeBinaryBulkSettings serialize_settings;
@ -597,10 +617,19 @@ void MergeTreeDataPartWriterWide::fillDataChecksums(IMergeTreeDataPart::Checksum
} }
} }
for (auto & stream : column_streams) for (auto & [stream_name, stream] : column_streams)
{ {
stream.second->preFinalize(); /// Remove checksums for old stream name if file was
stream.second->addToChecksums(checksums); /// renamed due to replacing the name to the hash of name.
const auto & full_stream_name = stream_name_to_full_name.at(stream_name);
if (stream_name != full_stream_name)
{
checksums_to_remove.insert(full_stream_name + stream->data_file_extension);
checksums_to_remove.insert(full_stream_name + stream->marks_file_extension);
}
stream->preFinalize();
stream->addToChecksums(checksums);
} }
} }
@ -632,11 +661,11 @@ void MergeTreeDataPartWriterWide::finishDataSerialization(bool sync)
} }
void MergeTreeDataPartWriterWide::fillChecksums(IMergeTreeDataPart::Checksums & checksums) void MergeTreeDataPartWriterWide::fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove)
{ {
// If we don't have anything to write, skip finalization. // If we don't have anything to write, skip finalization.
if (!columns_list.empty()) if (!columns_list.empty())
fillDataChecksums(checksums); fillDataChecksums(checksums, checksums_to_remove);
if (settings.rewrite_primary_key) if (settings.rewrite_primary_key)
fillPrimaryIndexChecksums(checksums); fillPrimaryIndexChecksums(checksums);
@ -666,10 +695,7 @@ void MergeTreeDataPartWriterWide::writeFinalMark(
{ {
bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes; bool is_offsets = !substream_path.empty() && substream_path.back().type == ISerialization::Substream::ArraySizes;
if (is_offsets) if (is_offsets)
{ offset_columns.insert(getStreamName(column, substream_path));
String stream_name = ISerialization::getFileNameForStream(column, substream_path);
offset_columns.insert(stream_name);
}
}); });
} }

View File

@ -29,14 +29,14 @@ public:
void write(const Block & block, const IColumn::Permutation * permutation) override; void write(const Block & block, const IColumn::Permutation * permutation) override;
void fillChecksums(IMergeTreeDataPart::Checksums & checksums) final; void fillChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove) final;
void finish(bool sync) final; void finish(bool sync) final;
private: private:
/// Finish serialization of data: write final mark if required and compute checksums /// Finish serialization of data: write final mark if required and compute checksums
/// Also validate written data in debug mode /// Also validate written data in debug mode
void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums); void fillDataChecksums(IMergeTreeDataPart::Checksums & checksums, NameSet & checksums_to_remove);
void finishDataSerialization(bool sync); void finishDataSerialization(bool sync);
/// Write data of one column. /// Write data of one column.
@ -101,6 +101,7 @@ private:
void adjustLastMarkIfNeedAndFlushToDisk(size_t new_rows_in_last_mark); void adjustLastMarkIfNeedAndFlushToDisk(size_t new_rows_in_last_mark);
ISerialization::OutputStreamGetter createStreamGetter(const NameAndTypePair & column, WrittenOffsetColumns & offset_columns) const; ISerialization::OutputStreamGetter createStreamGetter(const NameAndTypePair & column, WrittenOffsetColumns & offset_columns) const;
const String & getStreamName(const NameAndTypePair & column, const ISerialization::SubstreamPath & substream_path) const;
using SerializationState = ISerialization::SerializeBinaryBulkStatePtr; using SerializationState = ISerialization::SerializeBinaryBulkStatePtr;
using SerializationStates = std::unordered_map<String, SerializationState>; using SerializationStates = std::unordered_map<String, SerializationState>;
@ -110,6 +111,12 @@ private:
using ColumnStreams = std::map<String, StreamPtr>; using ColumnStreams = std::map<String, StreamPtr>;
ColumnStreams column_streams; ColumnStreams column_streams;
/// Some long column names may be replaced to hashes.
/// Below are mapping from original stream name to actual
/// stream name (probably hash of the stream) and vice versa.
std::unordered_map<String, String> full_name_to_stream_name;
std::unordered_map<String, String> stream_name_to_full_name;
/// Non written marks to disk (for each column). Waiting until all rows for /// Non written marks to disk (for each column). Waiting until all rows for
/// this marks will be written to disk. /// this marks will be written to disk.
using MarksForColumns = std::unordered_map<String, StreamsWithMarks>; using MarksForColumns = std::unordered_map<String, StreamsWithMarks>;

View File

@ -58,6 +58,7 @@ struct MergeTreeWriterSettings
, rewrite_primary_key(rewrite_primary_key_) , rewrite_primary_key(rewrite_primary_key_)
, blocks_are_granules_size(blocks_are_granules_size_) , blocks_are_granules_size(blocks_are_granules_size_)
, query_write_settings(query_write_settings_) , query_write_settings(query_write_settings_)
, max_threads_for_annoy_index_creation(global_settings.max_threads_for_annoy_index_creation)
{ {
} }
@ -75,6 +76,8 @@ struct MergeTreeWriterSettings
bool rewrite_primary_key; bool rewrite_primary_key;
bool blocks_are_granules_size; bool blocks_are_granules_size;
WriteSettings query_write_settings; WriteSettings query_write_settings;
size_t max_threads_for_annoy_index_creation;
}; };
} }

View File

@ -113,17 +113,20 @@ template <typename Distance>
MergeTreeIndexAggregatorAnnoy<Distance>::MergeTreeIndexAggregatorAnnoy( MergeTreeIndexAggregatorAnnoy<Distance>::MergeTreeIndexAggregatorAnnoy(
const String & index_name_, const String & index_name_,
const Block & index_sample_block_, const Block & index_sample_block_,
UInt64 trees_) UInt64 trees_,
size_t max_threads_for_creation_)
: index_name(index_name_) : index_name(index_name_)
, index_sample_block(index_sample_block_) , index_sample_block(index_sample_block_)
, trees(trees_) , trees(trees_)
, max_threads_for_creation(max_threads_for_creation_)
{} {}
template <typename Distance> template <typename Distance>
MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy<Distance>::getGranuleAndReset() MergeTreeIndexGranulePtr MergeTreeIndexAggregatorAnnoy<Distance>::getGranuleAndReset()
{ {
int threads = (max_threads_for_creation == 0) ? -1 : static_cast<int>(max_threads_for_creation);
// NOLINTNEXTLINE(*) // NOLINTNEXTLINE(*)
index->build(static_cast<int>(trees), /*number_of_threads=*/1); index->build(static_cast<int>(trees), threads);
auto granule = std::make_shared<MergeTreeIndexGranuleAnnoy<Distance>>(index_name, index_sample_block, index); auto granule = std::make_shared<MergeTreeIndexGranuleAnnoy<Distance>>(index_name, index_sample_block, index);
index = nullptr; index = nullptr;
return granule; return granule;
@ -282,20 +285,20 @@ std::vector<size_t> MergeTreeIndexConditionAnnoy::getUsefulRangesImpl(MergeTreeI
chassert(neighbors.size() == distances.size()); chassert(neighbors.size() == distances.size());
std::vector<size_t> granule_numbers; std::vector<size_t> granules;
granule_numbers.reserve(neighbors.size()); granules.reserve(neighbors.size());
for (size_t i = 0; i < neighbors.size(); ++i) for (size_t i = 0; i < neighbors.size(); ++i)
{ {
if (comparison_distance && distances[i] > comparison_distance) if (comparison_distance && distances[i] > comparison_distance)
continue; continue;
granule_numbers.push_back(neighbors[i] / index_granularity); granules.push_back(neighbors[i] / index_granularity);
} }
/// make unique /// make unique
std::sort(granule_numbers.begin(), granule_numbers.end()); std::sort(granules.begin(), granules.end());
granule_numbers.erase(std::unique(granule_numbers.begin(), granule_numbers.end()), granule_numbers.end()); granules.erase(std::unique(granules.begin(), granules.end()), granules.end());
return granule_numbers; return granules;
} }
MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_) MergeTreeIndexAnnoy::MergeTreeIndexAnnoy(const IndexDescription & index_, UInt64 trees_, const String & distance_function_)
@ -313,13 +316,13 @@ MergeTreeIndexGranulePtr MergeTreeIndexAnnoy::createIndexGranule() const
std::unreachable(); std::unreachable();
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexAnnoy::createIndexAggregator(const MergeTreeWriterSettings & settings) const
{ {
/// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171 /// TODO: Support more metrics. Available metrics: https://github.com/spotify/annoy/blob/master/src/annoymodule.cc#L151-L171
if (distance_function == DISTANCE_FUNCTION_L2) if (distance_function == DISTANCE_FUNCTION_L2)
return std::make_shared<MergeTreeIndexAggregatorAnnoy<Annoy::Euclidean>>(index.name, index.sample_block, trees); return std::make_shared<MergeTreeIndexAggregatorAnnoy<Annoy::Euclidean>>(index.name, index.sample_block, trees, settings.max_threads_for_annoy_index_creation);
else if (distance_function == DISTANCE_FUNCTION_COSINE) else if (distance_function == DISTANCE_FUNCTION_COSINE)
return std::make_shared<MergeTreeIndexAggregatorAnnoy<Annoy::Angular>>(index.name, index.sample_block, trees); return std::make_shared<MergeTreeIndexAggregatorAnnoy<Annoy::Angular>>(index.name, index.sample_block, trees, settings.max_threads_for_annoy_index_creation);
std::unreachable(); std::unreachable();
} }

View File

@ -48,7 +48,7 @@ struct MergeTreeIndexGranuleAnnoy final : public IMergeTreeIndexGranule
template <typename Distance> template <typename Distance>
struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator
{ {
MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, UInt64 trees); MergeTreeIndexAggregatorAnnoy(const String & index_name_, const Block & index_sample_block, UInt64 trees, size_t max_threads_for_creation);
~MergeTreeIndexAggregatorAnnoy() override = default; ~MergeTreeIndexAggregatorAnnoy() override = default;
bool empty() const override { return !index || index->get_n_items() == 0; } bool empty() const override { return !index || index->get_n_items() == 0; }
@ -58,6 +58,7 @@ struct MergeTreeIndexAggregatorAnnoy final : IMergeTreeIndexAggregator
const String index_name; const String index_name;
const Block index_sample_block; const Block index_sample_block;
const UInt64 trees; const UInt64 trees;
const size_t max_threads_for_creation;
AnnoyIndexWithSerializationPtr<Distance> index; AnnoyIndexWithSerializationPtr<Distance> index;
}; };
@ -96,7 +97,7 @@ public:
~MergeTreeIndexAnnoy() override = default; ~MergeTreeIndexAnnoy() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; }

View File

@ -65,7 +65,7 @@ bool MergeTreeIndexBloomFilter::mayBenefitFromIndexForIn(const ASTPtr & node) co
return false; return false;
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexBloomFilter::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
{ {
return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names); return std::make_shared<MergeTreeIndexAggregatorBloomFilter>(bits_per_row, hash_functions, index.column_names);
} }

View File

@ -18,7 +18,7 @@ public:
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query_info, ContextPtr context) const override;

View File

@ -679,7 +679,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexFullText::createIndexGranule() const
return std::make_shared<MergeTreeIndexGranuleFullText>(index.name, index.column_names.size(), params); return std::make_shared<MergeTreeIndexGranuleFullText>(index.name, index.column_names.size(), params);
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexFullText::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
{ {
return std::make_shared<MergeTreeIndexAggregatorFullText>(index.column_names, index.name, params, token_extractor.get()); return std::make_shared<MergeTreeIndexAggregatorFullText>(index.column_names, index.name, params, token_extractor.get());
} }

View File

@ -161,7 +161,7 @@ public:
~MergeTreeIndexFullText() override = default; ~MergeTreeIndexFullText() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition( MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override; const SelectQueryInfo & query, ContextPtr context) const override;

View File

@ -73,7 +73,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexHypothesis::createIndexGranule() const
return std::make_shared<MergeTreeIndexGranuleHypothesis>(index.name); return std::make_shared<MergeTreeIndexGranuleHypothesis>(index.name);
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexHypothesis::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
{ {
return std::make_shared<MergeTreeIndexAggregatorHypothesis>(index.name, index.sample_block.getNames().front()); return std::make_shared<MergeTreeIndexAggregatorHypothesis>(index.name, index.sample_block.getNames().front());
} }

View File

@ -67,7 +67,7 @@ public:
bool isMergeable() const override { return true; } bool isMergeable() const override { return true; }
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition( MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override; const SelectQueryInfo & query, ContextPtr context) const override;

View File

@ -708,14 +708,14 @@ MergeTreeIndexGranulePtr MergeTreeIndexInverted::createIndexGranule() const
return std::make_shared<MergeTreeIndexGranuleInverted>(index.name, index.column_names.size(), params); return std::make_shared<MergeTreeIndexGranuleInverted>(index.name, index.column_names.size(), params);
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
{ {
/// should not be called: createIndexAggregatorForPart should be used /// should not be called: createIndexAggregatorForPart should be used
assert(false); assert(false);
return nullptr; return nullptr;
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregatorForPart(const GinIndexStorePtr & store) const MergeTreeIndexAggregatorPtr MergeTreeIndexInverted::createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const
{ {
return std::make_shared<MergeTreeIndexAggregatorInverted>(store, index.column_names, index.name, params, token_extractor.get()); return std::make_shared<MergeTreeIndexAggregatorInverted>(store, index.column_names, index.name, params, token_extractor.get());
} }

View File

@ -167,8 +167,8 @@ public:
~MergeTreeIndexInverted() override = default; ~MergeTreeIndexInverted() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store) const override; MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & store, const MergeTreeWriterSettings & /*settings*/) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
bool mayBenefitFromIndexForIn(const ASTPtr & node) const override; bool mayBenefitFromIndexForIn(const ASTPtr & node) const override;

View File

@ -200,7 +200,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexMinMax::createIndexGranule() const
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexMinMax::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
{ {
return std::make_shared<MergeTreeIndexAggregatorMinMax>(index.name, index.sample_block); return std::make_shared<MergeTreeIndexAggregatorMinMax>(index.name, index.sample_block);
} }

View File

@ -75,7 +75,7 @@ public:
~MergeTreeIndexMinMax() override = default; ~MergeTreeIndexMinMax() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition( MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override; const SelectQueryInfo & query, ContextPtr context) const override;

View File

@ -698,7 +698,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexSet::createIndexGranule() const
return std::make_shared<MergeTreeIndexGranuleSet>(index.name, index.sample_block, max_rows); return std::make_shared<MergeTreeIndexGranuleSet>(index.name, index.sample_block, max_rows);
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexSet::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
{ {
return std::make_shared<MergeTreeIndexAggregatorSet>(index.name, index.sample_block, max_rows); return std::make_shared<MergeTreeIndexAggregatorSet>(index.name, index.sample_block, max_rows);
} }

View File

@ -145,7 +145,7 @@ public:
~MergeTreeIndexSet() override = default; ~MergeTreeIndexSet() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition( MergeTreeIndexConditionPtr createIndexCondition(
const SelectQueryInfo & query, ContextPtr context) const override; const SelectQueryInfo & query, ContextPtr context) const override;

View File

@ -320,20 +320,20 @@ std::vector<size_t> MergeTreeIndexConditionUSearch::getUsefulRangesImpl(MergeTre
std::vector<Float32> distances(result.size()); std::vector<Float32> distances(result.size());
result.dump_to(neighbors.data(), distances.data()); result.dump_to(neighbors.data(), distances.data());
std::vector<size_t> granule_numbers; std::vector<size_t> granules;
granule_numbers.reserve(neighbors.size()); granules.reserve(neighbors.size());
for (size_t i = 0; i < neighbors.size(); ++i) for (size_t i = 0; i < neighbors.size(); ++i)
{ {
if (comparison_distance && distances[i] > comparison_distance) if (comparison_distance && distances[i] > comparison_distance)
continue; continue;
granule_numbers.push_back(neighbors[i] / index_granularity); granules.push_back(neighbors[i] / index_granularity);
} }
/// make unique /// make unique
std::sort(granule_numbers.begin(), granule_numbers.end()); std::sort(granules.begin(), granules.end());
granule_numbers.erase(std::unique(granule_numbers.begin(), granule_numbers.end()), granule_numbers.end()); granules.erase(std::unique(granules.begin(), granules.end()), granules.end());
return granule_numbers; return granules;
} }
MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_, unum::usearch::scalar_kind_t scalar_kind_) MergeTreeIndexUSearch::MergeTreeIndexUSearch(const IndexDescription & index_, const String & distance_function_, unum::usearch::scalar_kind_t scalar_kind_)
@ -352,7 +352,7 @@ MergeTreeIndexGranulePtr MergeTreeIndexUSearch::createIndexGranule() const
std::unreachable(); std::unreachable();
} }
MergeTreeIndexAggregatorPtr MergeTreeIndexUSearch::createIndexAggregator() const MergeTreeIndexAggregatorPtr MergeTreeIndexUSearch::createIndexAggregator(const MergeTreeWriterSettings & /*settings*/) const
{ {
if (distance_function == DISTANCE_FUNCTION_L2) if (distance_function == DISTANCE_FUNCTION_L2)
return std::make_shared<MergeTreeIndexAggregatorUSearch<unum::usearch::metric_kind_t::l2sq_k>>(index.name, index.sample_block, scalar_kind); return std::make_shared<MergeTreeIndexAggregatorUSearch<unum::usearch::metric_kind_t::l2sq_k>>(index.name, index.sample_block, scalar_kind);

View File

@ -99,7 +99,7 @@ public:
~MergeTreeIndexUSearch() override = default; ~MergeTreeIndexUSearch() override = default;
MergeTreeIndexGranulePtr createIndexGranule() const override; MergeTreeIndexGranulePtr createIndexGranule() const override;
MergeTreeIndexAggregatorPtr createIndexAggregator() const override; MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const override;
MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override; MergeTreeIndexConditionPtr createIndexCondition(const SelectQueryInfo & query, ContextPtr context) const override;
bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; } bool mayBenefitFromIndexForIn(const ASTPtr & /*node*/) const override { return false; }

View File

@ -13,6 +13,7 @@
#include <Storages/MergeTree/MergeTreeDataPartChecksum.h> #include <Storages/MergeTree/MergeTreeDataPartChecksum.h>
#include <Storages/SelectQueryInfo.h> #include <Storages/SelectQueryInfo.h>
#include <Storages/MergeTree/MarkRange.h> #include <Storages/MergeTree/MarkRange.h>
#include <Storages/MergeTree/MergeTreeIOSettings.h>
#include <Storages/MergeTree/IDataPartStorage.h> #include <Storages/MergeTree/IDataPartStorage.h>
#include <Interpreters/ExpressionActions.h> #include <Interpreters/ExpressionActions.h>
#include <DataTypes/DataTypeLowCardinality.h> #include <DataTypes/DataTypeLowCardinality.h>
@ -164,11 +165,11 @@ struct IMergeTreeIndex
virtual MergeTreeIndexGranulePtr createIndexGranule() const = 0; virtual MergeTreeIndexGranulePtr createIndexGranule() const = 0;
virtual MergeTreeIndexAggregatorPtr createIndexAggregator() const = 0; virtual MergeTreeIndexAggregatorPtr createIndexAggregator(const MergeTreeWriterSettings & settings) const = 0;
virtual MergeTreeIndexAggregatorPtr createIndexAggregatorForPart([[maybe_unused]]const GinIndexStorePtr &store) const virtual MergeTreeIndexAggregatorPtr createIndexAggregatorForPart(const GinIndexStorePtr & /*store*/, const MergeTreeWriterSettings & settings) const
{ {
return createIndexAggregator(); return createIndexAggregator(settings);
} }
virtual MergeTreeIndexConditionPtr createIndexCondition( virtual MergeTreeIndexConditionPtr createIndexCondition(

View File

@ -907,13 +907,21 @@ UInt64 MergeTreeRangeReader::Stream::lastPartOffset() const
size_t MergeTreeRangeReader::Stream::ceilRowsToCompleteGranules(size_t rows_num) const size_t MergeTreeRangeReader::Stream::ceilRowsToCompleteGranules(size_t rows_num) const
{ {
/// FIXME suboptimal /// Find the first occurrence of mark that satisfies getRowsCountInRange(left, mark + 1) >= rows_num
size_t result = 0; /// in [current_mark, last_mark).
size_t from_mark = current_mark; assert(current_mark + 1 <= last_mark);
while (result < rows_num && from_mark < last_mark) size_t left_mark = current_mark;
result += index_granularity->getMarkRows(from_mark++); size_t right_mark = last_mark;
while (left_mark < right_mark)
return result; {
size_t mid_mark = left_mark + (right_mark - left_mark) / 2;
if (index_granularity->getRowsCountInRange(current_mark, mid_mark + 1) >= rows_num)
right_mark = mid_mark;
else
left_mark = mid_mark + 1;
}
size_t end_mark = (left_mark == last_mark) ? left_mark : left_mark + 1;
return index_granularity->getRowsCountInRange(current_mark, end_mark);
} }

View File

@ -206,35 +206,33 @@ void MergeTreeReaderWide::addStreams(
ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path) ISerialization::StreamCallback callback = [&] (const ISerialization::SubstreamPath & substream_path)
{ {
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums());
if (streams.contains(stream_name))
{
has_any_stream = true;
return;
}
bool data_file_exists = data_part_info_for_read->getChecksums().files.contains(stream_name + DATA_FILE_EXTENSION);
/** If data file is missing then we will not try to open it. /** If data file is missing then we will not try to open it.
* It is necessary since it allows to add new column to structure of the table without creating new files for old parts. * It is necessary since it allows to add new column to structure of the table without creating new files for old parts.
*/ */
if (!data_file_exists) if (!stream_name)
{ {
has_all_streams = false; has_all_streams = false;
return; return;
} }
if (streams.contains(*stream_name))
{
has_any_stream = true;
return;
}
has_any_stream = true; has_any_stream = true;
bool is_lc_dict = substream_path.size() > 1 && substream_path[substream_path.size() - 2].type == ISerialization::Substream::Type::DictionaryKeys; 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 context = data_part_info_for_read->getContext();
auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr; auto * load_marks_threadpool = settings.read_settings.load_marks_asynchronously ? &context->getLoadMarksThreadpool() : nullptr;
streams.emplace(stream_name, std::make_unique<MergeTreeReaderStream>( streams.emplace(*stream_name, std::make_unique<MergeTreeReaderStream>(
data_part_info_for_read, stream_name, DATA_FILE_EXTENSION, data_part_info_for_read, *stream_name, DATA_FILE_EXTENSION,
data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache, data_part_info_for_read->getMarksCount(), all_mark_ranges, settings, mark_cache,
uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(stream_name + DATA_FILE_EXTENSION), uncompressed_cache, data_part_info_for_read->getFileSizeOrZero(*stream_name + DATA_FILE_EXTENSION),
&data_part_info_for_read->getIndexGranularityInfo(), &data_part_info_for_read->getIndexGranularityInfo(),
profile_callback, clock_type, is_lc_dict, load_marks_threadpool)); profile_callback, clock_type, is_lc_dict, load_marks_threadpool));
}; };
@ -245,13 +243,14 @@ void MergeTreeReaderWide::addStreams(
partially_read_columns.insert(name_and_type.name); partially_read_columns.insert(name_and_type.name);
} }
static ReadBuffer * getStream( static ReadBuffer * getStream(
bool seek_to_start, bool seek_to_start,
const ISerialization::SubstreamPath & substream_path, const ISerialization::SubstreamPath & substream_path,
const MergeTreeDataPartChecksums & checksums,
MergeTreeReaderWide::FileStreams & streams, MergeTreeReaderWide::FileStreams & streams,
const NameAndTypePair & name_and_type, const NameAndTypePair & name_and_type,
size_t from_mark, bool seek_to_mark, size_t from_mark,
bool seek_to_mark,
size_t current_task_last_mark, size_t current_task_last_mark,
ISerialization::SubstreamsCache & cache) ISerialization::SubstreamsCache & cache)
{ {
@ -259,9 +258,11 @@ static ReadBuffer * getStream(
if (cache.contains(ISerialization::getSubcolumnNameForStream(substream_path))) if (cache.contains(ISerialization::getSubcolumnNameForStream(substream_path)))
return nullptr; return nullptr;
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, checksums);
if (!stream_name)
return nullptr;
auto it = streams.find(stream_name); auto it = streams.find(*stream_name);
if (it == streams.end()) if (it == streams.end())
return nullptr; return nullptr;
@ -288,7 +289,7 @@ void MergeTreeReaderWide::deserializePrefix(
ISerialization::DeserializeBinaryBulkSettings deserialize_settings; ISerialization::DeserializeBinaryBulkSettings deserialize_settings;
deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path) deserialize_settings.getter = [&](const ISerialization::SubstreamPath & substream_path)
{ {
return getStream(/* seek_to_start = */true, substream_path, streams, name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, cache); return getStream(/* seek_to_start = */true, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, 0, /* seek_to_mark = */false, current_task_last_mark, cache);
}; };
serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name]); serialization->deserializeBinaryBulkStatePrefix(deserialize_settings, deserialize_binary_bulk_state_map[name]);
} }
@ -307,15 +308,15 @@ void MergeTreeReaderWide::prefetchForColumn(
serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) serialization->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{ {
String stream_name = ISerialization::getFileNameForStream(name_and_type, substream_path); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(name_and_type, substream_path, data_part_info_for_read->getChecksums());
if (!prefetched_streams.contains(stream_name)) if (stream_name && !prefetched_streams.contains(*stream_name))
{ {
bool seek_to_mark = !continue_reading; bool seek_to_mark = !continue_reading;
if (ReadBuffer * buf = getStream(false, substream_path, streams, name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache)) if (ReadBuffer * buf = getStream(false, substream_path, data_part_info_for_read->getChecksums(), streams, name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache))
{ {
buf->prefetch(priority); buf->prefetch(priority);
prefetched_streams.insert(stream_name); prefetched_streams.insert(*stream_name);
} }
} }
}); });
@ -338,8 +339,9 @@ void MergeTreeReaderWide::readData(
bool seek_to_mark = !was_prefetched && !continue_reading; bool seek_to_mark = !was_prefetched && !continue_reading;
return getStream( return getStream(
/* seek_to_start = */false, substream_path, streams, name_and_type, from_mark, /* seek_to_start = */false, substream_path,
seek_to_mark, current_task_last_mark, cache); data_part_info_for_read->getChecksums(), streams,
name_and_type, from_mark, seek_to_mark, current_task_last_mark, cache);
}; };
deserialize_settings.continuous_reading = continue_reading; deserialize_settings.continuous_reading = continue_reading;

View File

@ -34,7 +34,8 @@ struct Settings;
M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \ M(UInt64, min_bytes_for_wide_part, 10485760, "Minimal uncompressed size in bytes to create part in wide format instead of compact", 0) \
M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \ M(UInt64, min_rows_for_wide_part, 0, "Minimal number of rows to create part in wide format instead of compact", 0) \
M(Float, ratio_of_defaults_for_sparse_serialization, 0.9375f, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \ M(Float, ratio_of_defaults_for_sparse_serialization, 0.9375f, "Minimal ratio of number of default values to number of all values in column to store it in sparse serializations. If >= 1, columns will be always written in full serialization.", 0) \
\ M(Bool, replace_long_file_name_to_hash, false, "If the file name for column is too long (more than 'max_file_name_length' bytes) replace it to SipHash128", 0) \
M(UInt64, max_file_name_length, 127, "The maximal length of the file name to keep it as is without hashing", 0) \
/** Merge settings. */ \ /** Merge settings. */ \
M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \ M(UInt64, merge_max_block_size, 8192, "How many rows in blocks should be formed for merge operations. By default has the same value as `index_granularity`.", 0) \
M(UInt64, merge_max_block_size_bytes, 10 * 1024 * 1024, "How many bytes in blocks should be formed for merge operations. By default has the same value as `index_granularity_bytes`.", 0) \ M(UInt64, merge_max_block_size_bytes, 10 * 1024 * 1024, "How many bytes in blocks should be formed for merge operations. By default has the same value as `index_granularity_bytes`.", 0) \

View File

@ -142,12 +142,16 @@ MergedBlockOutputStream::Finalizer MergedBlockOutputStream::finalizePartAsync(
{ {
/// Finish write and get checksums. /// Finish write and get checksums.
MergeTreeData::DataPart::Checksums checksums; MergeTreeData::DataPart::Checksums checksums;
NameSet checksums_to_remove;
if (additional_column_checksums) if (additional_column_checksums)
checksums = std::move(*additional_column_checksums); checksums = std::move(*additional_column_checksums);
/// Finish columns serialization. /// Finish columns serialization.
writer->fillChecksums(checksums); writer->fillChecksums(checksums, checksums_to_remove);
for (const auto & name : checksums_to_remove)
checksums.files.erase(name);
LOG_TRACE(&Poco::Logger::get("MergedBlockOutputStream"), "filled checksums {}", new_part->getNameWithState()); LOG_TRACE(&Poco::Logger::get("MergedBlockOutputStream"), "filled checksums {}", new_part->getNameWithState());

View File

@ -63,7 +63,11 @@ MergedColumnOnlyOutputStream::fillChecksums(
{ {
/// Finish columns serialization. /// Finish columns serialization.
MergeTreeData::DataPart::Checksums checksums; MergeTreeData::DataPart::Checksums checksums;
writer->fillChecksums(checksums); NameSet checksums_to_remove;
writer->fillChecksums(checksums, checksums_to_remove);
for (const auto & filename : checksums_to_remove)
all_checksums.files.erase(filename);
for (const auto & [projection_name, projection_part] : new_part->getProjectionParts()) for (const auto & [projection_name, projection_part] : new_part->getProjectionParts())
checksums.addFile( checksums.addFile(
@ -80,9 +84,7 @@ MergedColumnOnlyOutputStream::fillChecksums(
for (const String & removed_file : removed_files) for (const String & removed_file : removed_files)
{ {
new_part->getDataPartStorage().removeFileIfExists(removed_file); new_part->getDataPartStorage().removeFileIfExists(removed_file);
all_checksums.files.erase(removed_file);
if (all_checksums.files.contains(removed_file))
all_checksums.files.erase(removed_file);
} }
new_part->setColumns(columns, serialization_infos, metadata_snapshot->getMetadataVersion()); new_part->setColumns(columns, serialization_infos, metadata_snapshot->getMetadataVersion());

View File

@ -1,6 +1,5 @@
#include <Storages/MergeTree/MutateTask.h> #include <Storages/MergeTree/MutateTask.h>
#include "Common/Priority.h"
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Common/escapeForFileName.h> #include <Common/escapeForFileName.h>
#include <Storages/MergeTree/DataPartStorageOnDiskFull.h> #include <Storages/MergeTree/DataPartStorageOnDiskFull.h>
@ -523,7 +522,9 @@ static std::set<ProjectionDescriptionRawPtr> getProjectionsToRecalculate(
} }
static std::unordered_map<String, size_t> getStreamCounts( static std::unordered_map<String, size_t> getStreamCounts(
const MergeTreeDataPartPtr & data_part, const Names & column_names) const MergeTreeDataPartPtr & data_part,
const MergeTreeDataPartChecksums & source_part_checksums,
const Names & column_names)
{ {
std::unordered_map<String, size_t> stream_counts; std::unordered_map<String, size_t> stream_counts;
@ -533,8 +534,9 @@ static std::unordered_map<String, size_t> getStreamCounts(
{ {
auto callback = [&](const ISerialization::SubstreamPath & substream_path) auto callback = [&](const ISerialization::SubstreamPath & substream_path)
{ {
auto stream_name = ISerialization::getFileNameForStream(column_name, substream_path); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column_name, substream_path, source_part_checksums);
++stream_counts[stream_name]; if (stream_name)
++stream_counts[*stream_name];
}; };
serialization->enumerateStreams(callback); serialization->enumerateStreams(callback);
@ -544,7 +546,6 @@ static std::unordered_map<String, size_t> getStreamCounts(
return stream_counts; return stream_counts;
} }
/// Files, that we don't need to remove and don't need to hardlink, for example columns.txt and checksums.txt. /// Files, that we don't need to remove and don't need to hardlink, for example columns.txt and checksums.txt.
/// Because we will generate new versions of them after we perform mutation. /// Because we will generate new versions of them after we perform mutation.
static NameSet collectFilesToSkip( static NameSet collectFilesToSkip(
@ -572,9 +573,10 @@ static NameSet collectFilesToSkip(
if (isWidePart(source_part)) if (isWidePart(source_part))
{ {
auto new_stream_counts = getStreamCounts(new_part, new_part->getColumns().getNames()); auto new_stream_counts = getStreamCounts(new_part, source_part->checksums, new_part->getColumns().getNames());
auto source_updated_stream_counts = getStreamCounts(source_part, updated_header.getNames()); auto source_updated_stream_counts = getStreamCounts(source_part, source_part->checksums, updated_header.getNames());
auto new_updated_stream_counts = getStreamCounts(new_part, updated_header.getNames()); auto new_updated_stream_counts = getStreamCounts(new_part, source_part->checksums, updated_header.getNames());
/// Skip all modified files in new part. /// Skip all modified files in new part.
for (const auto & [stream_name, _] : new_updated_stream_counts) for (const auto & [stream_name, _] : new_updated_stream_counts)
@ -615,7 +617,7 @@ static NameToNameVector collectFilesForRenames(
const String & mrk_extension) const String & mrk_extension)
{ {
/// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes. /// Collect counts for shared streams of different columns. As an example, Nested columns have shared stream with array sizes.
auto stream_counts = getStreamCounts(source_part, source_part->getColumns().getNames()); auto stream_counts = getStreamCounts(source_part, source_part->checksums, source_part->getColumns().getNames());
NameToNameVector rename_vector; NameToNameVector rename_vector;
NameSet collected_names; NameSet collected_names;
@ -652,12 +654,13 @@ static NameToNameVector collectFilesForRenames(
{ {
ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path)
{ {
String stream_name = ISerialization::getFileNameForStream({command.column_name, command.data_type}, substream_path); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(command.column_name, substream_path, source_part->checksums);
/// Delete files if they are no longer shared with another column. /// Delete files if they are no longer shared with another column.
if (--stream_counts[stream_name] == 0) if (stream_name && --stream_counts[*stream_name] == 0)
{ {
add_rename(stream_name + ".bin", ""); add_rename(*stream_name + ".bin", "");
add_rename(stream_name + mrk_extension, ""); add_rename(*stream_name + mrk_extension, "");
} }
}; };
@ -671,13 +674,25 @@ static NameToNameVector collectFilesForRenames(
ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path) ISerialization::StreamCallback callback = [&](const ISerialization::SubstreamPath & substream_path)
{ {
String stream_from = ISerialization::getFileNameForStream(command.column_name, substream_path); String full_stream_from = ISerialization::getFileNameForStream(command.column_name, substream_path);
String stream_to = boost::replace_first_copy(stream_from, escaped_name_from, escaped_name_to); String full_stream_to = boost::replace_first_copy(full_stream_from, escaped_name_from, escaped_name_to);
auto stream_from = IMergeTreeDataPart::getStreamNameOrHash(full_stream_from, source_part->checksums);
if (!stream_from)
return;
String stream_to;
auto storage_settings = source_part->storage.getSettings();
if (storage_settings->replace_long_file_name_to_hash && full_stream_to.size() > storage_settings->max_file_name_length)
stream_to = sipHash128String(full_stream_to);
else
stream_to = full_stream_to;
if (stream_from != stream_to) if (stream_from != stream_to)
{ {
add_rename(stream_from + ".bin", stream_to + ".bin"); add_rename(*stream_from + ".bin", stream_to + ".bin");
add_rename(stream_from + mrk_extension, stream_to + mrk_extension); add_rename(*stream_from + mrk_extension, stream_to + mrk_extension);
} }
}; };
@ -690,8 +705,8 @@ static NameToNameVector collectFilesForRenames(
/// but were removed in new_part by MODIFY COLUMN from /// but were removed in new_part by MODIFY COLUMN from
/// type with higher number of streams (e.g. LowCardinality -> String). /// type with higher number of streams (e.g. LowCardinality -> String).
auto old_streams = getStreamCounts(source_part, source_part->getColumns().getNames()); auto old_streams = getStreamCounts(source_part, source_part->checksums, source_part->getColumns().getNames());
auto new_streams = getStreamCounts(new_part, source_part->getColumns().getNames()); auto new_streams = getStreamCounts(new_part, source_part->checksums, source_part->getColumns().getNames());
for (const auto & [old_stream, _] : old_streams) for (const auto & [old_stream, _] : old_streams)
{ {

View File

@ -15,6 +15,7 @@
#include <IO/HashingReadBuffer.h> #include <IO/HashingReadBuffer.h>
#include <IO/S3Common.h> #include <IO/S3Common.h>
#include <Common/CurrentMetrics.h> #include <Common/CurrentMetrics.h>
#include <Common/SipHash.h>
#include <Poco/Net/NetException.h> #include <Poco/Net/NetException.h>
#if USE_AZURE_BLOB_STORAGE #if USE_AZURE_BLOB_STORAGE
@ -38,6 +39,7 @@ namespace ErrorCodes
extern const int CANNOT_MUNMAP; extern const int CANNOT_MUNMAP;
extern const int CANNOT_MREMAP; extern const int CANNOT_MREMAP;
extern const int UNEXPECTED_FILE_IN_DATA_PART; extern const int UNEXPECTED_FILE_IN_DATA_PART;
extern const int NO_FILE_IN_DATA_PART;
extern const int NETWORK_ERROR; extern const int NETWORK_ERROR;
extern const int SOCKET_TIMEOUT; extern const int SOCKET_TIMEOUT;
} }
@ -200,7 +202,14 @@ static IMergeTreeDataPart::Checksums checkDataPart(
{ {
get_serialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path) get_serialization(column)->enumerateStreams([&](const ISerialization::SubstreamPath & substream_path)
{ {
String file_name = ISerialization::getFileNameForStream(column, substream_path) + ".bin"; auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(column, substream_path, ".bin", data_part_storage);
if (!stream_name)
throw Exception(ErrorCodes::NO_FILE_IN_DATA_PART,
"There is no file for column '{}' in data part '{}'",
column.name, data_part->name);
auto file_name = *stream_name + ".bin";
checksums_data.files[file_name] = checksum_compressed_file(data_part_storage, file_name); checksums_data.files[file_name] = checksum_compressed_file(data_part_storage, file_name);
}); });
} }

View File

@ -43,7 +43,7 @@ void NATSHandler::startLoop()
while (loop_state.load() == Loop::RUN && duration.count() < MAX_THREAD_WORK_DURATION_MS) while (loop_state.load() == Loop::RUN && duration.count() < MAX_THREAD_WORK_DURATION_MS)
{ {
uv_run(loop, UV_RUN_DEFAULT); uv_run(loop, UV_RUN_NOWAIT);
end_time = std::chrono::steady_clock::now(); end_time = std::chrono::steady_clock::now();
duration = std::chrono::duration_cast<std::chrono::milliseconds>(end_time - start_time); duration = std::chrono::duration_cast<std::chrono::milliseconds>(end_time - start_time);
} }

View File

@ -108,7 +108,6 @@ StorageNATS::StorageNATS(
} }
LOG_DEBUG(log, "Connect attempt #{} failed, error: {}. Reconnecting...", i + 1, nats_GetLastError(nullptr)); LOG_DEBUG(log, "Connect attempt #{} failed, error: {}. Reconnecting...", i + 1, nats_GetLastError(nullptr));
std::this_thread::sleep_for(std::chrono::milliseconds(configuration.reconnect_wait));
} }
} }
catch (...) catch (...)

View File

@ -269,6 +269,20 @@ void MaterializedPostgreSQLConsumer::readTupleData(
insertDefaultValue(buffer, column_idx); insertDefaultValue(buffer, column_idx);
break; break;
} }
case 'b': /// Binary data.
{
LOG_WARNING(log, "We do not yet process this format of data, will insert default value");
insertDefaultValue(buffer, column_idx);
break;
}
default:
{
LOG_WARNING(log, "Unexpected identifier: {}. This is a bug! Please report an issue on github", identifier);
chassert(false);
insertDefaultValue(buffer, column_idx);
break;
}
} }
}; };
@ -281,6 +295,10 @@ void MaterializedPostgreSQLConsumer::readTupleData(
} }
catch (...) catch (...)
{ {
LOG_ERROR(log,
"Got error while receiving value for column {}, will insert default value. Error: {}",
column_idx, getCurrentExceptionMessage(true));
insertDefaultValue(buffer, column_idx); insertDefaultValue(buffer, column_idx);
/// Let's collect only the first exception. /// Let's collect only the first exception.
/// This delaying of error throw is needed because /// This delaying of error throw is needed because

View File

@ -271,18 +271,21 @@ void StorageSystemPartsColumns::processNextStorage(
ColumnSize size; ColumnSize size;
NameAndTypePair subcolumn(column.name, name, column.type, data.type); NameAndTypePair subcolumn(column.name, name, column.type, data.type);
String file_name = ISerialization::getFileNameForStream(subcolumn, subpath);
auto bin_checksum = part->checksums.files.find(file_name + ".bin"); auto stream_name = IMergeTreeDataPart::getStreamNameForColumn(subcolumn, subpath, part->checksums);
if (bin_checksum != part->checksums.files.end()) if (stream_name)
{ {
size.data_compressed += bin_checksum->second.file_size; auto bin_checksum = part->checksums.files.find(*stream_name + ".bin");
size.data_uncompressed += bin_checksum->second.uncompressed_size; if (bin_checksum != part->checksums.files.end())
} {
size.data_compressed += bin_checksum->second.file_size;
size.data_uncompressed += bin_checksum->second.uncompressed_size;
}
auto mrk_checksum = part->checksums.files.find(file_name + part->index_granularity_info.mark_type.getFileExtension()); auto mrk_checksum = part->checksums.files.find(*stream_name + part->index_granularity_info.mark_type.getFileExtension());
if (mrk_checksum != part->checksums.files.end()) if (mrk_checksum != part->checksums.files.end())
size.marks += mrk_checksum->second.file_size; size.marks += mrk_checksum->second.file_size;
}
subcolumn_bytes_on_disk.push_back(size.data_compressed + size.marks); subcolumn_bytes_on_disk.push_back(size.data_compressed + size.marks);
subcolumn_data_compressed_bytes.push_back(size.data_compressed); subcolumn_data_compressed_bytes.push_back(size.data_compressed);

View File

@ -671,6 +671,8 @@ class MergeTreeSettingsRandomizer:
"compress_primary_key": lambda: random.randint(0, 1), "compress_primary_key": lambda: random.randint(0, 1),
"marks_compress_block_size": lambda: random.randint(8000, 100000), "marks_compress_block_size": lambda: random.randint(8000, 100000),
"primary_key_compress_block_size": lambda: random.randint(8000, 100000), "primary_key_compress_block_size": lambda: random.randint(8000, 100000),
"replace_long_file_name_to_hash": lambda: random.randint(0, 1),
"max_file_name_length": threshold_generator(0.3, 0.3, 0, 128),
} }
@staticmethod @staticmethod

View File

@ -1,5 +1,6 @@
<clickhouse> <clickhouse>
<merge_tree> <merge_tree>
<min_bytes_for_wide_part>0</min_bytes_for_wide_part> <min_bytes_for_wide_part>0</min_bytes_for_wide_part>
<replace_long_file_name_to_hash>0</replace_long_file_name_to_hash>
</merge_tree> </merge_tree>
</clickhouse> </clickhouse>

View File

@ -48,7 +48,7 @@ def test_nested_compression_codec(start_cluster):
column_array Array(Array(UInt64)) CODEC(T64, LZ4), column_array Array(Array(UInt64)) CODEC(T64, LZ4),
column_bad LowCardinality(Int64) CODEC(Delta) column_bad LowCardinality(Int64) CODEC(Delta)
) ENGINE = ReplicatedMergeTree('/t', '{}') ORDER BY tuple() PARTITION BY key ) ENGINE = ReplicatedMergeTree('/t', '{}') ORDER BY tuple() PARTITION BY key
SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0; SETTINGS min_rows_for_wide_part = 0, min_bytes_for_wide_part = 0, replace_long_file_name_to_hash = 0;
""".format( """.format(
i i
), ),

View File

@ -0,0 +1,5 @@
<clickhouse>
<merge_tree>
<replace_long_file_name_to_hash>0</replace_long_file_name_to_hash>
</merge_tree>
</clickhouse>

View File

@ -9,12 +9,20 @@ cluster = ClickHouseCluster(__file__)
node1 = cluster.add_instance( node1 = cluster.add_instance(
"node1", "node1",
main_configs=["configs/default_compression.xml", "configs/wide_parts_only.xml"], main_configs=[
"configs/default_compression.xml",
"configs/wide_parts_only.xml",
"configs/long_names.xml",
],
with_zookeeper=True, with_zookeeper=True,
) )
node2 = cluster.add_instance( node2 = cluster.add_instance(
"node2", "node2",
main_configs=["configs/default_compression.xml", "configs/wide_parts_only.xml"], main_configs=[
"configs/default_compression.xml",
"configs/wide_parts_only.xml",
"configs/long_names.xml",
],
with_zookeeper=True, with_zookeeper=True,
) )
node3 = cluster.add_instance( node3 = cluster.add_instance(

View File

@ -23,7 +23,7 @@ def test_file_path_escaping(started_cluster):
node.query( node.query(
""" """
CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32) CREATE TABLE test.`T.a_b,l-e!` (`~Id` UInt32)
ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0; ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0, replace_long_file_name_to_hash = 0;
""" """
) )
node.query("""INSERT INTO test.`T.a_b,l-e!` VALUES (1);""") node.query("""INSERT INTO test.`T.a_b,l-e!` VALUES (1);""")
@ -48,7 +48,7 @@ def test_file_path_escaping(started_cluster):
node.query( node.query(
""" """
CREATE TABLE `test 2`.`T.a_b,l-e!` UUID '12345678-1000-4000-8000-000000000001' (`~Id` UInt32) CREATE TABLE `test 2`.`T.a_b,l-e!` UUID '12345678-1000-4000-8000-000000000001' (`~Id` UInt32)
ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0; ENGINE = MergeTree() PARTITION BY `~Id` ORDER BY `~Id` SETTINGS min_bytes_for_wide_part = 0, replace_long_file_name_to_hash = 0;
""" """
) )
node.query("""INSERT INTO `test 2`.`T.a_b,l-e!` VALUES (1);""") node.query("""INSERT INTO `test 2`.`T.a_b,l-e!` VALUES (1);""")

View File

@ -2,5 +2,6 @@
<merge_tree> <merge_tree>
<min_rows_for_wide_part>0</min_rows_for_wide_part> <min_rows_for_wide_part>0</min_rows_for_wide_part>
<min_bytes_for_wide_part>0</min_bytes_for_wide_part> <min_bytes_for_wide_part>0</min_bytes_for_wide_part>
<replace_long_file_name_to_hash>0</replace_long_file_name_to_hash>
</merge_tree> </merge_tree>
</clickhouse> </clickhouse>

View File

@ -150,7 +150,7 @@ def partition_table_complex(started_cluster):
q("DROP TABLE IF EXISTS test.partition_complex") q("DROP TABLE IF EXISTS test.partition_complex")
q( q(
"CREATE TABLE test.partition_complex (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) " "CREATE TABLE test.partition_complex (p Date, k Int8, v1 Int8 MATERIALIZED k + 1) "
"ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1" "ENGINE = MergeTree PARTITION BY p ORDER BY k SETTINGS index_granularity=1, index_granularity_bytes=0, compress_marks=false, compress_primary_key=false, ratio_of_defaults_for_sparse_serialization=1, replace_long_file_name_to_hash=false"
) )
q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(31), 1)") q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(31), 1)")
q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(1), 2)") q("INSERT INTO test.partition_complex (p, k) VALUES(toDate(1), 2)")

View File

@ -4,7 +4,13 @@ DROP TABLE IF EXISTS test_00961;
CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32) CREATE TABLE test_00961 (d Date, a String, b UInt8, x String, y Int8, z UInt32)
ENGINE = MergeTree PARTITION BY d ORDER BY (a, b) ENGINE = MergeTree PARTITION BY d ORDER BY (a, b)
SETTINGS index_granularity = 111, min_bytes_for_wide_part = 0, compress_marks = 0, compress_primary_key = 0, index_granularity_bytes = '10Mi', ratio_of_defaults_for_sparse_serialization = 1; SETTINGS index_granularity = 111,
min_bytes_for_wide_part = 0,
compress_marks = 0,
compress_primary_key = 0,
index_granularity_bytes = '10Mi',
ratio_of_defaults_for_sparse_serialization = 1,
replace_long_file_name_to_hash = 0;
INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789); INSERT INTO test_00961 VALUES ('2000-01-01', 'Hello, world!', 123, 'xxx yyy', -123, 123456789);

View File

@ -10,7 +10,7 @@ $CLICKHOUSE_CLIENT -q "drop table if exists rmt sync;"
$CLICKHOUSE_CLIENT -q "CREATE TABLE rmt (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16)) $CLICKHOUSE_CLIENT -q "CREATE TABLE rmt (a UInt8, b Int16, c Float32, d String, e Array(UInt8), f Nullable(UUID), g Tuple(UInt8, UInt16))
ENGINE = ReplicatedMergeTree('/test/02253/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY a PARTITION BY b % 10 ENGINE = ReplicatedMergeTree('/test/02253/$CLICKHOUSE_TEST_ZOOKEEPER_PREFIX/rmt', '1') ORDER BY a PARTITION BY b % 10
SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0, SETTINGS old_parts_lifetime = 1, cleanup_delay_period = 0, cleanup_delay_period_random_add = 0,
cleanup_thread_preferred_points_per_iteration=0, min_bytes_for_wide_part=0, remove_empty_parts=0" cleanup_thread_preferred_points_per_iteration=0, min_bytes_for_wide_part=0, remove_empty_parts=0, replace_long_file_name_to_hash=0"
$CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO rmt SELECT rand(1), 0, 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(1000);" $CLICKHOUSE_CLIENT --insert_keeper_fault_injection_probability=0 -q "INSERT INTO rmt SELECT rand(1), 0, 1 / rand(3), toString(rand(4)), [rand(5), rand(6)], rand(7) % 2 ? NULL : generateUUIDv4(), (rand(8), rand(9)) FROM numbers(1000);"

View File

@ -0,0 +1,432 @@
One block
Parallel formatting: 0
JSON
{
"meta":
[
{
"name": "number",
"type": "UInt64"
},
{
"name": "res",
"type": "UInt8"
}
],
"data":
[
],
"rows": 0,
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
JSONEachRow
{"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "}
JSONCompact
{
"meta":
[
{
"name": "number",
"type": "UInt64"
},
{
"name": "res",
"type": "UInt8"
}
],
"data":
[
],
"rows": 0,
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
JSONCompactEachRow
["Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "]
JSONObjectEachRow
{
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
XML
<?xml version='1.0' encoding='UTF-8' ?>
<result>
<meta>
<columns>
<column>
<name>number</name>
<type>UInt64</type>
</column>
<column>
<name>res</name>
<type>UInt8</type>
</column>
</columns>
</meta>
<data>
</data>
<rows>0</rows>
<exception>Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) </exception>
</result>
Parallel formatting: 1
JSON
{
"meta":
[
{
"name": "number",
"type": "UInt64"
},
{
"name": "res",
"type": "UInt8"
}
],
"data":
[
],
"rows": 0,
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
JSONEachRow
{"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "}
JSONCompact
{
"meta":
[
{
"name": "number",
"type": "UInt64"
},
{
"name": "res",
"type": "UInt8"
}
],
"data":
[
],
"rows": 0,
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
JSONCompactEachRow
["Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "]
JSONObjectEachRow
{
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
XML
<?xml version='1.0' encoding='UTF-8' ?>
<result>
<meta>
<columns>
<column>
<name>number</name>
<type>UInt64</type>
</column>
<column>
<name>res</name>
<type>UInt8</type>
</column>
</columns>
</meta>
<data>
</data>
<rows>0</rows>
<exception>Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) </exception>
</result>
Several blocks
Without parallel formatting
JSON
{
"meta":
[
{
"name": "number",
"type": "UInt64"
},
{
"name": "res",
"type": "UInt8"
}
],
"data":
[
{
"number": "0",
"res": 0
},
{
"number": "1",
"res": 0
},
{
"number": "2",
"res": 0
},
{
"number": "3",
"res": 0
}
],
"rows": 4,
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
JSONEachRow
{"number":"0","res":0}
{"number":"1","res":0}
{"number":"2","res":0}
{"number":"3","res":0}
{"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "}
JSONCompact
{
"meta":
[
{
"name": "number",
"type": "UInt64"
},
{
"name": "res",
"type": "UInt8"
}
],
"data":
[
["0", 0],
["1", 0],
["2", 0],
["3", 0]
],
"rows": 4,
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
JSONCompactEachRow
["0", 0]
["1", 0]
["2", 0]
["3", 0]
["Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "]
JSONObjectEachRow
{
"row_1": {"number":"0","res":0},
"row_2": {"number":"1","res":0},
"row_3": {"number":"2","res":0},
"row_4": {"number":"3","res":0},
"exception": "Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) "
}
XML
<?xml version='1.0' encoding='UTF-8' ?>
<result>
<meta>
<columns>
<column>
<name>number</name>
<type>UInt64</type>
</column>
<column>
<name>res</name>
<type>UInt8</type>
</column>
</columns>
</meta>
<data>
<row>
<number>0</number>
<res>0</res>
</row>
<row>
<number>1</number>
<res>0</res>
</row>
<row>
<number>2</number>
<res>0</res>
</row>
<row>
<number>3</number>
<res>0</res>
</row>
</data>
<rows>4</rows>
<exception>Code: 395. : Value passed to 'throwIf' function is non-zero: while executing 'FUNCTION throwIf(greater(number, 3) :: 2) -> throwIf(greater(number, 3)) UInt8 : 1'. (FUNCTION_THROW_IF_VALUE_IS_NON_ZERO) </exception>
</result>
With parallel formatting
JSON
1
JSONCompact
1
JSONObjectEachRow
1
JSONEachRow
1
JSONCompactEachRow
1
Formatting error
Without parallel formatting
JSON
{
"meta":
[
{
"name": "x",
"type": "UInt32"
},
{
"name": "s",
"type": "String"
},
{
"name": "y",
"type": "Enum8('a' = 1)"
}
],
"data":
[
{
"x": 1,
"s": "str1",
"y": "a"
},
{
"x": 2,
"s": "str2",
"y": "a"
},
{
"x": 3,
"s": "str3",
"y": "a"
}
],
"rows": 3,
"exception": "Code: 36. : Unexpected value 99 in enum: While executing JSONRowOutputFormat. (BAD_ARGUMENTS) "
}
JSONEachRow
{"x":1,"s":"str1","y":"a"}
{"x":2,"s":"str2","y":"a"}
{"x":3,"s":"str3","y":"a"}
{"exception": "Code: 36. : Unexpected value 99 in enum: While executing JSONEachRowRowOutputFormat. (BAD_ARGUMENTS) "}
JSONCompact
{
"meta":
[
{
"name": "x",
"type": "UInt32"
},
{
"name": "s",
"type": "String"
},
{
"name": "y",
"type": "Enum8('a' = 1)"
}
],
"data":
[
[1, "str1", "a"],
[2, "str2", "a"],
[3, "str3", "a"]
],
"rows": 3,
"exception": "Code: 36. : Unexpected value 99 in enum: While executing JSONCompactRowOutputFormat. (BAD_ARGUMENTS) "
}
JSONCompactEachRow
[1, "str1", "a"]
[2, "str2", "a"]
[3, "str3", "a"]
["Code: 36. : Unexpected value 99 in enum: While executing JSONCompactEachRowRowOutputFormat. (BAD_ARGUMENTS) "]
JSONObjectEachRow
{
"row_1": {"x":1,"s":"str1","y":"a"},
"row_2": {"x":2,"s":"str2","y":"a"},
"row_3": {"x":3,"s":"str3","y":"a"},
"exception": "Code: 36. : Unexpected value 99 in enum: While executing JSONObjectEachRowRowOutputFormat. (BAD_ARGUMENTS) "
}
XML
<?xml version='1.0' encoding='UTF-8' ?>
<result>
<meta>
<columns>
<column>
<name>x</name>
<type>UInt32</type>
</column>
<column>
<name>s</name>
<type>String</type>
</column>
<column>
<name>y</name>
<type>Enum8('a' = 1)</type>
</column>
</columns>
</meta>
<data>
<row>
<x>1</x>
<s>str1</s>
<y>a</y>
</row>
<row>
<x>2</x>
<s>str2</s>
<y>a</y>
</row>
<row>
<x>3</x>
<s>str3</s>
<y>a</y>
</row>
</data>
<rows>3</rows>
<exception>Code: 36. : Unexpected value 99 in enum: While executing XMLRowOutputFormat. (BAD_ARGUMENTS) </exception>
</result>
With parallel formatting
JSON
1
JSONCompact
1
JSONObjectEachRow
1
JSONEachRow
1
JSONCompactEachRow
1
Test 1
1
1
Test 2
1
1
Test 3
1
1

View File

@ -0,0 +1,108 @@
#!/usr/bin/env bash
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
CH_URL="$CLICKHOUSE_URL&http_write_exception_in_output_format=1&allow_experimental_analyzer=0"
echo "One block"
for parallel in 0 1
do
echo "Parallel formatting: $parallel"
for format in JSON JSONEachRow JSONCompact JSONCompactEachRow JSONObjectEachRow XML
do
echo $format
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select number, throwIf(number > 3) as res from numbers(10) format $format settings output_format_parallel_formatting=$parallel" | sed "s/(version .*)//" | sed "s/DB::Exception//"
done
done
echo "Several blocks"
echo "Without parallel formatting"
for format in JSON JSONEachRow JSONCompact JSONCompactEachRow JSONObjectEachRow XML
do
echo $format
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select number, throwIf(number > 3) as res from system.numbers format $format settings max_block_size=1, output_format_parallel_formatting=0" | sed "s/(version .*)//" | sed "s/DB::Exception//"
done
echo "With parallel formatting"
for format in JSON JSONCompact JSONObjectEachRow
do
echo $format
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select number, throwIf(number > 3) as res from system.numbers format $format settings max_block_size=1, output_format_parallel_formatting=1" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
done
for format in JSONEachRow JSONCompactEachRow
do
echo $format
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select number, throwIf(number > 3) as res from system.numbers format $format settings max_block_size=1, output_format_parallel_formatting=1" | $CLICKHOUSE_LOCAL --input-format=LineAsString -q "select min(isValidJSON(line)) from table"
done
echo "Formatting error"
$CLICKHOUSE_CLIENT -q "drop table if exists test_02841"
$CLICKHOUSE_CLIENT -q "create table test_02841 (x UInt32, s String, y Enum('a' = 1)) engine=MergeTree order by x"
$CLICKHOUSE_CLIENT -q "system stop merges test_02841"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (1, 'str1', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (2, 'str2', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (3, 'str3', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (5, 'str5', 99)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (6, 'str6', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (7, 'str7', 1)"
echo "Without parallel formatting"
for format in JSON JSONEachRow JSONCompact JSONCompactEachRow JSONObjectEachRow XML
do
echo $format
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 order by x format $format settings output_format_parallel_formatting=0" | sed "s/(version .*)//" | sed "s/DB::Exception//"
done
echo "With parallel formatting"
for format in JSON JSONCompact JSONObjectEachRow
do
echo $format
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format $format settings output_format_parallel_formatting=1" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
done
for format in JSONEachRow JSONCompactEachRow
do
echo $format
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format $format settings output_format_parallel_formatting=1" | $CLICKHOUSE_LOCAL --input-format=LineAsString -q "select min(isValidJSON(line)) from table"
done
echo "Test 1"
$CLICKHOUSE_CLIENT -q "truncate table test_02841"
$CLICKHOUSE_CLIENT -q "insert into test_02841 select 1, repeat('aaaaa', 1000000), 1"
$CLICKHOUSE_CLIENT -q "insert into test_02841 select 2, repeat('aaaaa', 1000000), 99"
$CLICKHOUSE_CLIENT -q "insert into test_02841 select 3, repeat('aaaaa', 1000000), 1"
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format JSON settings output_format_parallel_formatting=0" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format JSON settings output_format_parallel_formatting=1" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
echo "Test 2"
$CLICKHOUSE_CLIENT -q "truncate table test_02841"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (1, 'str1', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (2, 'str2', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 select number, 'str_numbers_1', 1 from numbers(10000)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (3, 'str4', 99)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (4, 'str5', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 select number, 'str_numbers_2', 1 from numbers(10000)"
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format JSON settings output_format_parallel_formatting=0" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format JSON settings output_format_parallel_formatting=1" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
echo "Test 3"
$CLICKHOUSE_CLIENT -q "truncate table test_02841"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (1, 'str1', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (2, 'str2', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 select number, 'str_numbers_1', number > 9000 ? 99 : 1 from numbers(10000)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (3, 'str4', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 values (4, 'str5', 1)"
$CLICKHOUSE_CLIENT -q "insert into test_02841 select number, 'str_numbers_2', 1 from numbers(10000)"
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format JSON settings output_format_parallel_formatting=0" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
${CLICKHOUSE_CURL} -sS "$CH_URL" -d "select * from test_02841 format JSON settings output_format_parallel_formatting=1" | $CLICKHOUSE_LOCAL --input-format=JSONAsString -q "select isValidJSON(json) from table"
$CLICKHOUSE_CLIENT -q "drop table test_02841"

Some files were not shown because too many files have changed in this diff Show More