mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 23:52:03 +00:00
Merge branch 'master' into minio-integration
# Conflicts: # dbms/tests/integration/helpers/cluster.py
This commit is contained in:
commit
51848b87e0
@ -14,7 +14,6 @@ ClickHouse is an open-source column-oriented database management system that all
|
||||
|
||||
## 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 San Francisco](https://www.eventbrite.com/e/clickhouse-december-meetup-registration-78642047481) on December 3.
|
||||
|
||||
|
2
contrib/libunwind
vendored
2
contrib/libunwind
vendored
@ -1 +1 @@
|
||||
Subproject commit 5afe6d87ae9e66485c7fcb106d2f7c2c0359c8f6
|
||||
Subproject commit 68cffcbbd1840e14664a5f7f19c5e43f65c525b5
|
2
contrib/protobuf
vendored
2
contrib/protobuf
vendored
@ -1 +1 @@
|
||||
Subproject commit 12735370922a35f03999afff478e1c6d7aa917a4
|
||||
Subproject commit 0795fa6bc443666068bec56bf700e1f488f592f1
|
@ -1112,7 +1112,14 @@ private:
|
||||
/// Check if server send Exception packet
|
||||
auto packet_type = connection->checkPacket();
|
||||
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;
|
||||
}
|
||||
|
||||
connection->sendData(block);
|
||||
processed_rows += block.rows();
|
||||
|
@ -407,16 +407,16 @@ void HTTPHandler::processQuery(
|
||||
{
|
||||
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")
|
||||
{
|
||||
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
|
||||
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
|
||||
else
|
||||
|
@ -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,
|
||||
* but we are not going to do it, because settings is used everywhere as static struct fields.
|
||||
*
|
||||
* `flags` can be either 0 or IGNORABLE.
|
||||
* A setting is "IGNORABLE" if it doesn't affects the results of the queries and can be ignored without exception.
|
||||
* `flags` can be either 0 or IMPORTANT.
|
||||
* A setting is "IMPORTANT" if it affects the results of queries and can't be ignored by older versions.
|
||||
*/
|
||||
|
||||
#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, 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(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, 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) \
|
||||
@ -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(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(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, 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_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) \
|
||||
@ -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(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) \
|
||||
\
|
||||
@ -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_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_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_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(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(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) \
|
||||
\
|
||||
@ -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_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(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_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) \
|
||||
|
@ -286,7 +286,7 @@ enum class SettingsBinaryFormat
|
||||
* {
|
||||
* # define APPLY_FOR_MYSETTINGS(M) \
|
||||
* 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)
|
||||
*
|
||||
* DECLARE_SETTINGS_COLLECTION(MySettings, APPLY_FOR_MYSETTINGS)
|
||||
@ -316,7 +316,7 @@ private:
|
||||
|
||||
StringRef name;
|
||||
StringRef description;
|
||||
bool is_ignorable;
|
||||
bool is_important;
|
||||
IsChangedFunction is_changed;
|
||||
GetStringFunction get_string;
|
||||
GetFieldFunction get_field;
|
||||
|
@ -254,7 +254,7 @@ void SettingsCollection<Derived>::serialize(WriteBuffer & buf, SettingsBinaryFor
|
||||
{
|
||||
details::SettingsCollectionUtils::serializeName(member.name, buf);
|
||||
if (format >= SettingsBinaryFormat::STRINGS)
|
||||
details::SettingsCollectionUtils::serializeFlag(member.is_ignorable, buf);
|
||||
details::SettingsCollectionUtils::serializeFlag(member.is_important, buf);
|
||||
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 */)
|
||||
break;
|
||||
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)
|
||||
{
|
||||
member->deserialize(castToDerived(), buf, format);
|
||||
}
|
||||
else if (is_ignorable)
|
||||
else if (is_important)
|
||||
{
|
||||
details::SettingsCollectionUtils::throwNameNotFound(name);
|
||||
}
|
||||
else
|
||||
{
|
||||
details::SettingsCollectionUtils::warningNameNotFound(name);
|
||||
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_) \
|
||||
}; \
|
||||
constexpr int IGNORABLE = 1; \
|
||||
UNUSED(IGNORABLE); \
|
||||
constexpr int IMPORTANT = 1; \
|
||||
UNUSED(IMPORTANT); \
|
||||
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) \
|
||||
add({StringRef(#NAME, strlen(#NAME)), StringRef(DESCRIPTION, strlen(DESCRIPTION)), \
|
||||
FLAGS & IGNORABLE, \
|
||||
FLAGS & IMPORTANT, \
|
||||
[](const Derived & d) { return d.NAME.changed; }, \
|
||||
&Functions::NAME##_getString, &Functions::NAME##_getField, \
|
||||
&Functions::NAME##_setString, &Functions::NAME##_setField, \
|
||||
|
203
dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp
Normal file
203
dbms/src/DataStreams/ParallelParsingBlockInputStream.cpp
Normal 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;
|
||||
}
|
||||
|
||||
|
||||
}
|
258
dbms/src/DataStreams/ParallelParsingBlockInputStream.h
Normal file
258
dbms/src/DataStreams/ParallelParsingBlockInputStream.h
Normal 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();
|
||||
};
|
||||
|
||||
};
|
@ -125,7 +125,11 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadAll()
|
||||
* the necessity of holding process_list_element shared pointer.
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
|
@ -1,8 +1,10 @@
|
||||
#include <algorithm>
|
||||
#include <Common/config.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Core/Settings.h>
|
||||
#include <DataStreams/MaterializingBlockOutputStream.h>
|
||||
#include <DataStreams/ParallelParsingBlockInputStream.h>
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <Processors/Formats/IRowInputFormat.h>
|
||||
@ -93,7 +95,7 @@ BlockInputStreamPtr FormatFactory::getInput(
|
||||
|
||||
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)
|
||||
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);
|
||||
}
|
||||
|
||||
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));
|
||||
return std::make_shared<InputStreamFromInputFormat>(std::move(format));
|
||||
}
|
||||
@ -191,7 +224,7 @@ OutputFormatPtr FormatFactory::getOutputFormat(
|
||||
|
||||
void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator)
|
||||
{
|
||||
auto & target = dict[name].inout_creator;
|
||||
auto & target = dict[name].input_creator;
|
||||
if (target)
|
||||
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
|
||||
target = std::move(input_creator);
|
||||
@ -221,6 +254,13 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro
|
||||
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.
|
||||
|
||||
@ -249,6 +289,10 @@ void registerOutputFormatProcessorProtobuf(FormatFactory & factory);
|
||||
void registerInputFormatProcessorTemplate(FormatFactory & factory);
|
||||
void registerOutputFormatProcessorTemplate(FormatFactory &factory);
|
||||
|
||||
/// File Segmentation Engines for parallel reading
|
||||
|
||||
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory);
|
||||
|
||||
/// Output only (presentational) formats.
|
||||
|
||||
void registerOutputFormatNull(FormatFactory & factory);
|
||||
@ -299,6 +343,7 @@ FormatFactory::FormatFactory()
|
||||
registerInputFormatProcessorTemplate(*this);
|
||||
registerOutputFormatProcessorTemplate(*this);
|
||||
|
||||
registerFileSegmentationEngineTabSeparated(*this);
|
||||
|
||||
registerOutputFormatNull(*this);
|
||||
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Core/Types.h>
|
||||
#include <DataStreams/IBlockStream_fwd.h>
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
@ -41,6 +42,15 @@ public:
|
||||
/// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer.
|
||||
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.
|
||||
/// It's initial purpose was to flush Kafka message for each row.
|
||||
using WriteCallback = std::function<void()>;
|
||||
@ -77,10 +87,11 @@ private:
|
||||
|
||||
struct Creators
|
||||
{
|
||||
InputCreator inout_creator;
|
||||
InputCreator input_creator;
|
||||
OutputCreator output_creator;
|
||||
InputProcessorCreator input_processor_creator;
|
||||
OutputProcessorCreator output_processor_creator;
|
||||
FileSegmentationEngine file_segmentation_engine;
|
||||
};
|
||||
|
||||
using FormatsDictionary = std::unordered_map<String, Creators>;
|
||||
@ -114,6 +125,7 @@ public:
|
||||
/// Register format by its name.
|
||||
void registerInputFormat(const String & name, InputCreator input_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 registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator);
|
||||
|
@ -32,9 +32,9 @@ public:
|
||||
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)
|
||||
, in(in_)
|
||||
, in(std::move(in_))
|
||||
, brotli(std::make_unique<BrotliStateWrapper>())
|
||||
, in_available(0)
|
||||
, in_data(nullptr)
|
||||
@ -55,12 +55,12 @@ bool BrotliReadBuffer::nextImpl()
|
||||
|
||||
if (!in_available)
|
||||
{
|
||||
in.nextIfAtEnd();
|
||||
in_available = in.buffer().end() - in.position();
|
||||
in_data = reinterpret_cast<uint8_t *>(in.position());
|
||||
in->nextIfAtEnd();
|
||||
in_available = in->buffer().end() - 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);
|
||||
}
|
||||
@ -70,12 +70,12 @@ bool BrotliReadBuffer::nextImpl()
|
||||
|
||||
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);
|
||||
|
||||
if (brotli->result == BROTLI_DECODER_RESULT_SUCCESS)
|
||||
{
|
||||
if (in.eof())
|
||||
if (in->eof())
|
||||
{
|
||||
eof = true;
|
||||
return working_buffer.size() != 0;
|
||||
|
@ -11,7 +11,7 @@ class BrotliReadBuffer : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
BrotliReadBuffer(
|
||||
ReadBuffer & in_,
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
size_t alignment = 0);
|
||||
@ -21,7 +21,7 @@ public:
|
||||
private:
|
||||
bool nextImpl() override;
|
||||
|
||||
ReadBuffer & in;
|
||||
std::unique_ptr<ReadBuffer> in;
|
||||
|
||||
class BrotliStateWrapper;
|
||||
std::unique_ptr<BrotliStateWrapper> brotli;
|
||||
|
@ -12,6 +12,7 @@ enum class CompressionMethod
|
||||
/// This option corresponds to HTTP Content-Encoding: deflate.
|
||||
Zlib,
|
||||
Brotli,
|
||||
None
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -19,7 +19,6 @@ bool PeekableReadBuffer::peekNext()
|
||||
{
|
||||
checkStateCorrect();
|
||||
|
||||
size_t bytes_read = 0;
|
||||
Position copy_from = pos;
|
||||
size_t bytes_to_copy = sub_buf.available();
|
||||
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)
|
||||
if (checkpoint)
|
||||
copy_from = checkpoint;
|
||||
bytes_read = copy_from - sub_buf.buffer().begin();
|
||||
bytes_to_copy = sub_buf.buffer().end() - copy_from;
|
||||
if (!bytes_to_copy)
|
||||
{
|
||||
bytes += bytes_read;
|
||||
sub_buf.position() = copy_from;
|
||||
|
||||
/// 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())
|
||||
{
|
||||
bytes += bytes_read;
|
||||
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
|
||||
peeked_size -= offset;
|
||||
memmove(memory.data(), memory.data() + offset, peeked_size);
|
||||
bytes += offset;
|
||||
|
||||
if (need_update_checkpoint)
|
||||
checkpoint -= offset;
|
||||
|
@ -23,9 +23,11 @@
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
|
||||
#include <IO/CompressionMethod.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/ReadBufferFromMemory.h>
|
||||
#include <IO/VarInt.h>
|
||||
#include <IO/ZlibInflatingReadBuffer.h>
|
||||
|
||||
#ifdef __clang__
|
||||
#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.
|
||||
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...);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -90,6 +90,9 @@ public:
|
||||
++pos;
|
||||
}
|
||||
|
||||
virtual void sync() {}
|
||||
virtual void finalize() {}
|
||||
|
||||
private:
|
||||
/** Write the data in the buffer (from the beginning of the buffer to the current position).
|
||||
* Throw an exception if something is wrong.
|
||||
|
@ -51,7 +51,7 @@ private:
|
||||
/// Prepare an asynchronous request.
|
||||
void prepare();
|
||||
///
|
||||
void finalize();
|
||||
void finalize() override;
|
||||
|
||||
private:
|
||||
/// Buffer for asynchronous data writes.
|
||||
|
@ -25,7 +25,7 @@ public:
|
||||
|
||||
~WriteBufferFromHDFS() override;
|
||||
|
||||
void sync();
|
||||
void sync() override;
|
||||
};
|
||||
}
|
||||
#endif
|
||||
|
@ -113,8 +113,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
|
||||
response.set("Content-Encoding", "gzip");
|
||||
response_body_ostr = &(response.send());
|
||||
#endif
|
||||
out_raw.emplace(*response_body_ostr);
|
||||
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
|
||||
out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
|
||||
deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin());
|
||||
out = &*deflating_buf;
|
||||
}
|
||||
else if (compression_method == CompressionMethod::Zlib)
|
||||
@ -125,8 +125,8 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
|
||||
response.set("Content-Encoding", "deflate");
|
||||
response_body_ostr = &(response.send());
|
||||
#endif
|
||||
out_raw.emplace(*response_body_ostr);
|
||||
deflating_buf.emplace(*out_raw, compression_method, compression_level, working_buffer.size(), working_buffer.begin());
|
||||
out_raw = std::make_unique<WriteBufferFromOStream>(*response_body_ostr);
|
||||
deflating_buf.emplace(std::move(out_raw), compression_method, compression_level, working_buffer.size(), working_buffer.begin());
|
||||
out = &*deflating_buf;
|
||||
}
|
||||
#if USE_BROTLI
|
||||
@ -138,7 +138,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
|
||||
response.set("Content-Encoding", "br");
|
||||
response_body_ostr = &(response.send());
|
||||
#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());
|
||||
out = &*brotli_buf;
|
||||
}
|
||||
@ -155,7 +155,7 @@ void WriteBufferFromHTTPServerResponse::nextImpl()
|
||||
response_body_ostr = &(response.send());
|
||||
#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;
|
||||
}
|
||||
}
|
||||
|
@ -60,7 +60,7 @@ private:
|
||||
std::ostream * response_header_ostr = nullptr;
|
||||
#endif
|
||||
|
||||
std::optional<WriteBufferFromOStream> out_raw;
|
||||
std::unique_ptr<WriteBufferFromOStream> out_raw;
|
||||
std::optional<ZlibDeflatingWriteBuffer> deflating_buf;
|
||||
#if USE_BROTLI
|
||||
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
|
||||
/// to change response HTTP code.
|
||||
/// This method is idempotent.
|
||||
void finalize();
|
||||
void finalize() override;
|
||||
|
||||
/// Turn compression on or off.
|
||||
/// The setting has any effect only if HTTP headers haven't been sent yet.
|
||||
|
@ -43,7 +43,7 @@ public:
|
||||
void nextImpl() override;
|
||||
|
||||
/// Receives response from the server after sending all data.
|
||||
void finalize();
|
||||
void finalize() override;
|
||||
|
||||
~WriteBufferFromS3() override;
|
||||
|
||||
|
@ -20,11 +20,13 @@
|
||||
#include <Common/UInt128.h>
|
||||
#include <Common/intExp.h>
|
||||
|
||||
#include <IO/CompressionMethod.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteIntText.h>
|
||||
#include <IO/VarInt.h>
|
||||
#include <IO/DoubleConverter.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <IO/ZlibDeflatingWriteBuffer.h>
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
|
||||
@ -905,4 +907,16 @@ inline String toString(const T & x)
|
||||
writeText(x, buf);
|
||||
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...);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -6,14 +6,14 @@ namespace DB
|
||||
{
|
||||
|
||||
ZlibDeflatingWriteBuffer::ZlibDeflatingWriteBuffer(
|
||||
WriteBuffer & out_,
|
||||
std::unique_ptr<WriteBuffer> out_,
|
||||
CompressionMethod compression_method,
|
||||
int compression_level,
|
||||
size_t buf_size,
|
||||
char * existing_memory,
|
||||
size_t alignment)
|
||||
: BufferWithOwnMemory<WriteBuffer>(buf_size, existing_memory, alignment)
|
||||
, out(out_)
|
||||
, out(std::move(out_))
|
||||
{
|
||||
zstr.zalloc = nullptr;
|
||||
zstr.zfree = nullptr;
|
||||
@ -64,18 +64,18 @@ void ZlibDeflatingWriteBuffer::nextImpl()
|
||||
|
||||
do
|
||||
{
|
||||
out.nextIfAtEnd();
|
||||
zstr.next_out = reinterpret_cast<unsigned char *>(out.position());
|
||||
zstr.avail_out = out.buffer().end() - out.position();
|
||||
out->nextIfAtEnd();
|
||||
zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
|
||||
zstr.avail_out = out->buffer().end() - out->position();
|
||||
|
||||
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
|
||||
// for computing CRC32, and it looks like msan is unable to comprehend
|
||||
// it fully, so it complains about the resulting value depending on the
|
||||
// uninitialized padding of the input buffer.
|
||||
__msan_unpoison(out.position(), zstr.avail_out);
|
||||
__msan_unpoison(out->position(), zstr.avail_out);
|
||||
|
||||
if (rc != Z_OK)
|
||||
throw Exception(std::string("deflate failed: ") + zError(rc), ErrorCodes::ZLIB_DEFLATE_FAILED);
|
||||
@ -92,18 +92,18 @@ void ZlibDeflatingWriteBuffer::finish()
|
||||
|
||||
while (true)
|
||||
{
|
||||
out.nextIfAtEnd();
|
||||
zstr.next_out = reinterpret_cast<unsigned char *>(out.position());
|
||||
zstr.avail_out = out.buffer().end() - out.position();
|
||||
out->nextIfAtEnd();
|
||||
zstr.next_out = reinterpret_cast<unsigned char *>(out->position());
|
||||
zstr.avail_out = out->buffer().end() - out->position();
|
||||
|
||||
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
|
||||
// for computing CRC32, and it looks like msan is unable to comprehend
|
||||
// it fully, so it complains about the resulting value depending on the
|
||||
// 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)
|
||||
{
|
||||
|
@ -20,7 +20,7 @@ class ZlibDeflatingWriteBuffer : public BufferWithOwnMemory<WriteBuffer>
|
||||
{
|
||||
public:
|
||||
ZlibDeflatingWriteBuffer(
|
||||
WriteBuffer & out_,
|
||||
std::unique_ptr<WriteBuffer> out_,
|
||||
CompressionMethod compression_method,
|
||||
int compression_level,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
@ -37,7 +37,7 @@ public:
|
||||
private:
|
||||
void nextImpl() override;
|
||||
|
||||
WriteBuffer & out;
|
||||
std::unique_ptr<WriteBuffer> out;
|
||||
z_stream zstr;
|
||||
bool finished = false;
|
||||
};
|
||||
|
@ -5,13 +5,13 @@ namespace DB
|
||||
{
|
||||
|
||||
ZlibInflatingReadBuffer::ZlibInflatingReadBuffer(
|
||||
ReadBuffer & in_,
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
CompressionMethod compression_method,
|
||||
size_t buf_size,
|
||||
char * existing_memory,
|
||||
size_t alignment)
|
||||
: BufferWithOwnMemory<ReadBuffer>(buf_size, existing_memory, alignment)
|
||||
, in(in_)
|
||||
, in(std::move(in_))
|
||||
, eof(false)
|
||||
{
|
||||
zstr.zalloc = nullptr;
|
||||
@ -49,21 +49,21 @@ bool ZlibInflatingReadBuffer::nextImpl()
|
||||
|
||||
if (!zstr.avail_in)
|
||||
{
|
||||
in.nextIfAtEnd();
|
||||
zstr.next_in = reinterpret_cast<unsigned char *>(in.position());
|
||||
zstr.avail_in = in.buffer().end() - in.position();
|
||||
in->nextIfAtEnd();
|
||||
zstr.next_in = reinterpret_cast<unsigned char *>(in->position());
|
||||
zstr.avail_in = in->buffer().end() - in->position();
|
||||
}
|
||||
zstr.next_out = reinterpret_cast<unsigned char *>(internal_buffer.begin());
|
||||
zstr.avail_out = internal_buffer.size();
|
||||
|
||||
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);
|
||||
|
||||
if (rc == Z_STREAM_END)
|
||||
{
|
||||
if (in.eof())
|
||||
if (in->eof())
|
||||
{
|
||||
eof = true;
|
||||
return working_buffer.size() != 0;
|
||||
|
@ -21,7 +21,7 @@ class ZlibInflatingReadBuffer : public BufferWithOwnMemory<ReadBuffer>
|
||||
{
|
||||
public:
|
||||
ZlibInflatingReadBuffer(
|
||||
ReadBuffer & in_,
|
||||
std::unique_ptr<ReadBuffer> in_,
|
||||
CompressionMethod compression_method,
|
||||
size_t buf_size = DBMS_DEFAULT_BUFFER_SIZE,
|
||||
char * existing_memory = nullptr,
|
||||
@ -32,7 +32,7 @@ public:
|
||||
private:
|
||||
bool nextImpl() override;
|
||||
|
||||
ReadBuffer & in;
|
||||
std::unique_ptr<ReadBuffer> in;
|
||||
z_stream zstr;
|
||||
bool eof;
|
||||
};
|
||||
|
@ -22,8 +22,8 @@ try
|
||||
Stopwatch stopwatch;
|
||||
|
||||
{
|
||||
DB::WriteBufferFromFile buf("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);
|
||||
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(std::move(buf), DB::CompressionMethod::Gzip, /* compression_level = */ 3);
|
||||
|
||||
stopwatch.restart();
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
@ -40,8 +40,8 @@ try
|
||||
}
|
||||
|
||||
{
|
||||
DB::ReadBufferFromFile buf("test_zlib_buffers.gz");
|
||||
DB::ZlibInflatingReadBuffer inflating_buf(buf, DB::CompressionMethod::Gzip);
|
||||
auto buf = std::make_unique<DB::ReadBufferFromFile>("test_zlib_buffers.gz");
|
||||
DB::ZlibInflatingReadBuffer inflating_buf(std::move(buf), DB::CompressionMethod::Gzip);
|
||||
|
||||
stopwatch.restart();
|
||||
for (size_t i = 0; i < n; ++i)
|
||||
|
@ -22,7 +22,7 @@ public:
|
||||
|
||||
private:
|
||||
ASTPtr query;
|
||||
Context context;
|
||||
const Context & context;
|
||||
|
||||
BlockInputStreamPtr executeImpl();
|
||||
};
|
||||
|
@ -98,7 +98,10 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
{
|
||||
const auto & query = query_ptr->as<ASTInsertQuery &>();
|
||||
checkAccess(query);
|
||||
|
||||
BlockIO res;
|
||||
StoragePtr table = getTable(query);
|
||||
res.pipeline.addStorageHolder(table);
|
||||
|
||||
auto table_lock = table->lockStructureForShare(true, context.getInitialQueryId());
|
||||
|
||||
@ -134,7 +137,6 @@ BlockIO InterpreterInsertQuery::execute()
|
||||
out_wrapper->setProcessListElement(context.getProcessListElement());
|
||||
out = std::move(out_wrapper);
|
||||
|
||||
BlockIO res;
|
||||
res.out = std::move(out);
|
||||
|
||||
/// What type of query: INSERT or INSERT SELECT?
|
||||
|
@ -230,7 +230,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
: options(options_)
|
||||
/// 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())
|
||||
, context(context_)
|
||||
, context(std::make_shared<Context>(context_))
|
||||
, storage(storage_)
|
||||
, input(input_)
|
||||
, log(&Logger::get("InterpreterSelectQuery"))
|
||||
@ -238,7 +238,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
checkStackSize();
|
||||
|
||||
initSettings();
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
if (settings.max_subquery_depth && options.subquery_depth > settings.max_subquery_depth)
|
||||
throw Exception("Too deep subqueries. Maximum: " + settings.max_subquery_depth.toString(),
|
||||
@ -252,7 +252,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
@ -278,7 +278,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
{
|
||||
/// Read from subquery.
|
||||
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();
|
||||
}
|
||||
@ -288,48 +288,48 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
{
|
||||
/// Read from table function. propagate all settings from initSettings(),
|
||||
/// 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
|
||||
{
|
||||
String database_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);
|
||||
if (storage_values.getDatabaseName() == database_name && storage_values.getTableName() == table_name)
|
||||
{
|
||||
/// Read from view source.
|
||||
storage = context.getViewSource();
|
||||
storage = context->getViewSource();
|
||||
}
|
||||
}
|
||||
|
||||
if (!storage)
|
||||
{
|
||||
/// 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)
|
||||
table_lock = storage->lockStructureForShare(false, context.getInitialQueryId());
|
||||
table_lock = storage->lockStructureForShare(false, context->getInitialQueryId());
|
||||
|
||||
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());
|
||||
|
||||
/// Save scalar sub queries's results in the query context
|
||||
if (context.hasQueryContext())
|
||||
if (context->hasQueryContext())
|
||||
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_ptr, syntax_analyzer_result, context,
|
||||
query_ptr, syntax_analyzer_result, *context,
|
||||
NameSet(required_result_column_names.begin(), required_result_column_names.end()),
|
||||
options.subquery_depth, !options.only_analyze);
|
||||
|
||||
@ -346,8 +346,8 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
|
||||
/// Save the new temporary tables in the query context
|
||||
for (const auto & it : query_analyzer->getExternalTables())
|
||||
if (!context.tryGetExternalTable(it.first))
|
||||
context.addExternalTable(it.first, it.second);
|
||||
if (!context->tryGetExternalTable(it.first))
|
||||
context->addExternalTable(it.first, it.second);
|
||||
}
|
||||
|
||||
if (!options.only_analyze || options.modify_inplace)
|
||||
@ -358,7 +358,7 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
if (is_subquery)
|
||||
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||
table_expression,
|
||||
getSubqueryContext(context),
|
||||
getSubqueryContext(*context),
|
||||
options.subquery(),
|
||||
required_columns);
|
||||
}
|
||||
@ -378,10 +378,10 @@ InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
source_header = storage->getSampleBlockForColumns(required_columns);
|
||||
|
||||
/// 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->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());
|
||||
}
|
||||
}
|
||||
@ -450,27 +450,34 @@ Block InterpreterSelectQuery::getSampleBlock()
|
||||
BlockIO InterpreterSelectQuery::execute()
|
||||
{
|
||||
Pipeline pipeline;
|
||||
executeImpl(pipeline, input);
|
||||
BlockIO res;
|
||||
executeImpl(pipeline, input, res.pipeline);
|
||||
executeUnion(pipeline, getSampleBlock());
|
||||
|
||||
BlockIO res;
|
||||
res.in = pipeline.firstStream();
|
||||
res.pipeline.addInterpreterContext(context);
|
||||
res.pipeline.addStorageHolder(storage);
|
||||
return res;
|
||||
}
|
||||
|
||||
BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams()
|
||||
BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline)
|
||||
{
|
||||
///FIXME pipeline must be alive until query is finished
|
||||
Pipeline pipeline;
|
||||
executeImpl(pipeline, input);
|
||||
executeImpl(pipeline, input, parent_pipeline);
|
||||
unifyStreams(pipeline, getSampleBlock());
|
||||
parent_pipeline.addInterpreterContext(context);
|
||||
parent_pipeline.addStorageHolder(storage);
|
||||
return pipeline.streams;
|
||||
}
|
||||
|
||||
QueryPipeline InterpreterSelectQuery::executeWithProcessors()
|
||||
{
|
||||
QueryPipeline query_pipeline;
|
||||
query_pipeline.setMaxThreads(context.getSettingsRef().max_threads);
|
||||
executeImpl(query_pipeline, input);
|
||||
query_pipeline.setMaxThreads(context->getSettingsRef().max_threads);
|
||||
executeImpl(query_pipeline, input, query_pipeline);
|
||||
query_pipeline.addInterpreterContext(context);
|
||||
query_pipeline.addStorageHolder(storage);
|
||||
return query_pipeline;
|
||||
}
|
||||
|
||||
@ -478,13 +485,13 @@ QueryPipeline InterpreterSelectQuery::executeWithProcessors()
|
||||
Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
{
|
||||
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.
|
||||
|
||||
/// PREWHERE optimization.
|
||||
/// 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.prewhere());
|
||||
@ -498,7 +505,7 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
|
||||
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
|
||||
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};
|
||||
};
|
||||
|
||||
@ -507,14 +514,14 @@ Block InterpreterSelectQuery::getSampleBlockImpl()
|
||||
}
|
||||
|
||||
if (storage && !options.only_analyze)
|
||||
from_stage = storage->getQueryProcessingStage(context);
|
||||
from_stage = storage->getQueryProcessingStage(*context);
|
||||
|
||||
analysis_result = analyzeExpressions(
|
||||
getSelectQuery(),
|
||||
*query_analyzer,
|
||||
from_stage,
|
||||
options.to_stage,
|
||||
context,
|
||||
*context,
|
||||
storage,
|
||||
options.only_analyze,
|
||||
filter_info,
|
||||
@ -1007,7 +1014,7 @@ static InputSortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree,
|
||||
|
||||
|
||||
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.
|
||||
* 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.
|
||||
auto & query = getSelectQuery();
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
auto & expressions = analysis_result;
|
||||
|
||||
InputSortingInfoPtr input_sorting_info;
|
||||
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()))
|
||||
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)
|
||||
@ -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);
|
||||
|
||||
/** 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));
|
||||
}
|
||||
@ -1349,12 +1356,13 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS
|
||||
template <typename TPipeline>
|
||||
void InterpreterSelectQuery::executeFetchColumns(
|
||||
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;
|
||||
|
||||
auto & query = getSelectQuery();
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
/// Optimization for trivial query like SELECT count() FROM table.
|
||||
auto check_trivial_count_query = [&]() -> std::optional<AggregateDescription>
|
||||
@ -1418,11 +1426,11 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
if (storage)
|
||||
{
|
||||
/// 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;
|
||||
ExpressionActionsPtr actions;
|
||||
generateFilterActions(actions, storage, context, initial_required_columns);
|
||||
generateFilterActions(actions, storage, *context, initial_required_columns);
|
||||
auto required_columns_from_filter = actions->getRequiredColumns();
|
||||
|
||||
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; });
|
||||
}
|
||||
|
||||
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);
|
||||
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);
|
||||
|
||||
/// The set of required columns could be added as a result of adding an action to calculate ALIAS.
|
||||
required_columns = alias_actions->getRequiredColumns();
|
||||
@ -1542,7 +1550,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
if (prewhere_info)
|
||||
{
|
||||
/// 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())
|
||||
{
|
||||
if (action.type != ExpressionAction::REMOVE_COLUMN
|
||||
@ -1552,9 +1560,9 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
prewhere_info->prewhere_actions = std::move(new_actions);
|
||||
|
||||
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
|
||||
= 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.
|
||||
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;
|
||||
|
||||
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,
|
||||
* 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);
|
||||
|
||||
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
|
||||
subquery, getSubqueryContext(context),
|
||||
subquery, getSubqueryContext(*context),
|
||||
options.copy().subquery().noModify(), required_columns);
|
||||
|
||||
if (query_analyzer->hasAggregation())
|
||||
@ -1649,7 +1657,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
/// Just use pipeline from subquery.
|
||||
pipeline = interpreter_subquery->executeWithProcessors();
|
||||
else
|
||||
pipeline.streams = interpreter_subquery->executeWithMultipleStreams();
|
||||
pipeline.streams = interpreter_subquery->executeWithMultipleStreams(save_context_and_storage);
|
||||
}
|
||||
else if (storage)
|
||||
{
|
||||
@ -1676,9 +1684,9 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
bool use_pipes = pipeline_with_processors && storage->supportProcessorsPipeline();
|
||||
|
||||
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
|
||||
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)
|
||||
{
|
||||
@ -1757,7 +1765,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
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)
|
||||
{
|
||||
@ -1798,7 +1806,7 @@ void InterpreterSelectQuery::executeFetchColumns(
|
||||
auto header = stream->getHeader();
|
||||
auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name;
|
||||
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)
|
||||
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:
|
||||
* 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_bytes : SettingUInt64(0),
|
||||
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 (pipeline.streams.size() > 1)
|
||||
@ -1952,7 +1960,7 @@ void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const
|
||||
for (const auto & name : descr.argument_names)
|
||||
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:
|
||||
* 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_bytes : SettingUInt64(0),
|
||||
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);
|
||||
|
||||
@ -2029,7 +2037,7 @@ void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool ov
|
||||
* 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);
|
||||
|
||||
@ -2080,7 +2088,7 @@ void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bo
|
||||
* 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);
|
||||
|
||||
@ -2140,7 +2148,7 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
|
||||
{
|
||||
executeUnion(pipeline, {});
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
pipeline.firstStream() = std::make_shared<TotalsHavingBlockInputStream>(
|
||||
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)
|
||||
{
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
auto totals_having = std::make_shared<TotalsHavingTransform>(
|
||||
pipeline.getHeader(), overflow_row, expression,
|
||||
@ -2180,13 +2188,13 @@ void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificato
|
||||
for (const auto & name : key_names)
|
||||
keys.push_back(header.getPositionByName(name));
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
Aggregator::Params params(header, keys, aggregates,
|
||||
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
|
||||
SettingUInt64(0), SettingUInt64(0),
|
||||
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)
|
||||
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)
|
||||
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,
|
||||
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
|
||||
SettingUInt64(0), SettingUInt64(0),
|
||||
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);
|
||||
|
||||
@ -2251,9 +2259,9 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E
|
||||
void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoPtr input_sorting_info)
|
||||
{
|
||||
auto & query = getSelectQuery();
|
||||
SortDescription output_order_descr = getSortDescription(query, context);
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
UInt64 limit = getLimitForSorting(query, context);
|
||||
SortDescription output_order_descr = getSortDescription(query, *context);
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
UInt64 limit = getLimitForSorting(query, *context);
|
||||
|
||||
if (input_sorting_info)
|
||||
{
|
||||
@ -2305,17 +2313,17 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, InputSortingInfoP
|
||||
pipeline.firstStream() = std::make_shared<MergeSortingBlockInputStream>(
|
||||
pipeline.firstStream(), output_order_descr, settings.max_block_size, limit,
|
||||
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)
|
||||
{
|
||||
auto & query = getSelectQuery();
|
||||
SortDescription output_order_descr = getSortDescription(query, context);
|
||||
UInt64 limit = getLimitForSorting(query, context);
|
||||
SortDescription output_order_descr = getSortDescription(query, *context);
|
||||
UInt64 limit = getLimitForSorting(query, *context);
|
||||
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
/// TODO: Limits on sorting
|
||||
// IBlockInputStream::LocalLimits limits;
|
||||
@ -2382,7 +2390,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, InputSorting
|
||||
return std::make_shared<MergeSortingTransform>(
|
||||
header, output_order_descr, settings.max_block_size, limit,
|
||||
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)
|
||||
{
|
||||
auto & query = getSelectQuery();
|
||||
SortDescription order_descr = getSortDescription(query, context);
|
||||
UInt64 limit = getLimitForSorting(query, context);
|
||||
SortDescription order_descr = getSortDescription(query, *context);
|
||||
UInt64 limit = getLimitForSorting(query, *context);
|
||||
|
||||
/// If there are several streams, then we merge them into one
|
||||
if (pipeline.hasMoreThanOneStream())
|
||||
@ -2406,7 +2414,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline, const SortD
|
||||
{
|
||||
if (pipeline.hasMoreThanOneStream())
|
||||
{
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
/** MergingSortedBlockInputStream reads the sources sequentially.
|
||||
* 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)
|
||||
{
|
||||
auto & query = getSelectQuery();
|
||||
SortDescription order_descr = getSortDescription(query, context);
|
||||
UInt64 limit = getLimitForSorting(query, context);
|
||||
SortDescription order_descr = getSortDescription(query, *context);
|
||||
UInt64 limit = getLimitForSorting(query, *context);
|
||||
|
||||
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 (pipeline.getNumStreams() > 1)
|
||||
{
|
||||
const Settings & settings = context.getSettingsRef();
|
||||
const Settings & settings = context->getSettingsRef();
|
||||
|
||||
auto transform = std::make_shared<MergingSortedTransform>(
|
||||
pipeline.getHeader(),
|
||||
@ -2471,9 +2479,9 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or
|
||||
auto & query = getSelectQuery();
|
||||
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;
|
||||
|
||||
/// 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();
|
||||
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;
|
||||
|
||||
/// 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 (query.limitLength())
|
||||
{
|
||||
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
|
||||
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, *context);
|
||||
SortDescription sort_descr;
|
||||
if (query.limit_with_ties)
|
||||
{
|
||||
if (!query.orderBy())
|
||||
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)
|
||||
{
|
||||
@ -2567,7 +2575,7 @@ void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline)
|
||||
/// If there is LIMIT
|
||||
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
|
||||
{
|
||||
if (stream_type == QueryPipeline::StreamType::Totals)
|
||||
@ -2588,8 +2596,8 @@ void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline)
|
||||
Names columns;
|
||||
for (const auto & elem : query.limitBy()->children)
|
||||
columns.emplace_back(elem->getColumnName());
|
||||
UInt64 length = getLimitUIntValue(query.limitByLength(), context);
|
||||
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context) : 0);
|
||||
UInt64 length = getLimitUIntValue(query.limitByLength(), *context);
|
||||
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0);
|
||||
|
||||
pipeline.transform([&](auto & stream)
|
||||
{
|
||||
@ -2607,8 +2615,8 @@ void InterpreterSelectQuery::executeLimitBy(QueryPipeline & pipeline)
|
||||
for (const auto & elem : query.limitBy()->children)
|
||||
columns.emplace_back(elem->getColumnName());
|
||||
|
||||
UInt64 length = getLimitUIntValue(query.limitByLength(), context);
|
||||
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context) : 0);
|
||||
UInt64 length = getLimitUIntValue(query.limitByLength(), *context);
|
||||
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), *context) : 0);
|
||||
|
||||
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
|
||||
{
|
||||
@ -2672,12 +2680,12 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
|
||||
{
|
||||
if (!query.orderBy())
|
||||
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_offset;
|
||||
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context);
|
||||
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context);
|
||||
|
||||
pipeline.transform([&](auto & stream)
|
||||
{
|
||||
@ -2692,7 +2700,7 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline)
|
||||
auto & query = getSelectQuery();
|
||||
if (query.orderBy())
|
||||
{
|
||||
SortDescription order_descr = getSortDescription(query, context);
|
||||
SortDescription order_descr = getSortDescription(query, *context);
|
||||
SortDescription fill_descr;
|
||||
for (auto & desc : order_descr)
|
||||
{
|
||||
@ -2715,7 +2723,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPipeline & pipeline)
|
||||
auto & query = getSelectQuery();
|
||||
if (query.orderBy())
|
||||
{
|
||||
SortDescription order_descr = getSortDescription(query, context);
|
||||
SortDescription order_descr = getSortDescription(query, *context);
|
||||
SortDescription fill_descr;
|
||||
for (auto & desc : order_descr)
|
||||
{
|
||||
@ -2759,14 +2767,14 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline)
|
||||
|
||||
UInt64 limit_length;
|
||||
UInt64 limit_offset;
|
||||
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context);
|
||||
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, *context);
|
||||
|
||||
SortDescription order_descr;
|
||||
if (query.limit_with_ties)
|
||||
{
|
||||
if (!query.orderBy())
|
||||
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
|
||||
@ -2783,7 +2791,7 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline)
|
||||
|
||||
void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline)
|
||||
{
|
||||
if (!context.getSettingsRef().extremes)
|
||||
if (!context->getSettingsRef().extremes)
|
||||
return;
|
||||
|
||||
pipeline.transform([&](auto & stream)
|
||||
@ -2794,7 +2802,7 @@ void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline)
|
||||
|
||||
void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline)
|
||||
{
|
||||
if (!context.getSettingsRef().extremes)
|
||||
if (!context->getSettingsRef().extremes)
|
||||
return;
|
||||
|
||||
auto transform = std::make_shared<ExtremesTransform>(pipeline.getHeader());
|
||||
@ -2815,7 +2823,7 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline
|
||||
executeUnion(pipeline, {});
|
||||
|
||||
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)
|
||||
@ -2827,12 +2835,12 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pip
|
||||
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>(
|
||||
pipeline.getHeader(), subqueries_for_sets,
|
||||
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode),
|
||||
context);
|
||||
*context);
|
||||
|
||||
pipeline.addCreatingSetsTransform(std::move(creating_sets));
|
||||
}
|
||||
@ -2853,7 +2861,7 @@ void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline, Block header)
|
||||
auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name;
|
||||
|
||||
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();
|
||||
if (query.settings())
|
||||
InterpreterSetQuery(query.settings(), context).executeForCurrentContext();
|
||||
InterpreterSetQuery(query.settings(), *context).executeForCurrentContext();
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -69,7 +69,7 @@ public:
|
||||
BlockIO execute() override;
|
||||
|
||||
/// Execute the query and return multuple streams for parallel processing.
|
||||
BlockInputStreams executeWithMultipleStreams();
|
||||
BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline);
|
||||
|
||||
QueryPipeline executeWithProcessors() override;
|
||||
bool canExecuteWithProcessors() const override { return true; }
|
||||
@ -137,7 +137,7 @@ private:
|
||||
};
|
||||
|
||||
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
|
||||
{
|
||||
@ -199,7 +199,8 @@ private:
|
||||
template <typename TPipeline>
|
||||
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline,
|
||||
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 executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
|
||||
@ -261,7 +262,7 @@ private:
|
||||
|
||||
const SelectQueryOptions options;
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
std::shared_ptr<Context> context;
|
||||
SyntaxAnalyzerResultPtr syntax_analyzer_result;
|
||||
std::unique_ptr<SelectQueryExpressionAnalyzer> query_analyzer;
|
||||
SelectQueryInfo query_info;
|
||||
|
@ -34,7 +34,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
|
||||
const Names & required_result_column_names)
|
||||
: options(options_),
|
||||
query_ptr(query_ptr_),
|
||||
context(context_)
|
||||
context(std::make_shared<Context>(context_))
|
||||
{
|
||||
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.
|
||||
|
||||
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());
|
||||
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)
|
||||
{
|
||||
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())
|
||||
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>(
|
||||
ast.list_of_selects->children.at(query_num),
|
||||
context,
|
||||
*context,
|
||||
options,
|
||||
current_required_result_column_names));
|
||||
}
|
||||
@ -165,20 +165,23 @@ Block InterpreterSelectWithUnionQuery::getSampleBlock(
|
||||
}
|
||||
|
||||
|
||||
BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams()
|
||||
BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams(QueryPipeline & parent_pipeline)
|
||||
{
|
||||
BlockInputStreams nested_streams;
|
||||
|
||||
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());
|
||||
}
|
||||
|
||||
/// Unify data structure.
|
||||
if (nested_interpreters.size() > 1)
|
||||
{
|
||||
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;
|
||||
}
|
||||
@ -186,9 +189,10 @@ BlockInputStreams InterpreterSelectWithUnionQuery::executeWithMultipleStreams()
|
||||
|
||||
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;
|
||||
|
||||
if (nested_streams.empty())
|
||||
@ -206,8 +210,8 @@ BlockIO InterpreterSelectWithUnionQuery::execute()
|
||||
nested_streams.clear();
|
||||
}
|
||||
|
||||
BlockIO res;
|
||||
res.in = result_stream;
|
||||
res.pipeline.addInterpreterContext(context);
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -242,9 +246,11 @@ QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors()
|
||||
if (!pipelines.empty())
|
||||
{
|
||||
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;
|
||||
}
|
||||
|
||||
|
@ -29,7 +29,7 @@ public:
|
||||
BlockIO execute() override;
|
||||
|
||||
/// Execute the query without union of streams.
|
||||
BlockInputStreams executeWithMultipleStreams();
|
||||
BlockInputStreams executeWithMultipleStreams(QueryPipeline & parent_pipeline);
|
||||
|
||||
QueryPipeline executeWithProcessors() override;
|
||||
bool canExecuteWithProcessors() const override { return true; }
|
||||
@ -47,7 +47,7 @@ public:
|
||||
private:
|
||||
const SelectQueryOptions options;
|
||||
ASTPtr query_ptr;
|
||||
Context context;
|
||||
std::shared_ptr<Context> context;
|
||||
|
||||
std::vector<std::unique_ptr<InterpreterSelectQuery>> nested_interpreters;
|
||||
|
||||
|
@ -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).
|
||||
if (pos->type == TokenType::OpeningRoundBracket)
|
||||
if (allow_function_parameters && pos->type == TokenType::OpeningRoundBracket)
|
||||
{
|
||||
++pos;
|
||||
|
||||
|
@ -90,9 +90,12 @@ protected:
|
||||
*/
|
||||
class ParserFunction : public IParserBase
|
||||
{
|
||||
public:
|
||||
ParserFunction(bool allow_function_parameters_ = true) : allow_function_parameters(allow_function_parameters_) {}
|
||||
protected:
|
||||
const char * getName() const { return "function"; }
|
||||
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected);
|
||||
bool allow_function_parameters;
|
||||
};
|
||||
|
||||
class ParserCodecDeclarationList : public IParserBase
|
||||
|
@ -35,7 +35,7 @@ bool ParserInsertQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
|
||||
ParserToken s_rparen(TokenType::ClosingRoundBracket);
|
||||
ParserIdentifier name_p;
|
||||
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 table;
|
||||
|
@ -8,7 +8,6 @@
|
||||
#include <Common/PODArray.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -320,7 +320,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
|
||||
}
|
||||
|
||||
/// 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);
|
||||
if (buf.eof() || *buf.position() == ';')
|
||||
@ -328,7 +328,7 @@ bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_size, int balance)
|
||||
bool quoted = false;
|
||||
|
||||
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());
|
||||
if (buf.position() == buf.buffer().end())
|
||||
|
@ -60,7 +60,7 @@ private:
|
||||
|
||||
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:
|
||||
PeekableReadBuffer buf;
|
||||
|
@ -486,6 +486,10 @@ void QueryPipeline::unitePipelines(
|
||||
|
||||
processors.insert(processors.end(), pipeline.processors.begin(), pipeline.processors.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())
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -75,6 +76,8 @@ public:
|
||||
const Block & getHeader() const { return current_header; }
|
||||
|
||||
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.
|
||||
void setProgressCallback(const ProgressCallback & callback);
|
||||
@ -109,6 +112,12 @@ private:
|
||||
|
||||
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;
|
||||
|
||||
size_t max_threads = 0;
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Storages/AlterCommands.h>
|
||||
#include <Parsers/ASTCreateQuery.h>
|
||||
#include <Parsers/ASTSetQuery.h>
|
||||
#include <Common/StringUtils/StringUtils.h>
|
||||
#include <Common/quoteString.h>
|
||||
|
||||
#include <Processors/Executors/TreeExecutorBlockInputStream.h>
|
||||
@ -445,4 +446,21 @@ BlockInputStreams IStorage::read(
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <DataStreams/IBlockStream_fwd.h>
|
||||
#include <Databases/IDatabase.h>
|
||||
#include <Interpreters/CancellationCode.h>
|
||||
#include <IO/CompressionMethod.h>
|
||||
#include <Storages/IStorage_fwd.h>
|
||||
#include <Storages/SelectQueryInfo.h>
|
||||
#include <Storages/TableStructureLockHolder.h>
|
||||
@ -434,6 +435,8 @@ public:
|
||||
return {};
|
||||
}
|
||||
|
||||
static DB::CompressionMethod chooseCompressionMethod(const String & uri, const String & compression_method);
|
||||
|
||||
private:
|
||||
/// You always need to take the next three locks in this order.
|
||||
|
||||
|
@ -3269,6 +3269,11 @@ MergeTreeData::MutableDataPartPtr MergeTreeData::cloneAndLoadDataPartOnSameDisk(
|
||||
String tmp_dst_part_name = tmp_part_prefix + dst_part_name;
|
||||
|
||||
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());
|
||||
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();
|
||||
|
@ -8,6 +8,7 @@
|
||||
#include <Parsers/ASTIdentifier.h>
|
||||
|
||||
#include <IO/ReadBufferFromFile.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromFile.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
@ -127,9 +128,10 @@ StorageFile::StorageFile(
|
||||
const std::string & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
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_);
|
||||
setConstraints(constraints_);
|
||||
@ -178,7 +180,10 @@ StorageFile::StorageFile(
|
||||
class StorageFileBlockInputStream : public IBlockInputStream
|
||||
{
|
||||
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_))
|
||||
{
|
||||
if (storage->use_table_fd)
|
||||
@ -199,12 +204,12 @@ public:
|
||||
}
|
||||
|
||||
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
|
||||
{
|
||||
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);
|
||||
@ -235,7 +240,7 @@ public:
|
||||
private:
|
||||
std::shared_ptr<StorageFile> storage;
|
||||
Block sample_block;
|
||||
std::unique_ptr<ReadBufferFromFileDescriptor> read_buf;
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
BlockInputStreamPtr reader;
|
||||
|
||||
std::shared_lock<std::shared_mutex> shared_lock;
|
||||
@ -260,7 +265,7 @@ BlockInputStreams StorageFile::read(
|
||||
for (const auto & file_path : paths)
|
||||
{
|
||||
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));
|
||||
}
|
||||
return blocks_input;
|
||||
@ -270,7 +275,8 @@ BlockInputStreams StorageFile::read(
|
||||
class StorageFileBlockOutputStream : public IBlockOutputStream
|
||||
{
|
||||
public:
|
||||
explicit StorageFileBlockOutputStream(StorageFile & storage_)
|
||||
explicit StorageFileBlockOutputStream(StorageFile & storage_,
|
||||
const CompressionMethod compression_method)
|
||||
: storage(storage_), lock(storage.rwlock)
|
||||
{
|
||||
if (storage.use_table_fd)
|
||||
@ -280,13 +286,13 @@ public:
|
||||
* INSERT data; SELECT *; last SELECT returns only insert_data
|
||||
*/
|
||||
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
|
||||
{
|
||||
if (storage.paths.size() != 1)
|
||||
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);
|
||||
@ -317,7 +323,7 @@ public:
|
||||
private:
|
||||
StorageFile & storage;
|
||||
std::unique_lock<std::shared_mutex> lock;
|
||||
std::unique_ptr<WriteBufferFromFileDescriptor> write_buf;
|
||||
std::unique_ptr<WriteBuffer> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
|
||||
@ -325,7 +331,8 @@ BlockOutputStreamPtr StorageFile::write(
|
||||
const ASTPtr & /*query*/,
|
||||
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
|
||||
@ -361,9 +368,9 @@ void registerStorageFile(StorageFactory & factory)
|
||||
{
|
||||
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(
|
||||
"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);
|
||||
|
||||
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
|
||||
@ -371,6 +378,7 @@ void registerStorageFile(StorageFactory & factory)
|
||||
|
||||
int source_fd = -1;
|
||||
String source_path;
|
||||
String compression_method;
|
||||
if (engine_args.size() >= 2)
|
||||
{
|
||||
/// 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)
|
||||
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(
|
||||
source_path, source_fd,
|
||||
args.data_path,
|
||||
args.database_name, args.table_name, format_name, args.columns, args.constraints,
|
||||
args.context);
|
||||
args.context,
|
||||
compression_method);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -60,7 +60,8 @@ protected:
|
||||
const std::string & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_);
|
||||
Context & context_,
|
||||
const String & compression_method_);
|
||||
|
||||
private:
|
||||
std::string table_name;
|
||||
@ -69,6 +70,7 @@ private:
|
||||
Context & context_global;
|
||||
|
||||
int table_fd = -1;
|
||||
String compression_method;
|
||||
|
||||
std::vector<std::string> paths;
|
||||
|
||||
|
@ -7,14 +7,16 @@
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadBufferFromHDFS.h>
|
||||
#include <IO/WriteBufferFromHDFS.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/HDFSCommon.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataStreams/UnionBlockInputStream.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <DataStreams/OwningBlockInputStream.h>
|
||||
#include <DataStreams/IBlockInputStream.h>
|
||||
#include <Common/parseGlobs.h>
|
||||
#include <Poco/URI.h>
|
||||
#include <re2/re2.h>
|
||||
@ -36,12 +38,14 @@ StorageHDFS::StorageHDFS(const String & uri_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_)
|
||||
Context & context_,
|
||||
const String & compression_method_ = "")
|
||||
: uri(uri_)
|
||||
, format_name(format_name_)
|
||||
, table_name(table_name_)
|
||||
, database_name(database_name_)
|
||||
, context(context_)
|
||||
, compression_method(compression_method_)
|
||||
{
|
||||
setColumns(columns_);
|
||||
setConstraints(constraints_);
|
||||
@ -57,9 +61,11 @@ public:
|
||||
const String & format,
|
||||
const Block & sample_block,
|
||||
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);
|
||||
reader = std::make_shared<OwningBlockInputStream<ReadBuffer>>(input_stream, std::move(read_buf));
|
||||
}
|
||||
@ -99,10 +105,11 @@ public:
|
||||
HDFSBlockOutputStream(const String & uri,
|
||||
const String & format,
|
||||
const Block & sample_block_,
|
||||
const Context & context)
|
||||
const Context & context,
|
||||
const CompressionMethod compression_method)
|
||||
: 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);
|
||||
}
|
||||
|
||||
@ -130,7 +137,7 @@ public:
|
||||
|
||||
private:
|
||||
Block sample_block;
|
||||
std::unique_ptr<WriteBufferFromHDFS> write_buf;
|
||||
std::unique_ptr<WriteBuffer> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
|
||||
@ -203,7 +210,7 @@ BlockInputStreams StorageHDFS::read(
|
||||
for (const auto & res_path : res_paths)
|
||||
{
|
||||
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;
|
||||
@ -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*/)
|
||||
{
|
||||
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)
|
||||
@ -226,9 +237,9 @@ void registerStorageHDFS(StorageFactory & factory)
|
||||
{
|
||||
ASTs & engine_args = args.engine_args;
|
||||
|
||||
if (engine_args.size() != 2)
|
||||
if (engine_args.size() != 2 && engine_args.size() != 3)
|
||||
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);
|
||||
|
||||
@ -238,7 +249,14 @@ void registerStorageHDFS(StorageFactory & factory)
|
||||
|
||||
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);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -39,7 +39,8 @@ protected:
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_);
|
||||
Context & context_,
|
||||
const String & compression_method_);
|
||||
|
||||
private:
|
||||
String uri;
|
||||
@ -47,6 +48,7 @@ private:
|
||||
String table_name;
|
||||
String database_name;
|
||||
Context & context;
|
||||
String compression_method;
|
||||
|
||||
Logger * log = &Logger::get("StorageHDFS");
|
||||
};
|
||||
|
@ -1141,6 +1141,11 @@ bool StorageReplicatedMergeTree::tryExecutePartMutation(const StorageReplicatedM
|
||||
/// Can throw an exception.
|
||||
/// 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);
|
||||
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);
|
||||
|
||||
|
@ -6,7 +6,9 @@
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/WriteBufferFromS3.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
@ -35,11 +37,11 @@ namespace
|
||||
const Block & sample_block,
|
||||
const Context & context,
|
||||
UInt64 max_block_size,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const CompressionMethod compression_method)
|
||||
: 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);
|
||||
}
|
||||
|
||||
@ -70,7 +72,7 @@ namespace
|
||||
|
||||
private:
|
||||
String name;
|
||||
std::unique_ptr<ReadBufferFromS3> read_buf;
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
BlockInputStreamPtr reader;
|
||||
};
|
||||
|
||||
@ -82,10 +84,11 @@ namespace
|
||||
UInt64 min_upload_part_size,
|
||||
const Block & sample_block_,
|
||||
const Context & context,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const CompressionMethod compression_method)
|
||||
: 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);
|
||||
}
|
||||
|
||||
@ -113,7 +116,7 @@ namespace
|
||||
|
||||
private:
|
||||
Block sample_block;
|
||||
std::unique_ptr<WriteBufferFromS3> write_buf;
|
||||
std::unique_ptr<WriteBuffer> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
}
|
||||
@ -127,7 +130,8 @@ StorageS3::StorageS3(
|
||||
UInt64 min_upload_part_size_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_)
|
||||
Context & context_,
|
||||
const String & compression_method_ = "")
|
||||
: IStorage(columns_)
|
||||
, uri(uri_)
|
||||
, context_global(context_)
|
||||
@ -135,6 +139,7 @@ StorageS3::StorageS3(
|
||||
, database_name(database_name_)
|
||||
, table_name(table_name_)
|
||||
, min_upload_part_size(min_upload_part_size_)
|
||||
, compression_method(compression_method_)
|
||||
{
|
||||
setColumns(columns_);
|
||||
setConstraints(constraints_);
|
||||
@ -156,7 +161,8 @@ BlockInputStreams StorageS3::read(
|
||||
getHeaderBlock(column_names),
|
||||
context,
|
||||
max_block_size,
|
||||
ConnectionTimeouts::getHTTPTimeouts(context));
|
||||
ConnectionTimeouts::getHTTPTimeouts(context),
|
||||
IStorage::chooseCompressionMethod(uri.toString(), compression_method));
|
||||
|
||||
auto column_defaults = getColumns().getDefaults();
|
||||
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*/)
|
||||
{
|
||||
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)
|
||||
@ -182,9 +190,9 @@ void registerStorageS3(StorageFactory & factory)
|
||||
{
|
||||
ASTs & engine_args = args.engine_args;
|
||||
|
||||
if (engine_args.size() != 2)
|
||||
if (engine_args.size() != 2 && engine_args.size() != 3)
|
||||
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);
|
||||
|
||||
@ -197,6 +205,13 @@ void registerStorageS3(StorageFactory & factory)
|
||||
|
||||
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);
|
||||
});
|
||||
}
|
||||
|
@ -24,7 +24,8 @@ public:
|
||||
UInt64 min_upload_part_size_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_);
|
||||
Context & context_,
|
||||
const String & compression_method_);
|
||||
|
||||
String getName() const override
|
||||
{
|
||||
@ -61,6 +62,7 @@ private:
|
||||
String database_name;
|
||||
String table_name;
|
||||
UInt64 min_upload_part_size;
|
||||
String compression_method;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -5,8 +5,10 @@
|
||||
#include <Interpreters/evaluateConstantExpression.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <IO/WriteBufferFromHTTP.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
||||
@ -31,8 +33,9 @@ IStorageURLBase::IStorageURLBase(
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_)
|
||||
: uri(uri_), context_global(context_), format_name(format_name_), table_name(table_name_), database_name(database_name_)
|
||||
const ConstraintsDescription & constraints_,
|
||||
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_);
|
||||
setConstraints(constraints_);
|
||||
@ -51,10 +54,11 @@ namespace
|
||||
const Block & sample_block,
|
||||
const Context & context,
|
||||
UInt64 max_block_size,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const CompressionMethod compression_method)
|
||||
: 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);
|
||||
}
|
||||
|
||||
@ -85,7 +89,7 @@ namespace
|
||||
|
||||
private:
|
||||
String name;
|
||||
std::unique_ptr<ReadWriteBufferFromHTTP> read_buf;
|
||||
std::unique_ptr<ReadBuffer> read_buf;
|
||||
BlockInputStreamPtr reader;
|
||||
};
|
||||
|
||||
@ -96,10 +100,11 @@ namespace
|
||||
const String & format,
|
||||
const Block & sample_block_,
|
||||
const Context & context,
|
||||
const ConnectionTimeouts & timeouts)
|
||||
const ConnectionTimeouts & timeouts,
|
||||
const CompressionMethod compression_method)
|
||||
: 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);
|
||||
}
|
||||
|
||||
@ -127,7 +132,7 @@ namespace
|
||||
|
||||
private:
|
||||
Block sample_block;
|
||||
std::unique_ptr<WriteBufferFromHTTP> write_buf;
|
||||
std::unique_ptr<WriteBuffer> write_buf;
|
||||
BlockOutputStreamPtr writer;
|
||||
};
|
||||
}
|
||||
@ -177,8 +182,8 @@ BlockInputStreams IStorageURLBase::read(const Names & column_names,
|
||||
getHeaderBlock(column_names),
|
||||
context,
|
||||
max_block_size,
|
||||
ConnectionTimeouts::getHTTPTimeouts(context));
|
||||
|
||||
ConnectionTimeouts::getHTTPTimeouts(context),
|
||||
IStorage::chooseCompressionMethod(request_uri.toString(), compression_method));
|
||||
|
||||
auto column_defaults = getColumns().getDefaults();
|
||||
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*/)
|
||||
{
|
||||
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)
|
||||
@ -204,9 +211,9 @@ void registerStorageURL(StorageFactory & factory)
|
||||
{
|
||||
ASTs & engine_args = args.engine_args;
|
||||
|
||||
if (engine_args.size() != 2)
|
||||
if (engine_args.size() != 2 && engine_args.size() != 3)
|
||||
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);
|
||||
|
||||
@ -217,7 +224,19 @@ void registerStorageURL(StorageFactory & factory)
|
||||
|
||||
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);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -39,10 +39,12 @@ protected:
|
||||
const std::string & table_name_,
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_);
|
||||
const ConstraintsDescription & constraints_,
|
||||
const String & compression_method_);
|
||||
|
||||
Poco::URI uri;
|
||||
const Context & context_global;
|
||||
String compression_method;
|
||||
|
||||
private:
|
||||
String format_name;
|
||||
@ -80,8 +82,9 @@ public:
|
||||
const String & format_name_,
|
||||
const ColumnsDescription & columns_,
|
||||
const ConstraintsDescription & constraints_,
|
||||
Context & context_)
|
||||
: IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_)
|
||||
Context & context_,
|
||||
const String & compression_method_)
|
||||
: IStorageURLBase(uri_, context_, database_name_, table_name_, format_name_, columns_, constraints_, compression_method_)
|
||||
{
|
||||
}
|
||||
|
||||
|
@ -66,7 +66,9 @@ BlockInputStreams StorageView::read(
|
||||
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.
|
||||
/// Because method 'getSampleBlockForColumns' is used to obtain a structure of result in InterpreterSelectQuery.
|
||||
|
@ -7,6 +7,7 @@
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
#include <IO/CompressionMethod.h>
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <Poco/File.h>
|
||||
@ -31,7 +32,7 @@ StorageXDBC::StorageXDBC(
|
||||
const Context & context_,
|
||||
const BridgeHelperPtr bridge_helper_)
|
||||
/// 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_)
|
||||
, remote_database_name(remote_database_name_)
|
||||
, remote_table_name(remote_table_name_)
|
||||
|
@ -21,7 +21,7 @@ void StorageSystemFormats::fillData(MutableColumns & res_columns, const Context
|
||||
for (const auto & pair : formats)
|
||||
{
|
||||
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);
|
||||
res_columns[0]->insert(format_name);
|
||||
res_columns[1]->insert(has_input_format);
|
||||
|
@ -32,21 +32,27 @@ StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & ast_function, cons
|
||||
|
||||
ASTs & args = args_func.at(0)->children;
|
||||
|
||||
if (args.size() != 3)
|
||||
throw Exception("Table function '" + getName() + "' requires exactly 3 arguments: filename, format and structure.",
|
||||
if (args.size() != 3 && args.size() != 4)
|
||||
throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default auto).",
|
||||
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);
|
||||
|
||||
std::string filename = args[0]->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 compression_method;
|
||||
|
||||
if (args.size() == 4)
|
||||
{
|
||||
compression_method = args[3]->as<ASTLiteral &>().value.safeGet<String>();
|
||||
} else compression_method = "auto";
|
||||
|
||||
ColumnsDescription columns = parseColumnsListFromString(structure, context);
|
||||
|
||||
/// 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();
|
||||
|
||||
|
@ -16,6 +16,6 @@ class ITableFunctionFileLike : public ITableFunction
|
||||
private:
|
||||
StoragePtr executeImpl(const ASTPtr & ast_function, const Context & context, const std::string & table_name) const override;
|
||||
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;
|
||||
};
|
||||
}
|
||||
|
@ -6,7 +6,7 @@
|
||||
namespace DB
|
||||
{
|
||||
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,
|
||||
-1,
|
||||
@ -16,7 +16,8 @@ StoragePtr TableFunctionFile::getStorage(
|
||||
format,
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
global_context);
|
||||
global_context,
|
||||
compression_method);
|
||||
}
|
||||
|
||||
void registerTableFunctionFile(TableFunctionFactory & factory)
|
||||
|
@ -23,6 +23,6 @@ public:
|
||||
|
||||
private:
|
||||
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;
|
||||
};
|
||||
}
|
||||
|
@ -9,7 +9,7 @@
|
||||
namespace DB
|
||||
{
|
||||
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,
|
||||
getDatabaseName(),
|
||||
@ -17,7 +17,8 @@ StoragePtr TableFunctionHDFS::getStorage(
|
||||
format,
|
||||
columns,
|
||||
ConstraintsDescription{},
|
||||
global_context);
|
||||
global_context,
|
||||
compression_method);
|
||||
}
|
||||
|
||||
void registerTableFunctionHDFS(TableFunctionFactory & factory)
|
||||
|
@ -24,7 +24,7 @@ public:
|
||||
|
||||
private:
|
||||
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;
|
||||
};
|
||||
}
|
||||
|
||||
|
@ -7,11 +7,11 @@ namespace DB
|
||||
{
|
||||
|
||||
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);
|
||||
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)
|
||||
|
@ -25,7 +25,8 @@ private:
|
||||
const String & format,
|
||||
const ColumnsDescription & columns,
|
||||
Context & global_context,
|
||||
const std::string & table_name) const override;
|
||||
const std::string & table_name,
|
||||
const String & compression_method) const override;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -8,10 +8,10 @@
|
||||
namespace DB
|
||||
{
|
||||
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);
|
||||
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)
|
||||
|
@ -19,6 +19,6 @@ public:
|
||||
|
||||
private:
|
||||
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;
|
||||
};
|
||||
}
|
||||
|
@ -109,13 +109,13 @@ However, these libraries are optional and ClickHouse can well be built without t
|
||||
|
||||
# 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.
|
||||
|
||||
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`.
|
||||
|
||||
@ -131,9 +131,9 @@ cd 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 ..
|
||||
```
|
||||
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).
|
||||
|
||||
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.
|
||||
|
@ -109,13 +109,13 @@ Mac OS X: `brew install icu4c readline`
|
||||
|
||||
# Компилятор C++
|
||||
|
||||
В качестве компилятора C++ поддерживается GCC начиная с версии 7 или Clang начиная с версии 7.
|
||||
В качестве компилятора C++ поддерживается GCC начиная с версии 9 или Clang начиная с версии 8.
|
||||
|
||||
Официальные сборки от Яндекса, на данный момент, используют GCC, так как он генерирует слегка более производительный машинный код (разница в среднем до нескольких процентов по нашим бенчмаркам). Clang обычно более удобен для разработки. Впрочем, наша среда continuous integration проверяет около десятка вариантов сборки.
|
||||
|
||||
Для установки 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`.
|
||||
|
||||
@ -132,9 +132,9 @@ cd build
|
||||
Вы можете иметь несколько разных директорий (build_release, build_debug) для разных вариантов сборки.
|
||||
|
||||
Находясь в директории build, выполните конфигурацию сборки с помощью CMake.
|
||||
Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 7).
|
||||
Перед первым запуском необходимо выставить переменные окружения, отвечающие за выбор компилятора (в данном примере это - gcc версии 9).
|
||||
```
|
||||
export CC=gcc-7 CXX=g++-7
|
||||
export CC=gcc-9 CXX=g++-9
|
||||
cmake ..
|
||||
```
|
||||
Переменная CC отвечает за компилятор C (сокращение от слов C Compiler), переменная CXX отвечает за выбор компилятора C++ (символ X - это как плюс, но положенный набок, ради того, чтобы превратить его в букву).
|
||||
|
@ -27,6 +27,8 @@ from .hdfs_api import HDFSApi
|
||||
HELPERS_DIR = p.dirname(__file__)
|
||||
DEFAULT_ENV_NAME = 'env_file'
|
||||
|
||||
SANITIZER_SIGN = "=================="
|
||||
|
||||
|
||||
def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
|
||||
full_path = os.path.join(path, fname)
|
||||
@ -87,6 +89,7 @@ class ClickHouseCluster:
|
||||
# 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.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')
|
||||
self.docker_api_version = os.environ.get("DOCKER_API_VERSION")
|
||||
@ -451,6 +454,15 @@ class ClickHouseCluster:
|
||||
self.is_up = 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:
|
||||
subprocess_check_call(self.base_cmd + ['kill'])
|
||||
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
|
||||
@ -464,6 +476,10 @@ class ClickHouseCluster:
|
||||
instance.ip_address = 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):
|
||||
os.system(' '.join(self.base_cmd + ['exec', instance_name, '/bin/bash']))
|
||||
|
||||
|
@ -1,4 +1,6 @@
|
||||
#-*- coding: utf-8 -*-
|
||||
import StringIO
|
||||
import gzip
|
||||
import requests
|
||||
import subprocess
|
||||
from tempfile import NamedTemporaryFile
|
||||
@ -19,7 +21,7 @@ class HDFSApi(object):
|
||||
if response_data.status_code != 200:
|
||||
response_data.raise_for_status()
|
||||
|
||||
return response_data.text
|
||||
return response_data.content
|
||||
|
||||
# Requests can't put file
|
||||
def _curl_to_put(self, filename, path, params):
|
||||
@ -44,3 +46,12 @@ class HDFSApi(object):
|
||||
output = self._curl_to_put(fpath, path, additional_params)
|
||||
if "201 Created" not in 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()
|
||||
|
@ -112,4 +112,10 @@ def test_deep_structure(start_cluster):
|
||||
'''.format(pattern)) == '{}\n'.format(value)
|
||||
assert node.query('''
|
||||
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'
|
||||
|
@ -68,6 +68,14 @@ CREATE TABLE low_cardinality (d Date, x UInt32, s LowCardinality(String)) ENGINE
|
||||
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))''')
|
||||
|
||||
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
|
||||
|
||||
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')")
|
||||
time.sleep(0.5)
|
||||
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'
|
||||
|
@ -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'
|
||||
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():
|
||||
conn = pymysql.connect(user='root', password='clickhouse', host='127.0.0.1', port=3308)
|
||||
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):
|
||||
with conn.cursor() as cursor:
|
||||
cursor.execute(
|
||||
|
@ -133,4 +133,56 @@ def test_globs_in_read_table(started_cluster):
|
||||
("?", 0)]
|
||||
|
||||
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"
|
||||
|
@ -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>
|
@ -31,7 +31,7 @@ import kafka_pb2
|
||||
cluster = ClickHouseCluster(__file__)
|
||||
instance = cluster.add_instance('instance',
|
||||
config_dir='configs',
|
||||
main_configs=['configs/kafka.xml'],
|
||||
main_configs=['configs/kafka.xml', 'configs/log_conf.xml' ],
|
||||
with_kafka=True,
|
||||
clickhouse_path_dir='clickhouse_path')
|
||||
kafka_id = ''
|
||||
@ -557,7 +557,7 @@ def test_kafka_insert(kafka_cluster):
|
||||
kafka_check_result(result, True)
|
||||
|
||||
|
||||
@pytest.mark.timeout(180)
|
||||
@pytest.mark.timeout(240)
|
||||
def test_kafka_produce_consume(kafka_cluster):
|
||||
instance.query('''
|
||||
DROP TABLE IF EXISTS test.view;
|
||||
|
@ -0,0 +1 @@
|
||||
OK
|
48
dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh
Executable file
48
dbms/tests/queries/0_stateless/01019_parallel_parsing_cancel.sh
Executable 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"
|
||||
|
||||
|
@ -2,7 +2,7 @@ drop table if exists test_table_hdfs_syntax
|
||||
;
|
||||
create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('')
|
||||
; -- { serverError 42 }
|
||||
create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('','','')
|
||||
create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('','','', '')
|
||||
; -- { serverError 42 }
|
||||
drop table if exists test_table_hdfs_syntax
|
||||
;
|
||||
|
@ -2,7 +2,7 @@ drop table if exists test_table_s3_syntax
|
||||
;
|
||||
create table test_table_s3_syntax (id UInt32) ENGINE = S3('')
|
||||
; -- { serverError 42 }
|
||||
create table test_table_s3_syntax (id UInt32) ENGINE = S3('','','')
|
||||
create table test_table_s3_syntax (id UInt32) ENGINE = S3('','','','')
|
||||
; -- { serverError 42 }
|
||||
drop table if exists test_table_s3_syntax
|
||||
;
|
||||
|
@ -2,7 +2,7 @@ drop table if exists test_table_url_syntax
|
||||
;
|
||||
create table test_table_url_syntax (id UInt32) ENGINE = URL('')
|
||||
; -- { serverError 42 }
|
||||
create table test_table_url_syntax (id UInt32) ENGINE = URL('','','')
|
||||
create table test_table_url_syntax (id UInt32) ENGINE = URL('','','','')
|
||||
; -- { serverError 42 }
|
||||
drop table if exists test_table_url_syntax
|
||||
;
|
||||
|
@ -0,0 +1 @@
|
||||
-160.32605 37.705841
|
16
dbms/tests/queries/0_stateless/01034_values_parse_float_bug.sh
Executable file
16
dbms/tests/queries/0_stateless/01034_values_parse_float_bug.sh
Executable 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"
|
||||
|
@ -48,4 +48,4 @@ $ clickhouse-client --query "SELECT COUNT(*) FROM datasets.visits_v1"
|
||||
```
|
||||
|
||||
## 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.
|
||||
|
@ -753,12 +753,12 @@ This section contains the following parameters:
|
||||
|
||||
For example:
|
||||
|
||||
```xml
|
||||
```xml
|
||||
<node index="1">
|
||||
<host>example_host</host>
|
||||
<port>2181</port>
|
||||
</node>
|
||||
```
|
||||
```
|
||||
|
||||
The `index` attribute specifies the node order when trying to connect to the ZooKeeper cluster.
|
||||
|
||||
|
@ -994,4 +994,18 @@ Lower values mean higher priority. Threads with low `nice` priority values are e
|
||||
|
||||
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 -->
|
||||
|
@ -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.
|
||||
- [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.
|
||||
- [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 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.
|
||||
@ -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.
|
||||
|
||||
#### 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}
|
||||
|
||||
```sql
|
||||
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:
|
||||
|
||||
|
@ -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}
|
||||
|
||||
Returns an array of the same size as the original array containing the elements in reverse order.
|
||||
|
@ -61,10 +61,10 @@ However, you can delete old data using `ALTER TABLE ... DROP PARTITION`.
|
||||
|
||||
### 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.
|
||||
- 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:
|
||||
|
||||
|
@ -1,8 +1,8 @@
|
||||
# Планы разработки 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. Хранение данных, индексация.
|
||||
@ -14,7 +14,7 @@
|
||||
Задача "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 каталог баз данных.
|
||||
|
||||
@ -86,7 +86,7 @@
|
||||
|
||||
### 1.11. Виртуальная файловая система.
|
||||
|
||||
Нужно для Яндекс.Облака. Делает Александр Бурмак, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс.
|
||||
Нужно для Яндекс.Облака. Делает Александр, Яндекс.Облако, а также Олег Ершов, ВШЭ и Яндекс.
|
||||
|
||||
ClickHouse использует для хранения данных локальную файловую систему. Существует сценарий работы, в котором размещение старых (архивных) данных было бы выгодно на удалённой файловой системе. Если файловая система POSIX совместимая, то это не составляет проблем: ClickHouse успешно работает с Ceph, GlusterFS, MooseFS. Также востребованным является сценарий использования S3 (из-за доступности в облаке) или HDFS (для интеграции с Hadoop). Но эти файловые системы не являются POSIX совместимыми. Хотя для них существуют FUSE драйверы, но скорость работы сильно страдает и поддержка неполная.
|
||||
|
||||
@ -95,7 +95,7 @@ ClickHouse использует небольшое подмножество фу
|
||||
### 1.12. Экспериментальная реализация VFS поверх S3 и HDFS.
|
||||
|
||||
Нужно для Яндекс.Облака. Требует 1.11. Желательно 1.6 и 1.18.
|
||||
Делает Александр Бурмак, Яндекс.Облако (сначала часть для S3), а также Олег Ершов, ВШЭ и Яндекс.
|
||||
Делает Александр, Яндекс.Облако (сначала часть для S3), а также Олег Ершов, ВШЭ и Яндекс.
|
||||
|
||||
### 1.13. Ускорение запросов с FINAL.
|
||||
|
||||
@ -156,7 +156,7 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
### 2.8. Декларативный парсер запросов.
|
||||
|
||||
Низкий приоритет. Задачу хочет сделать [Иван Лежанкин](https://github.com/abyss7) в свободное время, но за последний год никаких поползновений не видно.
|
||||
Низкий приоритет. Задачу хочет сделать [Иван Лежанкин](https://github.com/abyss7) в свободное время, но пока ничего нет.
|
||||
|
||||
### 2.9. Логгировние в format-стиле.
|
||||
|
||||
@ -180,7 +180,7 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
### 2.14. Все функции с состоянием переделать на FunctionBuilder.
|
||||
|
||||
Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлимо.
|
||||
Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлемо.
|
||||
|
||||
### 2.15. Функция subscribe для IStorage.
|
||||
|
||||
@ -199,9 +199,9 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
Требует 3.1.
|
||||
|
||||
### 3.3. Исправить катастрофически отвратительно неприемлимый поиск по документации.
|
||||
### 3.3. Исправить катастрофически отвратительно неприемлемый поиск по документации.
|
||||
|
||||
[Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на треш-технологиях, которые трудно исправить.
|
||||
[Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на технологиях, не удовлетворяющих требованиям задачи, и эти технологии трудно исправить.
|
||||
|
||||
### 3.4. Добавить японский язык в документацию.
|
||||
|
||||
@ -226,7 +226,7 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
### 4.3. Ограничение числа одновременных скачиваний с реплик.
|
||||
|
||||
Изначально делал Олег Алексеенков, но решение оказалось неудачным, хотя там не так уж много доделывать.
|
||||
Изначально делал Олег Алексеенков, но пока решение не готово, хотя там не так уж много доделывать.
|
||||
|
||||
### 4.4. Ограничение сетевой полосы при репликации.
|
||||
|
||||
@ -257,14 +257,14 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
### 5.5. LTS релизы.
|
||||
|
||||
Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Задача полностью обречена на провал с практической точки зрения, но пользователям важен сам факт того, что она будет, а мы уважаем чувства пользователей. Первой LTS версией уже стала версия 19.14.
|
||||
Требует 7.5. Задачу хочет Метрика, Облако, БК, Маркет и Altinity. Первой LTS версией уже стала версия 19.14.
|
||||
|
||||
|
||||
## 6. Инструментирование.
|
||||
|
||||
### 6.1. Исправления сэмплирующего профайлера запросов.
|
||||
|
||||
[Никита Лапков](https://github.com/laplab), ноябрь 2019. Может не сделать, тогда будет делать Алексей Миловидов.
|
||||
Михаил Филимонов, Altinity. Ноябрь 2019.
|
||||
|
||||
### 6.2. Добавление memory profiler.
|
||||
|
||||
@ -311,7 +311,7 @@ ClickHouse использует небольшое подмножество фу
|
||||
### 7.2. LLVM в submodules.
|
||||
|
||||
Уже добавлено, но старой версии, и поэтому не используется. Надо обновить.
|
||||
Георгий Кондратьев. Возможно, он не сможет сделать эту задачу, тогда будет делать Алексей Миловидов.
|
||||
Георгий - очень опытный разработчик, либо будет делать Алексей Миловидов.
|
||||
|
||||
### 7.3. Обновление Poco.
|
||||
|
||||
@ -328,7 +328,7 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
### 7.6. Правильный статистический тест для comparison mode в clickhouse-performance-test.
|
||||
|
||||
Задачу начал делать Дмитрий Рубашкин. Сейчас продолжает [Александр Кузьменков](https://github.com/akuzm).
|
||||
Задачу начал делать Дмитрий Рубашкин (ВШЭ). Сейчас продолжает [Александр Кузьменков](https://github.com/akuzm).
|
||||
|
||||
### 7.7. Доделать тесты под MSan.
|
||||
|
||||
@ -337,7 +337,7 @@ ClickHouse использует небольшое подмножество фу
|
||||
|
||||
### 7.8. Добавить clang-tidy.
|
||||
|
||||
Уже есть PVS-Studio, но этого недостаточно.
|
||||
Уже есть PVS-Studio. Мы очень довольны, но этого недостаточно.
|
||||
|
||||
### 7.9. Проверки на стиль имён с помощью clang-tidy.
|
||||
|
||||
@ -390,7 +390,7 @@ Wolf Kreuzerkrieg. Возможно, его уже не интересует э
|
||||
|
||||
### 7.15. Замена libressl обратно на openssl.
|
||||
|
||||
Поводом использования libressl послужило желание Константина podshumok Игнатова из QRator и то, что тогда openssl был опозорен и libressl считалась адекватной альтернативой. Но сейчас ситуация изменилась - openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно.
|
||||
Поводом использования libressl послужило желание нашего хорошего друга из известной компании несколько лет назад. Но сейчас ситуация состоит в том, что openssl продолжает развиваться, а libressl не особо, и можно спокойно менять обратно.
|
||||
|
||||
### 7.16. tzdata внутри бинарника.
|
||||
|
||||
@ -470,12 +470,12 @@ Fuzzing тестирование - это тестирование случай
|
||||
|
||||
### 7.25. Синхронизация релизов в Аркадию.
|
||||
|
||||
Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в Яндекс.Морду, но обещает продолжать синхронизацию.
|
||||
Затем, возможно, [Иван Лежанкин](https://github.com/abyss7). Но сейчас приостановлено, так как Максим Ахмедов должен исправить регрессию производительности в анализе индекса.
|
||||
Изначально занимался Олег Алексеенков. Сейчас он перешёл работать в дружественный отдел, но обещает продолжать синхронизацию.
|
||||
Затем, возможно, [Иван Лежанкин](https://github.com/abyss7). Но сейчас приостановлено, так как Максим из YT должен исправить регрессию производительности в анализе индекса.
|
||||
|
||||
### 7.26. Побайтовая идентичность репозитория с Аркадией.
|
||||
|
||||
Команда DevTools. Фактически никто ничего не делает.
|
||||
Команда DevTools. Прогресс по задаче под вопросом.
|
||||
|
||||
### 7.27. Запуск автотестов в Аркадии.
|
||||
|
||||
@ -523,8 +523,7 @@ Fuzzing тестирование - это тестирование случай
|
||||
|
||||
### 7.37. Разобраться с repo.yandex.ru.
|
||||
|
||||
Есть жалобы на скорость загрузки. Подозрение, что repo.yandex.ru не является нормальным CDN. Отсутствует простой доступ к мониторингу и логам.
|
||||
Очень редко бывает нужно удалить пакет, но сделать это можно только через одного человека.
|
||||
Есть жалобы на скорость загрузки и неудобство maintenance, operations, visibility.
|
||||
|
||||
|
||||
## 8. Интеграция с внешними системами.
|
||||
@ -626,12 +625,12 @@ ClickHouse предоставляет возможность обратитьс
|
||||
|
||||
### 10.1. Исправление зависания в библиотеке доступа к YT.
|
||||
|
||||
Библиотека для доступа к YT обладает катастрофически отвратительно неприемлимым поведением и не переживает учения.
|
||||
Библиотека для доступа к YT не переживает учения.
|
||||
Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT.
|
||||
|
||||
### 10.2. Исправление SIGILL в библиотеке доступа к YT.
|
||||
|
||||
Код YT зачем-то использует SIGILL вместо abort. Это, опять же, происходит при учениях.
|
||||
Код YT использует SIGILL вместо abort. Это, опять же, происходит при учениях.
|
||||
Нужно для БК и Метрики. Поиск причин - [Александр Сапин](https://github.com/alesapin). Дальшейшее исправление возможно на стороне YT.
|
||||
|
||||
### 10.3. Возможность чтения данных из статических таблиц в YT словарях.
|
||||
@ -640,7 +639,7 @@ ClickHouse предоставляет возможность обратитьс
|
||||
|
||||
### 10.4. Словарь из YDB (KikiMR).
|
||||
|
||||
Нужно для Метрики, а делать будет таинственный незнакомец из команды KikiMR. Или он сейчас это прочитает и скажет "я никогда не буду делать эту задачу".
|
||||
Нужно для Метрики, а делать будет таинственный незнакомец из команды KikiMR (под вопросом).
|
||||
|
||||
### 10.5. Закрытие соединений и уменьшение числа соединений для MySQL и ODBC.
|
||||
|
||||
@ -658,7 +657,7 @@ ClickHouse предоставляет возможность обратитьс
|
||||
|
||||
### 10.9. Уменьшение блокировок для cache словарей за счёт одновременных запросов одного и того же.
|
||||
|
||||
Нужно для БК, но на самом деле они так просто думают, а все проблемы из-за низко-качественной библиотеки для доступа в YT.
|
||||
Нужно для БК, но мотивация задачи находится под вопросом, так как есть рабочее предположение о том, что данная задача не устраняет причину проблемы.
|
||||
|
||||
### 10.10. Возможность использования старых значений из cache словаря пока они перезапрашиваются.
|
||||
|
||||
@ -711,7 +710,7 @@ ClickHouse предоставляет возможность обратитьс
|
||||
|
||||
### 11.5. Поддержка TLS в clickhouse-cpp.
|
||||
|
||||
Библиотеку clickhouse-cpp разрабатывал Павел Артёмкин в свободное время.
|
||||
А знаете ли вы, что библиотеку clickhouse-cpp разрабатывал один хороший человек в свободное время?
|
||||
|
||||
### 11.6. Интеграционные тесты clickhouse-cpp.
|
||||
|
||||
@ -792,7 +791,7 @@ zhang2014
|
||||
|
||||
### 14.5. Поддержка задания множества как массива в правой части секции IN.
|
||||
|
||||
Василий Немков, Altinity, делал эту задачу, но зачем-то забросил её.
|
||||
Василий Немков, Altinity, делал эту задачу, но временно приостановил работу над ней в пользу других задач.
|
||||
|
||||
### 14.6. Глобальный scope для WITH.
|
||||
|
||||
@ -885,7 +884,7 @@ zhang2014
|
||||
|
||||
### 16.1. DateTime64.
|
||||
|
||||
Василий Немков.
|
||||
Василий Немков, Altinity, декабрь 2019.
|
||||
|
||||
### 16.2. Тип данных для JSON.
|
||||
|
||||
@ -975,7 +974,7 @@ ClickHouse не является geospatial СУБД. Тем не менее, в
|
||||
|
||||
### 19.3. Подключение YT Cypress или YDB как альтернативы ZooKeeper.
|
||||
|
||||
Hold. Полезно для Яндекс.Облака и БК, но есть риски, что будет вредно, а не полезно.
|
||||
Hold. Полезно для заказчиков внутри Яндекса, но есть риски.
|
||||
|
||||
### 19.4. internal_replication = 'auto'.
|
||||
|
||||
@ -1057,7 +1056,7 @@ Hold. Полезно для Яндекс.Облака и БК, но есть р
|
||||
|
||||
### 21.8.1. Отдельный аллокатор для кэшей с ASLR.
|
||||
|
||||
В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения оказались неудачными. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд.
|
||||
В прошлом году задачу пытался сделать Данила Кутенин с помощью lfalloc из Аркадии и mimalloc из Microsoft, но оба решения не были квалифицированы для использования в продакшене. Успешная реализация задачи 21.8 отменит необходимость в этой задаче, поэтому холд.
|
||||
|
||||
### 21.9. Исправить push-down выражений с помощью Processors.
|
||||
|
||||
@ -1119,7 +1118,7 @@ Constraints позволяют задать выражение, истиннос
|
||||
### 21.19. Оптимизация сортировки.
|
||||
|
||||
Василий Морозов, Арслан Гумеров, Альберт Кидрачев, ВШЭ.
|
||||
В прошлом году задачу начинал делать Евгений Правда, ВШЭ, но почти полностью не сделал её.
|
||||
В прошлом году задачу начинал делать другой человек, но не добился достаточного прогресса.
|
||||
|
||||
1. Оптимизация top sort.
|
||||
|
||||
@ -1204,10 +1203,12 @@ zhang2014.
|
||||
|
||||
[Виталий Баранов](https://github.com/vitlibar), почти всё готово.
|
||||
|
||||
### 22.12. Исправление катастрофически отвратительно неприемлимо низкой производительности чтения из Kafka.
|
||||
### 22.12. Исправление низкой производительности чтения из Kafka.
|
||||
|
||||
[Иван Лежанкин](https://github.com/abyss7).
|
||||
|
||||
Для ClickHouse нехарактерно наличие кода, обладающего столь низкой производительностью. Практики разработки не подразумевают, что такой код должен попасть в продакшен без надлежащего тестирования производительности.
|
||||
|
||||
### 22.13. Посмотреть, почему не работают некоторые collations.
|
||||
|
||||
[Иван Лежанкин](https://github.com/abyss7), совмещается с 7.1.
|
||||
@ -1218,9 +1219,11 @@ zhang2014.
|
||||
|
||||
[Иван Лежанкин](https://github.com/abyss7), если он не сдастся.
|
||||
|
||||
### 22.16. Исправление катастрофически отвратительно неприемлимо низкой производительности кодека DoubleDelta.
|
||||
### 22.16. Исправление низкой производительности кодека DoubleDelta.
|
||||
|
||||
Василий Немков, Altinity - сейчас старательно динамит эту задачу.
|
||||
Василий Немков, Altinity - временно приостановлено, но намерения остаются в силе.
|
||||
|
||||
Мы считаем важным, что код в ClickHouse содержит разумные оптимизации, основанные на анализе производительности. Но иногда бывают досадные исключения.
|
||||
|
||||
### 22.17. Консистентно работающий POPULATE для MaterializedView.
|
||||
|
||||
@ -1264,7 +1267,7 @@ zhang2014.
|
||||
|
||||
### 22.28. Изучить и исправить поведение работы с Kafka при ребалансировке.
|
||||
|
||||
[Иван Лежанкин](https://github.com/abyss7), если он не сдастся.
|
||||
[Иван Лежанкин](https://github.com/abyss7).
|
||||
|
||||
|
||||
## 23. Default Festival.
|
||||
@ -1288,7 +1291,7 @@ zhang2014.
|
||||
|
||||
### 23.5. Включение compile_expressions.
|
||||
|
||||
Требует 7.2. Задачу изначально делал Денис Скоробогатов, ВШЭ и Яндекс, затем доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin).
|
||||
Требует 7.2. Задачу изначально на 99% сделал Денис Скоробогатов, ВШЭ и Яндекс. Остальной процент доделывал Алексей Миловидов, а затем [Александр Сапин](https://github.com/alesapin).
|
||||
|
||||
### 23.6. Включение учёта порядка столбцов в CSV.
|
||||
|
||||
@ -1308,7 +1311,7 @@ zhang2014.
|
||||
|
||||
### 23.10. Включение mlock бинарника.
|
||||
|
||||
Возможность mlock бинарника сделал Олег Алексеенков. Поможет, когда на серверах кроме ClickHouse работает много треш-программ.
|
||||
Возможность mlock бинарника сделал Олег Алексеенков. Поможет, когда на серверах кроме ClickHouse работает много посторонних программ (мы иногда называем их в шутку "треш-программами").
|
||||
|
||||
|
||||
## 24. Экспериментальные задачи.
|
||||
@ -1544,25 +1547,25 @@ Amos Bird, но его решение слишком громоздкое и п
|
||||
|
||||
### 25.2. Вычитка и выкладка статьи про обфускацию данных на английском.
|
||||
|
||||
Эми Жанель Кришниевски, Александр Казаков, Алексей Миловидов, ноябрь 2019.
|
||||
Эми, Александр Казаков, Алексей Миловидов, ноябрь 2019.
|
||||
|
||||
### 25.3. Подготовка статьи "Секреты оптимизации производительности ClickHouse".
|
||||
|
||||
Алексей Миловидов, Леонид Клюев.
|
||||
Алексей Миловидов, Леонид.
|
||||
|
||||
### 25.4. Подготовка статьи "Профайлер запросов: трудный путь".
|
||||
|
||||
Алексей Миловидов, Леонид Клюев.
|
||||
Алексей Миловидов, Леонид.
|
||||
|
||||
### 25.5. Подготовка статьи "Тестирование ClickHouse, которое мы заслужили".
|
||||
|
||||
### 25.6. Перевод этих статей на английский.
|
||||
|
||||
Требует 25.3, 25.4, 25.5. Эми Жанель Кришниевски
|
||||
Требует 25.3, 25.4, 25.5. Эми
|
||||
|
||||
### 25.7. Перевод статьи Данилы Кутенина на английский.
|
||||
|
||||
Эми Жанель Кришниевски
|
||||
Эми
|
||||
|
||||
### 25.8. Выступление keynote на BDTC.
|
||||
|
||||
@ -1576,7 +1579,7 @@ Amos Bird, но его решение слишком громоздкое и п
|
||||
|
||||
### 25.10. Митапы в России и Беларуси: Москва x2 + митап для разработчиков или хакатон, Санкт-Петербург, Минск, Нижний Новгород, Екатеринбург, Новосибирск и/или Академгородок, Иннополис или Казань.
|
||||
|
||||
Екатерина Миназова - организация
|
||||
Екатерина - организация
|
||||
|
||||
### 25.11. Митапы зарубежные: восток США (Нью Йорк, возможно Raleigh), возможно северо-запад (Сиэтл), Китай (Пекин снова, возможно митап для разработчиков или хакатон), Лондон.
|
||||
|
||||
@ -1594,7 +1597,7 @@ Amos Bird, но его решение слишком громоздкое и п
|
||||
|
||||
Алексей Миловидов и все подготовленные докладчики
|
||||
|
||||
### 25.15. Конференции зарубежные: Percona, DataOps, возможно Big Data Warsaw, попытка попасть на более крупные.
|
||||
### 25.15. Конференции зарубежные: Percona, DataOps, попытка попасть на более крупные.
|
||||
|
||||
Алексей Миловидов и все подготовленные докладчики
|
||||
|
||||
@ -1649,4 +1652,4 @@ Amos Bird, но его решение слишком громоздкое и п
|
||||
|
||||
### 25.27. Обновить сайт ClickHouse.
|
||||
|
||||
Иван Блинков. Есть риск, что станет хуже.
|
||||
Иван Блинков. Есть риски.
|
||||
|
@ -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.
|
||||
|
@ -702,12 +702,12 @@ ClickHouse использует ZooKeeper для хранения метадан
|
||||
|
||||
Например:
|
||||
|
||||
```xml
|
||||
```xml
|
||||
<node index="1">
|
||||
<host>example_host</host>
|
||||
<port>2181</port>
|
||||
</node>
|
||||
```
|
||||
```
|
||||
|
||||
Атрибут `index` задает порядок опроса нод при попытках подключиться к кластеру ZooKeeper.
|
||||
|
||||
|
@ -188,7 +188,8 @@ ALTER TABLE [db].name DROP CONSTRAINT constraint_name;
|
||||
- [DETACH PARTITION](#alter_detach-partition) – перенести партицию в директорию `detached`;
|
||||
- [DROP PARTITION](#alter_drop-partition) – удалить партицию;
|
||||
- [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 INDEX IN PARTITION](#alter_clear-index-partition) - очистить построенные вторичные индексы для заданной партиции;
|
||||
- [FREEZE PARTITION](#alter_freeze-partition) – создать резервную копию партиции;
|
||||
@ -255,13 +256,29 @@ ALTER TABLE visits ATTACH PART 201901_2_2_0;
|
||||
|
||||
Это означает, что вы можете разместить данные в директории `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}
|
||||
|
||||
```sql
|
||||
ALTER TABLE table2 REPLACE PARTITION partition_expr FROM table1
|
||||
```
|
||||
|
||||
Копирует партицию из таблицы `table1` в таблицу `table2`. Данные из `table1` не удаляются.
|
||||
Копирует партицию из таблицы `table1` в таблицу `table2` с заменой существующих данных в `table2`. Данные из `table1` не удаляются.
|
||||
|
||||
Следует иметь в виду:
|
||||
|
||||
|
@ -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}
|
||||
|
||||
Возвращает массив того же размера, что и исходный массив, содержащий элементы в обратном порядке.
|
||||
|
@ -62,10 +62,10 @@ INSERT INTO [db.]table [(c1, c2, c3)] SELECT ...
|
||||
|
||||
### Замечания о производительности
|
||||
|
||||
`INSERT` сортирует входящие данные по первичному ключу и разбивает их на партиции по месяцам. Если вы вставляете данные за разные месяцы вперемешку, то это может значительно снизить производительность запроса `INSERT`. Чтобы избежать этого:
|
||||
`INSERT` сортирует входящие данные по первичному ключу и разбивает их на партиции по ключу партиционирования. Если вы вставляете данные в несколько партиций одновременно, то это может значительно снизить производительность запроса `INSERT`. Чтобы избежать этого:
|
||||
|
||||
- Добавляйте данные достаточно большими пачками. Например, по 100 000 строк.
|
||||
- Группируйте данные по месяцам самостоятельно перед загрузкой в ClickHouse.
|
||||
- Группируйте данные по ключу партиционирования самостоятельно перед загрузкой в ClickHouse.
|
||||
|
||||
Снижения производительности не будет, если:
|
||||
|
||||
|
@ -11,11 +11,8 @@ def concatenate(lang, docs_path, single_page_file):
|
||||
az_re = re.compile(r'[a-z]')
|
||||
|
||||
with open(proj_config) as cfg_file:
|
||||
files_to_concatenate = []
|
||||
for l in cfg_file:
|
||||
if '.md' in l and 'single_page' not in l:
|
||||
path = (l[l.index(':') + 1:]).strip(" '\n")
|
||||
files_to_concatenate.append(path)
|
||||
files_to_concatenate = [(l[l.index(':') + 1:]).strip(" '\n") for l in cfg_file
|
||||
if '.md' in l and 'single_page' not in l]
|
||||
|
||||
logging.info(
|
||||
str(len(files_to_concatenate)) +
|
||||
|
Loading…
Reference in New Issue
Block a user