Merge branch 'master' of github.com:ClickHouse/ClickHouse

This commit is contained in:
BayoNet 2019-11-22 18:04:33 +03:00
commit 08b873200d
101 changed files with 1750 additions and 369 deletions

View File

@ -14,7 +14,6 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Events ## Upcoming Events
* [ClickHouse Meetup in Ankara](https://www.eventbrite.com/e/clickhouse-meetup-ankara-create-blazing-fast-experiences-w-clickhouse-tickets-73100530655) on November 21.
* [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23. * [ClickHouse Meetup in Singapore](https://www.meetup.com/Singapore-Clickhouse-Meetup-Group/events/265085331/) on November 23.
* [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3. * [ClickHouse Meetup in San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3.

2
contrib/libunwind vendored

@ -1 +1 @@
Subproject commit 5afe6d87ae9e66485c7fcb106d2f7c2c0359c8f6 Subproject commit 68cffcbbd1840e14664a5f7f19c5e43f65c525b5

2
contrib/protobuf vendored

@ -1 +1 @@
Subproject commit 12735370922a35f03999afff478e1c6d7aa917a4 Subproject commit 0795fa6bc443666068bec56bf700e1f488f592f1

View File

@ -1112,7 +1112,14 @@ private:
/// Check if server send Exception packet /// Check if server send Exception packet
auto packet_type = connection->checkPacket(); auto packet_type = connection->checkPacket();
if (packet_type && *packet_type == Protocol::Server::Exception) if (packet_type && *packet_type == Protocol::Server::Exception)
{
/*
* We're exiting with error, so it makes sense to kill the
* input stream without waiting for it to complete.
*/
async_block_input->cancel(true);
return; return;
}
connection->sendData(block); connection->sendData(block);
processed_rows += block.rows(); processed_rows += block.rows();

View File

@ -407,16 +407,16 @@ void HTTPHandler::processQuery(
{ {
if (http_request_compression_method_str == "gzip") if (http_request_compression_method_str == "gzip")
{ {
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Gzip); in_post = std::make_unique<ZlibInflatingReadBuffer>(std::move(in_post_raw), CompressionMethod::Gzip);
} }
else if (http_request_compression_method_str == "deflate") else if (http_request_compression_method_str == "deflate")
{ {
in_post = std::make_unique<ZlibInflatingReadBuffer>(*in_post_raw, CompressionMethod::Zlib); in_post = std::make_unique<ZlibInflatingReadBuffer>(std::move(in_post_raw), CompressionMethod::Zlib);
} }
#if USE_BROTLI #if USE_BROTLI
else if (http_request_compression_method_str == "br") else if (http_request_compression_method_str == "br")
{ {
in_post = std::make_unique<BrotliReadBuffer>(*in_post_raw); in_post = std::make_unique<BrotliReadBuffer>(std::move(in_post_raw));
} }
#endif #endif
else else

View File

@ -41,8 +41,8 @@ struct Settings : public SettingsCollection<Settings>
* Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field, * Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field,
* but we are not going to do it, because settings is used everywhere as static struct fields. * but we are not going to do it, because settings is used everywhere as static struct fields.
* *
* `flags` can be either 0 or IGNORABLE. * `flags` can be either 0 or IMPORTANT.
* A setting is "IGNORABLE" if it doesn't affects the results of the queries and can be ignored without exception. * A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions.
*/ */
#define LIST_OF_SETTINGS(M) \ #define LIST_OF_SETTINGS(M) \
@ -72,7 +72,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \ M(SettingUInt64, distributed_connections_pool_size, DBMS_DEFAULT_DISTRIBUTED_CONNECTIONS_POOL_SIZE, "Maximum number of connections with one remote server in the pool.", 0) \
M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \ M(SettingUInt64, connections_with_failover_max_tries, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_MAX_TRIES, "The maximum number of attempts to connect to replicas.", 0) \
M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The mininum size of part to upload during multipart upload to S3.", 0) \ M(SettingUInt64, s3_min_upload_part_size, 512*1024*1024, "The mininum size of part to upload during multipart upload to S3.", 0) \
M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", 0) \ M(SettingBool, extremes, false, "Calculate minimums and maximums of the result columns. They can be output in JSON-formats.", IMPORTANT) \
M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \ M(SettingBool, use_uncompressed_cache, true, "Whether to use the cache of uncompressed blocks.", 0) \
M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \ M(SettingBool, replace_running_query, false, "Whether the running request should be canceled with the same id as the new one.", 0) \
M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \ M(SettingUInt64, background_pool_size, 16, "Number of threads performing background work for tables (for example, merging in merge tree). Only has meaning at server startup.", 0) \
@ -90,7 +90,7 @@ struct Settings : public SettingsCollection<Settings>
\ \
M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \ M(SettingLoadBalancing, load_balancing, LoadBalancing::RANDOM, "Which replicas (among healthy replicas) to preferably send a query to (on the first attempt) for distributed processing.", 0) \
\ \
M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = any are present.", 0) \ M(SettingTotalsMode, totals_mode, TotalsMode::AFTER_HAVING_EXCLUSIVE, "How to calculate TOTALS when HAVING is present, as well as when max_rows_to_group_by and group_by_overflow_mode = any are present.", IMPORTANT) \
M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \ M(SettingFloat, totals_auto_threshold, 0.5, "The threshold for totals_mode = 'auto'.", 0) \
\ \
M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \ M(SettingBool, allow_suspicious_low_cardinality_types, false, "In CREATE TABLE statement allows specifying LowCardinality modifier for types of small fixed size (8 or less). Enabling this may increase merge times and memory consumption.", 0) \
@ -111,6 +111,9 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \ M(SettingBool, distributed_group_by_no_merge, false, "Do not merge aggregation states from different servers for distributed query processing - in case it is for certain that there are different keys on different shards.", 0) \
M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \ M(SettingBool, optimize_skip_unused_shards, false, "Assumes that data is distributed by sharding_key. Optimization to skip unused shards if SELECT query filters by sharding_key.", 0) \
\ \
M(SettingBool, input_format_parallel_parsing, true, "Enable parallel parsing for some data formats.", 0) \
M(SettingUInt64, min_chunk_bytes_for_parallel_parsing, (1024 * 1024), "The minimum chunk size in bytes, which each thread will parse in parallel.", 0) \
\
M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \ M(SettingUInt64, merge_tree_min_rows_for_concurrent_read, (20 * 8192), "If at least as many lines are read from one file, the reading can be parallelized.", 0) \
M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \ M(SettingUInt64, merge_tree_min_bytes_for_concurrent_read, (24 * 10 * 1024 * 1024), "If at least as many bytes are read from one file, the reading can be parallelized.", 0) \
M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.", 0) \ M(SettingUInt64, merge_tree_min_rows_for_seek, 0, "You can skip reading more than that number of rows at the price of one seek per file.", 0) \
@ -146,7 +149,7 @@ struct Settings : public SettingsCollection<Settings>
\ \
M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \ M(SettingUInt64, log_queries_cut_to_length, 100000, "If query length is greater than specified threshold (in bytes), then cut query when writing to query log. Also limit length of printed query in ordinary text log.", 0) \
\ \
M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", 0) \ M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?", IMPORTANT) \
\ \
M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \ M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.", 0) \
\ \
@ -179,7 +182,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \ M(SettingBool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, CSVWithNames, TSVWithNames and TSKV formats).", 0) \
M(SettingBool, input_format_with_names_use_header, false, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \ M(SettingBool, input_format_with_names_use_header, false, "For TSVWithNames and CSVWithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \ M(SettingBool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \
M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", 0) \ M(SettingBool, input_format_defaults_for_omitted_fields, true, "For input data calculate default expressions for omitted fields (it works for JSONEachRow, CSV and TSV formats).", IMPORTANT) \
M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \ M(SettingBool, input_format_tsv_empty_as_default, false, "Treat empty fields in TSV input as default values.", 0) \
M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \ M(SettingBool, input_format_null_as_default, false, "For text input formats initialize null fields with default values if data type of this field is not nullable", 0) \
\ \
@ -209,10 +212,10 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ M(SettingUInt64, input_format_allow_errors_num, 0, "Maximum absolute amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \ M(SettingFloat, input_format_allow_errors_ratio, 0, "Maximum relative amount of errors while reading text formats (like CSV, TSV). In case of error, if at least absolute or relative amount of errors is lower than corresponding value, will skip until next line and continue.", 0) \
\ \
M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", 0) \ M(SettingBool, join_use_nulls, 0, "Use NULLs for non-joined rows of outer JOINs for types that can be inside Nullable. If false, use default value of corresponding columns data type.", IMPORTANT) \
\ \
M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \ M(SettingJoinStrictness, join_default_strictness, JoinStrictness::ALL, "Set default strictness in JOIN query. Possible values: empty string, 'ANY', 'ALL'. If empty, query without strictness will throw exception.", 0) \
M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.", 0) \ M(SettingBool, any_join_distinct_right_table_keys, false, "Enable old ANY JOIN logic with many-to-one left-to-right table keys mapping for all ANY JOINs. It leads to confusing not equal results for 't1 ANY LEFT JOIN t2' and 't2 ANY RIGHT JOIN t1'. ANY RIGHT JOIN needs one-to-many keys maping to be consistent with LEFT one.", IMPORTANT) \
\ \
M(SettingUInt64, preferred_block_size_bytes, 1000000, "", 0) \ M(SettingUInt64, preferred_block_size_bytes, 1000000, "", 0) \
\ \
@ -307,7 +310,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \ M(SettingUInt64, max_rows_in_join, 0, "Maximum size of the hash table for JOIN (in number of rows).", 0) \
M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \ M(SettingUInt64, max_bytes_in_join, 0, "Maximum size of the hash table for JOIN (in number of bytes in memory).", 0) \
M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \ M(SettingOverflowMode, join_overflow_mode, OverflowMode::THROW, "What to do when the limit is exceeded.", 0) \
M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", 0) \ M(SettingBool, join_any_take_last_row, false, "When disabled (default) ANY JOIN will take the first found row for a key. When enabled, it will take the last row seen if there are multiple rows for the same key.", IMPORTANT) \
M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join for LEFT and INNER JOINs.", 0) \ M(SettingBool, partial_merge_join, false, "Use partial merge join instead of hash join for LEFT and INNER JOINs.", 0) \
M(SettingBool, partial_merge_join_optimizations, false, "Enable optimizations in partial merge join", 0) \ M(SettingBool, partial_merge_join_optimizations, false, "Enable optimizations in partial merge join", 0) \
M(SettingUInt64, default_max_bytes_in_join, 100000000, "Maximum size of right-side table if limit's required but max_bytes_in_join is not set.", 0) \ M(SettingUInt64, default_max_bytes_in_join, 100000000, "Maximum size of right-side table if limit's required but max_bytes_in_join is not set.", 0) \

View File

@ -286,7 +286,7 @@ enum class SettingsBinaryFormat
* { * {
* # define APPLY_FOR_MYSETTINGS(M) \ * # define APPLY_FOR_MYSETTINGS(M) \
* M(SettingUInt64, a, 100, "Description of a", 0) \ * M(SettingUInt64, a, 100, "Description of a", 0) \
* M(SettingFloat, f, 3.11, "Description of f", IGNORABLE) // IGNORABLE - means the setting can be ignored by older versions) \ * M(SettingFloat, f, 3.11, "Description of f", IMPORTANT) // IMPORTANT - means the setting can't be ignored by older versions) \
* M(SettingString, s, "default", "Description of s", 0) * M(SettingString, s, "default", "Description of s", 0)
* *
* DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS) * DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS)
@ -316,7 +316,7 @@ private:
StringRef name; StringRef name;
StringRef description; StringRef description;
bool is_ignorable; bool is_important;
IsChangedFunction is_changed; IsChangedFunction is_changed;
GetStringFunction get_string; GetStringFunction get_string;
GetFieldFunction get_field; GetFieldFunction get_field;

View File

@ -254,7 +254,7 @@ void SettingsCollection<Derived>::serialize(WriteBuffer & buf, SettingsBinaryFor
{ {
details::SettingsCollectionUtils::serializeName(member.name, buf); details::SettingsCollectionUtils::serializeName(member.name, buf);
if (format >= SettingsBinaryFormat::STRINGS) if (format >= SettingsBinaryFormat::STRINGS)
details::SettingsCollectionUtils::serializeFlag(member.is_ignorable, buf); details::SettingsCollectionUtils::serializeFlag(member.is_important, buf);
member.serialize(castToDerived(), buf, format); member.serialize(castToDerived(), buf, format);
} }
} }
@ -272,18 +272,20 @@ void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFo
if (name.empty() /* empty string is a marker of the end of settings */) if (name.empty() /* empty string is a marker of the end of settings */)
break; break;
auto * member = the_members.find(name); auto * member = the_members.find(name);
bool is_ignorable = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : false; bool is_important = (format >= SettingsBinaryFormat::STRINGS) ? details::SettingsCollectionUtils::deserializeFlag(buf) : true;
if (member) if (member)
{ {
member->deserialize(castToDerived(), buf, format); member->deserialize(castToDerived(), buf, format);
} }
else if (is_ignorable) else if (is_important)
{
details::SettingsCollectionUtils::throwNameNotFound(name);
}
else
{ {
details::SettingsCollectionUtils::warningNameNotFound(name); details::SettingsCollectionUtils::warningNameNotFound(name);
details::SettingsCollectionUtils::skipValue(buf); details::SettingsCollectionUtils::skipValue(buf);
} }
else
details::SettingsCollectionUtils::throwNameNotFound(name);
} }
} }
@ -298,8 +300,8 @@ void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFo
{ \ { \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \ LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_DEFINE_FUNCTIONS_HELPER_) \
}; \ }; \
constexpr int IGNORABLE = 1; \ constexpr int IMPORTANT = 1; \
UNUSED(IGNORABLE); \ UNUSED(IMPORTANT); \
LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \ LIST_OF_SETTINGS_MACRO(IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_) \
} \ } \
/** \ /** \
@ -322,7 +324,7 @@ void SettingsCollection<Derived>::deserialize(ReadBuffer & buf, SettingsBinaryFo
#define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \ #define IMPLEMENT_SETTINGS_COLLECTION_ADD_MEMBER_INFO_HELPER_(TYPE, NAME, DEFAULT, DESCRIPTION, FLAGS) \
add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \ add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \
FLAGS & IGNORABLE, \ FLAGS & IMPORTANT, \
[](const Derived & d) { return d.NAME.changed; }, \ [](const Derived & d) { return d.NAME.changed; }, \
&Functions::NAME##_getString, &Functions::NAME##_getField, \ &Functions::NAME##_getString, &Functions::NAME##_getField, \
&Functions::NAME##_setString, &Functions::NAME##_setField, \ &Functions::NAME##_setString, &Functions::NAME##_setField, \

View File

@ -0,0 +1,203 @@
#include <DataStreams/ParallelParsingBlockInputStream.h>
#include "ParallelParsingBlockInputStream.h"
namespace DB
{
void ParallelParsingBlockInputStream::segmentatorThreadFunction()
{
setThreadName("Segmentator");
try
{
while (!finished)
{
const auto current_unit_number = segmentator_ticket_number % processing_units.size();
auto & unit = processing_units[current_unit_number];
{
std::unique_lock lock(mutex);
segmentator_condvar.wait(lock,
[&]{ return unit.status == READY_TO_INSERT || finished; });
}
if (finished)
{
break;
}
assert(unit.status == READY_TO_INSERT);
// Segmentating the original input.
unit.segment.resize(0);
const bool have_more_data = file_segmentation_engine(original_buffer,
unit.segment, min_chunk_bytes);
unit.is_last = !have_more_data;
unit.status = READY_TO_PARSE;
scheduleParserThreadForUnitWithNumber(current_unit_number);
++segmentator_ticket_number;
if (!have_more_data)
{
break;
}
}
}
catch (...)
{
onBackgroundException();
}
}
void ParallelParsingBlockInputStream::parserThreadFunction(size_t current_unit_number)
{
try
{
setThreadName("ChunkParser");
auto & unit = processing_units[current_unit_number];
/*
* This is kind of suspicious -- the input_process_creator contract with
* respect to multithreaded use is not clear, but we hope that it is
* just a 'normal' factory class that doesn't have any state, and so we
* can use it from multiple threads simultaneously.
*/
ReadBuffer read_buffer(unit.segment.data(), unit.segment.size(), 0);
auto parser = std::make_unique<InputStreamFromInputFormat>(
input_processor_creator(read_buffer, header, context,
row_input_format_params, format_settings));
unit.block_ext.block.clear();
unit.block_ext.block_missing_values.clear();
// We don't know how many blocks will be. So we have to read them all
// until an empty block occured.
Block block;
while (!finished && (block = parser->read()) != Block())
{
unit.block_ext.block.emplace_back(block);
unit.block_ext.block_missing_values.emplace_back(parser->getMissingValues());
}
// We suppose we will get at least some blocks for a non-empty buffer,
// except at the end of file. Also see a matching assert in readImpl().
assert(unit.is_last || unit.block_ext.block.size() > 0);
std::unique_lock lock(mutex);
unit.status = READY_TO_READ;
reader_condvar.notify_all();
}
catch (...)
{
onBackgroundException();
}
}
void ParallelParsingBlockInputStream::onBackgroundException()
{
tryLogCurrentException(__PRETTY_FUNCTION__);
std::unique_lock lock(mutex);
if (!background_exception)
{
background_exception = std::current_exception();
}
finished = true;
reader_condvar.notify_all();
segmentator_condvar.notify_all();
}
Block ParallelParsingBlockInputStream::readImpl()
{
if (isCancelledOrThrowIfKilled() || finished)
{
/**
* Check for background exception and rethrow it before we return.
*/
std::unique_lock lock(mutex);
if (background_exception)
{
lock.unlock();
cancel(false);
std::rethrow_exception(background_exception);
}
return Block{};
}
const auto current_unit_number = reader_ticket_number % processing_units.size();
auto & unit = processing_units[current_unit_number];
if (!next_block_in_current_unit.has_value())
{
// We have read out all the Blocks from the previous Processing Unit,
// wait for the current one to become ready.
std::unique_lock lock(mutex);
reader_condvar.wait(lock, [&](){ return unit.status == READY_TO_READ || finished; });
if (finished)
{
/**
* Check for background exception and rethrow it before we return.
*/
if (background_exception)
{
lock.unlock();
cancel(false);
std::rethrow_exception(background_exception);
}
return Block{};
}
assert(unit.status == READY_TO_READ);
next_block_in_current_unit = 0;
}
if (unit.block_ext.block.size() == 0)
{
/*
* Can we get zero blocks for an entire segment, when the format parser
* skips it entire content and does not create any blocks? Probably not,
* but if we ever do, we should add a loop around the above if, to skip
* these. Also see a matching assert in the parser thread.
*/
assert(unit.is_last);
finished = true;
return Block{};
}
assert(next_block_in_current_unit.value() < unit.block_ext.block.size());
Block res = std::move(unit.block_ext.block.at(*next_block_in_current_unit));
last_block_missing_values = std::move(unit.block_ext.block_missing_values[*next_block_in_current_unit]);
next_block_in_current_unit.value() += 1;
if (*next_block_in_current_unit == unit.block_ext.block.size())
{
// Finished reading this Processing Unit, move to the next one.
next_block_in_current_unit.reset();
++reader_ticket_number;
if (unit.is_last)
{
// It it was the last unit, we're finished.
finished = true;
}
else
{
// Pass the unit back to the segmentator.
std::unique_lock lock(mutex);
unit.status = READY_TO_INSERT;
segmentator_condvar.notify_all();
}
}
return res;
}
}

View File

@ -0,0 +1,258 @@
#pragma once
#include <DataStreams/IBlockInputStream.h>
#include <Formats/FormatFactory.h>
#include <Common/ThreadPool.h>
#include <Common/setThreadName.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ReadBuffer.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <Interpreters/Context.h>
namespace DB
{
/**
* ORDER-PRESERVING parallel parsing of data formats.
* It splits original data into chunks. Then each chunk is parsed by different thread.
* The number of chunks equals to the number or parser threads.
* The size of chunk is equal to min_chunk_bytes_for_parallel_parsing setting.
*
* This stream has three kinds of threads: one segmentator, multiple parsers,
* and one reader thread -- that is, the one from which readImpl() is called.
* They operate one after another on parts of data called "processing units".
* One unit consists of buffer with raw data from file, filled by segmentator
* thread. This raw data is then parsed by a parser thread to form a number of
* Blocks. These Blocks are returned to the parent stream from readImpl().
* After being read out, a processing unit is reused, to save on allocating
* memory for the raw buffer. The processing units are organized into a circular
* array to facilitate reuse and to apply backpressure on the segmentator thread
* -- after it runs out of processing units, it has to wait for the reader to
* read out the previous blocks.
* The outline of what the threads do is as follows:
* segmentator thread:
* 1) wait for the next processing unit to become empty
* 2) fill it with a part of input file
* 3) start a parser thread
* 4) repeat until eof
* parser thread:
* 1) parse the given raw buffer without any synchronization
* 2) signal that the given unit is ready to read
* 3) finish
* readImpl():
* 1) wait for the next processing unit to become ready to read
* 2) take the blocks from the processing unit to return them to the caller
* 3) signal that the processing unit is empty
* 4) repeat until it encounters unit that is marked as "past_the_end"
* All threads must also check for cancel/eof/exception flags.
*/
class ParallelParsingBlockInputStream : public IBlockInputStream
{
private:
using ReadCallback = std::function<void()>;
using InputProcessorCreator = std::function<InputFormatPtr(
ReadBuffer & buf,
const Block & header,
const Context & context,
const RowInputFormatParams & params,
const FormatSettings & settings)>;
public:
struct InputCreatorParams
{
const Block &sample;
const Context &context;
const RowInputFormatParams& row_input_format_params;
const FormatSettings &settings;
};
struct Params
{
ReadBuffer & read_buffer;
const InputProcessorCreator &input_processor_creator;
const InputCreatorParams &input_creator_params;
FormatFactory::FileSegmentationEngine file_segmentation_engine;
int max_threads;
size_t min_chunk_bytes;
};
explicit ParallelParsingBlockInputStream(const Params & params)
: header(params.input_creator_params.sample),
context(params.input_creator_params.context),
row_input_format_params(params.input_creator_params.row_input_format_params),
format_settings(params.input_creator_params.settings),
input_processor_creator(params.input_processor_creator),
min_chunk_bytes(params.min_chunk_bytes),
original_buffer(params.read_buffer),
// Subtract one thread that we use for segmentation and one for
// reading. After that, must have at least two threads left for
// parsing. See the assertion below.
pool(std::max(2, params.max_threads - 2)),
file_segmentation_engine(params.file_segmentation_engine)
{
// See comment above.
assert(params.max_threads >= 4);
// One unit for each thread, including segmentator and reader, plus a
// couple more units so that the segmentation thread doesn't spuriously
// bump into reader thread on wraparound.
processing_units.resize(params.max_threads + 2);
segmentator_thread = ThreadFromGlobalPool([this] { segmentatorThreadFunction(); });
}
String getName() const override { return "ParallelParsing"; }
~ParallelParsingBlockInputStream() override
{
finishAndWait();
}
void cancel(bool kill) override
{
/**
* Can be called multiple times, from different threads. Saturate the
* the kill flag with OR.
*/
if (kill)
is_killed = true;
is_cancelled = true;
/*
* The format parsers themselves are not being cancelled here, so we'll
* have to wait until they process the current block. Given that the
* chunk size is on the order of megabytes, this should't be too long.
* We can't call IInputFormat->cancel here, because the parser object is
* local to the parser thread, and we don't want to introduce any
* synchronization between parser threads and the other threads to get
* better performance. An ideal solution would be to add a callback to
* IInputFormat that checks whether it was cancelled.
*/
finishAndWait();
}
Block getHeader() const override
{
return header;
}
protected:
//Reader routine
Block readImpl() override;
const BlockMissingValues & getMissingValues() const override
{
return last_block_missing_values;
}
private:
const Block header;
const Context context;
const RowInputFormatParams row_input_format_params;
const FormatSettings format_settings;
const InputProcessorCreator input_processor_creator;
const size_t min_chunk_bytes;
/*
* This is declared as atomic to avoid UB, because parser threads access it
* without synchronization.
*/
std::atomic<bool> finished{false};
BlockMissingValues last_block_missing_values;
// Original ReadBuffer to read from.
ReadBuffer & original_buffer;
//Non-atomic because it is used in one thread.
std::optional<size_t> next_block_in_current_unit;
size_t segmentator_ticket_number{0};
size_t reader_ticket_number{0};
std::mutex mutex;
std::condition_variable reader_condvar;
std::condition_variable segmentator_condvar;
// There are multiple "parsers", that's why we use thread pool.
ThreadPool pool;
// Reading and segmentating the file
ThreadFromGlobalPool segmentator_thread;
// Function to segment the file. Then "parsers" will parse that segments.
FormatFactory::FileSegmentationEngine file_segmentation_engine;
enum ProcessingUnitStatus
{
READY_TO_INSERT,
READY_TO_PARSE,
READY_TO_READ
};
struct BlockExt
{
std::vector<Block> block;
std::vector<BlockMissingValues> block_missing_values;
};
struct ProcessingUnit
{
explicit ProcessingUnit()
: status(ProcessingUnitStatus::READY_TO_INSERT)
{
}
BlockExt block_ext;
Memory<> segment;
std::atomic<ProcessingUnitStatus> status;
bool is_last{false};
};
std::exception_ptr background_exception = nullptr;
// We use deque instead of vector, because it does not require a move
// constructor, which is absent for atomics that are inside ProcessingUnit.
std::deque<ProcessingUnit> processing_units;
void scheduleParserThreadForUnitWithNumber(size_t unit_number)
{
pool.scheduleOrThrowOnError(std::bind(&ParallelParsingBlockInputStream::parserThreadFunction, this, unit_number));
}
void finishAndWait()
{
finished = true;
{
std::unique_lock lock(mutex);
segmentator_condvar.notify_all();
reader_condvar.notify_all();
}
if (segmentator_thread.joinable())
segmentator_thread.join();
try
{
pool.wait();
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
}
void segmentatorThreadFunction();
void parserThreadFunction(size_t bucket_num);
// Save/log a background exception, set termination flag, wake up all
// threads. This function is used by segmentator and parsed threads.
// readImpl() is called from the main thread, so the exception handling
// is different.
void onBackgroundException();
};
};

View File

@ -125,7 +125,11 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadAll()
* the necessity of holding process_list_element shared pointer. * the necessity of holding process_list_element shared pointer.
*/ */
if (is_local) if (is_local)
return executeQuery(load_all_query, context, true).in; {
BlockIO res = executeQuery(load_all_query, context, true);
/// FIXME res.in may implicitly use some objects owned be res, but them will be destructed after return
return res.in;
}
return std::make_shared<RemoteBlockInputStream>(pool, load_all_query, sample_block, context); return std::make_shared<RemoteBlockInputStream>(pool, load_all_query, sample_block, context);
} }

View File

@ -1,8 +1,10 @@
#include <algorithm>
#include <Common/config.h> #include <Common/config.h>
#include <Common/Exception.h> #include <Common/Exception.h>
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Core/Settings.h> #include <Core/Settings.h>
#include <DataStreams/MaterializingBlockOutputStream.h> #include <DataStreams/MaterializingBlockOutputStream.h>
#include <DataStreams/ParallelParsingBlockInputStream.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <Processors/Formats/IRowInputFormat.h> #include <Processors/Formats/IRowInputFormat.h>
@ -93,7 +95,7 @@ BlockInputStreamPtr FormatFactory::getInput(
if (!getCreators(name).input_processor_creator) if (!getCreators(name).input_processor_creator)
{ {
const auto & input_getter = getCreators(name).inout_creator; const auto & input_getter = getCreators(name).input_creator;
if (!input_getter) if (!input_getter)
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT); throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
@ -103,6 +105,37 @@ BlockInputStreamPtr FormatFactory::getInput(
return input_getter(buf, sample, context, max_block_size, callback ? callback : ReadCallback(), format_settings); return input_getter(buf, sample, context, max_block_size, callback ? callback : ReadCallback(), format_settings);
} }
const Settings & settings = context.getSettingsRef();
const auto & file_segmentation_engine = getCreators(name).file_segmentation_engine;
// Doesn't make sense to use parallel parsing with less than four threads
// (segmentator + two parsers + reader).
if (settings.input_format_parallel_parsing
&& file_segmentation_engine
&& settings.max_threads >= 4)
{
const auto & input_getter = getCreators(name).input_processor_creator;
if (!input_getter)
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
FormatSettings format_settings = getInputFormatSetting(settings);
RowInputFormatParams row_input_format_params;
row_input_format_params.max_block_size = max_block_size;
row_input_format_params.allow_errors_num = format_settings.input_allow_errors_num;
row_input_format_params.allow_errors_ratio = format_settings.input_allow_errors_ratio;
row_input_format_params.callback = std::move(callback);
row_input_format_params.max_execution_time = settings.max_execution_time;
row_input_format_params.timeout_overflow_mode = settings.timeout_overflow_mode;
auto input_creator_params = ParallelParsingBlockInputStream::InputCreatorParams{sample, context, row_input_format_params, format_settings};
ParallelParsingBlockInputStream::Params params{buf, input_getter,
input_creator_params, file_segmentation_engine,
static_cast<int>(settings.max_threads),
settings.min_chunk_bytes_for_parallel_parsing};
return std::make_shared<ParallelParsingBlockInputStream>(params);
}
auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback)); auto format = getInputFormat(name, buf, sample, context, max_block_size, std::move(callback));
return std::make_shared<InputStreamFromInputFormat>(std::move(format)); return std::make_shared<InputStreamFromInputFormat>(std::move(format));
} }
@ -191,7 +224,7 @@ OutputFormatPtr FormatFactory::getOutputFormat(
void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator) void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator)
{ {
auto & target = dict[name].inout_creator; auto & target = dict[name].input_creator;
if (target) if (target)
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR); throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(input_creator); target = std::move(input_creator);
@ -221,6 +254,13 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro
target = std::move(output_creator); target = std::move(output_creator);
} }
void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine)
{
auto & target = dict[name].file_segmentation_engine;
if (target)
throw Exception("FormatFactory: File segmentation engine " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = file_segmentation_engine;
}
/// Formats for both input/output. /// Formats for both input/output.
@ -249,6 +289,10 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory);
void registerInputFormatProcessorTemplate(FormatFactory & factory); void registerInputFormatProcessorTemplate(FormatFactory & factory);
void registerOutputFormatProcessorTemplate(FormatFactory &factory); void registerOutputFormatProcessorTemplate(FormatFactory &factory);
/// File Segmentation Engines for parallel reading
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory);
/// Output only (presentational) formats. /// Output only (presentational) formats.
void registerOutputFormatNull(FormatFactory & factory); void registerOutputFormatNull(FormatFactory & factory);
@ -299,6 +343,7 @@ FormatFactory::FormatFactory()
registerInputFormatProcessorTemplate(*this); registerInputFormatProcessorTemplate(*this);
registerOutputFormatProcessorTemplate(*this); registerOutputFormatProcessorTemplate(*this);
registerFileSegmentationEngineTabSeparated(*this);
registerOutputFormatNull(*this); registerOutputFormatNull(*this);

View File

@ -2,6 +2,7 @@
#include <Core/Types.h> #include <Core/Types.h>
#include <DataStreams/IBlockStream_fwd.h> #include <DataStreams/IBlockStream_fwd.h>
#include <IO/BufferWithOwnMemory.h>
#include <functional> #include <functional>
#include <memory> #include <memory>
@ -41,6 +42,15 @@ public:
/// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer. /// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer.
using ReadCallback = std::function<void()>; using ReadCallback = std::function<void()>;
/** Fast reading data from buffer and save result to memory.
* Reads at least min_chunk_bytes and some more until the end of the chunk, depends on the format.
* Used in ParallelParsingBlockInputStream.
*/
using FileSegmentationEngine = std::function<bool(
ReadBuffer & buf,
DB::Memory<> & memory,
size_t min_chunk_bytes)>;
/// This callback allows to perform some additional actions after writing a single row. /// This callback allows to perform some additional actions after writing a single row.
/// It's initial purpose was to flush Kafka message for each row. /// It's initial purpose was to flush Kafka message for each row.
using WriteCallback = std::function<void()>; using WriteCallback = std::function<void()>;
@ -77,10 +87,11 @@ private:
struct Creators struct Creators
{ {
InputCreator inout_creator; InputCreator input_creator;
OutputCreator output_creator; OutputCreator output_creator;
InputProcessorCreator input_processor_creator; InputProcessorCreator input_processor_creator;
OutputProcessorCreator output_processor_creator; OutputProcessorCreator output_processor_creator;
FileSegmentationEngine file_segmentation_engine;
}; };
using FormatsDictionary = std::unordered_map<String, Creators>; using FormatsDictionary = std::unordered_map<String, Creators>;
@ -114,6 +125,7 @@ public:
/// Register format by its name. /// Register format by its name.
void registerInputFormat(const String & name, InputCreator input_creator); void registerInputFormat(const String & name, InputCreator input_creator);
void registerOutputFormat(const String & name, OutputCreator output_creator); void registerOutputFormat(const String & name, OutputCreator output_creator);
void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine);
void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator); void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator);
void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator); void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator);

View File

@ -32,9 +32,9 @@ public:
BrotliDecoderResult result; BrotliDecoderResult result;
}; };
BrotliReadBuffer::BrotliReadBuffer(ReadBuffer &in_, size_t buf_size, char *existing_memory, size_t alignment) BrotliReadBuffer::BrotliReadBuffer(std::unique_ptr<ReadBuffer> in_, size_t buf_size, char *existing_memory, size_t alignment)
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment) : BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment)
, in(in_) , in(std::move(in_))
, brotli(std::make_unique<BrotliStateWrapper>()) , brotli(std::make_unique<BrotliStateWrapper>())
, in_available(0) , in_available(0)
, in_data(nullptr) , in_data(nullptr)
@ -55,12 +55,12 @@ bool BrotliReadBuffer::nextImpl()
if (!in_available) if (!in_available)
{ {
in.nextIfAtEnd(); in->nextIfAtEnd();
in_available = in.buffer().end() - in.position(); in_available = in->buffer().end() - in->position();
in_data = reinterpret_cast<uint8_t *>(in.position()); in_data = reinterpret_cast<uint8_t *>(in->position());
} }
if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in.eof())) if (brotli->result == BROTLI_DECODER_RESULT_NEEDS_MORE_INPUT && (!in_available || in->eof()))
{ {
throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED); throw Exception("brotli decode error", ErrorCodes::BROTLI_READ_FAILED);
} }
@ -70,12 +70,12 @@ bool BrotliReadBuffer::nextImpl()
brotli->result = BrotliDecoderDecompressStream(brotli->state, &in_available, &in_data, &out_capacity, &out_data, nullptr); brotli->result = BrotliDecoderDecompressStream(brotli->state, &in_available, &in_data, &out_capacity, &out_data, nullptr);
in.position() = in.buffer().end() - in_available; in->position() = in->buffer().end() - in_available;
working_buffer.resize(internal_buffer.size() - out_capacity); working_buffer.resize(internal_buffer.size() - out_capacity);
if (brotli->result == BROTLI_DECODER_RESULT_SUCCESS) if (brotli->result == BROTLI_DECODER_RESULT_SUCCESS)
{ {
if (in.eof()) if (in->eof())
{ {
eof = true; eof = true;
return working_buffer.size() != 0; return working_buffer.size() != 0;

View File

@ -11,7 +11,7 @@ class BrotliReadBuffer : public BufferWithOwnMemory<ReadBuffer>
{ {
public: public:
BrotliReadBuffer( BrotliReadBuffer(
ReadBuffer & in_, std::unique_ptr<ReadBuffer> in_,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr, char * existing_memory = nullptr,
size_t alignment = 0); size_t alignment = 0);
@ -21,7 +21,7 @@ public:
private: private:
bool nextImpl() override; bool nextImpl() override;
ReadBuffer & in; std::unique_ptr<ReadBuffer> in;
class BrotliStateWrapper; class BrotliStateWrapper;
std::unique_ptr<BrotliStateWrapper> brotli; std::unique_ptr<BrotliStateWrapper> brotli;

View File

@ -12,6 +12,7 @@ enum class CompressionMethod
/// This option corresponds to HTTP Content-Encoding: deflate. /// This option corresponds to HTTP Content-Encoding: deflate.
Zlib, Zlib,
Brotli, Brotli,
None
}; };
} }

View File

@ -19,7 +19,6 @@ bool PeekableReadBuffer::peekNext()
{ {
checkStateCorrect(); checkStateCorrect();
size_t bytes_read = 0;
Position copy_from = pos; Position copy_from = pos;
size_t bytes_to_copy = sub_buf.available(); size_t bytes_to_copy = sub_buf.available();
if (useSubbufferOnly()) if (useSubbufferOnly())
@ -27,11 +26,9 @@ bool PeekableReadBuffer::peekNext()
/// Don't have to copy all data from sub-buffer if there is no data in own memory (checkpoint and pos are in sub-buffer) /// Don't have to copy all data from sub-buffer if there is no data in own memory (checkpoint and pos are in sub-buffer)
if (checkpoint) if (checkpoint)
copy_from = checkpoint; copy_from = checkpoint;
bytes_read = copy_from - sub_buf.buffer().begin();
bytes_to_copy = sub_buf.buffer().end() - copy_from; bytes_to_copy = sub_buf.buffer().end() - copy_from;
if (!bytes_to_copy) if (!bytes_to_copy)
{ {
bytes += bytes_read;
sub_buf.position() = copy_from; sub_buf.position() = copy_from;
/// Both checkpoint and pos are at the end of sub-buffer. Just load next part of data. /// Both checkpoint and pos are at the end of sub-buffer. Just load next part of data.
@ -50,7 +47,6 @@ bool PeekableReadBuffer::peekNext()
if (useSubbufferOnly()) if (useSubbufferOnly())
{ {
bytes += bytes_read;
sub_buf.position() = copy_from; sub_buf.position() = copy_from;
} }
@ -198,7 +194,6 @@ void PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append)
/// Move unread data to the beginning of own memory instead of resize own memory /// Move unread data to the beginning of own memory instead of resize own memory
peeked_size -= offset; peeked_size -= offset;
memmove(memory.data(), memory.data() + offset, peeked_size); memmove(memory.data(), memory.data() + offset, peeked_size);
bytes += offset;
if (need_update_checkpoint) if (need_update_checkpoint)
checkpoint -= offset; checkpoint -= offset;

View File

@ -23,9 +23,11 @@
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
#include <IO/CompressionMethod.h>
#include <IO/ReadBuffer.h> #include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromMemory.h> #include <IO/ReadBufferFromMemory.h>
#include <IO/VarInt.h> #include <IO/VarInt.h>
#include <IO/ZlibInflatingReadBuffer.h>
#ifdef __clang__ #ifdef __clang__
#pragma clang diagnostic push #pragma clang diagnostic push
@ -911,4 +913,15 @@ void skipToNextLineOrEOF(ReadBuffer & buf);
/// Skip to next character after next unescaped \n. If no \n in stream, skip to end. Does not throw on invalid escape sequences. /// Skip to next character after next unescaped \n. If no \n in stream, skip to end. Does not throw on invalid escape sequences.
void skipToUnescapedNextLineOrEOF(ReadBuffer & buf); void skipToUnescapedNextLineOrEOF(ReadBuffer & buf);
template <class TReadBuffer, class... Types>
std::unique_ptr<ReadBuffer> getReadBuffer(const DB::CompressionMethod method, Types&&... args)
{
if (method == DB::CompressionMethod::Gzip)
{
auto read_buf = std::make_unique<TReadBuffer>(std::forward<Types>(args)...);
return std::make_unique<ZlibInflatingReadBuffer>(std::move(read_buf), method);
}
return std::make_unique<TReadBuffer>(args...);
}
} }

View File

@ -90,6 +90,9 @@ public:
++pos; ++pos;
} }
virtual void sync() {}
virtual void finalize() {}
private: private:
/** Write the data in the buffer (from the beginning of the buffer to the current position). /** Write the data in the buffer (from the beginning of the buffer to the current position).
* Throw an exception if something is wrong. * Throw an exception if something is wrong.

View File

@ -51,7 +51,7 @@ private:
/// Prepare an asynchronous request. /// Prepare an asynchronous request.
void prepare(); void prepare();
/// ///
void finalize(); void finalize() override;
private: private:
/// Buffer for asynchronous data writes. /// Buffer for asynchronous data writes.

View File

@ -25,7 +25,7 @@ public:
~WriteBufferFromHDFS() override; ~WriteBufferFromHDFS() override;
void sync(); void sync() override;
}; };
} }
#endif #endif

View File

@ -113,8 +113,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response.set("Content-Encoding", "gzip"); response.set("Content-Encoding", "gzip");
response_body_ostr = &(response.send()); response_body_ostr = &(response.send());
#endif #endif
out_raw.emplace(*response_body_ostr); out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin()); deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf; out = &*deflating_buf;
} }
else if (compression_method == CompressionMethod::Zlib) else if (compression_method == CompressionMethod::Zlib)
@ -125,8 +125,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response.set("Content-Encoding", "deflate"); response.set("Content-Encoding", "deflate");
response_body_ostr = &(response.send()); response_body_ostr = &(response.send());
#endif #endif
out_raw.emplace(*response_body_ostr); out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin()); deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin());
out = &*deflating_buf; out = &*deflating_buf;
} }
#if USE_BROTLI #if USE_BROTLI
@ -138,7 +138,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response.set("Content-Encoding", "br"); response.set("Content-Encoding", "br");
response_body_ostr = &(response.send()); response_body_ostr = &(response.send());
#endif #endif
out_raw.emplace(*response_body_ostr); out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin()); brotli_buf.emplace(*out_raw, compression_level, working_buffer.size(), working_buffer.begin());
out = &*brotli_buf; out = &*brotli_buf;
} }
@ -155,7 +155,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
response_body_ostr = &(response.send()); response_body_ostr = &(response.send());
#endif #endif
out_raw.emplace(*response_body_ostr, working_buffer.size(), working_buffer.begin()); out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr, working_buffer.size(), working_buffer.begin());
out = &*out_raw; out = &*out_raw;
} }
} }

View File

@ -60,7 +60,7 @@ private:
std::ostream * response_header_ostr = nullptr; std::ostream * response_header_ostr = nullptr;
#endif #endif
std::optional<WriteBufferFromOStream> out_raw; std::unique_ptr<WriteBufferFromOStream> out_raw;
std::optional<ZlibDeflatingWriteBuffer> deflating_buf; std::optional<ZlibDeflatingWriteBuffer> deflating_buf;
#if USE_BROTLI #if USE_BROTLI
std::optional<BrotliWriteBuffer> brotli_buf; std::optional<BrotliWriteBuffer> brotli_buf;
@ -109,7 +109,7 @@ public:
/// Use after the data has possibly been sent and no error happened (and thus you do not plan /// Use after the data has possibly been sent and no error happened (and thus you do not plan
/// to change response HTTP code. /// to change response HTTP code.
/// This method is idempotent. /// This method is idempotent.
void finalize(); void finalize() override;
/// Turn compression on or off. /// Turn compression on or off.
/// The setting has any effect only if HTTP headers haven't been sent yet. /// The setting has any effect only if HTTP headers haven't been sent yet.

View File

@ -43,7 +43,7 @@ public:
void nextImpl() override; void nextImpl() override;
/// Receives response from the server after sending all data. /// Receives response from the server after sending all data.
void finalize(); void finalize() override;
~WriteBufferFromS3() override; ~WriteBufferFromS3() override;

View File

@ -20,11 +20,13 @@
#include <Common/UInt128.h> #include <Common/UInt128.h>
#include <Common/intExp.h> #include <Common/intExp.h>
#include <IO/CompressionMethod.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteIntText.h> #include <IO/WriteIntText.h>
#include <IO/VarInt.h> #include <IO/VarInt.h>
#include <IO/DoubleConverter.h> #include <IO/DoubleConverter.h>
#include <IO/WriteBufferFromString.h> #include <IO/WriteBufferFromString.h>
#include <IO/ZlibDeflatingWriteBuffer.h>
#include <Formats/FormatSettings.h> #include <Formats/FormatSettings.h>
@ -905,4 +907,16 @@ inline String toString(const T & x)
writeText(x, buf); writeText(x, buf);
return buf.str(); return buf.str();
} }
template <class TWriteBuffer, class... Types>
std::unique_ptr<WriteBuffer> getWriteBuffer(const DB::CompressionMethod method, Types&&... args)
{
if (method == DB::CompressionMethod::Gzip)
{
auto write_buf = std::make_unique<TWriteBuffer>(std::forward<Types>(args)...);
return std::make_unique<ZlibDeflatingWriteBuffer>(std::move(write_buf), method, 1 /* compression level */);
}
return std::make_unique<TWriteBuffer>(args...);
}
} }

View File

@ -6,14 +6,14 @@ namespace DB
{ {
ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer( ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
WriteBuffer & out_, std::unique_ptr<WriteBuffer> out_,
CompressionMethod compression_method, CompressionMethod compression_method,
int compression_level, int compression_level,
size_t buf_size, size_t buf_size,
char * existing_memory, char * existing_memory,
size_t alignment) size_t alignment)
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment) : BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
, out(out_) , out(std::move(out_))
{ {
zstr.zalloc = nullptr; zstr.zalloc = nullptr;
zstr.zfree = nullptr; zstr.zfree = nullptr;
@ -64,18 +64,18 @@ void ZlibDeflatingWriteBuffer::nextImpl()
do do
{ {
out.nextIfAtEnd(); out->nextIfAtEnd();
zstr.next_out = reinterpret_cast<unsigned char *>(out.position()); zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
zstr.avail_out = out.buffer().end() - out.position(); zstr.avail_out = out->buffer().end() - out->position();
int rc = deflate(&zstr, Z_NO_FLUSH); int rc = deflate(&zstr, Z_NO_FLUSH);
out.position() = out.buffer().end() - zstr.avail_out; out->position() = out->buffer().end() - zstr.avail_out;
// Unpoison the result of deflate explicitly. It uses some custom SSE algo // Unpoison the result of deflate explicitly. It uses some custom SSE algo
// for computing CRC32, and it looks like msan is unable to comprehend // for computing CRC32, and it looks like msan is unable to comprehend
// it fully, so it complains about the resulting value depending on the // it fully, so it complains about the resulting value depending on the
// uninitialized padding of the input buffer. // uninitialized padding of the input buffer.
__msan_unpoison(out.position(), zstr.avail_out); __msan_unpoison(out->position(), zstr.avail_out);
if (rc != Z_OK) if (rc != Z_OK)
throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED); throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED);
@ -92,18 +92,18 @@ void ZlibDeflatingWriteBuffer::finish()
while (true) while (true)
{ {
out.nextIfAtEnd(); out->nextIfAtEnd();
zstr.next_out = reinterpret_cast<unsigned char *>(out.position()); zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
zstr.avail_out = out.buffer().end() - out.position(); zstr.avail_out = out->buffer().end() - out->position();
int rc = deflate(&zstr, Z_FINISH); int rc = deflate(&zstr, Z_FINISH);
out.position() = out.buffer().end() - zstr.avail_out; out->position() = out->buffer().end() - zstr.avail_out;
// Unpoison the result of deflate explicitly. It uses some custom SSE algo // Unpoison the result of deflate explicitly. It uses some custom SSE algo
// for computing CRC32, and it looks like msan is unable to comprehend // for computing CRC32, and it looks like msan is unable to comprehend
// it fully, so it complains about the resulting value depending on the // it fully, so it complains about the resulting value depending on the
// uninitialized padding of the input buffer. // uninitialized padding of the input buffer.
__msan_unpoison(out.position(), zstr.avail_out); __msan_unpoison(out->position(), zstr.avail_out);
if (rc == Z_STREAM_END) if (rc == Z_STREAM_END)
{ {

View File

@ -20,7 +20,7 @@ class ZlibDeflatingWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
{ {
public: public:
ZlibDeflatingWriteBuffer( ZlibDeflatingWriteBuffer(
WriteBuffer & out_, std::unique_ptr<WriteBuffer> out_,
CompressionMethod compression_method, CompressionMethod compression_method,
int compression_level, int compression_level,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
@ -37,7 +37,7 @@ public:
private: private:
void nextImpl() override; void nextImpl() override;
WriteBuffer & out; std::unique_ptr<WriteBuffer> out;
z_stream zstr; z_stream zstr;
bool finished = false; bool finished = false;
}; };

View File

@ -5,13 +5,13 @@ namespace DB
{ {
ZlibInflatingReadBuffer::ZlibInflatingReadBuffer( ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
ReadBuffer & in_, std::unique_ptr<ReadBuffer> in_,
CompressionMethod compression_method, CompressionMethod compression_method,
size_t buf_size, size_t buf_size,
char * existing_memory, char * existing_memory,
size_t alignment) size_t alignment)
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment) : BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment)
, in(in_) , in(std::move(in_))
, eof(false) , eof(false)
{ {
zstr.zalloc = nullptr; zstr.zalloc = nullptr;
@ -49,21 +49,21 @@ bool ZlibInflatingReadBuffer::nextImpl()
if (!zstr.avail_in) if (!zstr.avail_in)
{ {
in.nextIfAtEnd(); in->nextIfAtEnd();
zstr.next_in = reinterpret_cast<unsigned char *>(in.position()); zstr.next_in = reinterpret_cast<unsigned char *>(in->position());
zstr.avail_in = in.buffer().end() - in.position(); zstr.avail_in = in->buffer().end() - in->position();
} }
zstr.next_out = reinterpret_cast<unsigned char *>(internal_buffer.begin()); zstr.next_out = reinterpret_cast<unsigned char *>(internal_buffer.begin());
zstr.avail_out = internal_buffer.size(); zstr.avail_out = internal_buffer.size();
int rc = inflate(&zstr, Z_NO_FLUSH); int rc = inflate(&zstr, Z_NO_FLUSH);
in.position() = in.buffer().end() - zstr.avail_in; in->position() = in->buffer().end() - zstr.avail_in;
working_buffer.resize(internal_buffer.size() - zstr.avail_out); working_buffer.resize(internal_buffer.size() - zstr.avail_out);
if (rc == Z_STREAM_END) if (rc == Z_STREAM_END)
{ {
if (in.eof()) if (in->eof())
{ {
eof = true; eof = true;
return working_buffer.size() != 0; return working_buffer.size() != 0;

View File

@ -21,7 +21,7 @@ class ZlibInflatingReadBuffer : public BufferWithOwnMemory<ReadBuffer>
{ {
public: public:
ZlibInflatingReadBuffer( ZlibInflatingReadBuffer(
ReadBuffer & in_, std::unique_ptr<ReadBuffer> in_,
CompressionMethod compression_method, CompressionMethod compression_method,
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE, size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
char * existing_memory = nullptr, char * existing_memory = nullptr,
@ -32,7 +32,7 @@ public:
private: private:
bool nextImpl() override; bool nextImpl() override;
ReadBuffer & in; std::unique_ptr<ReadBuffer> in;
z_stream zstr; z_stream zstr;
bool eof; bool eof;
}; };

View File

@ -22,8 +22,8 @@ try
Stopwatch stopwatch; Stopwatch stopwatch;
{ {
DB::WriteBufferFromFile buf("test_zlib_buffers.gz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC); auto buf = std::make_unique<DB::WriteBufferFromFile>("test_zlib_buffers.gz", DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_CREAT | O_TRUNC);
DB::ZlibDeflatingWriteBuffer deflating_buf(buf, DB::CompressionMethod::Gzip, /* compression_level = */ 3); DB::ZlibDeflatingWriteBuffer deflating_buf(std::move(buf), DB::CompressionMethod::Gzip, /* compression_level = */ 3);
stopwatch.restart(); stopwatch.restart();
for (size_t i = 0; i < n; ++i) for (size_t i = 0; i < n; ++i)
@ -40,8 +40,8 @@ try
} }
{ {
DB::ReadBufferFromFile buf("test_zlib_buffers.gz"); auto buf = std::make_unique<DB::ReadBufferFromFile>("test_zlib_buffers.gz");
DB::ZlibInflatingReadBuffer inflating_buf(buf, DB::CompressionMethod::Gzip); DB::ZlibInflatingReadBuffer inflating_buf(std::move(buf), DB::CompressionMethod::Gzip);
stopwatch.restart(); stopwatch.restart();
for (size_t i = 0; i < n; ++i) for (size_t i = 0; i < n; ++i)

View File

@ -22,7 +22,7 @@ public:
private: private:
ASTPtr query; ASTPtr query;
Context context; const Context & context;
BlockInputStreamPtr executeImpl(); BlockInputStreamPtr executeImpl();
}; };

View File

@ -98,7 +98,10 @@ BlockIO InterpreterInsertQuery::execute()
{ {
const auto & query = query_ptr->as<ASTInsertQuery &>(); const auto & query = query_ptr->as<ASTInsertQuery &>();
checkAccess(query); checkAccess(query);
BlockIO res;
StoragePtr table = getTable(query); StoragePtr table = getTable(query);
res.pipeline.addStorageHolder(table);
auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId()); auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId());
@ -134,7 +137,6 @@ BlockIO InterpreterInsertQuery::execute()
out_wrapper->setProcessListElement(context.getProcessListElement()); out_wrapper->setProcessListElement(context.getProcessListElement());
out = std::move(out_wrapper); out = std::move(out_wrapper);
BlockIO res;
res.out = std::move(out); res.out = std::move(out);
/// What type of query: INSERT or INSERT SELECT? /// What type of query: INSERT or INSERT SELECT?

View File

@ -230,7 +230,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
: options(options_) : options(options_)
/// NOTE: the query almost always should be cloned because it will be modified during analysis. /// NOTE: the query almost always should be cloned because it will be modified during analysis.
, query_ptr(options.modify_inplace ? query_ptr_ : query_ptr_->clone()) , query_ptr(options.modify_inplace ? query_ptr_ : query_ptr_->clone())
, context(context_) , context(std::make_shared<Context>(context_))
, storage(storage_) , storage(storage_)
, input(input_) , input(input_)
, log(&Logger::get("InterpreterSelectQuery")) , log(&Logger::get("InterpreterSelectQuery"))
@ -238,7 +238,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
checkStackSize(); checkStackSize();
initSettings(); initSettings();
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
if (settings.max_subquery_depth && options.subquery_depth > settings.max_subquery_depth) if (settings.max_subquery_depth && options.subquery_depth > settings.max_subquery_depth)
throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(), throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(),
@ -252,7 +252,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (settings.allow_experimental_multiple_joins_emulation) if (settings.allow_experimental_multiple_joins_emulation)
{ {
JoinToSubqueryTransformVisitor::Data join_to_subs_data{context}; JoinToSubqueryTransformVisitor::Data join_to_subs_data{*context};
JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr); JoinToSubqueryTransformVisitor(join_to_subs_data).visit(query_ptr);
} }
@ -278,7 +278,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
{ {
/// Read from subquery. /// Read from subquery.
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>( interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
table_expression, getSubqueryContext(context), options.subquery(), required_columns); table_expression, getSubqueryContext(*context), options.subquery(), required_columns);
source_header = interpreter_subquery->getSampleBlock(); source_header = interpreter_subquery->getSampleBlock();
} }
@ -288,48 +288,48 @@ InterpreterSelectQuery::InterpreterSelectQuery(
{ {
/// Read from table function. propagate all settings from initSettings(), /// Read from table function. propagate all settings from initSettings(),
/// alternative is to call on current `context`, but that can potentially pollute it. /// alternative is to call on current `context`, but that can potentially pollute it.
storage = getSubqueryContext(context).executeTableFunction(table_expression); storage = getSubqueryContext(*context).executeTableFunction(table_expression);
} }
else else
{ {
String database_name; String database_name;
String table_name; String table_name;
getDatabaseAndTableNames(query, database_name, table_name, context); getDatabaseAndTableNames(query, database_name, table_name, *context);
if (auto view_source = context.getViewSource()) if (auto view_source = context->getViewSource())
{ {
auto & storage_values = static_cast<const StorageValues &>(*view_source); auto & storage_values = static_cast<const StorageValues &>(*view_source);
if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name) if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name)
{ {
/// Read from view source. /// Read from view source.
storage = context.getViewSource(); storage = context->getViewSource();
} }
} }
if (!storage) if (!storage)
{ {
/// Read from table. Even without table expression (implicit SELECT ... FROM system.one). /// Read from table. Even without table expression (implicit SELECT ... FROM system.one).
storage = context.getTable(database_name, table_name); storage = context->getTable(database_name, table_name);
} }
} }
} }
if (storage) if (storage)
table_lock = storage->lockStructureForShare(false, context.getInitialQueryId()); table_lock = storage->lockStructureForShare(false, context->getInitialQueryId());
auto analyze = [&] () auto analyze = [&] ()
{ {
syntax_analyzer_result = SyntaxAnalyzer(context, options).analyze( syntax_analyzer_result = SyntaxAnalyzer(*context, options).analyze(
query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList()); query_ptr, source_header.getNamesAndTypesList(), required_result_column_names, storage, NamesAndTypesList());
/// Save scalar sub queries's results in the query context /// Save scalar sub queries's results in the query context
if (context.hasQueryContext()) if (context->hasQueryContext())
for (const auto & it : syntax_analyzer_result->getScalars()) for (const auto & it : syntax_analyzer_result->getScalars())
context.getQueryContext().addScalar(it.first, it.second); context->getQueryContext().addScalar(it.first, it.second);
query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>( query_analyzer = std::make_unique<SelectQueryExpressionAnalyzer>(
query_ptr, syntax_analyzer_result, context, query_ptr, syntax_analyzer_result, *context,
NameSet(required_result_column_names.begin(), required_result_column_names.end()), NameSet(required_result_column_names.begin(), required_result_column_names.end()),
options.subquery_depth, !options.only_analyze); options.subquery_depth, !options.only_analyze);
@ -346,8 +346,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
/// Save the new temporary tables in the query context /// Save the new temporary tables in the query context
for (const auto & it : query_analyzer->getExternalTables()) for (const auto & it : query_analyzer->getExternalTables())
if (!context.tryGetExternalTable(it.first)) if (!context->tryGetExternalTable(it.first))
context.addExternalTable(it.first, it.second); context->addExternalTable(it.first, it.second);
} }
if (!options.only_analyze || options.modify_inplace) if (!options.only_analyze || options.modify_inplace)
@ -358,7 +358,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
if (is_subquery) if (is_subquery)
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>( interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
table_expression, table_expression,
getSubqueryContext(context), getSubqueryContext(*context),
options.subquery(), options.subquery(),
required_columns); required_columns);
} }
@ -378,10 +378,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
source_header = storage->getSampleBlockForColumns(required_columns); source_header = storage->getSampleBlockForColumns(required_columns);
/// Fix source_header for filter actions. /// Fix source_header for filter actions.
if (context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter"))
{ {
filter_info = std::make_shared<FilterInfo>(); filter_info = std::make_shared<FilterInfo>();
filter_info->column_name = generateFilterActions(filter_info->actions, storage, context, required_columns); filter_info->column_name = generateFilterActions(filter_info->actions, storage, *context, required_columns);
source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns()); source_header = storage->getSampleBlockForColumns(filter_info->actions->getRequiredColumns());
} }
} }
@ -450,27 +450,34 @@ Block InterpreterSelectQuery::getSampleBlock()
BlockIO InterpreterSelectQuery::execute() BlockIO InterpreterSelectQuery::execute()
{ {
Pipeline pipeline; Pipeline pipeline;
executeImpl(pipeline, input); BlockIO res;
executeImpl(pipeline, input, res.pipeline);
executeUnion(pipeline, getSampleBlock()); executeUnion(pipeline, getSampleBlock());
BlockIO res;
res.in = pipeline.firstStream(); res.in = pipeline.firstStream();
res.pipeline.addInterpreterContext(context);
res.pipeline.addStorageHolder(storage);
return res; return res;
} }
BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams() BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline)
{ {
///FIXME pipeline must be alive until query is finished
Pipeline pipeline; Pipeline pipeline;
executeImpl(pipeline, input); executeImpl(pipeline, input, parent_pipeline);
unifyStreams(pipeline, getSampleBlock()); unifyStreams(pipeline, getSampleBlock());
parent_pipeline.addInterpreterContext(context);
parent_pipeline.addStorageHolder(storage);
return pipeline.streams; return pipeline.streams;
} }
QueryPipeline InterpreterSelectQuery::executeWithProcessors() QueryPipeline InterpreterSelectQuery::executeWithProcessors()
{ {
QueryPipeline query_pipeline; QueryPipeline query_pipeline;
query_pipeline.setMaxThreads(context.getSettingsRef().max_threads); query_pipeline.setMaxThreads(context->getSettingsRef().max_threads);
executeImpl(query_pipeline, input); executeImpl(query_pipeline, input, query_pipeline);
query_pipeline.addInterpreterContext(context);
query_pipeline.addStorageHolder(storage);
return query_pipeline; return query_pipeline;
} }
@ -478,13 +485,13 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors()
Block InterpreterSelectQuery::getSampleBlockImpl() Block InterpreterSelectQuery::getSampleBlockImpl()
{ {
auto & query = getSelectQuery(); auto & query = getSelectQuery();
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
/// Do all AST changes here, because actions from analysis_result will be used later in readImpl. /// Do all AST changes here, because actions from analysis_result will be used later in readImpl.
/// PREWHERE optimization. /// PREWHERE optimization.
/// Turn off, if the table filter (row-level security) is applied. /// Turn off, if the table filter (row-level security) is applied.
if (storage && !context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) if (storage && !context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter"))
{ {
query_analyzer->makeSetsForIndex(query.where()); query_analyzer->makeSetsForIndex(query.where());
query_analyzer->makeSetsForIndex(query.prewhere()); query_analyzer->makeSetsForIndex(query.prewhere());
@ -498,7 +505,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable /// Try transferring some condition from WHERE to PREWHERE if enabled and viable
if (settings.optimize_move_to_prewhere && query.where() && !query.prewhere() && !query.final()) if (settings.optimize_move_to_prewhere && query.where() && !query.prewhere() && !query.final())
MergeTreeWhereOptimizer{current_info, context, merge_tree, MergeTreeWhereOptimizer{current_info, *context, merge_tree,
syntax_analyzer_result->requiredSourceColumns(), log}; syntax_analyzer_result->requiredSourceColumns(), log};
}; };
@ -507,14 +514,14 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
} }
if (storage && !options.only_analyze) if (storage && !options.only_analyze)
from_stage = storage->getQueryProcessingStage(context); from_stage = storage->getQueryProcessingStage(*context);
analysis_result = analyzeExpressions( analysis_result = analyzeExpressions(
getSelectQuery(), getSelectQuery(),
*query_analyzer, *query_analyzer,
from_stage, from_stage,
options.to_stage, options.to_stage,
context, *context,
storage, storage,
options.only_analyze, options.only_analyze,
filter_info, filter_info,
@ -1007,7 +1014,7 @@ static InputSortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree,
template <typename TPipeline> template <typename TPipeline>
void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input) void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, QueryPipeline & save_context_and_storage)
{ {
/** Streams of data. When the query is executed in parallel, we have several data streams. /** Streams of data. When the query is executed in parallel, we have several data streams.
* If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then * If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then
@ -1023,14 +1030,14 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
/// Now we will compose block streams that perform the necessary actions. /// Now we will compose block streams that perform the necessary actions.
auto & query = getSelectQuery(); auto & query = getSelectQuery();
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
auto & expressions = analysis_result; auto & expressions = analysis_result;
InputSortingInfoPtr input_sorting_info; InputSortingInfoPtr input_sorting_info;
if (settings.optimize_read_in_order && storage && query.orderBy() && !query_analyzer->hasAggregation() && !query.final() && !query.join()) if (settings.optimize_read_in_order && storage && query.orderBy() && !query_analyzer->hasAggregation() && !query.final() && !query.join())
{ {
if (const auto * merge_tree_data = dynamic_cast<const MergeTreeData *>(storage.get())) if (const auto * merge_tree_data = dynamic_cast<const MergeTreeData *>(storage.get()))
input_sorting_info = optimizeReadInOrder(*merge_tree_data, query, context, syntax_analyzer_result); input_sorting_info = optimizeReadInOrder(*merge_tree_data, query, *context, syntax_analyzer_result);
} }
if (options.only_analyze) if (options.only_analyze)
@ -1090,7 +1097,7 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE); throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE);
/** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */ /** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */
executeFetchColumns(from_stage, pipeline, input_sorting_info, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere); executeFetchColumns(from_stage, pipeline, input_sorting_info, expressions.prewhere_info, expressions.columns_to_remove_after_prewhere, save_context_and_storage);
LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(options.to_stage)); LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(options.to_stage));
} }
@ -1349,12 +1356,13 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
template <typename TPipeline> template <typename TPipeline>
void InterpreterSelectQuery::executeFetchColumns( void InterpreterSelectQuery::executeFetchColumns(
QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, QueryProcessingStage::Enum processing_stage, TPipeline & pipeline,
const InputSortingInfoPtr & input_sorting_info, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere) const InputSortingInfoPtr & input_sorting_info, const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere,
QueryPipeline & save_context_and_storage)
{ {
constexpr bool pipeline_with_processors = std::is_same<TPipeline, QueryPipeline>::value; constexpr bool pipeline_with_processors = std::is_same<TPipeline, QueryPipeline>::value;
auto & query = getSelectQuery(); auto & query = getSelectQuery();
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
/// Optimization for trivial query like SELECT count() FROM table. /// Optimization for trivial query like SELECT count() FROM table.
auto check_trivial_count_query = [&]() -> std::optional<AggregateDescription> auto check_trivial_count_query = [&]() -> std::optional<AggregateDescription>
@ -1418,11 +1426,11 @@ void InterpreterSelectQuery::executeFetchColumns(
if (storage) if (storage)
{ {
/// Append columns from the table filter to required /// Append columns from the table filter to required
if (context.hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter")) if (context->hasUserProperty(storage->getDatabaseName(), storage->getTableName(), "filter"))
{ {
auto initial_required_columns = required_columns; auto initial_required_columns = required_columns;
ExpressionActionsPtr actions; ExpressionActionsPtr actions;
generateFilterActions(actions, storage, context, initial_required_columns); generateFilterActions(actions, storage, *context, initial_required_columns);
auto required_columns_from_filter = actions->getRequiredColumns(); auto required_columns_from_filter = actions->getRequiredColumns();
for (const auto & column : required_columns_from_filter) for (const auto & column : required_columns_from_filter)
@ -1521,8 +1529,8 @@ void InterpreterSelectQuery::executeFetchColumns(
= ext::map<NameSet>(required_columns_after_prewhere, [](const auto & it) { return it.name; }); = ext::map<NameSet>(required_columns_after_prewhere, [](const auto & it) { return it.name; });
} }
auto syntax_result = SyntaxAnalyzer(context).analyze(required_columns_all_expr, required_columns_after_prewhere, {}, storage); auto syntax_result = SyntaxAnalyzer(*context).analyze(required_columns_all_expr, required_columns_after_prewhere, {}, storage);
alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, context).getActions(true); alias_actions = ExpressionAnalyzer(required_columns_all_expr, syntax_result, *context).getActions(true);
/// The set of required columns could be added as a result of adding an action to calculate ALIAS. /// The set of required columns could be added as a result of adding an action to calculate ALIAS.
required_columns = alias_actions->getRequiredColumns(); required_columns = alias_actions->getRequiredColumns();
@ -1542,7 +1550,7 @@ void InterpreterSelectQuery::executeFetchColumns(
if (prewhere_info) if (prewhere_info)
{ {
/// Don't remove columns which are needed to be aliased. /// Don't remove columns which are needed to be aliased.
auto new_actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), context); auto new_actions = std::make_shared<ExpressionActions>(prewhere_info->prewhere_actions->getRequiredColumnsWithTypes(), *context);
for (const auto & action : prewhere_info->prewhere_actions->getActions()) for (const auto & action : prewhere_info->prewhere_actions->getActions())
{ {
if (action.type != ExpressionAction::REMOVE_COLUMN if (action.type != ExpressionAction::REMOVE_COLUMN
@ -1552,9 +1560,9 @@ void InterpreterSelectQuery::executeFetchColumns(
prewhere_info->prewhere_actions = std::move(new_actions); prewhere_info->prewhere_actions = std::move(new_actions);
auto analyzed_result auto analyzed_result
= SyntaxAnalyzer(context).analyze(required_columns_from_prewhere_expr, storage->getColumns().getAllPhysical()); = SyntaxAnalyzer(*context).analyze(required_columns_from_prewhere_expr, storage->getColumns().getAllPhysical());
prewhere_info->alias_actions prewhere_info->alias_actions
= ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, context).getActions(true, false); = ExpressionAnalyzer(required_columns_from_prewhere_expr, analyzed_result, *context).getActions(true, false);
/// Add (physical?) columns required by alias actions. /// Add (physical?) columns required by alias actions.
auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns(); auto required_columns_from_alias = prewhere_info->alias_actions->getRequiredColumns();
@ -1597,7 +1605,7 @@ void InterpreterSelectQuery::executeFetchColumns(
UInt64 max_block_size = settings.max_block_size; UInt64 max_block_size = settings.max_block_size;
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
/** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size, /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size,
* then as the block size we will use limit + offset (not to read more from the table than requested), * then as the block size we will use limit + offset (not to read more from the table than requested),
@ -1638,7 +1646,7 @@ void InterpreterSelectQuery::executeFetchColumns(
throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR); throw Exception("Subquery expected", ErrorCodes::LOGICAL_ERROR);
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>( interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
subquery, getSubqueryContext(context), subquery, getSubqueryContext(*context),
options.copy().subquery().noModify(), required_columns); options.copy().subquery().noModify(), required_columns);
if (query_analyzer->hasAggregation()) if (query_analyzer->hasAggregation())
@ -1649,7 +1657,7 @@ void InterpreterSelectQuery::executeFetchColumns(
/// Just use pipeline from subquery. /// Just use pipeline from subquery.
pipeline = interpreter_subquery->executeWithProcessors(); pipeline = interpreter_subquery->executeWithProcessors();
else else
pipeline.streams = interpreter_subquery->executeWithMultipleStreams(); pipeline.streams = interpreter_subquery->executeWithMultipleStreams(save_context_and_storage);
} }
else if (storage) else if (storage)
{ {
@ -1676,9 +1684,9 @@ void InterpreterSelectQuery::executeFetchColumns(
bool use_pipes = pipeline_with_processors && storage->supportProcessorsPipeline(); bool use_pipes = pipeline_with_processors && storage->supportProcessorsPipeline();
if (use_pipes) if (use_pipes)
pipes = storage->readWithProcessors(required_columns, query_info, context, processing_stage, max_block_size, max_streams); pipes = storage->readWithProcessors(required_columns, query_info, *context, processing_stage, max_block_size, max_streams);
else else
streams = storage->read(required_columns, query_info, context, processing_stage, max_block_size, max_streams); streams = storage->read(required_columns, query_info, *context, processing_stage, max_block_size, max_streams);
if (streams.empty() && !use_pipes) if (streams.empty() && !use_pipes)
{ {
@ -1757,7 +1765,7 @@ void InterpreterSelectQuery::executeFetchColumns(
limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed; limits.speed_limits.timeout_before_checking_execution_speed = settings.timeout_before_checking_execution_speed;
} }
QuotaForIntervals & quota = context.getQuota(); QuotaForIntervals & quota = context->getQuota();
for (auto & stream : streams) for (auto & stream : streams)
{ {
@ -1798,7 +1806,7 @@ void InterpreterSelectQuery::executeFetchColumns(
auto header = stream->getHeader(); auto header = stream->getHeader();
auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name;
if (!blocksHaveEqualStructure(first_header, header)) if (!blocksHaveEqualStructure(first_header, header))
stream = std::make_shared<ConvertingBlockInputStream>(context, stream, first_header, mode); stream = std::make_shared<ConvertingBlockInputStream>(*context, stream, first_header, mode);
} }
} }
@ -1886,7 +1894,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre
for (const auto & name : descr.argument_names) for (const auto & name : descr.argument_names)
descr.arguments.push_back(header.getPositionByName(name)); descr.arguments.push_back(header.getPositionByName(name));
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
/** Two-level aggregation is useful in two cases: /** Two-level aggregation is useful in two cases:
* 1. Parallel aggregation is done, and the results should be merged in parallel. * 1. Parallel aggregation is done, and the results should be merged in parallel.
@ -1899,7 +1907,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0),
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0),
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
/// If there are several sources, then we perform parallel aggregation /// If there are several sources, then we perform parallel aggregation
if (pipeline.streams.size() > 1) if (pipeline.streams.size() > 1)
@ -1952,7 +1960,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const
for (const auto & name : descr.argument_names) for (const auto & name : descr.argument_names)
descr.arguments.push_back(header_before_aggregation.getPositionByName(name)); descr.arguments.push_back(header_before_aggregation.getPositionByName(name));
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
/** Two-level aggregation is useful in two cases: /** Two-level aggregation is useful in two cases:
* 1. Parallel aggregation is done, and the results should be merged in parallel. * 1. Parallel aggregation is done, and the results should be merged in parallel.
@ -1965,7 +1973,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0),
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0), allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0),
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final); auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
@ -2029,7 +2037,7 @@ void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool ov
* but it can work more slowly. * but it can work more slowly.
*/ */
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
Aggregator::Params params(header, keys, aggregates, overflow_row, settings.max_threads); Aggregator::Params params(header, keys, aggregates, overflow_row, settings.max_threads);
@ -2080,7 +2088,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bo
* but it can work more slowly. * but it can work more slowly.
*/ */
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
Aggregator::Params params(header_before_merge, keys, aggregates, overflow_row, settings.max_threads); Aggregator::Params params(header_before_merge, keys, aggregates, overflow_row, settings.max_threads);
@ -2140,7 +2148,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
{ {
executeUnion(pipeline, {}); executeUnion(pipeline, {});
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
pipeline.firstStream() = std::make_shared<TotalsHavingBlockInputStream>( pipeline.firstStream() = std::make_shared<TotalsHavingBlockInputStream>(
pipeline.firstStream(), pipeline.firstStream(),
@ -2154,7 +2162,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final) void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final)
{ {
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
auto totals_having = std::make_shared<TotalsHavingTransform>( auto totals_having = std::make_shared<TotalsHavingTransform>(
pipeline.getHeader(), overflow_row, expression, pipeline.getHeader(), overflow_row, expression,
@ -2180,13 +2188,13 @@ void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificato
for (const auto & name : key_names) for (const auto & name : key_names)
keys.push_back(header.getPositionByName(name)); keys.push_back(header.getPositionByName(name));
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
Aggregator::Params params(header, keys, aggregates, Aggregator::Params params(header, keys, aggregates,
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
SettingUInt64(0), SettingUInt64(0), SettingUInt64(0), SettingUInt64(0),
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
if (modificator == Modificator::ROLLUP) if (modificator == Modificator::ROLLUP)
pipeline.firstStream() = std::make_shared<RollupBlockInputStream>(pipeline.firstStream(), params); pipeline.firstStream() = std::make_shared<RollupBlockInputStream>(pipeline.firstStream(), params);
@ -2209,13 +2217,13 @@ void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modif
for (const auto & name : key_names) for (const auto & name : key_names)
keys.push_back(header_before_transform.getPositionByName(name)); keys.push_back(header_before_transform.getPositionByName(name));
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
Aggregator::Params params(header_before_transform, keys, aggregates, Aggregator::Params params(header_before_transform, keys, aggregates,
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode, false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
SettingUInt64(0), SettingUInt64(0), SettingUInt64(0), SettingUInt64(0),
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set, settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data); context->getTemporaryPath(), settings.max_threads, settings.min_free_disk_space_for_temporary_data);
auto transform_params = std::make_shared<AggregatingTransformParams>(params, true); auto transform_params = std::make_shared<AggregatingTransformParams>(params, true);
@ -2251,9 +2259,9 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E
void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoPtr input_sorting_info) void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoPtr input_sorting_info)
{ {
auto & query = getSelectQuery(); auto & query = getSelectQuery();
SortDescription output_order_descr = getSortDescription(query, context); SortDescription output_order_descr = getSortDescription(query, *context);
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
UInt64 limit = getLimitForSorting(query, context); UInt64 limit = getLimitForSorting(query, *context);
if (input_sorting_info) if (input_sorting_info)
{ {
@ -2305,17 +2313,17 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoP
pipeline.firstStream() = std::make_shared<MergeSortingBlockInputStream>( pipeline.firstStream() = std::make_shared<MergeSortingBlockInputStream>(
pipeline.firstStream(), output_order_descr, settings.max_block_size, limit, pipeline.firstStream(), output_order_descr, settings.max_block_size, limit,
settings.max_bytes_before_remerge_sort, settings.max_bytes_before_remerge_sort,
settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data);
} }
} }
void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info) void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSortingInfoPtr input_sorting_info)
{ {
auto & query = getSelectQuery(); auto & query = getSelectQuery();
SortDescription output_order_descr = getSortDescription(query, context); SortDescription output_order_descr = getSortDescription(query, *context);
UInt64 limit = getLimitForSorting(query, context); UInt64 limit = getLimitForSorting(query, *context);
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
/// TODO: Limits on sorting /// TODO: Limits on sorting
// IBlockInputStream::LocalLimits limits; // IBlockInputStream::LocalLimits limits;
@ -2382,7 +2390,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
return std::make_shared<MergeSortingTransform>( return std::make_shared<MergeSortingTransform>(
header, output_order_descr, settings.max_block_size, limit, header, output_order_descr, settings.max_block_size, limit,
settings.max_bytes_before_remerge_sort, settings.max_bytes_before_remerge_sort,
settings.max_bytes_before_external_sort, context.getTemporaryPath(), settings.min_free_disk_space_for_temporary_data); settings.max_bytes_before_external_sort, context->getTemporaryPath(), settings.min_free_disk_space_for_temporary_data);
}); });
} }
@ -2390,8 +2398,8 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
{ {
auto & query = getSelectQuery(); auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query, context); SortDescription order_descr = getSortDescription(query, *context);
UInt64 limit = getLimitForSorting(query, context); UInt64 limit = getLimitForSorting(query, *context);
/// If there are several streams, then we merge them into one /// If there are several streams, then we merge them into one
if (pipeline.hasMoreThanOneStream()) if (pipeline.hasMoreThanOneStream())
@ -2406,7 +2414,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline, const SortD
{ {
if (pipeline.hasMoreThanOneStream()) if (pipeline.hasMoreThanOneStream())
{ {
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
/** MergingSortedBlockInputStream reads the sources sequentially. /** MergingSortedBlockInputStream reads the sources sequentially.
* To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream.
@ -2425,8 +2433,8 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline, const SortD
void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline) void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline)
{ {
auto & query = getSelectQuery(); auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query, context); SortDescription order_descr = getSortDescription(query, *context);
UInt64 limit = getLimitForSorting(query, context); UInt64 limit = getLimitForSorting(query, *context);
executeMergeSorted(pipeline, order_descr, limit); executeMergeSorted(pipeline, order_descr, limit);
} }
@ -2436,7 +2444,7 @@ void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline, const
/// If there are several streams, then we merge them into one /// If there are several streams, then we merge them into one
if (pipeline.getNumStreams() > 1) if (pipeline.getNumStreams() > 1)
{ {
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
auto transform = std::make_shared<MergingSortedTransform>( auto transform = std::make_shared<MergingSortedTransform>(
pipeline.getHeader(), pipeline.getHeader(),
@ -2471,9 +2479,9 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or
auto & query = getSelectQuery(); auto & query = getSelectQuery();
if (query.distinct) if (query.distinct)
{ {
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
UInt64 limit_for_distinct = 0; UInt64 limit_for_distinct = 0;
/// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows. /// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows.
@ -2493,9 +2501,9 @@ void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool befo
auto & query = getSelectQuery(); auto & query = getSelectQuery();
if (query.distinct) if (query.distinct)
{ {
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
UInt64 limit_for_distinct = 0; UInt64 limit_for_distinct = 0;
/// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows. /// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows.
@ -2545,13 +2553,13 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
/// If there is LIMIT /// If there is LIMIT
if (query.limitLength()) if (query.limitLength())
{ {
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
SortDescription sort_descr; SortDescription sort_descr;
if (query.limit_with_ties) if (query.limit_with_ties)
{ {
if (!query.orderBy()) if (!query.orderBy())
throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
sort_descr = getSortDescription(query, context); sort_descr = getSortDescription(query, *context);
} }
pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) pipeline.transform([&, limit = limit_length + limit_offset](auto & stream)
{ {
@ -2567,7 +2575,7 @@ void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline)
/// If there is LIMIT /// If there is LIMIT
if (query.limitLength()) if (query.limitLength())
{ {
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
pipeline.addSimpleTransform([&, limit = limit_length + limit_offset](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr pipeline.addSimpleTransform([&, limit = limit_length + limit_offset](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{ {
if (stream_type == QueryPipeline::StreamType::Totals) if (stream_type == QueryPipeline::StreamType::Totals)
@ -2588,8 +2596,8 @@ void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline)
Names columns; Names columns;
for (const auto & elem : query.limitBy()->children) for (const auto & elem : query.limitBy()->children)
columns.emplace_back(elem->getColumnName()); columns.emplace_back(elem->getColumnName());
UInt64 length = getLimitUIntValue(query.limitByLength(), context); UInt64 length = getLimitUIntValue(query.limitByLength(), *context);
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context) : 0); UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0);
pipeline.transform([&](auto & stream) pipeline.transform([&](auto & stream)
{ {
@ -2607,8 +2615,8 @@ void InterpreterSelectQuery::executeLimitBy(QueryPipeline & pipeline)
for (const auto & elem : query.limitBy()->children) for (const auto & elem : query.limitBy()->children)
columns.emplace_back(elem->getColumnName()); columns.emplace_back(elem->getColumnName());
UInt64 length = getLimitUIntValue(query.limitByLength(), context); UInt64 length = getLimitUIntValue(query.limitByLength(), *context);
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context) : 0); UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0);
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{ {
@ -2672,12 +2680,12 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
{ {
if (!query.orderBy()) if (!query.orderBy())
throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
order_descr = getSortDescription(query, context); order_descr = getSortDescription(query, *context);
} }
UInt64 limit_length; UInt64 limit_length;
UInt64 limit_offset; UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context); std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context);
pipeline.transform([&](auto & stream) pipeline.transform([&](auto & stream)
{ {
@ -2692,7 +2700,7 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline)
auto & query = getSelectQuery(); auto & query = getSelectQuery();
if (query.orderBy()) if (query.orderBy())
{ {
SortDescription order_descr = getSortDescription(query, context); SortDescription order_descr = getSortDescription(query, *context);
SortDescription fill_descr; SortDescription fill_descr;
for (auto & desc : order_descr) for (auto & desc : order_descr)
{ {
@ -2715,7 +2723,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPipeline & pipeline)
auto & query = getSelectQuery(); auto & query = getSelectQuery();
if (query.orderBy()) if (query.orderBy())
{ {
SortDescription order_descr = getSortDescription(query, context); SortDescription order_descr = getSortDescription(query, *context);
SortDescription fill_descr; SortDescription fill_descr;
for (auto & desc : order_descr) for (auto & desc : order_descr)
{ {
@ -2759,14 +2767,14 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline)
UInt64 limit_length; UInt64 limit_length;
UInt64 limit_offset; UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context); std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context);
SortDescription order_descr; SortDescription order_descr;
if (query.limit_with_ties) if (query.limit_with_ties)
{ {
if (!query.orderBy()) if (!query.orderBy())
throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR);
order_descr = getSortDescription(query, context); order_descr = getSortDescription(query, *context);
} }
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
@ -2783,7 +2791,7 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline)
void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline)
{ {
if (!context.getSettingsRef().extremes) if (!context->getSettingsRef().extremes)
return; return;
pipeline.transform([&](auto & stream) pipeline.transform([&](auto & stream)
@ -2794,7 +2802,7 @@ void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline)
void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline) void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline)
{ {
if (!context.getSettingsRef().extremes) if (!context->getSettingsRef().extremes)
return; return;
auto transform = std::make_shared<ExtremesTransform>(pipeline.getHeader()); auto transform = std::make_shared<ExtremesTransform>(pipeline.getHeader());
@ -2815,7 +2823,7 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline
executeUnion(pipeline, {}); executeUnion(pipeline, {});
pipeline.firstStream() = std::make_shared<CreatingSetsBlockInputStream>( pipeline.firstStream() = std::make_shared<CreatingSetsBlockInputStream>(
pipeline.firstStream(), subqueries_for_sets, context); pipeline.firstStream(), subqueries_for_sets, *context);
} }
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets) void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets)
@ -2827,12 +2835,12 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip
executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0); executeMergeSorted(pipeline, query_info.input_sorting_info->order_key_prefix_descr, 0);
} }
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
auto creating_sets = std::make_shared<CreatingSetsTransform>( auto creating_sets = std::make_shared<CreatingSetsTransform>(
pipeline.getHeader(), subqueries_for_sets, pipeline.getHeader(), subqueries_for_sets,
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode), SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode),
context); *context);
pipeline.addCreatingSetsTransform(std::move(creating_sets)); pipeline.addCreatingSetsTransform(std::move(creating_sets));
} }
@ -2853,7 +2861,7 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header)
auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name; auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name;
if (!blocksHaveEqualStructure(header, stream_header)) if (!blocksHaveEqualStructure(header, stream_header))
stream = std::make_shared<ConvertingBlockInputStream>(context, stream, header, mode); stream = std::make_shared<ConvertingBlockInputStream>(*context, stream, header, mode);
} }
} }
@ -2868,7 +2876,7 @@ void InterpreterSelectQuery::initSettings()
{ {
auto & query = getSelectQuery(); auto & query = getSelectQuery();
if (query.settings()) if (query.settings())
InterpreterSetQuery(query.settings(), context).executeForCurrentContext(); InterpreterSetQuery(query.settings(), *context).executeForCurrentContext();
} }
} }

View File

@ -69,7 +69,7 @@ public:
BlockIO execute() override; BlockIO execute() override;
/// Execute the query and return multuple streams for parallel processing. /// Execute the query and return multuple streams for parallel processing.
BlockInputStreams executeWithMultipleStreams(); BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline);
QueryPipeline executeWithProcessors() override; QueryPipeline executeWithProcessors() override;
bool canExecuteWithProcessors() const override { return true; } bool canExecuteWithProcessors() const override { return true; }
@ -137,7 +137,7 @@ private:
}; };
template <typename TPipeline> template <typename TPipeline>
void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input); void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, QueryPipeline & save_context_and_storage);
struct AnalysisResult struct AnalysisResult
{ {
@ -199,7 +199,8 @@ private:
template <typename TPipeline> template <typename TPipeline>
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline,
const InputSortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info, const InputSortingInfoPtr & sorting_info, const PrewhereInfoPtr & prewhere_info,
const Names & columns_to_remove_after_prewhere); const Names & columns_to_remove_after_prewhere,
QueryPipeline & save_context_and_storage);
void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter); void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter);
void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
@ -261,7 +262,7 @@ private:
const SelectQueryOptions options; const SelectQueryOptions options;
ASTPtr query_ptr; ASTPtr query_ptr;
Context context; std::shared_ptr<Context> context;
SyntaxAnalyzerResultPtr syntax_analyzer_result; SyntaxAnalyzerResultPtr syntax_analyzer_result;
std::unique_ptr<SelectQueryExpressionAnalyzer> query_analyzer; std::unique_ptr<SelectQueryExpressionAnalyzer> query_analyzer;
SelectQueryInfo query_info; SelectQueryInfo query_info;

View File

@ -34,7 +34,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
const Names & required_result_column_names) const Names & required_result_column_names)
: options(options_), : options(options_),
query_ptr(query_ptr_), query_ptr(query_ptr_),
context(context_) context(std::make_shared<Context>(context_))
{ {
const auto & ast = query_ptr->as<ASTSelectWithUnionQuery &>(); const auto & ast = query_ptr->as<ASTSelectWithUnionQuery &>();
@ -57,7 +57,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
/// We use it to determine positions of 'required_result_column_names' in SELECT clause. /// We use it to determine positions of 'required_result_column_names' in SELECT clause.
Block full_result_header = InterpreterSelectQuery( Block full_result_header = InterpreterSelectQuery(
ast.list_of_selects->children.at(0), context, options.copy().analyze().noModify()).getSampleBlock(); ast.list_of_selects->children.at(0), *context, options.copy().analyze().noModify()).getSampleBlock();
std::vector<size_t> positions_of_required_result_columns(required_result_column_names.size()); std::vector<size_t> positions_of_required_result_columns(required_result_column_names.size());
for (size_t required_result_num = 0, size = required_result_column_names.size(); required_result_num < size; ++required_result_num) for (size_t required_result_num = 0, size = required_result_column_names.size(); required_result_num < size; ++required_result_num)
@ -66,7 +66,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
for (size_t query_num = 1; query_num < num_selects; ++query_num) for (size_t query_num = 1; query_num < num_selects; ++query_num)
{ {
Block full_result_header_for_current_select = InterpreterSelectQuery( Block full_result_header_for_current_select = InterpreterSelectQuery(
ast.list_of_selects->children.at(query_num), context, options.copy().analyze().noModify()).getSampleBlock(); ast.list_of_selects->children.at(query_num), *context, options.copy().analyze().noModify()).getSampleBlock();
if (full_result_header_for_current_select.columns() != full_result_header.columns()) if (full_result_header_for_current_select.columns() != full_result_header.columns())
throw Exception("Different number of columns in UNION ALL elements:\n" throw Exception("Different number of columns in UNION ALL elements:\n"
@ -88,7 +88,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
nested_interpreters.emplace_back(std::make_unique<InterpreterSelectQuery>( nested_interpreters.emplace_back(std::make_unique<InterpreterSelectQuery>(
ast.list_of_selects->children.at(query_num), ast.list_of_selects->children.at(query_num),
context, *context,
options, options,
current_required_result_column_names)); current_required_result_column_names));
} }
@ -165,20 +165,23 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(
} }
BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams() BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline)
{ {
BlockInputStreams nested_streams; BlockInputStreams nested_streams;
for (auto & interpreter : nested_interpreters) for (auto & interpreter : nested_interpreters)
{ {
BlockInputStreams streams = interpreter->executeWithMultipleStreams(); BlockInputStreams streams = interpreter->executeWithMultipleStreams(parent_pipeline);
nested_streams.insert(nested_streams.end(), streams.begin(), streams.end()); nested_streams.insert(nested_streams.end(), streams.begin(), streams.end());
} }
/// Unify data structure. /// Unify data structure.
if (nested_interpreters.size() > 1) if (nested_interpreters.size() > 1)
{
for (auto & stream : nested_streams) for (auto & stream : nested_streams)
stream = std::make_shared<ConvertingBlockInputStream>(context, stream, result_header, ConvertingBlockInputStream::MatchColumnsMode::Position); stream = std::make_shared<ConvertingBlockInputStream>(*context, stream, result_header,ConvertingBlockInputStream::MatchColumnsMode::Position);
parent_pipeline.addInterpreterContext(context);
}
return nested_streams; return nested_streams;
} }
@ -186,9 +189,10 @@ BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams()
BlockIO InterpreterSelectWithUnionQuery::execute() BlockIO InterpreterSelectWithUnionQuery::execute()
{ {
const Settings & settings = context.getSettingsRef(); const Settings & settings = context->getSettingsRef();
BlockInputStreams nested_streams = executeWithMultipleStreams(); BlockIO res;
BlockInputStreams nested_streams = executeWithMultipleStreams(res.pipeline);
BlockInputStreamPtr result_stream; BlockInputStreamPtr result_stream;
if (nested_streams.empty()) if (nested_streams.empty())
@ -206,8 +210,8 @@ BlockIO InterpreterSelectWithUnionQuery::execute()
nested_streams.clear(); nested_streams.clear();
} }
BlockIO res;
res.in = result_stream; res.in = result_stream;
res.pipeline.addInterpreterContext(context);
return res; return res;
} }
@ -242,9 +246,11 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors()
if (!pipelines.empty()) if (!pipelines.empty())
{ {
auto common_header = getCommonHeaderForUnion(headers); auto common_header = getCommonHeaderForUnion(headers);
main_pipeline.unitePipelines(std::move(pipelines), common_header, context); main_pipeline.unitePipelines(std::move(pipelines), common_header, *context);
} }
main_pipeline.addInterpreterContext(context);
return main_pipeline; return main_pipeline;
} }

View File

@ -29,7 +29,7 @@ public:
BlockIO execute() override; BlockIO execute() override;
/// Execute the query without union of streams. /// Execute the query without union of streams.
BlockInputStreams executeWithMultipleStreams(); BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline);
QueryPipeline executeWithProcessors() override; QueryPipeline executeWithProcessors() override;
bool canExecuteWithProcessors() const override { return true; } bool canExecuteWithProcessors() const override { return true; }
@ -47,7 +47,7 @@ public:
private: private:
const SelectQueryOptions options; const SelectQueryOptions options;
ASTPtr query_ptr; ASTPtr query_ptr;
Context context; std::shared_ptr<Context> context;
std::vector<std::unique_ptr<InterpreterSelectQuery>> nested_interpreters; std::vector<std::unique_ptr<InterpreterSelectQuery>> nested_interpreters;

View File

@ -252,7 +252,7 @@ bool ParserFunction::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
} }
/// The parametric aggregate function has two lists (parameters and arguments) in parentheses. Example: quantile(0.9)(x). /// The parametric aggregate function has two lists (parameters and arguments) in parentheses. Example: quantile(0.9)(x).
if (pos->type == TokenType::OpeningRoundBracket) if (allow_function_parameters && pos->type == TokenType::OpeningRoundBracket)
{ {
++pos; ++pos;

View File

@ -90,9 +90,12 @@ protected:
*/ */
class ParserFunction : public IParserBase class ParserFunction : public IParserBase
{ {
public:
ParserFunction(bool allow_function_parameters_ = true) : allow_function_parameters(allow_function_parameters_) {}
protected: protected:
const char * getName() const { return "function"; } const char * getName() const { return "function"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected); bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
bool allow_function_parameters;
}; };
class ParserCodecDeclarationList : public IParserBase class ParserCodecDeclarationList : public IParserBase

View File

@ -35,7 +35,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserToken s_rparen(TokenType::ClosingRoundBracket); ParserToken s_rparen(TokenType::ClosingRoundBracket);
ParserIdentifier name_p; ParserIdentifier name_p;
ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false); ParserList columns_p(std::make_unique<ParserCompoundIdentifier>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserFunction table_function_p; ParserFunction table_function_p{false};
ASTPtr database; ASTPtr database;
ASTPtr table; ASTPtr table;

View File

@ -8,7 +8,6 @@
#include <Common/PODArray.h> #include <Common/PODArray.h>
#include <Common/UTF8Helpers.h> #include <Common/UTF8Helpers.h>
namespace DB namespace DB
{ {

View File

@ -384,4 +384,81 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
} }
} }
void saveUpToPosition(ReadBuffer & in, DB::Memory<> & memory, char * current)
{
assert(current >= in.position());
assert(current <= in.buffer().end());
const int old_bytes = memory.size();
const int additional_bytes = current - in.position();
const int new_bytes = old_bytes + additional_bytes;
/// There are no new bytes to add to memory.
/// No need to do extra stuff.
if (new_bytes == 0)
return;
memory.resize(new_bytes);
memcpy(memory.data() + old_bytes, in.position(), additional_bytes);
in.position() = current;
}
bool loadAtPosition(ReadBuffer & in, DB::Memory<> & memory, char * & current)
{
assert(current <= in.buffer().end());
if (current < in.buffer().end())
{
return true;
}
saveUpToPosition(in, memory, current);
bool loaded_more = !in.eof();
assert(in.position() == in.buffer().begin());
current = in.position();
return loaded_more;
}
bool fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
bool need_more_data = true;
char * pos = in.position();
while (loadAtPosition(in, memory, pos) && need_more_data)
{
pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end());
if (pos == in.buffer().end())
{
continue;
}
if (*pos == '\\')
{
++pos;
if (loadAtPosition(in, memory, pos))
{
++pos;
}
}
else if (*pos == '\n' || *pos == '\r')
{
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
{
need_more_data = false;
}
++pos;
}
}
saveUpToPosition(in, memory, pos);
return loadAtPosition(in, memory, pos);
}
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory)
{
// We can use the same segmentation engine for TSKV.
for (auto name : {"TabSeparated", "TSV", "TSKV"})
{
factory.registerFileSegmentationEngine(name, &fileSegmentationEngineTabSeparatedImpl);
}
}
} }

View File

@ -320,7 +320,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
} }
/// Can be used in fileSegmentationEngine for parallel parsing of Values /// Can be used in fileSegmentationEngine for parallel parsing of Values
bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_size, int balance) bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_bytes, int balance)
{ {
skipWhitespaceIfAny(buf); skipWhitespaceIfAny(buf);
if (buf.eof() || *buf.position() == ';') if (buf.eof() || *buf.position() == ';')
@ -328,7 +328,7 @@ bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_size, int balance)
bool quoted = false; bool quoted = false;
size_t chunk_begin_buf_count = buf.count(); size_t chunk_begin_buf_count = buf.count();
while (!buf.eof() && (balance || buf.count() - chunk_begin_buf_count < min_chunk_size)) while (!buf.eof() && (balance || buf.count() - chunk_begin_buf_count < min_chunk_bytes))
{ {
buf.position() = find_first_symbols<'\\', '\'', ')', '('>(buf.position(), buf.buffer().end()); buf.position() = find_first_symbols<'\\', '\'', ')', '('>(buf.position(), buf.buffer().end());
if (buf.position() == buf.buffer().end()) if (buf.position() == buf.buffer().end())

View File

@ -60,7 +60,7 @@ private:
void readSuffix(); void readSuffix();
bool skipToNextRow(size_t min_chunk_size = 0, int balance = 0); bool skipToNextRow(size_t min_chunk_bytes = 0, int balance = 0);
private: private:
PeekableReadBuffer buf; PeekableReadBuffer buf;

View File

@ -486,6 +486,10 @@ void QueryPipeline::unitePipelines(
processors.insert(processors.end(), pipeline.processors.begin(), pipeline.processors.end()); processors.insert(processors.end(), pipeline.processors.begin(), pipeline.processors.end());
streams.insert(streams.end(), pipeline.streams.begin(), pipeline.streams.end()); streams.insert(streams.end(), pipeline.streams.begin(), pipeline.streams.end());
table_locks.insert(table_locks.end(), std::make_move_iterator(pipeline.table_locks.begin()), std::make_move_iterator(pipeline.table_locks.end()));
interpreter_context.insert(interpreter_context.end(), pipeline.interpreter_context.begin(), pipeline.interpreter_context.end());
storage_holder.insert(storage_holder.end(), pipeline.storage_holder.begin(), pipeline.storage_holder.end());
} }
if (!extremes.empty()) if (!extremes.empty())

View File

@ -6,6 +6,7 @@
#include <DataStreams/IBlockInputStream.h> #include <DataStreams/IBlockInputStream.h>
#include <DataStreams/IBlockOutputStream.h> #include <DataStreams/IBlockOutputStream.h>
#include <Storages/IStorage_fwd.h>
namespace DB namespace DB
{ {
@ -75,6 +76,8 @@ public:
const Block & getHeader() const { return current_header; } const Block & getHeader() const { return current_header; }
void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); } void addTableLock(const TableStructureReadLockHolder & lock) { table_locks.push_back(lock); }
void addInterpreterContext(std::shared_ptr<Context> context) { interpreter_context.emplace_back(std::move(context)); }
void addStorageHolder(StoragePtr storage) { storage_holder.emplace_back(std::move(storage)); }
/// For compatibility with IBlockInputStream. /// For compatibility with IBlockInputStream.
void setProgressCallback(const ProgressCallback & callback); void setProgressCallback(const ProgressCallback & callback);
@ -109,6 +112,12 @@ private:
TableStructureReadLocks table_locks; TableStructureReadLocks table_locks;
/// Some Streams (or Processors) may implicitly use Context or temporary Storage created by Interpreter.
/// But lifetime of Streams is not nested in lifetime of Interpreters, so we have to store it here,
/// because QueryPipeline is alive until query is finished.
std::vector<std::shared_ptr<Context>> interpreter_context;
std::vector<StoragePtr> storage_holder;
IOutputFormat * output_format = nullptr; IOutputFormat * output_format = nullptr;
size_t max_threads = 0; size_t max_threads = 0;

View File

@ -3,6 +3,7 @@
#include <Storages/AlterCommands.h> #include <Storages/AlterCommands.h>
#include <Parsers/ASTCreateQuery.h> #include <Parsers/ASTCreateQuery.h>
#include <Parsers/ASTSetQuery.h> #include <Parsers/ASTSetQuery.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/quoteString.h> #include <Common/quoteString.h>
#include <Processors/Executors/TreeExecutorBlockInputStream.h> #include <Processors/Executors/TreeExecutorBlockInputStream.h>
@ -445,4 +446,21 @@ BlockInputStreams IStorage::read(
return res; return res;
} }
DB::CompressionMethod IStorage::chooseCompressionMethod(const String & uri, const String & compression_method)
{
if (compression_method == "auto" || compression_method == "")
{
if (endsWith(uri, ".gz"))
return DB::CompressionMethod::Gzip;
else
return DB::CompressionMethod::None;
}
else if (compression_method == "gzip")
return DB::CompressionMethod::Gzip;
else if (compression_method == "none")
return DB::CompressionMethod::None;
else
throw Exception("Only auto, none, gzip supported as compression method", ErrorCodes::NOT_IMPLEMENTED);
}
} }

View File

@ -5,6 +5,7 @@
#include <DataStreams/IBlockStream_fwd.h> #include <DataStreams/IBlockStream_fwd.h>
#include <Databases/IDatabase.h> #include <Databases/IDatabase.h>
#include <Interpreters/CancellationCode.h> #include <Interpreters/CancellationCode.h>
#include <IO/CompressionMethod.h>
#include <Storages/IStorage_fwd.h> #include <Storages/IStorage_fwd.h>
#include <Storages/SelectQueryInfo.h> #include <Storages/SelectQueryInfo.h>
#include <Storages/TableStructureLockHolder.h> #include <Storages/TableStructureLockHolder.h>
@ -434,6 +435,8 @@ public:
return {}; return {};
} }
static DB::CompressionMethod chooseCompressionMethod(const String & uri, const String & compression_method);
private: private:
/// You always need to take the next three locks in this order. /// You always need to take the next three locks in this order.

View File

@ -3269,6 +3269,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
String tmp_dst_part_name = tmp_part_prefix + dst_part_name; String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
auto reservation = src_part->disk->reserve(src_part->bytes_on_disk); auto reservation = src_part->disk->reserve(src_part->bytes_on_disk);
if (!reservation)
{
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(src_part->bytes_on_disk) + ", not enough space",
ErrorCodes::NOT_ENOUGH_SPACE);
}
String dst_part_path = getFullPathOnDisk(reservation->getDisk()); String dst_part_path = getFullPathOnDisk(reservation->getDisk());
Poco::Path dst_part_absolute_path = Poco::Path(dst_part_path + tmp_dst_part_name).absolute(); Poco::Path dst_part_absolute_path = Poco::Path(dst_part_path + tmp_dst_part_name).absolute();
Poco::Path src_part_absolute_path = Poco::Path(src_part->getFullPath()).absolute(); Poco::Path src_part_absolute_path = Poco::Path(src_part->getFullPath()).absolute();

View File

@ -8,6 +8,7 @@
#include <Parsers/ASTIdentifier.h> #include <Parsers/ASTIdentifier.h>
#include <IO/ReadBufferFromFile.h> #include <IO/ReadBufferFromFile.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromFile.h> #include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
@ -127,9 +128,10 @@ StorageFile::StorageFile(
const std::string & format_name_, const std::string & format_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
Context & context_) Context & context_,
const String & compression_method_ = "")
: :
table_name(table_name_), database_name(database_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_) table_name(table_name_), database_name(database_name_), format_name(format_name_), context_global(context_), table_fd(table_fd_), compression_method(compression_method_)
{ {
setColumns(columns_); setColumns(columns_);
setConstraints(constraints_); setConstraints(constraints_);
@ -178,7 +180,10 @@ StorageFile::StorageFile(
class StorageFileBlockInputStream : public IBlockInputStream class StorageFileBlockInputStream : public IBlockInputStream
{ {
public: public:
StorageFileBlockInputStream(std::shared_ptr<StorageFile> storage_, const Context & context, UInt64 max_block_size, std::string file_path) StorageFileBlockInputStream(std::shared_ptr<StorageFile> storage_,
const Context & context, UInt64 max_block_size,
std::string file_path,
const CompressionMethod compression_method)
: storage(std::move(storage_)) : storage(std::move(storage_))
{ {
if (storage->use_table_fd) if (storage->use_table_fd)
@ -199,12 +204,12 @@ public:
} }
storage->table_fd_was_used = true; storage->table_fd_was_used = true;
read_buf = std::make_unique<ReadBufferFromFileDescriptor>(storage->table_fd); read_buf = getReadBuffer<ReadBufferFromFileDescriptor>(compression_method, storage->table_fd);
} }
else else
{ {
shared_lock = std::shared_lock(storage->rwlock); shared_lock = std::shared_lock(storage->rwlock);
read_buf = std::make_unique<ReadBufferFromFile>(file_path); read_buf = getReadBuffer<ReadBufferFromFile>(compression_method, file_path);
} }
reader = FormatFactory::instance().getInput(storage->format_name, *read_buf, storage->getSampleBlock(), context, max_block_size); reader = FormatFactory::instance().getInput(storage->format_name, *read_buf, storage->getSampleBlock(), context, max_block_size);
@ -235,7 +240,7 @@ public:
private: private:
std::shared_ptr<StorageFile> storage; std::shared_ptr<StorageFile> storage;
Block sample_block; Block sample_block;
std::unique_ptr<ReadBufferFromFileDescriptor> read_buf; std::unique_ptr<ReadBuffer> read_buf;
BlockInputStreamPtr reader; BlockInputStreamPtr reader;
std::shared_lock<std::shared_mutex> shared_lock; std::shared_lock<std::shared_mutex> shared_lock;
@ -260,7 +265,7 @@ BlockInputStreams StorageFile::read(
for (const auto & file_path : paths) for (const auto & file_path : paths)
{ {
BlockInputStreamPtr cur_block = std::make_shared<StorageFileBlockInputStream>( BlockInputStreamPtr cur_block = std::make_shared<StorageFileBlockInputStream>(
std::static_pointer_cast<StorageFile>(shared_from_this()), context, max_block_size, file_path); std::static_pointer_cast<StorageFile>(shared_from_this()), context, max_block_size, file_path, IStorage::chooseCompressionMethod(file_path, compression_method));
blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared<AddingDefaultsBlockInputStream>(cur_block, column_defaults, context)); blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared<AddingDefaultsBlockInputStream>(cur_block, column_defaults, context));
} }
return blocks_input; return blocks_input;
@ -270,7 +275,8 @@ BlockInputStreams StorageFile::read(
class StorageFileBlockOutputStream : public IBlockOutputStream class StorageFileBlockOutputStream : public IBlockOutputStream
{ {
public: public:
explicit StorageFileBlockOutputStream(StorageFile & storage_) explicit StorageFileBlockOutputStream(StorageFile & storage_,
const CompressionMethod compression_method)
: storage(storage_), lock(storage.rwlock) : storage(storage_), lock(storage.rwlock)
{ {
if (storage.use_table_fd) if (storage.use_table_fd)
@ -280,13 +286,13 @@ public:
* INSERT data; SELECT *; last SELECT returns only insert_data * INSERT data; SELECT *; last SELECT returns only insert_data
*/ */
storage.table_fd_was_used = true; storage.table_fd_was_used = true;
write_buf = std::make_unique<WriteBufferFromFileDescriptor>(storage.table_fd); write_buf = getWriteBuffer<WriteBufferFromFileDescriptor>(compression_method, storage.table_fd);
} }
else else
{ {
if (storage.paths.size() != 1) if (storage.paths.size() != 1)
throw Exception("Table '" + storage.table_name + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED); throw Exception("Table '" + storage.table_name + "' is in readonly mode because of globs in filepath", ErrorCodes::DATABASE_ACCESS_DENIED);
write_buf = std::make_unique<WriteBufferFromFile>(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); write_buf = getWriteBuffer<WriteBufferFromFile>(compression_method, storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT);
} }
writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global);
@ -317,7 +323,7 @@ public:
private: private:
StorageFile & storage; StorageFile & storage;
std::unique_lock<std::shared_mutex> lock; std::unique_lock<std::shared_mutex> lock;
std::unique_ptr<WriteBufferFromFileDescriptor> write_buf; std::unique_ptr<WriteBuffer> write_buf;
BlockOutputStreamPtr writer; BlockOutputStreamPtr writer;
}; };
@ -325,7 +331,8 @@ BlockOutputStreamPtr StorageFile::write(
const ASTPtr & /*query*/, const ASTPtr & /*query*/,
const Context & /*context*/) const Context & /*context*/)
{ {
return std::make_shared<StorageFileBlockOutputStream>(*this); return std::make_shared<StorageFileBlockOutputStream>(*this,
IStorage::chooseCompressionMethod(paths[0], compression_method));
} }
Strings StorageFile::getDataPaths() const Strings StorageFile::getDataPaths() const
@ -361,9 +368,9 @@ void registerStorageFile(StorageFactory & factory)
{ {
ASTs & engine_args = args.engine_args; ASTs & engine_args = args.engine_args;
if (!(engine_args.size() == 1 || engine_args.size() == 2)) if (!(engine_args.size() >= 1 && engine_args.size() <= 3))
throw Exception( throw Exception(
"Storage File requires 1 or 2 arguments: name of used format and source.", "Storage File requires from 1 to 3 arguments: name of used format, source and compression_method.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
@ -371,6 +378,7 @@ void registerStorageFile(StorageFactory & factory)
int source_fd = -1; int source_fd = -1;
String source_path; String source_path;
String compression_method;
if (engine_args.size() >= 2) if (engine_args.size() >= 2)
{ {
/// Will use FD if engine_args[1] is int literal or identifier with std* name /// Will use FD if engine_args[1] is int literal or identifier with std* name
@ -397,13 +405,19 @@ void registerStorageFile(StorageFactory & factory)
else if (type == Field::Types::String) else if (type == Field::Types::String)
source_path = literal->value.get<String>(); source_path = literal->value.get<String>();
} }
if (engine_args.size() == 3)
{
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
compression_method = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
} else compression_method = "auto";
} }
return StorageFile::create( return StorageFile::create(
source_path, source_fd, source_path, source_fd,
args.data_path, args.data_path,
args.database_name, args.table_name, format_name, args.columns, args.constraints, args.database_name, args.table_name, format_name, args.columns, args.constraints,
args.context); args.context,
compression_method);
}); });
} }

View File

@ -60,7 +60,8 @@ protected:
const std::string & format_name_, const std::string & format_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
Context & context_); Context & context_,
const String & compression_method_);
private: private:
std::string table_name; std::string table_name;
@ -69,6 +70,7 @@ private:
Context & context_global; Context & context_global;
int table_fd = -1; int table_fd = -1;
String compression_method;
std::vector<std::string> paths; std::vector<std::string> paths;

View File

@ -7,14 +7,16 @@
#include <Interpreters/Context.h> #include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadBufferFromHDFS.h> #include <IO/ReadBufferFromHDFS.h>
#include <IO/WriteBufferFromHDFS.h> #include <IO/WriteBufferFromHDFS.h>
#include <IO/WriteHelpers.h>
#include <IO/HDFSCommon.h> #include <IO/HDFSCommon.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
#include <DataStreams/IBlockOutputStream.h> #include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/UnionBlockInputStream.h> #include <DataStreams/UnionBlockInputStream.h>
#include <DataStreams/IBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h> #include <DataStreams/OwningBlockInputStream.h>
#include <DataStreams/IBlockInputStream.h>
#include <Common/parseGlobs.h> #include <Common/parseGlobs.h>
#include <Poco/URI.h> #include <Poco/URI.h>
#include <re2/re2.h> #include <re2/re2.h>
@ -36,12 +38,14 @@ StorageHDFS::StorageHDFS(const String & uri_,
const String & format_name_, const String & format_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
Context & context_) Context & context_,
const String & compression_method_ = "")
: uri(uri_) : uri(uri_)
, format_name(format_name_) , format_name(format_name_)
, table_name(table_name_) , table_name(table_name_)
, database_name(database_name_) , database_name(database_name_)
, context(context_) , context(context_)
, compression_method(compression_method_)
{ {
setColumns(columns_); setColumns(columns_);
setConstraints(constraints_); setConstraints(constraints_);
@ -57,9 +61,11 @@ public:
const String & format, const String & format,
const Block & sample_block, const Block & sample_block,
const Context & context, const Context & context,
UInt64 max_block_size) UInt64 max_block_size,
const CompressionMethod compression_method)
{ {
std::unique_ptr<ReadBuffer> read_buf = std::make_unique<ReadBufferFromHDFS>(uri); auto read_buf = getReadBuffer<ReadBufferFromHDFS>(compression_method, uri);
auto input_stream = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); auto input_stream = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
reader = std::make_shared<OwningBlockInputStream<ReadBuffer>>(input_stream, std::move(read_buf)); reader = std::make_shared<OwningBlockInputStream<ReadBuffer>>(input_stream, std::move(read_buf));
} }
@ -99,10 +105,11 @@ public:
HDFSBlockOutputStream(const String & uri, HDFSBlockOutputStream(const String & uri,
const String & format, const String & format,
const Block & sample_block_, const Block & sample_block_,
const Context & context) const Context & context,
const CompressionMethod compression_method)
: sample_block(sample_block_) : sample_block(sample_block_)
{ {
write_buf = std::make_unique<WriteBufferFromHDFS>(uri); write_buf = getWriteBuffer<WriteBufferFromHDFS>(compression_method, uri);
writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context);
} }
@ -130,7 +137,7 @@ public:
private: private:
Block sample_block; Block sample_block;
std::unique_ptr<WriteBufferFromHDFS> write_buf; std::unique_ptr<WriteBuffer> write_buf;
BlockOutputStreamPtr writer; BlockOutputStreamPtr writer;
}; };
@ -203,7 +210,7 @@ BlockInputStreams StorageHDFS::read(
for (const auto & res_path : res_paths) for (const auto & res_path : res_paths)
{ {
result.push_back(std::make_shared<HDFSBlockInputStream>(uri_without_path + res_path, format_name, getSampleBlock(), context_, result.push_back(std::make_shared<HDFSBlockInputStream>(uri_without_path + res_path, format_name, getSampleBlock(), context_,
max_block_size)); max_block_size, IStorage::chooseCompressionMethod(res_path, compression_method)));
} }
return result; return result;
@ -217,7 +224,11 @@ void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_d
BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Context & /*context*/) BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Context & /*context*/)
{ {
return std::make_shared<HDFSBlockOutputStream>(uri, format_name, getSampleBlock(), context); return std::make_shared<HDFSBlockOutputStream>(uri,
format_name,
getSampleBlock(),
context,
IStorage::chooseCompressionMethod(uri, compression_method));
} }
void registerStorageHDFS(StorageFactory & factory) void registerStorageHDFS(StorageFactory & factory)
@ -226,9 +237,9 @@ void registerStorageHDFS(StorageFactory & factory)
{ {
ASTs & engine_args = args.engine_args; ASTs & engine_args = args.engine_args;
if (engine_args.size() != 2) if (engine_args.size() != 2 && engine_args.size() != 3)
throw Exception( throw Exception(
"Storage HDFS requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); "Storage HDFS requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
@ -238,7 +249,14 @@ void registerStorageHDFS(StorageFactory & factory)
String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>(); String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); String compression_method;
if (engine_args.size() == 3)
{
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
compression_method = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
} else compression_method = "auto";
return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context, compression_method);
}); });
} }

View File

@ -39,7 +39,8 @@ protected:
const String & format_name_, const String & format_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
Context & context_); Context & context_,
const String & compression_method_);
private: private:
String uri; String uri;
@ -47,6 +48,7 @@ private:
String table_name; String table_name;
String database_name; String database_name;
Context & context; Context & context;
String compression_method;
Logger * log = &Logger::get("StorageHDFS"); Logger * log = &Logger::get("StorageHDFS");
}; };

View File

@ -1141,6 +1141,11 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
/// Can throw an exception. /// Can throw an exception.
/// Once we mutate part, we must reserve space on the same disk, because mutations can possibly create hardlinks. /// Once we mutate part, we must reserve space on the same disk, because mutations can possibly create hardlinks.
DiskSpace::ReservationPtr reserved_space = source_part->disk->reserve(estimated_space_for_result); DiskSpace::ReservationPtr reserved_space = source_part->disk->reserve(estimated_space_for_result);
if (!reserved_space)
{
throw Exception("Cannot reserve " + formatReadableSizeWithBinarySuffix(estimated_space_for_result) + ", not enough space",
ErrorCodes::NOT_ENOUGH_SPACE);
}
auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY); auto table_lock = lockStructureForShare(false, RWLockImpl::NO_QUERY);

View File

@ -6,7 +6,9 @@
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <IO/ReadBufferFromS3.h> #include <IO/ReadBufferFromS3.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromS3.h> #include <IO/WriteBufferFromS3.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
@ -35,11 +37,11 @@ namespace
const Block & sample_block, const Block & sample_block,
const Context & context, const Context & context,
UInt64 max_block_size, UInt64 max_block_size,
const ConnectionTimeouts & timeouts) const ConnectionTimeouts & timeouts,
const CompressionMethod compression_method)
: name(name_) : name(name_)
{ {
read_buf = std::make_unique<ReadBufferFromS3>(uri, timeouts); read_buf = getReadBuffer<ReadBufferFromS3>(compression_method, uri, timeouts);
reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
} }
@ -70,7 +72,7 @@ namespace
private: private:
String name; String name;
std::unique_ptr<ReadBufferFromS3> read_buf; std::unique_ptr<ReadBuffer> read_buf;
BlockInputStreamPtr reader; BlockInputStreamPtr reader;
}; };
@ -82,10 +84,11 @@ namespace
UInt64 min_upload_part_size, UInt64 min_upload_part_size,
const Block & sample_block_, const Block & sample_block_,
const Context & context, const Context & context,
const ConnectionTimeouts & timeouts) const ConnectionTimeouts & timeouts,
const CompressionMethod compression_method)
: sample_block(sample_block_) : sample_block(sample_block_)
{ {
write_buf = std::make_unique<WriteBufferFromS3>(uri, min_upload_part_size, timeouts); write_buf = getWriteBuffer<WriteBufferFromS3>(compression_method, uri, min_upload_part_size, timeouts);
writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context);
} }
@ -113,7 +116,7 @@ namespace
private: private:
Block sample_block; Block sample_block;
std::unique_ptr<WriteBufferFromS3> write_buf; std::unique_ptr<WriteBuffer> write_buf;
BlockOutputStreamPtr writer; BlockOutputStreamPtr writer;
}; };
} }
@ -127,7 +130,8 @@ StorageS3::StorageS3(
UInt64 min_upload_part_size_, UInt64 min_upload_part_size_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
Context & context_) Context & context_,
const String & compression_method_ = "")
: IStorage(columns_) : IStorage(columns_)
, uri(uri_) , uri(uri_)
, context_global(context_) , context_global(context_)
@ -135,6 +139,7 @@ StorageS3::StorageS3(
, database_name(database_name_) , database_name(database_name_)
, table_name(table_name_) , table_name(table_name_)
, min_upload_part_size(min_upload_part_size_) , min_upload_part_size(min_upload_part_size_)
, compression_method(compression_method_)
{ {
setColumns(columns_); setColumns(columns_);
setConstraints(constraints_); setConstraints(constraints_);
@ -156,7 +161,8 @@ BlockInputStreams StorageS3::read(
getHeaderBlock(column_names), getHeaderBlock(column_names),
context, context,
max_block_size, max_block_size,
ConnectionTimeouts::getHTTPTimeouts(context)); ConnectionTimeouts::getHTTPTimeouts(context),
IStorage::chooseCompressionMethod(uri.toString(), compression_method));
auto column_defaults = getColumns().getDefaults(); auto column_defaults = getColumns().getDefaults();
if (column_defaults.empty()) if (column_defaults.empty())
@ -173,7 +179,9 @@ void StorageS3::rename(const String & /*new_path_to_db*/, const String & new_dat
BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/) BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const Context & /*context*/)
{ {
return std::make_shared<StorageS3BlockOutputStream>( return std::make_shared<StorageS3BlockOutputStream>(
uri, format_name, min_upload_part_size, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); uri, format_name, min_upload_part_size, getSampleBlock(), context_global,
ConnectionTimeouts::getHTTPTimeouts(context_global),
IStorage::chooseCompressionMethod(uri.toString(), compression_method));
} }
void registerStorageS3(StorageFactory & factory) void registerStorageS3(StorageFactory & factory)
@ -182,9 +190,9 @@ void registerStorageS3(StorageFactory & factory)
{ {
ASTs & engine_args = args.engine_args; ASTs & engine_args = args.engine_args;
if (engine_args.size() != 2) if (engine_args.size() != 2 && engine_args.size() != 3)
throw Exception( throw Exception(
"Storage S3 requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); "Storage S3 requires 2 or 3 arguments: url, name of used format and compression_method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
@ -197,6 +205,13 @@ void registerStorageS3(StorageFactory & factory)
UInt64 min_upload_part_size = args.local_context.getSettingsRef().s3_min_upload_part_size; UInt64 min_upload_part_size = args.local_context.getSettingsRef().s3_min_upload_part_size;
String compression_method;
if (engine_args.size() == 3)
{
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
compression_method = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
} else compression_method = "auto";
return StorageS3::create(uri, args.database_name, args.table_name, format_name, min_upload_part_size, args.columns, args.constraints, args.context); return StorageS3::create(uri, args.database_name, args.table_name, format_name, min_upload_part_size, args.columns, args.constraints, args.context);
}); });
} }

View File

@ -24,7 +24,8 @@ public:
UInt64 min_upload_part_size_, UInt64 min_upload_part_size_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
Context & context_); Context & context_,
const String & compression_method_);
String getName() const override String getName() const override
{ {
@ -61,6 +62,7 @@ private:
String database_name; String database_name;
String table_name; String table_name;
UInt64 min_upload_part_size; UInt64 min_upload_part_size;
String compression_method;
}; };
} }

View File

@ -5,8 +5,10 @@
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTLiteral.h> #include <Parsers/ASTLiteral.h>
#include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h> #include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/WriteBufferFromHTTP.h> #include <IO/WriteBufferFromHTTP.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h> #include <Formats/FormatFactory.h>
@ -31,8 +33,9 @@ IStorageURLBase::IStorageURLBase(
const std::string & table_name_, const std::string & table_name_,
const String & format_name_, const String & format_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_) const ConstraintsDescription & constraints_,
: uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_), database_name(database_name_) const String & compression_method_)
: uri(uri_), context_global(context_), compression_method(compression_method_), format_name(format_name_), table_name(table_name_), database_name(database_name_)
{ {
setColumns(columns_); setColumns(columns_);
setConstraints(constraints_); setConstraints(constraints_);
@ -51,10 +54,11 @@ namespace
const Block & sample_block, const Block & sample_block,
const Context & context, const Context & context,
UInt64 max_block_size, UInt64 max_block_size,
const ConnectionTimeouts & timeouts) const ConnectionTimeouts & timeouts,
const CompressionMethod compression_method)
: name(name_) : name(name_)
{ {
read_buf = std::make_unique<ReadWriteBufferFromHTTP>(uri, method, callback, timeouts, context.getSettingsRef().max_http_get_redirects); read_buf = getReadBuffer<ReadWriteBufferFromHTTP>(compression_method, uri, method, callback, timeouts, context.getSettingsRef().max_http_get_redirects);
reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size); reader = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
} }
@ -85,7 +89,7 @@ namespace
private: private:
String name; String name;
std::unique_ptr<ReadWriteBufferFromHTTP> read_buf; std::unique_ptr<ReadBuffer> read_buf;
BlockInputStreamPtr reader; BlockInputStreamPtr reader;
}; };
@ -96,10 +100,11 @@ namespace
const String & format, const String & format,
const Block & sample_block_, const Block & sample_block_,
const Context & context, const Context & context,
const ConnectionTimeouts & timeouts) const ConnectionTimeouts & timeouts,
const CompressionMethod compression_method)
: sample_block(sample_block_) : sample_block(sample_block_)
{ {
write_buf = std::make_unique<WriteBufferFromHTTP>(uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts); write_buf = getWriteBuffer<WriteBufferFromHTTP>(compression_method, uri, Poco::Net::HTTPRequest::HTTP_POST, timeouts);
writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context); writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context);
} }
@ -127,7 +132,7 @@ namespace
private: private:
Block sample_block; Block sample_block;
std::unique_ptr<WriteBufferFromHTTP> write_buf; std::unique_ptr<WriteBuffer> write_buf;
BlockOutputStreamPtr writer; BlockOutputStreamPtr writer;
}; };
} }
@ -177,8 +182,8 @@ BlockInputStreams IStorageURLBase::read(const Names & column_names,
getHeaderBlock(column_names), getHeaderBlock(column_names),
context, context,
max_block_size, max_block_size,
ConnectionTimeouts::getHTTPTimeouts(context)); ConnectionTimeouts::getHTTPTimeouts(context),
IStorage::chooseCompressionMethod(request_uri.toString(), compression_method));
auto column_defaults = getColumns().getDefaults(); auto column_defaults = getColumns().getDefaults();
if (column_defaults.empty()) if (column_defaults.empty())
@ -195,7 +200,9 @@ void IStorageURLBase::rename(const String & /*new_path_to_db*/, const String & n
BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const Context & /*context*/) BlockOutputStreamPtr IStorageURLBase::write(const ASTPtr & /*query*/, const Context & /*context*/)
{ {
return std::make_shared<StorageURLBlockOutputStream>( return std::make_shared<StorageURLBlockOutputStream>(
uri, format_name, getSampleBlock(), context_global, ConnectionTimeouts::getHTTPTimeouts(context_global)); uri, format_name, getSampleBlock(), context_global,
ConnectionTimeouts::getHTTPTimeouts(context_global),
IStorage::chooseCompressionMethod(uri.toString(), compression_method));
} }
void registerStorageURL(StorageFactory & factory) void registerStorageURL(StorageFactory & factory)
@ -204,9 +211,9 @@ void registerStorageURL(StorageFactory & factory)
{ {
ASTs & engine_args = args.engine_args; ASTs & engine_args = args.engine_args;
if (engine_args.size() != 2) if (engine_args.size() != 2 && engine_args.size() != 3)
throw Exception( throw Exception(
"Storage URL requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); "Storage URL requires 2 or 3 arguments: url, name of used format and optional compression method.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context); engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
@ -217,7 +224,19 @@ void registerStorageURL(StorageFactory & factory)
String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>(); String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
return StorageURL::create(uri, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context); String compression_method;
if (engine_args.size() == 3)
{
engine_args[2] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[2], args.local_context);
compression_method = engine_args[2]->as<ASTLiteral &>().value.safeGet<String>();
} else compression_method = "auto";
return StorageURL::create(
uri,
args.database_name, args.table_name,
format_name,
args.columns, args.constraints, args.context,
compression_method);
}); });
} }
} }

View File

@ -39,10 +39,12 @@ protected:
const std::string & table_name_, const std::string & table_name_,
const String & format_name_, const String & format_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_); const ConstraintsDescription & constraints_,
const String & compression_method_);
Poco::URI uri; Poco::URI uri;
const Context & context_global; const Context & context_global;
String compression_method;
private: private:
String format_name; String format_name;
@ -80,8 +82,9 @@ public:
const String & format_name_, const String & format_name_,
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
Context & context_) Context & context_,
: IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_) const String & compression_method_)
: IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_, compression_method_)
{ {
} }

View File

@ -66,7 +66,9 @@ BlockInputStreams StorageView::read(
current_inner_query = new_inner_query; current_inner_query = new_inner_query;
} }
res = InterpreterSelectWithUnionQuery(current_inner_query, context, {}, column_names).executeWithMultipleStreams(); QueryPipeline pipeline;
/// FIXME res may implicitly use some objects owned be pipeline, but them will be destructed after return
res = InterpreterSelectWithUnionQuery(current_inner_query, context, {}, column_names).executeWithMultipleStreams(pipeline);
/// It's expected that the columns read from storage are not constant. /// It's expected that the columns read from storage are not constant.
/// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery. /// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery.

View File

@ -7,6 +7,7 @@
#include <Poco/Util/AbstractConfiguration.h> #include <Poco/Util/AbstractConfiguration.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <IO/CompressionMethod.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h> #include <IO/ReadWriteBufferFromHTTP.h>
#include <Poco/File.h> #include <Poco/File.h>
@ -31,7 +32,7 @@ StorageXDBC::StorageXDBC(
const Context & context_, const Context & context_,
const BridgeHelperPtr bridge_helper_) const BridgeHelperPtr bridge_helper_)
/// Please add support for constraints as soon as StorageODBC or JDBC will support insertion. /// Please add support for constraints as soon as StorageODBC or JDBC will support insertion.
: IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_, ConstraintsDescription{}) : IStorageURLBase(Poco::URI(), context_, database_name_, table_name_, IXDBCBridgeHelper::DEFAULT_FORMAT, columns_, ConstraintsDescription{}, "" /* CompressionMethod */)
, bridge_helper(bridge_helper_) , bridge_helper(bridge_helper_)
, remote_database_name(remote_database_name_) , remote_database_name(remote_database_name_)
, remote_table_name(remote_table_name_) , remote_table_name(remote_table_name_)

View File

@ -21,7 +21,7 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns, const Context
for (const auto & pair : formats) for (const auto & pair : formats)
{ {
const auto & [format_name, creators] = pair; const auto & [format_name, creators] = pair;
UInt64 has_input_format(creators.inout_creator != nullptr || creators.input_processor_creator != nullptr); UInt64 has_input_format(creators.input_creator != nullptr || creators.input_processor_creator != nullptr);
UInt64 has_output_format(creators.output_creator != nullptr || creators.output_processor_creator != nullptr); UInt64 has_output_format(creators.output_creator != nullptr || creators.output_processor_creator != nullptr);
res_columns[0]->insert(format_name); res_columns[0]->insert(format_name);
res_columns[1]->insert(has_input_format); res_columns[1]->insert(has_input_format);

View File

@ -32,21 +32,27 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons
ASTs & args = args_func.at(0)->children; ASTs & args = args_func.at(0)->children;
if (args.size() != 3) if (args.size() != 3 && args.size() != 4)
throw Exception("Table function '" + getName() + "' requires exactly 3 arguments: filename, format and structure.", throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default auto).",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (size_t i = 0; i < 3; ++i) for (size_t i = 0; i < args.size(); ++i)
args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context); args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(args[i], context);
std::string filename = args[0]->as<ASTLiteral &>().value.safeGet<String>(); std::string filename = args[0]->as<ASTLiteral &>().value.safeGet<String>();
std::string format = args[1]->as<ASTLiteral &>().value.safeGet<String>(); std::string format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
std::string structure = args[2]->as<ASTLiteral &>().value.safeGet<String>(); std::string structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
std::string compression_method;
if (args.size() == 4)
{
compression_method = args[3]->as<ASTLiteral &>().value.safeGet<String>();
} else compression_method = "auto";
ColumnsDescription columns = parseColumnsListFromString(structure, context); ColumnsDescription columns = parseColumnsListFromString(structure, context);
/// Create table /// Create table
StoragePtr storage = getStorage(filename, format, columns, const_cast<Context &>(context), table_name); StoragePtr storage = getStorage(filename, format, columns, const_cast<Context &>(context), table_name, compression_method);
storage->startup(); storage->startup();

View File

@ -16,6 +16,6 @@ class ITableFunctionFileLike : public ITableFunction
private: private:
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override; StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
virtual StoragePtr getStorage( virtual StoragePtr getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const = 0; const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const = 0;
}; };
} }

View File

@ -6,7 +6,7 @@
namespace DB namespace DB
{ {
StoragePtr TableFunctionFile::getStorage( StoragePtr TableFunctionFile::getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const
{ {
return StorageFile::create(source, return StorageFile::create(source,
-1, -1,
@ -16,7 +16,8 @@ StoragePtr TableFunctionFile::getStorage(
format, format,
columns, columns,
ConstraintsDescription{}, ConstraintsDescription{},
global_context); global_context,
compression_method);
} }
void registerTableFunctionFile(TableFunctionFactory & factory) void registerTableFunctionFile(TableFunctionFactory & factory)

View File

@ -23,6 +23,6 @@ public:
private: private:
StoragePtr getStorage( StoragePtr getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override; const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const std::string & compression_method) const override;
}; };
} }

View File

@ -9,7 +9,7 @@
namespace DB namespace DB
{ {
StoragePtr TableFunctionHDFS::getStorage( StoragePtr TableFunctionHDFS::getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const
{ {
return StorageHDFS::create(source, return StorageHDFS::create(source,
getDatabaseName(), getDatabaseName(),
@ -17,7 +17,8 @@ StoragePtr TableFunctionHDFS::getStorage(
format, format,
columns, columns,
ConstraintsDescription{}, ConstraintsDescription{},
global_context); global_context,
compression_method);
} }
void registerTableFunctionHDFS(TableFunctionFactory & factory) void registerTableFunctionHDFS(TableFunctionFactory & factory)

View File

@ -24,7 +24,7 @@ public:
private: private:
StoragePtr getStorage( StoragePtr getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override; const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override;
}; };
} }

View File

@ -7,11 +7,11 @@ namespace DB
{ {
StoragePtr TableFunctionS3::getStorage( StoragePtr TableFunctionS3::getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const
{ {
Poco::URI uri(source); Poco::URI uri(source);
UInt64 min_upload_part_size = global_context.getSettingsRef().s3_min_upload_part_size; UInt64 min_upload_part_size = global_context.getSettingsRef().s3_min_upload_part_size;
return StorageS3::create(uri, getDatabaseName(), table_name, format, min_upload_part_size, columns, ConstraintsDescription{}, global_context); return StorageS3::create(uri, getDatabaseName(), table_name, format, min_upload_part_size, columns, ConstraintsDescription{}, global_context, compression_method);
} }
void registerTableFunctionS3(TableFunctionFactory & factory) void registerTableFunctionS3(TableFunctionFactory & factory)

View File

@ -25,7 +25,8 @@ private:
const String & format, const String & format,
const ColumnsDescription & columns, const ColumnsDescription & columns,
Context & global_context, Context & global_context,
const std::string & table_name) const override; const std::string & table_name,
const String & compression_method) const override;
}; };
} }

View File

@ -8,10 +8,10 @@
namespace DB namespace DB
{ {
StoragePtr TableFunctionURL::getStorage( StoragePtr TableFunctionURL::getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const
{ {
Poco::URI uri(source); Poco::URI uri(source);
return StorageURL::create(uri, getDatabaseName(), table_name, format, columns, ConstraintsDescription{}, global_context); return StorageURL::create(uri, getDatabaseName(), table_name, format, columns, ConstraintsDescription{}, global_context, compression_method);
} }
void registerTableFunctionURL(TableFunctionFactory & factory) void registerTableFunctionURL(TableFunctionFactory & factory)

View File

@ -19,6 +19,6 @@ public:
private: private:
StoragePtr getStorage( StoragePtr getStorage(
const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name) const override; const String & source, const String & format, const ColumnsDescription & columns, Context & global_context, const std::string & table_name, const String & compression_method) const override;
}; };
} }

View File

@ -109,13 +109,13 @@ However, these libraries are optional and ClickHouse can well be built without t
# C++ Compiler # C++ Compiler
Compilers GCC starting from version 7 and Clang version 7 or above are supported for building ClickHouse. Compilers GCC starting from version 9 and Clang version 8 or above are supported for building ClickHouse.
Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations. Official Yandex builds currently use GCC because it generates machine code of slightly better performance (yielding a difference of up to several percent according to our benchmarks). And Clang is more convenient for development usually. Though, our continuous integration (CI) platform runs checks for about a dozen of build combinations.
To install GCC on Ubuntu run: `sudo apt install gcc g++` To install GCC on Ubuntu run: `sudo apt install gcc g++`
Check the version of gcc: `gcc --version`. If it is below 7, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-7. Check the version of gcc: `gcc --version`. If it is below 9, then follow the instruction here: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9.
To install GCC on Mac OS X run: `brew install gcc`. To install GCC on Mac OS X run: `brew install gcc`.
@ -131,9 +131,9 @@ cd build
``` ```
You can have several different directories (build_release, build_debug, etc.) for different types of build. You can have several different directories (build_release, build_debug, etc.) for different types of build.
While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 7 gcc compiler in this example). While inside the `build` directory, configure your build by running CMake. Before the first run you need to define environment variables that specify compiler (version 9 gcc compiler in this example).
``` ```
export CC=gcc-7 CXX=g++-7 export CC=gcc-9 CXX=g++-9
cmake .. cmake ..
``` ```
The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building. The `CC` variable specifies the compiler for C (short for C Compiler), and `CXX` variable instructs which C++ compiler is to be used for building.
@ -265,4 +265,4 @@ Testing will commence as soon as Yandex employees label your PR with a tag "can
The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the "Details" link next to "ClickHouse build check" entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear). The system will prepare ClickHouse binary builds for your pull request individually. To retrieve these builds click the "Details" link next to "ClickHouse build check" entry in the list of checks. There you will find direct links to the built .deb packages of ClickHouse which you can deploy even on your production servers (if you have no fear).
Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways. Most probably some of the builds will fail at first times. This is due to the fact that we check builds both with gcc as well as with clang, with almost all of existing warnings (always with the `-Werror` flag) enabled for clang. On that same page you can find all of the build logs so that you do not have to build ClickHouse in all of the possible ways.

View File

@ -109,13 +109,13 @@ Mac OS X: `brew install icu4c readline`
# Компилятор C++ # Компилятор C++
В качестве компилятора C++ поддерживается GCC начиная с версии 7 или Clang начиная с версии 7. В качестве компилятора C++ поддерживается GCC начиная с версии 9 или Clang начиная с версии 8.
Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки. Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки.
Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`. Для установки GCC под Ubuntu, выполните: `sudo apt install gcc g++`.
Проверьте версию gcc: `gcc --version`. Если версия меньше 7, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-7 Проверьте версию gcc: `gcc --version`. Если версия меньше 9, то следуйте инструкции: https://clickhouse.yandex/docs/en/development/build/#install-gcc-9
Для установки GCC под Mac OS X, выполните `brew install gcc`. Для установки GCC под Mac OS X, выполните `brew install gcc`.
@ -132,9 +132,9 @@ cd build
Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки. Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки.
Находясь в директории build, выполните конфигурацию сборки с помощью CMake. Находясь в директории build, выполните конфигурацию сборки с помощью CMake.
Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 7). Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9).
``` ```
export CC=gcc-7 CXX=g++-7 export CC=gcc-9 CXX=g++-9
cmake .. cmake ..
``` ```
Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву). Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву).

View File

@ -29,6 +29,9 @@ from .hdfs_api import HDFSApi
HELPERS_DIR = p.dirname(__file__) HELPERS_DIR = p.dirname(__file__)
DEFAULT_ENV_NAME = 'env_file' DEFAULT_ENV_NAME = 'env_file'
SANITIZER_SIGN = "=================="
def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME): def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
full_path = os.path.join(path, fname) full_path = os.path.join(path, fname)
with open(full_path, 'w') as f: with open(full_path, 'w') as f:
@ -81,6 +84,7 @@ class ClickHouseCluster:
# docker-compose removes everything non-alphanumeric from project names so we do it too. # docker-compose removes everything non-alphanumeric from project names so we do it too.
self.project_name = re.sub(r'[^a-z0-9]', '', self.project_name.lower()) self.project_name = re.sub(r'[^a-z0-9]', '', self.project_name.lower())
self.instances_dir = p.join(self.base_dir, '_instances' + ('' if not self.name else '_' + self.name)) self.instances_dir = p.join(self.base_dir, '_instances' + ('' if not self.name else '_' + self.name))
self.docker_logs_path = p.join(self.instances_dir, 'docker.log')
custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST') custom_dockerd_host = custom_dockerd_host or os.environ.get('CLICKHOUSE_TESTS_DOCKERD_HOST')
self.docker_api_version = os.environ.get("DOCKER_API_VERSION") self.docker_api_version = os.environ.get("DOCKER_API_VERSION")
@ -395,6 +399,15 @@ class ClickHouseCluster:
def shutdown(self, kill=True): def shutdown(self, kill=True):
sanitizer_assert_instance = None
with open(self.docker_logs_path, "w+") as f:
subprocess.check_call(self.base_cmd + ['logs'], stdout=f)
f.seek(0)
for line in f:
if SANITIZER_SIGN in line:
sanitizer_assert_instance = line.split('|')[0].strip()
break
if kill: if kill:
subprocess_check_call(self.base_cmd + ['kill']) subprocess_check_call(self.base_cmd + ['kill'])
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans']) subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
@ -407,6 +420,9 @@ class ClickHouseCluster:
instance.ip_address = None instance.ip_address = None
instance.client = None instance.client = None
if sanitizer_assert_instance is not None:
raise Exception("Sanitizer assert found in {} for instance {}".format(self.docker_logs_path, sanitizer_assert_instance))
def open_bash_shell(self, instance_name): def open_bash_shell(self, instance_name):
os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash'])) os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash']))

View File

@ -1,4 +1,6 @@
#-*- coding: utf-8 -*- #-*- coding: utf-8 -*-
import StringIO
import gzip
import requests import requests
import subprocess import subprocess
from tempfile import NamedTemporaryFile from tempfile import NamedTemporaryFile
@ -19,7 +21,7 @@ class HDFSApi(object):
if response_data.status_code != 200: if response_data.status_code != 200:
response_data.raise_for_status() response_data.raise_for_status()
return response_data.text return response_data.content
# Requests can't put file # Requests can't put file
def _curl_to_put(self, filename, path, params): def _curl_to_put(self, filename, path, params):
@ -44,3 +46,12 @@ class HDFSApi(object):
output = self._curl_to_put(fpath, path, additional_params) output = self._curl_to_put(fpath, path, additional_params)
if "201 Created" not in output: if "201 Created" not in output:
raise Exception("Can't create file on hdfs:\n {}".format(output)) raise Exception("Can't create file on hdfs:\n {}".format(output))
def write_gzip_data(self, path, content):
out = StringIO.StringIO()
with gzip.GzipFile(fileobj=out, mode="w") as f:
f.write(content)
self.write_data(path, out.getvalue())
def read_gzip_data(self, path):
return gzip.GzipFile(fileobj=StringIO.StringIO(self.read_data(path))).read()

View File

@ -34,7 +34,7 @@ RUN apt-get update \
ENV TZ=Europe/Moscow ENV TZ=Europe/Moscow
RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone RUN ln -snf /usr/share/zoneinfo/$TZ /etc/localtime && echo $TZ > /etc/timezone
RUN pip install pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout RUN pip install urllib3==1.23 pytest docker-compose==1.22.0 docker dicttoxml kazoo PyMySQL psycopg2==2.7.5 pymongo tzlocal kafka-python protobuf redis aerospike pytest-timeout minio
ENV DOCKER_CHANNEL stable ENV DOCKER_CHANNEL stable
ENV DOCKER_VERSION 17.09.1-ce ENV DOCKER_VERSION 17.09.1-ce

View File

@ -112,4 +112,10 @@ def test_deep_structure(start_cluster):
'''.format(pattern)) == '{}\n'.format(value) '''.format(pattern)) == '{}\n'.format(value)
assert node.query(''' assert node.query('''
select count(*) from file('{}{}', 'TSV', 'text String, number Float64') select count(*) from file('{}{}', 'TSV', 'text String, number Float64')
'''.format(path_to_userfiles_from_defaut_config, pattern)) == '{}\n'.format(value) '''.format(path_to_userfiles_from_defaut_config, pattern)) == '{}\n'.format(value)
def test_table_function(start_cluster):
node.exec_in_container(['bash', '-c', 'mkdir -p {}some/path/to/'.format(path_to_userfiles_from_defaut_config)])
node.exec_in_container(['bash', '-c', 'touch {}some/path/to/data.CSV'.format(path_to_userfiles_from_defaut_config)])
node.query("insert into table function file('some/path/to/data.CSV', CSV, 'n UInt8, s String') select number, concat('str_', toString(number)) from numbers(100000)")
assert node.query("select count() from file('some/path/to/data.CSV', CSV, 'n UInt8, s String')").rstrip() == '100000'

View File

@ -68,6 +68,14 @@ CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE
shard1.query(''' shard1.query('''
CREATE TABLE low_cardinality_all (d Date, x UInt32, s LowCardinality(String)) ENGINE = Distributed('shard_with_low_cardinality', 'default', 'low_cardinality', sipHash64(s))''') CREATE TABLE low_cardinality_all (d Date, x UInt32, s LowCardinality(String)) ENGINE = Distributed('shard_with_low_cardinality', 'default', 'low_cardinality', sipHash64(s))''')
node1.query('''
CREATE TABLE table_function (n UInt8, s String) ENGINE = MergeTree() ORDER BY n''')
node2.query('''
CREATE TABLE table_function (n UInt8, s String) ENGINE = MergeTree() ORDER BY n''')
yield cluster yield cluster
finally: finally:
@ -189,3 +197,7 @@ def test_inserts_low_cardinality(started_cluster):
instance.query("INSERT INTO low_cardinality_all (d,x,s) VALUES ('2018-11-12',1,'123')") instance.query("INSERT INTO low_cardinality_all (d,x,s) VALUES ('2018-11-12',1,'123')")
time.sleep(0.5) time.sleep(0.5)
assert instance.query("SELECT count(*) FROM low_cardinality_all").strip() == '1' assert instance.query("SELECT count(*) FROM low_cardinality_all").strip() == '1'
def test_table_function(started_cluster):
node1.query("insert into table function cluster('shard_with_local_replica', 'default', 'table_function') select number, concat('str_', toString(number)) from numbers(100000)")
assert node1.query("select count() from cluster('shard_with_local_replica', 'default', 'table_function')").rstrip() == '100000'

View File

@ -108,10 +108,28 @@ def test_insert_select_with_mysql_style_table(started_cluster):
assert node1.query("SELECT sum(`float`) FROM `clickhouse_mysql`.`{}`".format('test_mysql\`_style_table')).rstrip() == '30000' assert node1.query("SELECT sum(`float`) FROM `clickhouse_mysql`.`{}`".format('test_mysql\`_style_table')).rstrip() == '30000'
mysql_connection.close() mysql_connection.close()
def test_table_function(started_cluster):
mysql_connection = get_mysql_conn()
create_normal_mysql_table(mysql_connection, 'table_function')
table_function = get_mysql_table_function_expr('table_function')
assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '0'
node1.query("INSERT INTO {} (id, name, money) select number, concat('name_', toString(number)), 3 from numbers(10000)".format('TABLE FUNCTION ' + table_function))
assert node1.query("SELECT count() FROM {}".format(table_function)).rstrip() == '10000'
assert node1.query("SELECT sum(c) FROM ("
"SELECT count() as c FROM {} WHERE id % 3 == 0"
" UNION ALL SELECT count() as c FROM {} WHERE id % 3 == 1"
" UNION ALL SELECT count() as c FROM {} WHERE id % 3 == 2)".format(table_function, table_function, table_function)).rstrip() == '10000'
assert node1.query("SELECT sum(`money`) FROM {}".format(table_function)).rstrip() == '30000'
mysql_connection.close()
def get_mysql_conn(): def get_mysql_conn():
conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308) conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308)
return conn return conn
def get_mysql_table_function_expr(table_name):
return "mysql('mysql1:3306', 'clickhouse', '{}', 'root', 'clickhouse')".format(table_name)
def create_mysql_db(conn, name): def create_mysql_db(conn, name):
with conn.cursor() as cursor: with conn.cursor() as cursor:
cursor.execute( cursor.execute(

View File

@ -133,4 +133,56 @@ def test_globs_in_read_table(started_cluster):
("?", 0)] ("?", 0)]
for pattern, value in test_requests: for pattern, value in test_requests:
assert node1.query("select * from hdfs('hdfs://hdfs1:9000" + globs_dir + pattern + "', 'TSV', 'id UInt64, text String, number Float64')") == value * some_data assert node1.query("select * from hdfs('hdfs://hdfs1:9000" + globs_dir + pattern + "', 'TSV', 'id UInt64, text String, number Float64')") == value * some_data
def test_read_write_gzip_table(started_cluster):
hdfs_api = HDFSApi("root")
data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n"
hdfs_api.write_gzip_data("/simple_table_function.gz", data)
assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data
assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64')") == data
def test_read_write_gzip_table_with_parameter_gzip(started_cluster):
hdfs_api = HDFSApi("root")
data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n"
hdfs_api.write_gzip_data("/simple_table_function", data)
assert hdfs_api.read_gzip_data("/simple_table_function") == data
assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64', 'gzip')") == data
def test_read_write_table_with_parameter_none(started_cluster):
hdfs_api = HDFSApi("root")
data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n"
hdfs_api.write_data("/simple_table_function.gz", data)
assert hdfs_api.read_data("/simple_table_function.gz") == data
assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'none')") == data
def test_read_write_gzip_table_with_parameter_auto_gz(started_cluster):
hdfs_api = HDFSApi("root")
data = "1\tHello Jessica\t555.222\n2\tI rolled a joint\t777.333\n"
hdfs_api.write_gzip_data("/simple_table_function.gz", data)
assert hdfs_api.read_gzip_data("/simple_table_function.gz") == data
assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function.gz', 'TSV', 'id UInt64, text String, number Float64', 'auto')") == data
def test_write_gz_storage(started_cluster):
hdfs_api = HDFSApi("root")
node1.query("create table GZHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage.gz', 'TSV')")
node1.query("insert into GZHDFSStorage values (1, 'Mark', 72.53)")
assert hdfs_api.read_gzip_data("/storage.gz") == "1\tMark\t72.53\n"
assert node1.query("select * from GZHDFSStorage") == "1\tMark\t72.53\n"
def test_write_gzip_storage(started_cluster):
hdfs_api = HDFSApi("root")
node1.query("create table GZIPHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/gzip_storage', 'TSV', 'gzip')")
node1.query("insert into GZIPHDFSStorage values (1, 'Mark', 72.53)")
assert hdfs_api.read_gzip_data("/gzip_storage") == "1\tMark\t72.53\n"
assert node1.query("select * from GZIPHDFSStorage") == "1\tMark\t72.53\n"

View File

@ -0,0 +1,11 @@
<yandex>
<logger>
<level>trace</level>
<log>/var/log/clickhouse-server/log.log</log>
<errorlog>/var/log/clickhouse-server/log.err.log</errorlog>
<size>1000M</size>
<count>10</count>
<stderr>/var/log/clickhouse-server/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/stdout.log</stdout>
</logger>
</yandex>

View File

@ -31,7 +31,7 @@ import kafka_pb2
cluster = ClickHouseCluster(__file__) cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance('instance', instance = cluster.add_instance('instance',
config_dir='configs', config_dir='configs',
main_configs=['configs/kafka.xml'], main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ],
with_kafka=True, with_kafka=True,
clickhouse_path_dir='clickhouse_path') clickhouse_path_dir='clickhouse_path')
kafka_id = '' kafka_id = ''
@ -557,7 +557,7 @@ def test_kafka_insert(kafka_cluster):
kafka_check_result(result, True) kafka_check_result(result, True)
@pytest.mark.timeout(180) @pytest.mark.timeout(240)
def test_kafka_produce_consume(kafka_cluster): def test_kafka_produce_consume(kafka_cluster):
instance.query(''' instance.query('''
DROP TABLE IF EXISTS test.view; DROP TABLE IF EXISTS test.view;

View File

@ -0,0 +1,48 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS a;"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS b;"
$CLICKHOUSE_CLIENT --query="CREATE TABLE a (x UInt64) ENGINE = Memory;"
$CLICKHOUSE_CLIENT --query="CREATE TABLE b (x UInt64) ENGINE = Memory;"
function thread1()
{
while true; do
seq 1 11000000 | $CLICKHOUSE_CLIENT --query_id=11 --query="INSERT INTO a(x) FORMAT TSV"
sleep 1
$CLICKHOUSE_CLIENT --query="kill query where query_id='22'" SYNC
done
}
function thread2()
{
while true; do
seq 1 11000000 | $CLICKHOUSE_CLIENT --query_id=22 --query="INSERT INTO b(x) FORMAT TSV"
sleep 1
$CLICKHOUSE_CLIENT --query="kill query where query_id='11'" SYNC
done
}
# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout
export -f thread1;
export -f thread2;
TIMEOUT=20
timeout $TIMEOUT bash -c thread1 2>&1 > /dev/null &
timeout $TIMEOUT bash -c thread2 2>&1 > /dev/null &
wait
echo OK
$CLICKHOUSE_CLIENT --query "DROP TABLE a"
$CLICKHOUSE_CLIENT --query "DROP TABLE b"

View File

@ -2,7 +2,7 @@ drop table if exists test_table_hdfs_syntax
; ;
create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('') create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('')
; -- { serverError 42 } ; -- { serverError 42 }
create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('','','') create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('','','', '')
; -- { serverError 42 } ; -- { serverError 42 }
drop table if exists test_table_hdfs_syntax drop table if exists test_table_hdfs_syntax
; ;

View File

@ -2,7 +2,7 @@ drop table if exists test_table_s3_syntax
; ;
create table test_table_s3_syntax (id UInt32) ENGINE = S3('') create table test_table_s3_syntax (id UInt32) ENGINE = S3('')
; -- { serverError 42 } ; -- { serverError 42 }
create table test_table_s3_syntax (id UInt32) ENGINE = S3('','','') create table test_table_s3_syntax (id UInt32) ENGINE = S3('','','','')
; -- { serverError 42 } ; -- { serverError 42 }
drop table if exists test_table_s3_syntax drop table if exists test_table_s3_syntax
; ;

View File

@ -2,7 +2,7 @@ drop table if exists test_table_url_syntax
; ;
create table test_table_url_syntax (id UInt32) ENGINE = URL('') create table test_table_url_syntax (id UInt32) ENGINE = URL('')
; -- { serverError 42 } ; -- { serverError 42 }
create table test_table_url_syntax (id UInt32) ENGINE = URL('','','') create table test_table_url_syntax (id UInt32) ENGINE = URL('','','','')
; -- { serverError 42 } ; -- { serverError 42 }
drop table if exists test_table_url_syntax drop table if exists test_table_url_syntax
; ;

View File

@ -0,0 +1 @@
-160.32605 37.705841

View File

@ -0,0 +1,16 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS values_floats"
${CLICKHOUSE_CLIENT} --query="CREATE TABLE values_floats (a Float32, b Float64) ENGINE = Memory"
${CLICKHOUSE_CLIENT} --query="SELECT '(-160.32605134916085,37.70584056842162),' FROM numbers(1000000)" | ${CLICKHOUSE_CLIENT} --query="INSERT INTO values_floats FORMAT Values"
${CLICKHOUSE_CLIENT} --query="SELECT DISTINCT round(a, 6), round(b, 6) FROM values_floats"
${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS values_floats"

View File

@ -48,4 +48,4 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
``` ```
## Queries ## Queries
Examples of queries to these tables (they are named `test.hits` and `test.visits`) can be found among [stateful tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) and in some [performance tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance/test_hits) of ClickHouse. Examples of queries to these tables (they are named `test.hits` and `test.visits`) can be found among [stateful tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) and in some [performance tests](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance) of ClickHouse.

View File

@ -753,12 +753,12 @@ This section contains the following parameters:
For example: For example:
```xml ```xml
<node index="1"> <node index="1">
<host>example_host</host> <host>example_host</host>
<port>2181</port> <port>2181</port>
</node> </node>
``` ```
The `index` attribute specifies the node order when trying to connect to the ZooKeeper cluster. The `index` attribute specifies the node order when trying to connect to the ZooKeeper cluster.

View File

@ -994,4 +994,18 @@ Lower values mean higher priority. Threads with low `nice` priority values are e
Default value: 0. Default value: 0.
## input_format_parallel_parsing
- Type: bool
- Default value: True
Enable order-preserving parallel parsing of data formats. Supported only for TSV format.
## min_chunk_bytes_for_parallel_parsing
- Type: unsigned int
- Default value: 1 MiB
The minimum chunk size in bytes, which each thread will parse in parallel.
[Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) <!-- hide --> [Original article](https://clickhouse.yandex/docs/en/operations/settings/settings/) <!-- hide -->

View File

@ -189,7 +189,8 @@ The following operations with [partitions](../operations/table_engines/custom_pa
- [DETACH PARTITION](#alter_detach-partition) Moves a partition to the `detached` directory and forget it. - [DETACH PARTITION](#alter_detach-partition) Moves a partition to the `detached` directory and forget it.
- [DROP PARTITION](#alter_drop-partition) Deletes a partition. - [DROP PARTITION](#alter_drop-partition) Deletes a partition.
- [ATTACH PART|PARTITION](#alter_attach-partition) Adds a part or partition from the `detached` directory to the table. - [ATTACH PART|PARTITION](#alter_attach-partition) Adds a part or partition from the `detached` directory to the table.
- [REPLACE PARTITION](#alter_replace-partition) - Copies the data partition from one table to another. - [ATTACH PARTITION FROM](#alter_attach-partition-from) Copies the data partition from one table to another and adds.
- [REPLACE PARTITION](#alter_replace-partition) - Copies the data partition from one table to another and replaces.
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Resets the value of a specified column in a partition. - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) - Resets the value of a specified column in a partition.
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the specified secondary index in a partition. - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - Resets the specified secondary index in a partition.
- [FREEZE PARTITION](#alter_freeze-partition) Creates a backup of a partition. - [FREEZE PARTITION](#alter_freeze-partition) Creates a backup of a partition.
@ -256,13 +257,26 @@ This query is replicated. Each replica checks whether there is data in the `deta
So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas. So you can put data to the `detached` directory on one replica, and use the `ALTER ... ATTACH` query to add it to the table on all replicas.
#### ATTACH PARTITION FROM {#alter_attach-partition-from}
```sql
ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1
```
This query copies the data partition from the `table1` to `table2` adds data to exsisting in the `table2`. Note that data won't be deleted from `table1`.
For the query to run successfully, the following conditions must be met:
- Both tables must have the same structure.
- Both tables must have the same partition key.
#### REPLACE PARTITION {#alter_replace-partition} #### REPLACE PARTITION {#alter_replace-partition}
```sql ```sql
ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1
``` ```
This query copies the data partition from the `table1` to `table2`. Note that data won't be deleted from `table1`. This query copies the data partition from the `table1` to `table2` and replaces existing partition in the `table2`. Note that data won't be deleted from `table1`.
For the query to run successfully, the following conditions must be met: For the query to run successfully, the following conditions must be met:

View File

@ -772,6 +772,22 @@ SELECT arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10])
└─────────────────────────────────────────────────────────────┘ └─────────────────────────────────────────────────────────────┘
``` ```
## arrayFlatten(arr) {#array_functions-arrayflatten}
The `arrayFlatten` (or `flatten` alias) method will collapse the elements of an array to create a single array.
Example:
```sql
SELECT arrayFlatten([[1, 2, 3], [4, 5]])
```
```text
┌─arrayFlatten([[1, 2, 3], [4, 5]])─┐
│ [1,2,3,4,5] │
└───────────────────────────────────┘
```
## arrayReverse(arr) {#array_functions-arrayreverse} ## arrayReverse(arr) {#array_functions-arrayreverse}
Returns an array of the same size as the original array containing the elements in reverse order. Returns an array of the same size as the original array containing the elements in reverse order.

View File

@ -61,10 +61,10 @@ However, you can delete old data using `ALTER TABLE ... DROP PARTITION`.
### Performance Considerations ### Performance Considerations
`INSERT` sorts the input data by primary key and splits them into partitions by month. If you insert data for mixed months, it can significantly reduce the performance of the `INSERT` query. To avoid this: `INSERT` sorts the input data by primary key and splits them into partitions by a partition key. If you insert data into several partitions at once, it can significantly reduce the performance of the `INSERT` query. To avoid this:
- Add data in fairly large batches, such as 100,000 rows at a time. - Add data in fairly large batches, such as 100,000 rows at a time.
- Group data by month before uploading it to ClickHouse. - Group data by a partition key before uploading it to ClickHouse.
Performance will not decrease if: Performance will not decrease if:

View File

@ -1,8 +1,8 @@
# Планы разработки ClickHouse 2020. # Планы разработки ClickHouse 2020.
Здесь собраны важные задачи на 2020 год. Также многие из них присутствуют в GitHub Issues или (редко) в Яндекс.Трекере, а также (очень редко) в Яндекс.Goals. Мы предпочитаем использовать Яндекс.Трекер только для секретных задач, а все публичные задачи размещать в GitHub Issues для того, чтобы сделать разработку более открытой наружу. В любом случае, для работы над задачами, таск-трекеры уступают в удобстве plaintext файлу. Здесь собраны важные задачи на 2020 год. Многие из них присутствуют в GitHub Issues. Данный текст следует рассматривать как рабочий черновик со сводкой и кратким описанием задач, ссылками и материалами для быстрого доступа на одной странице. Классификация задач условная.
Классификация задач условная и за её основу взята известная [классификация животных](https://ru.wikipedia.org/wiki/%D0%9A%D0%BB%D0%B0%D1%81%D1%81%D0%B8%D1%84%D0%B8%D0%BA%D0%B0%D1%86%D0%B8%D1%8F_%D0%B6%D0%B8%D0%B2%D0%BE%D1%82%D0%BD%D1%8B%D1%85_(%D0%91%D0%BE%D1%80%D1%85%D0%B5%D1%81)). Так как ClickHouse - open-source продукт, мы хотим, чтобы рабочий процесс был также максимально открытым. В связи с этим, вам следует ожидать наличия на данной странице несколько большего количества деталей описания рабочего процесса, чем вы могли бы предположить - настолько близко, насколько рабочий процесс видят разработчики. Так как неотъемлимой частью процесса разработки является исправление недостатков продукта и улучшение качества кода, на данной странице вы найдёте весьма подробные описания таких деталей. Для других open-source продуктов такой подход к разработке обычно нехарактерен. Благодаря тому, что для каждой задачи указаны её зависимости, вы сможете понять, какие подготовительные работы требуются, что позволяет более точно понимать сроки реализации.
## 1. Хранение данных, индексация. ## 1. Хранение данных, индексация.
@ -14,12 +14,16 @@
Задача "normalized z-Order curve" в перспективе может быть полезна для БК и Метрики, так как позволяет смешивать OrderID и PageID и избежать дублирования данных. Задача "normalized z-Order curve" в перспективе может быть полезна для БК и Метрики, так как позволяет смешивать OrderID и PageID и избежать дублирования данных.
В задаче также вводится способ индексации путём обращения функции нескольких аргументов на интервале, что имеет смысл для дальнейшего развития. В задаче также вводится способ индексации путём обращения функции нескольких аргументов на интервале, что имеет смысл для дальнейшего развития.
Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь доделывает [Ольга Хвостикова](https://github.com/stavrolia), но задача прокрастинирована из-за задачи 25.9. Сейчас оказалось, что задача 25.9 тоже не готова. Будем надеятся на реализацию к концу ноября. Впрочем, [Андрей Чулков](https://github.com/achulkov2) скоро выйдет на работу стажёром на пол ставки и сможет помочь её доделать. Изначально делал [Андрей Чулков](https://github.com/achulkov2), ВШЭ, теперь доделывает [Ольга Хвостикова](https://github.com/stavrolia), но сроки немного сдвинуты из-за задачи 25.9. Будем надеятся на реализацию к концу ноября. Впрочем, [Андрей Чулков](https://github.com/achulkov2) скоро сможет помочь её доделать.
### 1.2. Wait-free каталог баз данных. ### 1.2. Wait-free каталог баз данных.
Делает [Александр Токмаков](https://github.com/tavplubix), первый рабочий вариант в декабре 2019. Нужно для DataLens и Яндекс.Метрики. Делает [Александр Токмаков](https://github.com/tavplubix), первый рабочий вариант в декабре 2019. Нужно для DataLens и Яндекс.Метрики.
Манипуляции с каталогом баз данных: запросы CREATE TABLE, DROP TABLE, RENAME TABLE и DATABASE, требуют синхронизации с помощью блокировок. Эта синхронизация становится весьма сложной, так как на неё полагается много внутренних структур данных.
Предлагается реализовать альтернативный подход, в котором таблицы и базы данных являются всего лишь ссылками на persistent объекты. Подробное описание задачи: https://github.com/yandex/ClickHouse/issues/6787
### 1.3. Неблокирующие ALTER. ### 1.3. Неблокирующие ALTER.
И полностью immutable куски. Делает [Александр Сапин](https://github.com/alesapin). Готов приступить к задаче в конце ноября 2019. Нужно для Яндекс.Метрики. И полностью immutable куски. Делает [Александр Сапин](https://github.com/alesapin). Готов приступить к задаче в конце ноября 2019. Нужно для Яндекс.Метрики.
@ -36,6 +40,12 @@
Делает [Антон Попов](https://github.com/CurtizJ), первый рабочий вариант в декабре. Пререквизит чтобы снизить сложность мелких INSERT, что в свою очередь нужно для 1.12, иначе задача 1.12 не сможет нормально работать. Особенно нужно для Яндекс.Облака. Делает [Антон Попов](https://github.com/CurtizJ), первый рабочий вариант в декабре. Пререквизит чтобы снизить сложность мелких INSERT, что в свою очередь нужно для 1.12, иначе задача 1.12 не сможет нормально работать. Особенно нужно для Яндекс.Облака.
Данные в таблицах типа MergeTree в ClickHouse хранятся в виде набора независимых "кусков". Внутри куска, каждый столбец, а также индекс, хранится в отдельных файлах. Это сделано для возможности быстрых манипуляций со столбцами (пример - запрос ALTER DROP COLUMN). При вставке данных (INSERT), создаётся новый кусок. Для таблиц с большим количеством столбцов, запросы INSERT с маленьким количеством строк являются неэффективными, так как требуют создания большого количества файлов в файловой системе. Это является врождённой особенностью ClickHouse - одной из первой проблем, с которыми сталкиваются пользователи. Пользователям приходится буферизовывать данные и собирать их в более крупные пачки перед вставкой в ClickHouse.
Для смягчения эффекта от этой проблемы, в ClickHouse существуют таблицы типа Buffer. Они накапливают данные в оперативке перед записью в другую таблицу. Впрочем, таблицы Buffer не являются полноценным решением проблемы из-за: - наличия блокировок при вставке; - переупорядочивание вставляемых данных; - неатомарность перекладывания данных из Buffer в результирующую таблицу.
Вместо этого предлагается разрешить кускам таблиц типа MergeTree располагать данные в разных форматах. А именно: - в оперативной памяти; - на диске со всеми столбцами в одном файле; - на диске со столбцами в отдельных файлах: в зависимости от размера куска и прошедшего времени. Для размещения кусков в оперативной памяти, придётся также реализовать опциональную поддержку write-ahead log с настраиваемыми правилами по сбросу на диск. Это позволит избавиться от проблем с мелкими вставками для MergeTree таблиц. Для ReplicatedMergeTree таблиц, это решит проблему лишь частично.
### 1.7. Буферизация и WAL в MergeTree. ### 1.7. Буферизация и WAL в MergeTree.
Требует 1.6. Требует 1.6.
@ -48,18 +58,44 @@
В очереди. В очереди.
Сейчас пользователь может задать в таблице выражение, которое определяет, сколько времени хранятся данные. Обычно это выражение задаётся относительно значения столбца с датой - например: удалять данные через три месяца. https://clickhouse.yandex/docs/ru/operations/table_engines/mergetree/#table_engine-mergetree-ttl
Это может быть задано для всей таблицы (тогда строки целиком удаляются после указанного времени) или для отдельных столбцов (тогда данные столбца физически удаляются с диска, а строки в таблице остаются; при чтении значений столбца, они читаются как значения по-умолчанию).
Но пользователи также хотят более продвинутый вариант этой функциональности: не удалять строки или столбцы целиком, а прореживать их - оставлять меньшее количество строк.
И тут есть несколько вариантов:
1. По прошествии времени, оставлять каждую N-ую строку.
2. По прошествии времени, выполнять агрегацию данных, заменяя значения некоторых столбцов на значения агрегатных функций от множества значений в нескольких строках.
Пункт 1 не представляет интереса, так как уже реализован с помощью TTL выражений для удаления данных. В качестве этого выражения можно прописать, например, cityHash64(*) % 10 = 0 ? now() : event_time + INTERVAL 3 MONTH. Правда как-то неудобно получается.
А вот пункт 2 требуется продумать. Не очевидно даже, какой лучше использовать синтаксис для этого при создании таблицы. Но мы придумаем - сразу видно несколько вариантов.
Частный случай такой задачи уже есть в https://clickhouse.yandex/docs/ru/operations/table_engines/graphitemergetree/ Но это было сделано для конкретной задачи. А надо обобщить.
### 1.10. Пережатие старых данных в фоне. ### 1.10. Пережатие старых данных в фоне.
Будет делать Кирилл Барухов, ВШЭ, экспериментальная реализация к весне 2020. Нужно для Яндекс.Метрики. Будет делать Кирилл Барухов, ВШЭ, экспериментальная реализация к весне 2020. Нужно для Яндекс.Метрики.
Алгоритмы сжатия типа LZ77 позволяют потратить больше времени на сжатие данных, чтобы сжать данные сильнее, но при этом без проигрыша по скорости разжатия данных. В частности, этим свойством обладает LZ4 и ZSTD, которые используются в ClickHouse. Это позволяет использовать свободные ресурсы CPU, когда сервер не нагружен, для пережатия данных, чтобы данные занимали меньше места на дисках, и при этом сохранить или даже улучшить скорость обработки запросов.
В то же время, ClickHouse обычно используется для "импульсного" сценария нагрузки. Запрос от пользователя обрабатывается максимально быстро, используя все ресурсы CPU, но в среднем по времени, сервер недостаточно нагружен.
Предлагается добавить в ClickHouse настройки по пережатию данных и фоновые потоки, выполняющие эту задачу.
### 1.11. Виртуальная файловая система. ### 1.11. Виртуальная файловая система.
Нужно для Яндекс.Облака. Делает Александр Бурмак, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс. Нужно для Яндекс.Облака. Делает Александр, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс.
ClickHouse использует для хранения данных локальную файловую систему. Существует сценарий работы, в котором размещение старых (архивных) данных было бы выгодно на удалённой файловой системе. Если файловая система POSIX совместимая, то это не составляет проблем: ClickHouse успешно работает с Ceph, GlusterFS, MooseFS. Также востребованным является сценарий использования S3 (из-за доступности в облаке) или HDFS (для интеграции с Hadoop). Но эти файловые системы не являются POSIX совместимыми. Хотя для них существуют FUSE драйверы, но скорость работы сильно страдает и поддержка неполная.
ClickHouse использует небольшое подмножество функций ФС, но в то же время, и некоторые специфические части: симлинки и хардлинки, O_DIRECT. Предлагается выделить всё взаимодействие с файловой системой в отдельный интерфейс.
### 1.12. Экспериментальная реализация VFS поверх S3 и HDFS. ### 1.12. Экспериментальная реализация VFS поверх S3 и HDFS.
Нужно для Яндекс.Облака. Требует 1.11. Желательно 1.6 и 1.18. Нужно для Яндекс.Облака. Требует 1.11. Желательно 1.6 и 1.18.
Делает Александр Бурмак, Яндекс.Облако (сначала часть для S3), а также Олег Ершов, ВШЭ и Яндекс. Делает Александр, Яндекс.Облако (сначала часть для S3), а также Олег Ершов, ВШЭ и Яндекс.
### 1.13. Ускорение запросов с FINAL. ### 1.13. Ускорение запросов с FINAL.
@ -120,7 +156,7 @@
### 2.8. Декларативный парсер запросов. ### 2.8. Декларативный парсер запросов.
Низкий приоритет. Задачу хочет сделать [Иван Лежанкин](https://github.com/abyss7) в свободное время, но за последний год никаких поползновений не видно. Низкий приоритет. Задачу хочет сделать [Иван Лежанкин](https://github.com/abyss7) в свободное время, но пока ничего нет.
### 2.9. Логгировние в format-стиле. ### 2.9. Логгировние в format-стиле.
@ -144,7 +180,7 @@
### 2.14. Все функции с состоянием переделать на FunctionBuilder. ### 2.14. Все функции с состоянием переделать на FunctionBuilder.
Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлимо. Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлемо.
### 2.15. Функция subscribe для IStorage. ### 2.15. Функция subscribe для IStorage.
@ -163,9 +199,9 @@
Требует 3.1. Требует 3.1.
### 3.3. Исправить катастрофически отвратительно неприемлимый поиск по документации. ### 3.3. Исправить катастрофически отвратительно неприемлемый поиск по документации.
[Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на треш-технологиях, которые трудно исправить. [Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на технологиях, не удовлетворяющих требованиям задачи, и эти технологии трудно исправить.
### 3.4. Добавить японский язык в документацию. ### 3.4. Добавить японский язык в документацию.
@ -182,9 +218,15 @@
[Никита Лапков](https://github.com/laplab), весна 2020. Нужно для Яндекс.Метрики. Требует 4.1. [Никита Лапков](https://github.com/laplab), весна 2020. Нужно для Яндекс.Метрики. Требует 4.1.
Если распределённый запрос затрагивает большое количество серверов, то время выполнения запросов часто становится большим из-за tail latencies - случайных редких замедлений отдельных серверов. Эту проблему можно избежать, отправляя один и тот же запрос сразу на несколько реплик, и используя данные с наиболее быстрой.
Задача скрывает в себе много тонкостей, связанных с обработкой стадий выполнения запроса (соединение, обмен handshake, отправка запроса, получение заголовка результата, получение пакетов прогресса, получение данных), правильной возможностью настройки таймаутов, правильной отменой запросов.
Сейчас для распределённых запросов используется по потоку на соединение. Это позволяет хорошо распараллелить вычисления над полученными данными и утилизировать сеть, но становится сильно избыточным для больших кластеров. Для примера, создание 1000 потоков для чтения данных из 1000 серверов кластера - лишь расходует ресурсы и увеличивает время выполнения запроса. Вместо этого необходимо использовать количество потоков не большее количества процессорных ядер, и мультиплексировать в одном потоке общение с серверами. Реализация нетривиальна, так как мультиплексировать необходимо каждую стадию общения по сети, включая установку соединения и обмен handshake.
### 4.3. Ограничение числа одновременных скачиваний с реплик. ### 4.3. Ограничение числа одновременных скачиваний с реплик.
Изначально делал Олег Алексеенков, но решение оказалось неудачным, хотя там не так уж много доделывать. Изначально делал Олег Алексеенков, но пока решение не готово, хотя там не так уж много доделывать.
### 4.4. Ограничение сетевой полосы при репликации. ### 4.4. Ограничение сетевой полосы при репликации.
@ -215,14 +257,14 @@
### 5.5. LTS релизы. ### 5.5. LTS релизы.
Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Задача полностью обречена на провал с практической точки зрения, но пользователям важен сам факт того, что она будет, а мы уважаем чувства пользователей. Первой LTS версией уже стала версия 19.14. Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Первой LTS версией уже стала версия 19.14.
## 6. Инструментирование. ## 6. Инструментирование.
### 6.1. Исправления сэмплирующего профайлера запросов. ### 6.1. Исправления сэмплирующего профайлера запросов.
[Никита Лапков](https://github.com/laplab), ноябрь 2019. Может не сделать, тогда будет делать Алексей Миловидов. Михаил Филимонов, Altinity. Ноябрь 2019.
### 6.2. Добавление memory profiler. ### 6.2. Добавление memory profiler.
@ -236,6 +278,8 @@
Делает Андрей Скобцов, ВШЭ. Делает Андрей Скобцов, ВШЭ.
В Linux существует возможность получать в программе информацию о счётчиках производительности и событиях, относящихся к CPU и ядру ОС. Подробнее смотрите `man perf_event_open`. Предлагается добавить эти метрики в ClickHouse для инструментирования запросов.
### 6.5. Эксперименты с LLVM X-Ray. ### 6.5. Эксперименты с LLVM X-Ray.
Требует 2.2. Требует 2.2.
@ -267,7 +311,7 @@
### 7.2. LLVM в submodules. ### 7.2. LLVM в submodules.
Уже добавлено, но старой версии, и поэтому не используется. Надо обновить. Уже добавлено, но старой версии, и поэтому не используется. Надо обновить.
Георгий Кондратьев. Возможно, он не сможет сделать эту задачу, тогда будет делать Алексей Миловидов. Георгий - очень опытный разработчик, либо будет делать Алексей Миловидов.
### 7.3. Обновление Poco. ### 7.3. Обновление Poco.
@ -284,7 +328,7 @@
### 7.6. Правильный статистический тест для comparison mode в clickhouse-performance-test. ### 7.6. Правильный статистический тест для comparison mode в clickhouse-performance-test.
Задачу начал делать Дмитрий Рубашкин. Сейчас продолжает [Александр Кузьменков](https://github.com/akuzm). Задачу начал делать Дмитрий Рубашкин (ВШЭ). Сейчас продолжает [Александр Кузьменков](https://github.com/akuzm).
### 7.7. Доделать тесты под MSan. ### 7.7. Доделать тесты под MSan.
@ -293,7 +337,7 @@
### 7.8. Добавить clang-tidy. ### 7.8. Добавить clang-tidy.
Уже есть PVS-Studio, но этого недостаточно. Уже есть PVS-Studio. Мы очень довольны, но этого недостаточно.
### 7.9. Проверки на стиль имён с помощью clang-tidy. ### 7.9. Проверки на стиль имён с помощью clang-tidy.
@ -319,9 +363,34 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
Тагир Кускаров, ВШЭ. Посмотрим на https://github.com/AmokHuginnsson/replxx Тагир Кускаров, ВШЭ. Посмотрим на https://github.com/AmokHuginnsson/replxx
Для ввода запросов в интерактивном режиме в клиенте командной строки clickhouse-client используется библиотека readline или libedit.
Библиотеки readline и libedit обладает следующими недостатками:
- (исправлено в новых версиях readline) Очень низкая производительность вставки больших кусков текста. Вставка каждого следующего символа имеет сложность O(n = количество предыдущих символов) и при вставке 1 МБ текста, скорость падает до десятков байт в секунду.
- Крайне сложно или невозможно реализовать подсветку синтаксиса по мере набора текста, а также autocomplete без нажатия дополнительных клавиш для вызова.
- Лицензия GPL (для readline) препятствует её включению в кодовую базу продукта.
- Плохо работает навигация по истории, если история вкючает запросы, не помещающиеся на экран.
- История сохраняется лишь при завершении работы клиента.
- При параллельной работе нескольких клиентов с одним файлом истории, сохраняется история только одного из клиентов.
- Плохо работает история для многострочных запросов.
- Излишняя экономия пересылаемых данных, что часто приводит к остаткам мусора в терминале.
Кроме того, имеются следующие сложно достижимые достоинства:
- Поддержка right-to-left текста;
- Поддержка editrc конфигураций.
В качестве альтернатив можно рассмотреть следующие варианты:
- Linenoise от Salvatore Sanfilippo. Достоинства: простота и компактность кода; высокая скорость работы. Недостатки: отсутствует поддержка Unicode; отсутствует автоматический перенос текста, что затрудняет работу с многострочными запросами.
- Linenoise с патчами для поддержки Unicode. Недостаток: теряется преимущество по скорости работы.
- Fish shell. Не является библиотекой, но представляет собой отличный пример, как можно реализовать подстветку синтаксиса и удобный autocomplete. Поддерживает Unicode, но работает весьма медленно.
- Python Prompt Toolkit. Не является подходящим решением для интеграции в C++ проект. Хорошие возможности по подсветке синтаксиса и autocomplete.
Вместо этого предлагается в качестве примера изучить прототип текстового редактора Kilo: https://viewsourcecode.org/snaptoken/kilo/ и реализовать всю необходимую функциональность.
### 7.15. Замена libressl обратно на openssl. ### 7.15. Замена libressl обратно на openssl.
Поводом использования libressl послужило желание Константина podshumok Игнатова из QRator и то, что тогда openssl был опозорен и libressl считалась адекватной альтернативой. Но сейчас ситуация изменилась - openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно. Поводом использования libressl послужило желание нашего хорошего друга из известной компании несколько лет назад. Но сейчас ситуация состоит в том, что openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно.
### 7.16. tzdata внутри бинарника. ### 7.16. tzdata внутри бинарника.
@ -359,22 +428,54 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
Дарья Петрова, УрФУ. Дарья Петрова, УрФУ.
Над ClickHouse одновременно работает большое количество разработчиков, которые оформляют свои изменения в виде pull requests. Когда непомерженных pull requests много, то возникает сложность с организацией работы - непонятно, на какой pull request смотреть в первую очередь.
Предлагается реализовать простое одностраничное веб-приложение, в котором отображается список pull requests со следующей информацией:
- размер diff - количество изменённых строк;
- как давно было последнее обновление;
- типы изменённых файлов: C++, документация, скрипты сборки;
- наличие добавленных тестов;
- есть ли описание для changelog;
- изменены ли submodules;
- был ли разрешён запуск проверок CI;
- статусы проверок CI;
- количество approve от ревьюеров;
Статусы проверок - наиболее важная часть. Так как для каждого PR выполняется несколько десятков проверок и наиболее медленные работают до нескольких часов, придётся:
- отображать сразу все проверки для каждого PR в виде красивой разноцветной матрицы с информацией по наведению мыши;
- отсортировать проверки по важности: например, если у внешнего разработчика проходят все проверки кроме стиля кода, то мы можем взять это в работу сами;
- если для предыдущего коммита проверка была завершена, а для последнего коммита ещё только идёт - то можно отображать в таблице статус предыдущей проверки более блёклым цветом.
Предлагается реализовать несколько вариантов сортировок. Очевидное - по времени обновления, более интересно - некое ранжирование с целью выяснить, "что лучше взять в работу прямо сейчас".
Похожие продукты уже есть, например: http://prs.mozilla.io/yandex:ClickHouse К сожалению, этот продукт заброшен, да и делает не совсем то, что нужно. По своему усмотрению, можно взять из него что-нибудь полезное.
### 7.23. Функции для fuzzing. ### 7.23. Функции для fuzzing.
Андрей Некрашевич, ВШЭ. Андрей Некрашевич, ВШЭ.
Fuzzing тестирование - это тестирование случайными данными. Мы рассмотрим несколько подходов к этой задачи:
1. Добавление в SQL диалект ClickHouse функций для генерации случайных данных (пример - случайные бинарные строки заданной длины, случайные валидные UTF-8 строки) и "порчи" данных (например, поменять значения случайных бит с заданной частотой). Это будет использовано для тестирования SQL-функций ClickHouse.
### 7.24. Fuzzing лексера и парсера запросов; кодеков и форматов. ### 7.24. Fuzzing лексера и парсера запросов; кодеков и форматов.
Андрей Некрашевич, ВШЭ. Андрей Некрашевич, ВШЭ.
Продолжение 7.23.
2. Использование AFL или LibFuzzer для тестирования отдельных частей кодовой базы ClickHouse.
3. Генерация и выполнение случайных синтаксически корректных запросов на случайных данных.
### 7.25. Синхронизация релизов в Аркадию. ### 7.25. Синхронизация релизов в Аркадию.
Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в Яндекс.Морду, но обещает продолжать синхронизацию. Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в дружественный отдел, но обещает продолжать синхронизацию.
Затем, возможно, [Иван Лежанкин](https://github.com/abyss7). Но сейчас приостановлено, так как Максим Ахмедов должен исправить регрессию производительности в анализе индекса. Затем, возможно, [Иван Лежанкин](https://github.com/abyss7). Но сейчас приостановлено, так как Максим из YT должен исправить регрессию производительности в анализе индекса.
### 7.26. Побайтовая идентичность репозитория с Аркадией. ### 7.26. Побайтовая идентичность репозитория с Аркадией.
Команда DevTools. Фактически никто ничего не делает. Команда DevTools. Прогресс по задаче под вопросом.
### 7.27. Запуск автотестов в Аркадии. ### 7.27. Запуск автотестов в Аркадии.
@ -396,6 +497,12 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
Роман Ильговский. Нужно для Яндекс.Метрики. Роман Ильговский. Нужно для Яндекс.Метрики.
Имея SQL запрос, требуется вывести структуру таблиц, на которых этот запрос будет выполнен, и заполнить эти таблицы случайными данными, такими, что результат этого запроса зависит от выбора подмножества данных.
Для примера, если есть запрос `SELECT SearchPhrase, count(*) FROM table WHERE CounterID = 34 AND SearchPhrase LIKE '%ClickHouse%'`, то мы можем сделать вывод, что CounterID имеет числовой тип, а SearchPhrase - строковый. Заполнить таблицу данными, на которых отдельные условия `CounterID = 34` и `SearchPhrase LIKE '%ClickHouse%'` для некоторых строк выполнены, а для некоторых строк не выполнены.
Обфускация запросов: имея секретные запросы и структуру таблиц, заменить имена полей и константы, чтобы запросы можно было использовать в качестве публично доступных тестов.
### 7.33. Выкладывать патч релизы в репозиторий автоматически. ### 7.33. Выкладывать патч релизы в репозиторий автоматически.
[Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно. [Александр Сапин](https://github.com/alesapin). Может делегировать эту задачу кому угодно.
@ -416,8 +523,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
### 7.37. Разобраться с repo.yandex.ru. ### 7.37. Разобраться с repo.yandex.ru.
Есть жалобы на скорость загрузки. Подозрение, что repo.yandex.ru не является нормальным CDN. Отсутствует простой доступ к мониторингу и логам. Есть жалобы на скорость загрузки и неудобство maintenance, operations, visibility.
Очень редко бывает нужно удалить пакет, но сделать это можно только через одного человека.
## 8. Интеграция с внешними системами. ## 8. Интеграция с внешними системами.
@ -471,16 +577,26 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
Павел Круглов, ВШЭ и Яндекс. Павел Круглов, ВШЭ и Яндекс.
Формат Apache Avro является компактным структурированным построчным бинарным форматом данных с внешней схемой. Этот формат часто используется совместно с Kafka и поддержка его в качестве одного из форматов ввода-вывода в ClickHouse является востребованной пользователями.
### 8.17. ClickHouse как MySQL реплика. ### 8.17. ClickHouse как MySQL реплика.
Ильяс Адюгамов, ВШЭ. Ильяс Адюгамов, ВШЭ.
Реализовать возможность подписаться на row-based репликацию MySQL и сохранять полученные данные в CollapsingMergeTree или ReplacingMergeTree таблицы. Сторонние решения для этой задачи уже существуют: https://www.altinity.com/blog/2018/6/30/realtime-mysql-clickhouse-replication-in-practice Также существует стороннее решение для PostgreSQL: https://github.com/mkabilov/pg2ch
Встроенная в ClickHouse возможность работать в качестве реплики MySQL даст преимущества для дальнейшего развития.
### 8.18. ClickHouse как Federated MySQL. ### 8.18. ClickHouse как Federated MySQL.
### 8.19. Интеграция с RabbitMQ. ### 8.19. Интеграция с RabbitMQ.
Ксения Сумарокова, ВШЭ. Ксения Сумарокова, ВШЭ.
В ClickHouse часто используется потоковый импорт данных из распределённой очереди. Наиболее популярно использование совместно с Kafka. Эта возможность уже есть.
Следующей по востребованности является система очередей RabbitMQ. Её поддержка в ClickHouse отсутствует.
### 8.20. Интеграция с SQS. ### 8.20. Интеграция с SQS.
### 8.21. Поддержка произвольного количества языков для имён регионов. ### 8.21. Поддержка произвольного количества языков для имён регионов.
@ -498,6 +614,10 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
Валерий Батурин, ВШЭ. Валерий Батурин, ВШЭ.
ClickHouse предоставляет возможность обратиться к внешней базе данных из языка запросов. Это реализовано в виде табличных функций. В параметрах к табличной функции указывается адрес удалённой базы данных (хост, порт), а также аутентификационные данные (имя пользователя, пароль). Аутентификационные данные указываются в запросе в открытом виде и, таким образом, попадают в историю запросов и в логи, что компрометирует безопасность системы.
Вместо этого предлагается описывать необходимые данные в конфигурационном файле сервера или в отдельном сервисе и ссылаться на них по именам.
### 9.3. Поддержка TLS для ZooKeeper. ### 9.3. Поддержка TLS для ZooKeeper.
@ -505,12 +625,12 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
### 10.1. Исправление зависания в библиотеке доступа к YT. ### 10.1. Исправление зависания в библиотеке доступа к YT.
Библиотека для доступа к YT обладает катастрофически отвратительно неприемлимым поведением и не переживает учения. Библиотека для доступа к YT не переживает учения.
Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT. Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT.
### 10.2. Исправление SIGILL в библиотеке доступа к YT. ### 10.2. Исправление SIGILL в библиотеке доступа к YT.
Код YT зачем-то использует SIGILL вместо abort. Это, опять же, происходит при учениях. Код YT использует SIGILL вместо abort. Это, опять же, происходит при учениях.
Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT. Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT.
### 10.3. Возможность чтения данных из статических таблиц в YT словарях. ### 10.3. Возможность чтения данных из статических таблиц в YT словарях.
@ -519,7 +639,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
### 10.4. Словарь из YDB (KikiMR). ### 10.4. Словарь из YDB (KikiMR).
Нужно для Метрики, а делать будет таинственный незнакомец из команды KikiMR. Или он сейчас это прочитает и скажет "я никогда не буду делать эту задачу". Нужно для Метрики, а делать будет таинственный незнакомец из команды KikiMR (под вопросом).
### 10.5. Закрытие соединений и уменьшение числа соединений для MySQL и ODBC. ### 10.5. Закрытие соединений и уменьшение числа соединений для MySQL и ODBC.
@ -537,7 +657,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
### 10.9. Уменьшение блокировок для cache словарей за счёт одновременных запросов одного и того же. ### 10.9. Уменьшение блокировок для cache словарей за счёт одновременных запросов одного и того же.
Нужно для БК, но на самом деле они так просто думают, а все проблемы из-за низко-качественной библиотеки для доступа в YT. Нужно для БК, но мотивация задачи находится под вопросом, так как есть рабочее предположение о том, что данная задача не устраняет причину проблемы.
### 10.10. Возможность использования старых значений из cache словаря пока они перезапрашиваются. ### 10.10. Возможность использования старых значений из cache словаря пока они перезапрашиваются.
@ -563,10 +683,16 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
Никита Васильев, ВШЭ и Яндекс. Никита Васильев, ВШЭ и Яндекс.
Реализовать в ClickHouse специализированный движок таблиц, подходящий для быстрых key-value запросов и оптимизированный для расположения данных на SSD. Это может быть: реализация на основе RocksDB; сериализованные RowBinary данные с индексом в оперативке; секретная очень эффективная структура данных, о которой я расскажу.
Использовать эту структуру данных как отдельный вид словарей, как источник для cache словарей или как дополнительный уровень кэширования для cache словарей.
### 10.17. Локальный дамп состояния словаря для быстрого старта сервера. ### 10.17. Локальный дамп состояния словаря для быстрого старта сервера.
### 10.18. Таблица Join или словарь на удалённом сервере как key-value БД для cache словаря. ### 10.18. Таблица Join или словарь на удалённом сервере как key-value БД для cache словаря.
### 10.19. Возможность зарегистрировать некоторые функции, использующие словари, под пользовательскими именами.
## 11. Интерфейсы. ## 11. Интерфейсы.
@ -574,7 +700,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
### 11.2. Возможность использовать движок JDBC из коробки. ### 11.2. Возможность использовать движок JDBC из коробки.
Нужно разобраться, как упаковывать Java в статический бинарник, возможно AppImage. Нужно разобраться, как упаковывать Java в статический бинарник, возможно AppImage. Или предоставить максимально простую инструкцию по установке jdbc-bridge. Может быть будет заинтересован Александр Крашенинников, Badoo, так как он разработал jdbc-bridge.
### 11.3. Интеграционные тесты ODBC драйвера путём подключения ClickHouse к самому себе через ODBC. ### 11.3. Интеграционные тесты ODBC драйвера путём подключения ClickHouse к самому себе через ODBC.
@ -584,7 +710,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
### 11.5. Поддержка TLS в clickhouse-cpp. ### 11.5. Поддержка TLS в clickhouse-cpp.
Библиотеку clickhouse-cpp разрабатывал Павел Артёмкин в свободное время. А знаете ли вы, что библиотеку clickhouse-cpp разрабатывал один хороший человек в свободное время?
### 11.6. Интеграционные тесты clickhouse-cpp. ### 11.6. Интеграционные тесты clickhouse-cpp.
@ -594,6 +720,8 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
Элбакян Мовсес Андраникович, ВШЭ. Элбакян Мовсес Андраникович, ВШЭ.
В ClickHouse в прошлом году добавили поддержку wire-протокола MySQL. PostgreSQL, так же как MySQL, использует несложный протокол общения между клиентом и сервером, но свой собственный. Поддержка этого протокола является востребованной и откроет новые возможности для ClickHouse.
### 11.9. Доработки ODBC драйвера. ### 11.9. Доработки ODBC драйвера.
Денис Глазачев, Altinity. Денис Глазачев, Altinity.
@ -602,6 +730,8 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
zhang2014 zhang2014
Возможность описать в конфигурационном файле handler (путь в URL) для HTTP запросов к серверу, которому соответствует некоторый параметризованный запрос. Пользователь может вызвать этот обработчик и не должен передавать SQL запрос.
## 12. Управление пользователями и доступом. ## 12. Управление пользователями и доступом.
@ -661,7 +791,7 @@ zhang2014
### 14.5. Поддержка задания множества как массива в правой части секции IN. ### 14.5. Поддержка задания множества как массива в правой части секции IN.
Василий Немков, Altinity, делал эту задачу, но зачем-то забросил её. Василий Немков, Altinity, делал эту задачу, но временно приостановил работу над ней в пользу других задач.
### 14.6. Глобальный scope для WITH. ### 14.6. Глобальный scope для WITH.
@ -673,6 +803,10 @@ zhang2014
Софья Борзенкова, ВШЭ. Софья Борзенкова, ВШЭ.
В ClickHouse поддерживается вычисление COUNT(DISTINCT x). Предлагается добавить возможность использования модификатора DISTINCT для всех агрегатных функций. Например, AVG(DISTINCT x) - вычислить среднее значение для всех различных значений x. Под вопросом вариант, в котором фильтрация уникальных значений выполняется по одному выражению, а агрегация по другому.
Результат некоторых агрегатных функций зависит от порядка данных. Предлагается реализовать модификатор ORDER BY, задающий порядок явно. Пример: groupArray(x ORDER BY y, z).
### 14.9. Поддержка запроса EXPLAIN. ### 14.9. Поддержка запроса EXPLAIN.
Требует 2.1. [Николай Кочетов](https://github.com/KochetovNicolai). Требует 2.1. [Николай Кочетов](https://github.com/KochetovNicolai).
@ -685,6 +819,10 @@ zhang2014
Сложная задача, так как вводит новый класс функций и требует его обработку в оптимизаторе запросов. Сложная задача, так как вводит новый класс функций и требует его обработку в оптимизаторе запросов.
В time-series СУБД нужны функции, которые зависят от последовательности значений. Или даже от последовательности значений и их меток времени. Примеры: moving average, exponential smoothing, derivative, Holt-Winters forecast. Вычисление таких функций поддерживается в ClickHouse лишь частично. Так, ClickHouse поддерживает тип данных "массив" и позволяет реализовать эти функции как функции, принимающие массивы. Но гораздо удобнее для пользователя было бы иметь возможность применить такие функции к таблице (промежуточному результату запроса после сортировки).
Это требует введение нового класса функций (помимо обычных и агрегатных функций) - такие функции будут иметь в коде ClickHouse свой собственный интерфейс, и их вычисление придётся отдельно учитывать в конвейере выполнения запросов. Для примера, вычисление обычных функций тривиально распараллеливается по процессорным ядрам и по серверам; вычисление агрегатных функций распараллеливается с некоторыми особенностями (работа с промежуточными состояниями вычислений, операция merge); а для функций по обработке временных рядов этот вопрос остаётся открытым - возможно, их придётся вычислять на одном сервере и в одном потоке.
### 14.13. Применимость функций высшего порядка для кортежей и Nested. ### 14.13. Применимость функций высшего порядка для кортежей и Nested.
### 14.14. Неявные преобразования типов констант. ### 14.14. Неявные преобразования типов констант.
@ -746,7 +884,7 @@ zhang2014
### 16.1. DateTime64. ### 16.1. DateTime64.
Василий Немков. Василий Немков, Altinity, декабрь 2019.
### 16.2. Тип данных для JSON. ### 16.2. Тип данных для JSON.
@ -770,10 +908,16 @@ zhang2014
[Андрей Чулков](https://github.com/achulkov2), Антон Кваша, Артур Петуховский, ВШЭ. [Андрей Чулков](https://github.com/achulkov2), Антон Кваша, Артур Петуховский, ВШЭ.
Будет основано на коде от Арслана Урташева. Будет основано на коде от Арслана Урташева.
ClickHouse не является geospatial СУБД. Тем не менее, в ClickHouse есть несколько функций для таких задач. Например, функция `pointInPolygon` позволяет быстро проверить попадание точек в полигон на плоскости. При этом, полигон задаётся в явном виде и должен быть константным для вызова функции (то есть - проверяется принадлежность многих точек одному полигону). Эта функциональность нужна, например, для рекламного таргетинга мобильных устройств по координатам.
Похожая, но более сложная задача, которую ClickHouse пока не умеет решать - определение полигона среди множества полигонов, в которые попадают точки. Для примера: определение района города по географическим координатам. Для решения этой задачи нужно будет реализовать поддержку словарей с полигонами, в которых данные проиндексированы для быстрого поиска.
### 17.2. GIS типы данных и операции. ### 17.2. GIS типы данных и операции.
Алексей Коряков, Алексей Илюхов, ВШЭ, Яндекс.Карты. Алексей Коряков, Алексей Илюхов, ВШЭ, Яндекс.Карты.
Реализовать в ClickHouse типы данных для задач обработки геоинформационных данных: Point, Line, MultiLine, Polygon и операции над ними - проверка вхождения, пересечения. Вариантом минимум будет реализация этих операций в евклидовой системе координат. Дополнительно - на сфере и WGS84.
### 17.3. Ускорение greatCircleDistance. ### 17.3. Ускорение greatCircleDistance.
[Ольга Хвостикова](https://github.com/stavrolia), основано на коде Андрея Аксёнова, получено разрешение на использование кода. [Ольга Хвостикова](https://github.com/stavrolia), основано на коде Андрея Аксёнова, получено разрешение на использование кода.
@ -797,6 +941,8 @@ zhang2014
Артём Цыганов, Руденский Константин Игоревич, Семёнов Денис, ВШЭ. Артём Цыганов, Руденский Константин Игоревич, Семёнов Денис, ВШЭ.
Предлагается реализовать в ClickHouse статистические тесты (Analysis of Variance, тесты нормальности распределения и т. п.) в виде агрегатных функций. Пример: `welchTTest(value, sample_idx)`.
### 18.3. Инфраструктура для тренировки моделей в ClickHouse. ### 18.3. Инфраструктура для тренировки моделей в ClickHouse.
В очереди. Возможно, Александр Кожихов. У него сначала идёт задача 24.26. В очереди. Возможно, Александр Кожихов. У него сначала идёт задача 24.26.
@ -808,13 +954,27 @@ zhang2014
Александра Латышева, ВШЭ и Яндекс. Александра Латышева, ВШЭ и Яндекс.
Репликация данных в ClickHouse по-умолчанию является асинхронной без выделенного мастера. Это значит, что клиент, осуществляющий вставку данных, получает успешный ответ после того, как данные попали на один сервер; репликация данных по остальным серверам осуществляется в другой момент времени. Это ненадёжно, потому что допускает потерю только что вставленных данных при потере лишь одного сервера.
Для решения этой проблемы, в ClickHouse есть возможность включить "кворумную" вставку. Это значит, что клиент, осуществляющий вставку данных, получает успешный ответ после того, как данные попали на несколько (кворум) серверов. Обеспечивается линеаризуемость: клиент, получает успешный ответ после того, как данные попали на несколько реплик, *которые содержат все предыдущие данные, вставленные с кворумом* (такие реплики можно называть "синхронными"), и при запросе SELECT можно выставить настройку, разрешающую только чтение с синхронных реплик.
Если бы свойства линеаризуемости не было, то для трёх серверов A, B, C, значения кворума = 2, и для трёх вставок данных 1, 2, 3, возможна ситуация, что первая вставка прошла на серверы A и B, вторая прошла на серверы B и C, а третья - на серверы A и C, и теперь ни один из серверов не содержит полный набор данных 1, 2, 3.
Как ни странно, такое свойство не нужно большинству пользователей. Оно запрещает параллельно выполняющиеся вставки. А пользователи хотят вставлять данные надёжно (на более чем одну реплику), но не важно, в каком порядке. Предлагается сделать опцию, которая отключает линеаризуемость.
Иногда пользователь хочет реализовать кворумную вставку вручную: просто соединиться с несколькими репликами и вставть на них одинаковые данные (чтобы обеспечить надёжную вставку, не ориентируясь на то, как работает механизм репликации). Сейчас ожидания пользователя не оправдываются. В ClickHouse есть механизм дедупликации для обеспечения идемпотентности вставок. Вторая вставка с такими же данными (пусть даже на другую реплику) будет проигнорирована. Надо сделать так, чтобы вместо этого, вставка одинаковых данных на другую реплику, имела такой же эффект, как если бы эти данные были получены с помощью механизма репликации.
### 19.2. Подключение Etcd или Consul как альтернативы ZooKeeper. ### 19.2. Подключение Etcd или Consul как альтернативы ZooKeeper.
Алексей Лёвушкин, ВШЭ и Яндекс. Алексей Лёвушкин, ВШЭ и Яндекс.
Для координации реплик в ClickHouse используется ZooKeeper. Многие пользователи ClickHouse хотели бы иметь возможность использовать для координации некоторые другие системы вместо ZooKeeper. Рассматриваемыми вариантами таких систем являются Etcd, Consul, FoundationDB. Это весьма проблематично, так как эти системы существенно отличаются по интерфейсам и возможностям. Тем не менее, для того, чтобы эта задача стала возможной, в ClickHouse обобщён интерфейс взаимодействия с ZooKeeper, и теперь на его место можно подставлять другие реализации.
В прошлом году, Алексей добавил модельную реализацию (mock) интерфейса ZooKeeper для тестирования. Сейчас предлагается сделать реализацию поверх Etcd, а также расширить возможности тестовой реализации.
### 19.3. Подключение YT Cypress или YDB как альтернативы ZooKeeper. ### 19.3. Подключение YT Cypress или YDB как альтернативы ZooKeeper.
Hold. Полезно для Яндекс.Облака и БК, но есть риски, что будет вредно, а не полезно. Hold. Полезно для заказчиков внутри Яндекса, но есть риски.
### 19.4. internal_replication = 'auto'. ### 19.4. internal_replication = 'auto'.
@ -822,6 +982,10 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р
В очереди, возможно Валерий Батурин, ВШЭ. В очереди, возможно Валерий Батурин, ВШЭ.
Репликация в ClickHouse работает на уровне отдельных таблиц. Это является очень гибким решением: на одном сервере одна из таблиц может быть не реплицирована, другая иметь двухкратную репликацию, а третья - реплицирована по всем серверам. Но если все таблицы в базе данных реплицированы одинаковым образом. то это затрудняет управление кластером. Например, при восстановлени сервера, требуется отдельно создавать реплику для каждой таблицы.
Предлагается реализовать "движок баз данных", который осуществляет репликацию метаданных (множество имеющихся таблиц и лог DDL операций над ними: CREATE, DROP, RENAME, ALTER). Пользователь сможет создать реплицируемую базу данных; при её создании или восстановлении на другом сервере, все реплицируемые таблицы будут созданы автоматически.
### 19.6. Одновременный выбор кусков для слияния многими репликами, отказ от leader election в ZK. ### 19.6. Одновременный выбор кусков для слияния многими репликами, отказ от leader election в ZK.
### 19.7. Возможность записи данных при недоступности ZK и отказ от линейного порядка кусков в большинстве случаев. ### 19.7. Возможность записи данных при недоступности ZK и отказ от линейного порядка кусков в большинстве случаев.
@ -866,6 +1030,12 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р
Дмитрий Рубашкин, ВШЭ. Помогает Антон Попов. Дмитрий Рубашкин, ВШЭ. Помогает Антон Попов.
Если таблица имеет ключ сортировки, то возможно эффективное чтение упорядоченных данных. Если запрос содержит операцию GROUP BY, содержащую по крайней мере префикс от ключа сортировки таблицы, либо инъективные функции от него, то возможно более эффективное выполнение GROUP BY: промежуточный результат агрегации финализируется и отправляется клиенту как только в потоке данных при чтении из таблицы встретился следующий ключ.
Аналогичную оптимизацию следует реализовать для DISTINCT и LIMIT BY.
В прошлом году, аналогичное решение сделали для операции ORDER BY.
### 21.5. Распараллеливание INSERT при INSERT SELECT, если это необходимо. ### 21.5. Распараллеливание INSERT при INSERT SELECT, если это необходимо.
### 21.6. Уменьшение числа потоков для SELECT в случае тривиального INSERT SELECT. ### 21.6. Уменьшение числа потоков для SELECT в случае тривиального INSERT SELECT.
@ -878,9 +1048,15 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р
Михаил Кот, ВШЭ. Задача сложная и рискованная. Михаил Кот, ВШЭ. Задача сложная и рискованная.
Для выделения памяти, аллокаторы запрашивают её у операционной системы (`mmap`). Это возможно только для достаточно крупных кусков памяти является довольно медленной операцией. Поэтому, современные аллокаторы кэшируют крупные куски памяти в программе. При вызове free, кусок памяти, как правило, не отдаётся ОС, а остаётся для последующего переиспользования. Для выделения мелких кусков памяти, крупные куски разбиваются с помощью специальных структур данных (free-list, heap, bitmap). Для уменьшения contention в многопоточных программах, эти структуры также делаются thread-локальными.
Часто в программе есть кэши некоторых данных. Например - кэш данных после разжатия, использующийся чтобы сэкономить на повторных запросах одних и тех же данных. При вытеснении из кэша, блок данных освобождается (`free`) и данные, бывшие в кэше, становятся недоступными для переиспользования. Но если принимать во внимание то, как работает аллокатор памяти, то оказывается, что после освобождения памяти, данные всё ещё остаются доступными в программе. И если этот кусок памяти не будет выделен аллокатором снова, его можно было бы продолжить использовать в качестве кэша. Иными словами, в программе есть domain-specific кэш, а аллокатор имеет свой кэш, и они не знают друг о друге.
Для domain-specific кэшей (как например, кэш разжатых данных) выгодно, чтобы они использовали как можно больший объём свободной памяти. Но в этом случае, памяти может не хватить для других структур данных в программе. Если аллокатор памяти знает про кэш, то выделение памяти можно было бы делать путём вытеснения данных из кэша.
### 21.8.1. Отдельный аллокатор для кэшей с ASLR. ### 21.8.1. Отдельный аллокатор для кэшей с ASLR.
В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения оказались неудачными. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд. В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения не были квалифицированы для использования в продакшене. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд.
### 21.9. Исправить push-down выражений с помощью Processors. ### 21.9. Исправить push-down выражений с помощью Processors.
@ -894,10 +1070,25 @@ Amos Bird.
Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ
Реализовать в ClickHouse оптимизации запросов, основанные на упрощении отдельных небольших кусков выражений (так называемые "peephole" оптимизации). Примеры:
- Замена цепочек if на multiIf.
- Удаление min/max/any-агрегатов от выражений от ключей GROUP BY.
- Вынесение арифметических операций из агрегатных функций;
- Вынесение любых функций наружу any, anyLast.
- При GROUP BY по transform или if по строкам, замена строк на Enum.
### 21.12. Алгебраические оптимизации запросов. ### 21.12. Алгебраические оптимизации запросов.
Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ Руслан Камалов, Михаил Малафеев, Виктор Гришанин, ВШЭ
Реализовать в ClickHouse оптимизации запросов, основанные на алгебраических свойствах функций. Примеры:
- Обращение инъективных функций в сравнениях на равенство.
- Вынесение инъективных функцию наружу uniq.
- Удаление монотонных функций из ORDER BY.
- Удаление избыточных выражений из ORDER BY.
- Удаление из GROUP BY функций от других ключей GROUP BY.
- Удаление дублирующихся DISTINCT, ORDER BY из подзапросов.
### 21.13. Fusion агрегатных функций. ### 21.13. Fusion агрегатных функций.
После или совместно с 21.11. После или совместно с 21.11.
@ -906,6 +1097,10 @@ Amos Bird.
Мария Нефедова, ВШЭ. Мария Нефедова, ВШЭ.
Constraints позволяют задать выражение, истинность которого проверяется при вставке данных в таблицу. Предположение о том, что выражение истинно, может использоваться и для оптимизации запросов. Например, встретив в запросе точно такое же выражение, можно заменить его на константу 1.
Если выражение содержит равенство, то встретив в запросе одну из частей равенства, её можно заменить на другую часть равенства, если это сделает проще чтение данных или вычисление выражения. Например, задан constraint: `URLDomain = domain(URL)`. Значит, выражение `domain(URL)` можно заменить на `URLDomain`.
### 21.15. Многоступенчатое чтение данных вместо PREWHERE. ### 21.15. Многоступенчатое чтение данных вместо PREWHERE.
Требует 2.1 и 21.10. Требует 2.1 и 21.10.
@ -923,7 +1118,29 @@ Amos Bird.
### 21.19. Оптимизация сортировки. ### 21.19. Оптимизация сортировки.
Василий Морозов, Арслан Гумеров, Альберт Кидрачев, ВШЭ. Василий Морозов, Арслан Гумеров, Альберт Кидрачев, ВШЭ.
В прошлом году задачу начинал делать Евгений Правда, ВШЭ, но почти полностью не сделал её. В прошлом году задачу начинал делать другой человек, но не добился достаточного прогресса.
1. Оптимизация top sort.
В ClickHouse используется неоптимальный вариант top sort. Суть его в том, что из каждого блока достаётся top N записей, а затем, все блоки мержатся. Но доставание top N записей у каждого следующего блока бессмысленно, если мы знаем, что из них в глобальный top N войдёт меньше. Конечно нужно реализовать вариацию на тему priority queue (heap) с быстрым пропуском целых блоков, если ни одна строка не попадёт в накопленный top.
2. Рекурсивный вариант сортировки по кортежам.
Для сортировки по кортежам используется обычная сортировка с компаратором, который в цикле по элементам кортежа делает виртуальные вызовы `IColumn::compareAt`. Это неоптимально - как из-за короткого цикла по неизвестному в compile-time количеству элементов, так и из-за виртуальных вызовов. Чтобы обойтись без виртуальных вызовов, есть метод `IColumn::getPermutation`. Он используется в случае сортировки по одному столбцу. Есть вариант, что в случае сортировки по кортежу, что-то похожее тоже можно применить... например, сделать метод `updatePermutation`, принимающий аргументы offset и limit, и допереставляющий перестановку в диапазоне значений, в которых предыдущий столбец имел равные значения.
3. RadixSort для сортировки.
Один наш знакомый начал делать задачу по попытке использования RadixSort для сортировки столбцов. Был сделан вариант indirect сортировки (для `getPermutation`), но не оптимизирован до конца - есть лишние ненужные перекладывания элементов. Для того, чтобы его оптимизировать, придётся добавить немного шаблонной магии (на последнем шаге что-то не копировать, вместо перекладывания индексов - складывать их в готовое место). Также этот человек добавил метод MSD Radix Sort для реализации radix partial sort. Но даже не проверил производительность.
Наиболее содержательная часть задачи может состоять в применении Radix Sort для сортировки кортежей, расположенных в оперативке в виде Structure Of Arrays неизвестного в compile-time размера. Это может работать хуже, чем то, что описано в пункте 2... Но попробовать не помешает.
4. Three-way comparison sort.
Виртуальный метод `compareAt` возвращает -1, 0, 1. Но алгоритмы сортировки сравнениями обычно рассчитаны на `operator<` и не могут получить преимущества от three-way comparison. А можно ли написать так, чтобы преимущество было?
5. pdq partial sort
Хороший алгоритм сортировки сравнениями `pdqsort` не имеет варианта partial sort. Заметим, что на практике, почти все сортировки в запросах ClickHouse являются partial_sort, так как `ORDER BY` почти всегда идёт с `LIMIT`. Кстати, Данила Кутенин уже попробовал это и показал, что в тривиальном случае преимущества нет. Но не очевидно, что нельзя сделать лучше.
### 21.20. Использование материализованных представлений для оптимизации запросов. ### 21.20. Использование материализованных представлений для оптимизации запросов.
@ -986,10 +1203,12 @@ zhang2014.
[Виталий Баранов](https://github.com/vitlibar), почти всё готово. [Виталий Баранов](https://github.com/vitlibar), почти всё готово.
### 22.12. Исправление катастрофически отвратительно неприемлимо низкой производительности чтения из Kafka. ### 22.12. Исправление низкой производительности чтения из Kafka.
[Иван Лежанкин](https://github.com/abyss7). [Иван Лежанкин](https://github.com/abyss7).
Для ClickHouse нехарактерно наличие кода, обладающего столь низкой производительностью. Практики разработки не подразумевают, что такой код должен попасть в продакшен без надлежащего тестирования производительности.
### 22.13. Посмотреть, почему не работают некоторые collations. ### 22.13. Посмотреть, почему не работают некоторые collations.
[Иван Лежанкин](https://github.com/abyss7), совмещается с 7.1. [Иван Лежанкин](https://github.com/abyss7), совмещается с 7.1.
@ -1000,9 +1219,11 @@ zhang2014.
[Иван Лежанкин](https://github.com/abyss7), если он не сдастся. [Иван Лежанкин](https://github.com/abyss7), если он не сдастся.
### 22.16. Исправление катастрофически отвратительно неприемлимо низкой производительности кодека DoubleDelta. ### 22.16. Исправление низкой производительности кодека DoubleDelta.
Василий Немков, Altinity - сейчас старательно динамит эту задачу. Василий Немков, Altinity - временно приостановлено, но намерения остаются в силе.
Мы считаем важным, что код в ClickHouse содержит разумные оптимизации, основанные на анализе производительности. Но иногда бывают досадные исключения.
### 22.17. Консистентно работающий POPULATE для MaterializedView. ### 22.17. Консистентно работающий POPULATE для MaterializedView.
@ -1046,7 +1267,7 @@ zhang2014.
### 22.28. Изучить и исправить поведение работы с Kafka при ребалансировке. ### 22.28. Изучить и исправить поведение работы с Kafka при ребалансировке.
[Иван Лежанкин](https://github.com/abyss7), если он не сдастся. [Иван Лежанкин](https://github.com/abyss7).
## 23. Default Festival. ## 23. Default Festival.
@ -1070,7 +1291,7 @@ zhang2014.
### 23.5. Включение compile_expressions. ### 23.5. Включение compile_expressions.
Требует 7.2. Задачу изначально делал Денис Скоробогатов, ВШЭ и Яндекс, затем доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin). Требует 7.2. Задачу изначально на 99% сделал Денис Скоробогатов, ВШЭ и Яндекс. Остальной процент доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin).
### 23.6. Включение учёта порядка столбцов в CSV. ### 23.6. Включение учёта порядка столбцов в CSV.
@ -1090,7 +1311,7 @@ zhang2014.
### 23.10. Включение mlock бинарника. ### 23.10. Включение mlock бинарника.
Возможность mlock бинарника сделал Олег Алексеенков. Поможет, когда на серверах кроме ClickHouse работает много треш-программ. Возможность mlock бинарника сделал Олег Алексеенков. Поможет, когда на серверах кроме ClickHouse работает много посторонних программ (мы иногда называем их в шутку "треш-программами").
## 24. Экспериментальные задачи. ## 24. Экспериментальные задачи.
@ -1099,38 +1320,89 @@ zhang2014.
Антон Мамонов, УрФУ, Яндекс. Антон Мамонов, УрФУ, Яндекс.
Внутри ClickHouse есть богатые возможности по интроспекции и профилированию. Эти возможности доступны через системные таблицы и использовать их приходится путём формулирования SQL запросов. Это неудобно.
Вместо этого предлагается сделать, чтобы ClickHouse отдавал HTML страницу, реализующую интерактивный web-интерфейс со следующими возможностями:
- отображение состояния кластеров (какие кластеры известны, статус каждого сервера);
- графики нагрузки текущего сервера или выбранного сервера кластера;
- обновляемый список запросов;
- просмотр лога запросов с наиболее востребованными фильтрациями по одной кнопке;
- просмотр лога на кластере, например - последние ошибки;
- просмотр метрик использования ресурсов, flame graph и pprof-граф для выбранных запросов;
- отчёт по использованию кластера (пример: количество ядер CPU по пользователям за сегодня).
### 24.2. Экспериментальные алгоритмы сжатия. ### 24.2. Экспериментальные алгоритмы сжатия.
Анастасия Наумова, ВШЭ. Анастасия Наумова, ВШЭ.
ClickHouse поддерживает LZ4 и ZSTD для сжатия данных. Эти алгоритмы являются парето-оптимальными по соотношению скорости и коэффициентам сжатия среди достаточно известных. Тем не менее, существуют менее известные алгоритмы сжатия, которые могут превзойти их по какому-либо критерию. Из потенциально более быстрых по сравнимом коэффициенте сжатия: Lizard, LZSSE, density. Из более сильных: bsc и csc. Необходимо изучить эти алгоритмы, добавить их поддержку в ClickHouse и исследовать их работу на тестовых датасетах.
### 24.3. Экспериментальные кодеки. ### 24.3. Экспериментальные кодеки.
Вероника Фалчикова, Лада Торчик, ВШЭ. Вероника Фалчикова, Лада Торчик, ВШЭ.
Существуют специализированные алгоритмы кодирования числовых последовательностей: Group VarInt, MaskedVByte, PFOR. Необходимо изучить наиболее эффективные реализации этих алгоритмов. Примеры вы сможете найти на https://github.com/lemire и https://github.com/powturbo/ а также https://github.com/schizofreny/middle-out
Внедрить их в ClickHouse в виде кодеков и изучить их работу на тестовых датасетах.
### 24.4. Шифрование в ClickHouse на уровне кусков данных. ### 24.4. Шифрование в ClickHouse на уровне кусков данных.
Yuchen Dong, ICS. Yuchen Dong, ICS.
Данные в ClickHouse хранятся без шифрования. При наличии доступа к дискам, злоумышленник может прочитать данные. Предлагается реализовать два подхода к шифрованию:
1. Шифрование блоков данных.
Шифрование данных столбцов на диске требуется реализовать в виде кодеков. Это позволит применять шифрование к отдельным столбцам; применять его после сжатия данных (эффективно, но менее безопасно) или без сжатия. Потребуется проработать работу с ключами: получение ключей из отдельного сервиса, правильная работа с ключами в оперативке. Отдельным вопросом стоит шифрование индексов.
### 24.5. Поддержка функций шифрования для отдельных значений. ### 24.5. Поддержка функций шифрования для отдельных значений.
Yuchen Dong, ICS. Yuchen Dong, ICS.
Смотрите также 24.5.
2. Шифрование отдельных значений.
Для этого требуется реализовать функции шифрования и расшифрования, доступные из SQL. Для шифрования реализовать возможность добавления нужного количества случайных бит для исключения одинаковых зашифрованных значений на одинаковых данных. Это позволит реализовать возможность "забывания" данных без удаления строк таблицы: можно шифровать данные разных клиентов разными ключами, и для того, чтобы забыть данные одного клиента, потребуется всего лишь удалить ключ.
### 24.6. Userspace RAID. ### 24.6. Userspace RAID.
Глеб Новиков, ВШЭ. Глеб Новиков, ВШЭ.
RAID позволяет одновременно увеличить надёжность хранения данных на дисках и увеличить скорость работы дискового массива. Обычно RAID настраивается с помощью встроенных возможностей ядра Linux (mdraid) или с помощью hardware контроллера. У этого есть следующие ограничения:
1. Иногда (в облачной инфраструктуре некоторых компаний) сервер предоставляется с отдельными дисками, подмонтированными в виде отдельных разделов (JBOD), без возможности создания RAID.
2. В ClickHouse для обеспечения избыточности обычно используется репликация между серверами. Но при восстановлении одного из дисков RAID не используются данные с реплик, а в случае отказа одного из дисков в RAID-0, приходится передавать с реплики все данные, а не только данные, соответствующие одному из дисков. Это происходит, потому что RAID не интегрирован в ClickHouse и "не знает" про его особенности.
3. Отсутствуют продвинутые варианты обеспечения избыточности, как например, LRC.
Для преодоления этих ограничений, предлагается реализовать в ClickHouse встроенный алгоритм расположения данных на дисках.
### 24.7. Вероятностные структуры данных для фильтрации по подзапросам. ### 24.7. Вероятностные структуры данных для фильтрации по подзапросам.
Рузель Ибрагимов, ВШЭ и Яндекс. Рузель Ибрагимов, ВШЭ и Яндекс.
Частой задачей является выполнение запроса с фильтрацией по множеству, полученному по подзапросу. Пример: найти пользователей, которые заходили на сайт сегодня и заходили неделю назад. Это выражается в виде запроса: `SELECT UserID FROM table WHERE EventDate = today() AND UserID IN (SELECT ...)`. При выполнении этого запроса, сначала выполняется подзапрос в правой части `IN` и формируется хэш-таблица в оперативке; затем эта хэш-таблица используется для фильтрации.
Иногда объём данных достаточно большой, и хэш-таблица не помещается в оперативку. В этом случае можно рассмотреть в качестве варианта приближённый рассчёт: найти пользователей, которые заходили на сайт сегодня и наверное заходили неделю назад. Для этого можно вместо хэш-таблицы использовать Bloom Filter. Другая задача: найти пользователей, которые встречались, скорее всего, не менее некоторого количества раз. Для этого можно использовать Counting Bloom Filter. Также следует изучить структуры данных Quotient Filter и Cuckoo Filer, а ещё - секретный алгоритм Chaotic Map от Андрея Плахова.
Предлагается реализовать это в языке запросов ClickHouse с помощью специального синтаксиса, например `x IN BLOOM FILTER (n, m) (SELECT ...)`.
### 24.8. Специализация векторизованного кода для AVX/AVX2/AVX512 и ARM NEON. ### 24.8. Специализация векторизованного кода для AVX/AVX2/AVX512 и ARM NEON.
Дмитрий Ковальков, ВШЭ и Яндекс. Дмитрий Ковальков, ВШЭ и Яндекс.
Подавляющее большинство кода ClickHouse написана для x86_64 с набором инструкций до SSE 4.2 включительно. Лишь отдельные редкие функции поддерживают AVX/AVX2/AVX512 с динамической диспетчеризацией.
В первой части задачи, следует добавить в ClickHouse реализации некоторых примитивов, оптимизированные под более новый набор инструкций. Например, AVX2 реализацию генератора случайных чисел pcg: https://github.com/lemire/simdpcg
Во второй части задачи, предлагается адаптировать существующие куски кода, использующие SSE intrinsics на AVX/AVX2 и сравнить производительность. Также рассматривается оптимизация под ARM NEON.
### 24.9. Общий подход к CPU dispatching в фабрике функций. ### 24.9. Общий подход к CPU dispatching в фабрике функций.
Дмитрий Ковальков, ВШЭ и Яндекс. Дмитрий Ковальков, ВШЭ и Яндекс.
Продолжение 24.8.
### 24.10. Поддержка типов half/bfloat16/unum. ### 24.10. Поддержка типов half/bfloat16/unum.
Рустам Гусейн-заде, ВШЭ. Рустам Гусейн-заде, ВШЭ.
@ -1139,12 +1411,30 @@ Yuchen Dong, ICS.
Игорь Минеев, ВШЭ. Игорь Минеев, ВШЭ.
ClickHouse предоставляет достаточно богатый набор встроенных функций языка запросов, но не позволяет пользователю добавлять свои функции без редактировния исходников и перекомпиляции системы. Это мотивировано следующими потенциальными проблемами:
1. ClickHouse является array-oriented системой, и все функции внутри кода принимают для обработки целые массивы, а не отдельные значения. Это усложняет внутренний интерфейс и делает его менее удобным для пользователя.
2. Предоставление возможности подключения UDF в виде shared библиотек, потребовало бы фиксировать этот интерфейс или поддерживать обратную совместимость, тогда как мы бы хотели, при разработке ClickHouse, менять этот интерфейс по своему усмотрению без оглядки.
3. Сложность внутренних структур данных повышает вероятность ошибок типа buffer overflow и повреждения памяти, что сильно затруднит сопровождение ClickHouse с пользовательскими функциями.
Тем не менее, можно выбрать более аккуратный подход, избегающий непосредственной линковки с shared библиотеками.
Сначала можно реализовать поддержку UDF в виде выражений, составленных из простых функций ClickHouse. В ClickHouse есть встроенная кодогенерация на LLVM, что позволит таким функциям работать весьма эффективно. Но этот подход весьма ограничен и поэтому не является исчерпывающим.
Затем предлагается реализовать поддержку UDF в виде исходников на C++, которые компилируются в runtime, с использованием заголовочных файлов ClickHouse. Требование компиляции из исходников вместо shared библиотек, позволит ослабить необходимость в поддержке совместимости ABI.
Для безопасности, потребуется исследовать возможность размещения буферов данных в shared memory для выполнения UDF в отдельных процессах с изоляцией по памяти. Возможно, для этого пригодится интеграция с Apache Arrow.
Также рассматривается возможность написания UDF на Rust, а также использование Web Assembly. Отдельно можно рассмотреть подключение NumPy и R и других технологий, которые предоставляют операции над целыми массивами.
### 24.12. GPU offloading. ### 24.12. GPU offloading.
Риск состоит в том, что даже известные GPU базы, такие как OmniSci, работают медленнее, чем ClickHouse. Риск состоит в том, что даже известные GPU базы, такие как OmniSci, работают медленнее, чем ClickHouse.
Преимущество возможно только на полной сортировке и JOIN. Преимущество возможно только на полной сортировке и JOIN.
Алексей Соловей, nVidia и Рита Коннова, ВШЭ. Алексей Соловей, nVidia и Рита Коннова, ВШЭ.
В компании nVidia сделали прототип offloading вычисления GROUP BY с некоторыми из агрегатных функций в ClickHouse и обещат предоставить исходники в публичный доступ для дальнейшего развития. Предлагается изучить этот прототип и расширить его применимость для более широкого сценария использования. В качестве альтернативы, предлагается изучить исходные коды системы `OmniSci` или `Alenka` или библиотеку `CUB` https://nvlabs.github.io/cub/ и применить некоторые из алгоритмов в ClickHouse.
### 24.13. Stream запросы. ### 24.13. Stream запросы.
Пререквизит для ClickHouse как CEP-системы. Пререквизит для ClickHouse как CEP-системы.
@ -1181,14 +1471,32 @@ Yuchen Dong, ICS.
Эльмир Марданов, ВШЭ. Эльмир Марданов, ВШЭ.
ClickHouse является строго типизированной системой. Для того, чтобы прочитать данные в каком либо формате (например, CSV), требуется заранее указать типы данных. Если при чтении формата выясняется, что данные не могут быть прочитаны в рамках заданных типов, то кидается исключение.
ClickHouse также может использоваться для быстрой аналитики по локальным файлам, без загрузки их в базу данных (программа `clickhouse-local`). В этом случае, его использование может заменить `awk`, `sed`, `grep`. Но остаётся неудобство - необходимость указания типов данных.
Предлагается реализовать функциональность вывода типов по первому блоку данных путём применения эвристик и постепенного расширения типов.
Другая экспериментальная задача - реализация эвристик для обработки данных в неизвестном построчном текстовом формате. Детектирование CSV, TSV, JSON, детектирование разделителей и форматов значений.
### 24.23. Минимальная поддержка транзакций для множества вставок/чтений. ### 24.23. Минимальная поддержка транзакций для множества вставок/чтений.
Максим Кузнецов, ВШЭ. Максим Кузнецов, ВШЭ.
Таблицы типа MergeTree состоят из набора независимых неизменяемых "кусков" данных. При вставках данных (INSERT), формируются новые куски. При модификациях данных (слияние кусков), формируются новые куски, а старые - становятся неактивными и перестают использоваться следующими запросами. Чтение данных (SELECT) производится из снэпшота множества кусков на некоторый момент времени. Таким образом, чтения и вставки не блокируют друг друга.
Если же выполняется несколько запросов SELECT, то чтение данных может осуществляться из снэпшотов по состоянию на несколько разных моментов времени и быть неконсистентным. Пример: пользователю отображается отчёт из нескольких графиков и таблиц, но из-за того, что между разными запросами, данные успели обновиться, отображаемые данные не соответствуют друг другу.
Пример с другой стороны - пользователь хочет осуществить несколько вставок (INSERT) в одну или несколько таблиц, но так, чтобы данные появились в них атомарно с точки зрения других запросов (SELECT).
Для решения этих проблем, предлагается ввести глобальные метки времени для кусков данных (сейчас уже есть инкрементальные номера кусков, но они выделяются в рамках одной таблицы). Первым шагом сделаем эти метки времени в рамках сервера. Вторым шагом сделаем метки времени в рамках всех серверов, но неточные на основе локальных часов. Третьим шагом сделаем метки времени, выдаваемые сервисом координации.
### 24.24. Реализация алгоритмов differential privacy. ### 24.24. Реализация алгоритмов differential privacy.
Артём Вишняков, ВШЭ. Артём Вишняков, ВШЭ.
https://github.com/yandex/ClickHouse/issues/6874
### 24.25. Интеграция в ClickHouse функциональности обработки HTTP User Agent. ### 24.25. Интеграция в ClickHouse функциональности обработки HTTP User Agent.
Есть хороший код в Яндекс.Метрике. Получено согласие от руководства. Есть хороший код в Яндекс.Метрике. Получено согласие от руководства.
@ -1202,6 +1510,8 @@ Yuchen Dong, ICS.
ucasFL, ICS. ucasFL, ICS.
Алгоритмы min-hash и sim-hash позволяют вычислить для текста несколько хэш-значений таких, что при небольшом изменении текста, по крайней мере один из хэшей не меняется. Вычисления можно реализовать на n-грамах и словарных шинглах. Предлагается добавить поддержку этих алгоритмов в виде функций в ClickHouse и изучить их применимость для задачи нечёткого поиска полудубликатов.
### 24.28. Другой sketch для квантилей. ### 24.28. Другой sketch для квантилей.
Похоже на quantileTiming, но с логарифмическими корзинами. Похоже на quantileTiming, но с логарифмическими корзинами.
@ -1221,6 +1531,12 @@ Amos Bird, но его решение слишком громоздкое и п
Мария Конькова, ВШЭ и Яндекс. Мария Конькова, ВШЭ и Яндекс.
Также смотрите 24.29. Также смотрите 24.29.
В ClickHouse есть два основных протокола: родной протокол общения между серверами и HTTP/1.1 протокол. HTTP/1.1 протокол удобен для работы из самых разных языков программирования, но, в отличие от родного протокола, не поддерживает двусторонний обмен информацией во время запроса:
- передачу информации о прогрессе во время выполнения запроса;
- передачу логов во время выполнения запроса;
- отмену выполнения запроса в тот момент как данные ещё не начали передаваться;
Рассматривается вариант - поддержка GRPC в ClickHouse. Здесь есть неочевидные моменты, такие как - эффективная передача массивов данных в column-oriented формате - насколько удобно будет обернуть это в GRPC.
## 25. DevRel ## 25. DevRel
@ -1231,25 +1547,25 @@ Amos Bird, но его решение слишком громоздкое и п
### 25.2. Вычитка и выкладка статьи про обфускацию данных на английском. ### 25.2. Вычитка и выкладка статьи про обфускацию данных на английском.
Эми Жанель Кришниевски, Александр Казаков, Алексей Миловидов, ноябрь 2019. Эми, Александр Казаков, Алексей Миловидов, ноябрь 2019.
### 25.3. Подготовка статьи "Секреты оптимизации производительности ClickHouse". ### 25.3. Подготовка статьи "Секреты оптимизации производительности ClickHouse".
Алексей Миловидов, Леонид Клюев. Алексей Миловидов, Леонид.
### 25.4. Подготовка статьи "Профайлер запросов: трудный путь". ### 25.4. Подготовка статьи "Профайлер запросов: трудный путь".
Алексей Миловидов, Леонид Клюев. Алексей Миловидов, Леонид.
### 25.5. Подготовка статьи "Тестирование ClickHouse, которое мы заслужили". ### 25.5. Подготовка статьи "Тестирование ClickHouse, которое мы заслужили".
### 25.6. Перевод этих статей на английский. ### 25.6. Перевод этих статей на английский.
Требует 25.3, 25.4, 25.5. Эми Жанель Кришниевски Требует 25.3, 25.4, 25.5. Эми
### 25.7. Перевод статьи Данилы Кутенина на английский. ### 25.7. Перевод статьи Данилы Кутенина на английский.
Эми Жанель Кришниевски Эми
### 25.8. Выступление keynote на BDTC. ### 25.8. Выступление keynote на BDTC.
@ -1263,7 +1579,7 @@ Amos Bird, но его решение слишком громоздкое и п
### 25.10. Митапы в России и Беларуси: Москва x2 + митап для разработчиков или хакатон, Санкт-Петербург, Минск, Нижний Новгород, Екатеринбург, Новосибирск и/или Академгородок, Иннополис или Казань. ### 25.10. Митапы в России и Беларуси: Москва x2 + митап для разработчиков или хакатон, Санкт-Петербург, Минск, Нижний Новгород, Екатеринбург, Новосибирск и/или Академгородок, Иннополис или Казань.
Екатерина Миназова - организация Екатерина - организация
### 25.11. Митапы зарубежные: восток США (Нью Йорк, возможно Raleigh), возможно северо-запад (Сиэтл), Китай (Пекин снова, возможно митап для разработчиков или хакатон), Лондон. ### 25.11. Митапы зарубежные: восток США (Нью Йорк, возможно Raleigh), возможно северо-запад (Сиэтл), Китай (Пекин снова, возможно митап для разработчиков или хакатон), Лондон.
@ -1281,12 +1597,23 @@ Amos Bird, но его решение слишком громоздкое и п
Алексей Миловидов и все подготовленные докладчики Алексей Миловидов и все подготовленные докладчики
### 25.15. Конференции зарубежные: Percona, DataOps, возможно Big Data Warsaw, попытка попасть на более крупные. ### 25.15. Конференции зарубежные: Percona, DataOps, попытка попасть на более крупные.
Алексей Миловидов и все подготовленные докладчики Алексей Миловидов и все подготовленные докладчики
### 25.16. Сайт play.clickhouse. ### 25.16. Сайт play.clickhouse.
Цель состоит в реализации сайта, на котором можно попробовать задавать произвольные запросы к временному экземпляру ClickHouse и изучать его поведение. Из похожих проектов можно отметить: [Compiler Explorer](https://godbolt.org/), http://ideone.com/, [SQLFiddle](http://sqlfiddle.com/), [DB-Fiddle](https://www.db-fiddle.com/).
С помощью такого сайта можно решать следующие задачи:
- ознакомление с языком запросов ClickHouse;
- демонстрация примеров из документации;
- демонстрация скорости работы на тестовых датасетах;
- сравнение поведения разных версий ClickHouse друг с другом;
- демонстрация неожиданного поведения или багов;
Требуется проработать вопрос безопасности и изоляции инстансов (поднятие в контейнерах с ограничениями по сети), подключение тестовых датасетов с помощью copy-on-write файловой системы; органичения ресурсов.
### 25.17. Взаимодействие с ВУЗами: ВШЭ, УрФУ, ICS Beijing. ### 25.17. Взаимодействие с ВУЗами: ВШЭ, УрФУ, ICS Beijing.
Алексей Миловидов и вся группа разработки Алексей Миловидов и вся группа разработки
@ -1301,6 +1628,8 @@ Amos Bird, но его решение слишком громоздкое и п
Матвей Бубнов, УрФУ Матвей Бубнов, УрФУ
Существуют мало известные специализированные СУБД, способные конкурировать с ClickHouse по скорости обработки некоторых классов запросов. Пример: `TDEngine` и `DolphinDB`, `VictoriaMetrics`, а также `Apache Doris` и `LocustDB`. Предлагается изучить и классифицировать архитектурные особенности этих систем - их особенности и преимущества. Установить эти системы, загрузить тестовые данные, изучить производительность. Проанализировать, за счёт чего достигаются преимущества.
### 25.21. Повторное награждение контрибьюторов в Китае. ### 25.21. Повторное награждение контрибьюторов в Китае.
### 25.22. On-site помощь с ClickHouse компаниям в дни рядом с мероприятиями. ### 25.22. On-site помощь с ClickHouse компаниям в дни рядом с мероприятиями.
@ -1323,4 +1652,4 @@ Amos Bird, но его решение слишком громоздкое и п
### 25.27. Обновить сайт ClickHouse. ### 25.27. Обновить сайт ClickHouse.
Иван Блинков. Есть риск, что станет хуже. Иван Блинков. Есть риски.

View File

@ -49,4 +49,4 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
``` ```
## Запросы ## Запросы
Примеры запросов к этим таблицам (они называются `test.hits` и `test.visits`) можно найти среди [stateful тестов](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) и в некоторых [performance тестах](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance/test_hits) ClickHouse. Примеры запросов к этим таблицам (они называются `test.hits` и `test.visits`) можно найти среди [stateful тестов](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/queries/1_stateful) и в некоторых [performance тестах](https://github.com/ClickHouse/ClickHouse/tree/master/dbms/tests/performance) ClickHouse.

View File

@ -702,12 +702,12 @@ ClickHouse использует ZooKeeper для хранения метадан
Например: Например:
```xml ```xml
<node index="1"> <node index="1">
<host>example_host</host> <host>example_host</host>
<port>2181</port> <port>2181</port>
</node> </node>
``` ```
Атрибут `index` задает порядок опроса нод при попытках подключиться к кластеру ZooKeeper. Атрибут `index` задает порядок опроса нод при попытках подключиться к кластеру ZooKeeper.

View File

@ -188,7 +188,8 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name;
- [DETACH PARTITION](#alter_detach-partition) перенести партицию в директорию `detached`; - [DETACH PARTITION](#alter_detach-partition) перенести партицию в директорию `detached`;
- [DROP PARTITION](#alter_drop-partition) удалить партицию; - [DROP PARTITION](#alter_drop-partition) удалить партицию;
- [ATTACH PARTITION|PART](#alter_attach-partition) добавить партицию/кусок в таблицу из директории `detached`; - [ATTACH PARTITION|PART](#alter_attach-partition) добавить партицию/кусок в таблицу из директории `detached`;
- [REPLACE PARTITION](#alter_replace-partition) скопировать партицию из другой таблицы; - [ATTACH PARTITION FROM](#alter_attach-partition-from) скопировать партицию из другой таблицы;
- [REPLACE PARTITION](#alter_replace-partition) скопировать партицию из другой таблицы с заменой;
- [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) удалить все значения в столбце для заданной партиции; - [CLEAR COLUMN IN PARTITION](#alter_clear-column-partition) удалить все значения в столбце для заданной партиции;
- [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить построенные вторичные индексы для заданной партиции; - [CLEAR INDEX IN PARTITION](#alter_clear-index-partition) - очистить построенные вторичные индексы для заданной партиции;
- [FREEZE PARTITION](#alter_freeze-partition) создать резервную копию партиции; - [FREEZE PARTITION](#alter_freeze-partition) создать резервную копию партиции;
@ -255,13 +256,29 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0;
Это означает, что вы можете разместить данные в директории `detached` на одной реплике и с помощью запроса `ALTER ... ATTACH` добавить их в таблицу на всех репликах. Это означает, что вы можете разместить данные в директории `detached` на одной реплике и с помощью запроса `ALTER ... ATTACH` добавить их в таблицу на всех репликах.
#### ATTACH PARTITION FROM {#alter_attach-partition-from}
```sql
ALTER TABLE table2 ATTACH PARTITION partition_expr FROM table1
```
Копирует партицию из таблицы `table1` в таблицу `table2` и добавляет к существующим данным `table2`. Данные из `table1` не удаляются.
Следует иметь в виду:
- Таблицы должны иметь одинаковую структуру.
- Для таблиц должен быть задан одинаковый ключ партиционирования.
Подробнее о том, как корректно задать имя партиции, см. в разделе [Как задавать имя партиции в запросах ALTER](#alter-how-to-specify-part-expr).
#### REPLACE PARTITION {#alter_replace-partition} #### REPLACE PARTITION {#alter_replace-partition}
```sql ```sql
ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1 ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1
``` ```
Копирует партицию из таблицы `table1` в таблицу `table2`. Данные из `table1` не удаляются. Копирует партицию из таблицы `table1` в таблицу `table2` с заменой существующих данных в `table2`. Данные из `table1` не удаляются.
Следует иметь в виду: Следует иметь в виду:

View File

@ -778,6 +778,22 @@ SELECT arrayReduce('uniqUpTo(3)', [1, 2, 3, 4, 5, 6, 7, 8, 9, 10])
└─────────────────────────────────────────────────────────────┘ └─────────────────────────────────────────────────────────────┘
``` ```
## arrayFlatten(arr) {#array_functions-arrayflatten}
Функция `arrayFlatten` (или `flatten`) соеденит вложенные массивы и вернет массив из их элементов.
Пример:
```sql
SELECT arrayFlatten([[1, 2, 3], [4, 5]])
```
```text
┌─arrayFlatten([[1, 2, 3], [4, 5]])─┐
│ [1,2,3,4,5] │
└───────────────────────────────────┘
```
## arrayReverse(arr) {#array_functions-arrayreverse} ## arrayReverse(arr) {#array_functions-arrayreverse}
Возвращает массив того же размера, что и исходный массив, содержащий элементы в обратном порядке. Возвращает массив того же размера, что и исходный массив, содержащий элементы в обратном порядке.

View File

@ -62,10 +62,10 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ...
### Замечания о производительности ### Замечания о производительности
`INSERT` сортирует входящие данные по первичному ключу и разбивает их на партиции по месяцам. Если вы вставляете данные за разные месяцы вперемешку, то это может значительно снизить производительность запроса `INSERT`. Чтобы избежать этого: `INSERT` сортирует входящие данные по первичному ключу и разбивает их на партиции по ключу партиционирования. Если вы вставляете данные в несколько партиций одновременно, то это может значительно снизить производительность запроса `INSERT`. Чтобы избежать этого:
- Добавляйте данные достаточно большими пачками. Например, по 100 000 строк. - Добавляйте данные достаточно большими пачками. Например, по 100 000 строк.
- Группируйте данные по месяцам самостоятельно перед загрузкой в ClickHouse. - Группируйте данные по ключу партиционирования самостоятельно перед загрузкой в ClickHouse.
Снижения производительности не будет, если: Снижения производительности не будет, если:

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