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

This commit is contained in:
Alexey Milovidov 2019-11-23 03:13:09 +03:00
commit e1d8e42cf4
106 changed files with 2025 additions and 863 deletions

View File

@ -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

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

2
contrib/protobuf vendored

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

View File

@ -1112,7 +1112,14 @@ private:
/// Check if server send Exception packet
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();

View File

@ -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

View File

@ -41,8 +41,8 @@ struct Settings : public SettingsCollection<Settings>
* Note: as an alternative, we could implement settings to be completely dynamic in form of map: String -> Field,
* 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) \

View File

@ -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;

View File

@ -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, \

View File

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

View File

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

View File

@ -125,7 +125,11 @@ BlockInputStreamPtr ClickHouseDictionarySource::loadAll()
* the necessity of holding process_list_element shared pointer.
*/
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);
}

View File

@ -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);

View File

@ -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);

View File

@ -971,8 +971,16 @@ public:
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isStringOrFixedString(arguments[0].type))
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
{
if (this->getName().find("OrZero") != std::string::npos ||
this->getName().find("OrNull") != std::string::npos)
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName() +
". Conversion functions with postfix 'OrZero' or 'OrNull' should take String argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
else
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (arguments.size() == 2)
{

View File

@ -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;

View File

@ -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;

View File

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

View File

@ -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;

View File

@ -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...);
}
}

View File

@ -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.

View File

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

View File

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

View File

@ -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;
}
}

View File

@ -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.

View File

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

View File

@ -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...);
}
}

View File

@ -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)
{

View File

@ -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;
};

View File

@ -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;

View File

@ -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;
};

View File

@ -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)

View File

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

View File

@ -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?

View File

@ -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();
}
}

View File

@ -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;

View File

@ -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;
}

View File

@ -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;

View File

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

View File

@ -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

View File

@ -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;

View File

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

View File

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

View File

@ -320,7 +320,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
}
/// Can be used in fileSegmentationEngine for parallel parsing of Values
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())

View File

@ -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;

View File

@ -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())

View File

@ -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;

View File

@ -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);
}
}

View File

@ -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.

View File

@ -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();

View File

@ -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);
});
}

View File

@ -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;

View File

@ -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);
});
}

View File

@ -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");
};

View File

@ -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);

View File

@ -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);
});
}

View File

@ -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;
};
}

View File

@ -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);
});
}
}

View File

@ -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_)
{
}

View File

@ -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.

View File

@ -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_)

View File

@ -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);

View File

@ -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();

View File

@ -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;
};
}

View File

@ -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)

View File

@ -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;
};
}

View File

@ -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)

View File

@ -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;
};
}

View File

@ -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)

View File

@ -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;
};
}

View File

@ -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)

View File

@ -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;
};
}

View File

@ -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.

View File

@ -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 - это как плюс, но положенный набок, ради того, чтобы превратить его в букву).

View File

@ -14,9 +14,9 @@ Don't use Docker from your system repository.
* [pip](https://pypi.python.org/pypi/pip) and `libpq-dev`. To install: `sudo apt-get install python-pip libpq-dev`
* [py.test](https://docs.pytest.org/) testing framework. To install: `sudo -H pip install pytest`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout`
* [docker-compose](https://docs.docker.com/compose/) and additional python libraries. To install: `sudo -H pip install docker-compose docker dicttoxml kazoo PyMySQL psycopg2 pymongo tzlocal kafka-python protobuf pytest-timeout minio`
(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout`
(highly not recommended) If you really want to use OS packages on modern debian/ubuntu instead of "pip": `sudo apt install -y docker docker-compose python-pytest python-dicttoxml python-docker python-pymysql python-pymongo python-tzlocal python-kazoo python-psycopg2 python-kafka python-pytest-timeout python-minio`
If you want to run the tests under a non-privileged user, you must add this user to `docker` group: `sudo usermod -aG docker $USER` and re-login.
(You must close all your sessions (for example, restart your computer))

View File

@ -1,34 +1,35 @@
import base64
import distutils.dir_util
import errno
import os
import os.path as p
import pwd
import re
import subprocess
import shutil
import distutils.dir_util
import socket
import subprocess
import time
import errno
from dicttoxml import dicttoxml
import pymysql
import urllib
import xml.dom.minidom
import logging
import docker
import psycopg2
import pymongo
import pymysql
from dicttoxml import dicttoxml
from kazoo.client import KazooClient
from kazoo.exceptions import KazooException
import psycopg2
import requests
import base64
import pymongo
import urllib
from minio import Minio
import docker
from docker.errors import ContainerError
from .client import Client, CommandRequest
from .client import Client
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)
with open(full_path, 'w') as f:
@ -36,16 +37,19 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
f.write("=".join([var, value]) + "\n")
return full_path
def subprocess_check_call(args):
# Uncomment for debugging
# print('run:', ' ' . join(args))
subprocess.check_call(args)
def subprocess_call(args):
# Uncomment for debugging
# print('run:', ' ' . join(args))
subprocess.call(args)
def get_odbc_bridge_path():
path = os.environ.get('CLICKHOUSE_TESTS_ODBC_BRIDGE_BIN_PATH')
if path is None:
@ -71,16 +75,21 @@ class ClickHouseCluster:
self.base_dir = p.dirname(base_path)
self.name = name if name is not None else ''
self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/')
self.server_bin_path = p.realpath(server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse'))
self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR',
'/etc/clickhouse-server/')
self.server_bin_path = p.realpath(
server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse'))
self.odbc_bridge_bin_path = p.realpath(odbc_bridge_bin_path or get_odbc_bridge_path())
self.client_bin_path = p.realpath(client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client'))
self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(HELPERS_DIR, 'zookeeper_config.xml')
self.client_bin_path = p.realpath(
client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client'))
self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(
HELPERS_DIR, 'zookeeper_config.xml')
self.project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name
# 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")
@ -105,17 +114,29 @@ class ClickHouseCluster:
self.with_net_trics = False
self.with_redis = False
self.with_minio = False
self.minio_host = "minio1"
self.minio_bucket = "root"
self.minio_port = 9001
self.minio_client = None # type: Minio
self.minio_redirect_host = "redirect"
self.minio_redirect_port = 80
self.docker_client = None
self.is_up = False
def get_client_cmd(self):
cmd = self.client_bin_path
if p.basename(cmd) == 'clickhouse':
cmd += " client"
return cmd
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False, with_redis=False, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test", stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]):
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={},
with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None,
with_odbc_drivers=False, with_postgres=False, with_hdfs=False, with_mongo=False,
with_redis=False, with_minio=False,
hostname=None, env_variables={}, image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]):
"""Add an instance to the cluster.
name - the name of the instance directory and the value of the 'instance' macro in ClickHouse.
@ -133,9 +154,11 @@ class ClickHouseCluster:
instance = ClickHouseInstance(
self, self.base_dir, name, config_dir, main_configs, user_configs, macros, with_zookeeper,
self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, self.base_configs_dir, self.server_bin_path,
self.zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, with_minio,
self.base_configs_dir, self.server_bin_path,
self.odbc_bridge_bin_path, clickhouse_path_dir, with_odbc_drivers, hostname=hostname,
env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address, ipv6_address=ipv6_address,
env_variables=env_variables, image=image, stay_alive=stay_alive, ipv4_address=ipv4_address,
ipv6_address=ipv6_address,
with_installed_binary=with_installed_binary, tmpfs=tmpfs)
self.instances[name] = instance
@ -150,14 +173,14 @@ class ClickHouseCluster:
self.with_zookeeper = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')])
self.base_zookeeper_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')]
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_zookeeper.yml')]
cmds.append(self.base_zookeeper_cmd)
if with_mysql and not self.with_mysql:
self.with_mysql = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')])
self.base_mysql_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')]
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mysql.yml')]
cmds.append(self.base_mysql_cmd)
@ -165,7 +188,7 @@ class ClickHouseCluster:
self.with_postgres = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')])
self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]
cmds.append(self.base_postgres_cmd)
if with_odbc_drivers and not self.with_odbc_drivers:
@ -181,28 +204,29 @@ class ClickHouseCluster:
self.with_postgres = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')])
self.base_postgres_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]
self.project_name, '--file',
p.join(HELPERS_DIR, 'docker_compose_postgres.yml')]
cmds.append(self.base_postgres_cmd)
if with_kafka and not self.with_kafka:
self.with_kafka = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')])
self.base_kafka_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')]
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_kafka.yml')]
cmds.append(self.base_kafka_cmd)
if with_hdfs and not self.with_hdfs:
self.with_hdfs = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')])
self.base_hdfs_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')]
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_hdfs.yml')]
cmds.append(self.base_hdfs_cmd)
if with_mongo and not self.with_mongo:
self.with_mongo = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')])
self.base_mongo_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')]
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_mongo.yml')]
cmds.append(self.base_mongo_cmd)
if self.with_net_trics:
@ -213,12 +237,17 @@ class ClickHouseCluster:
self.with_redis = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')])
self.base_redis_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')]
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_redis.yml')]
if with_minio and not self.with_minio:
self.with_minio = True
self.base_cmd.extend(['--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')])
self.base_minio_cmd = ['docker-compose', '--project-directory', self.base_dir, '--project-name',
self.project_name, '--file', p.join(HELPERS_DIR, 'docker_compose_minio.yml')]
cmds.append(self.base_minio_cmd)
return instance
def get_instance_docker_id(self, instance_name):
# According to how docker-compose names containers.
return self.project_name + '_' + instance_name + '_1'
@ -246,7 +275,7 @@ class ClickHouseCluster:
subprocess.check_call(self.base_cmd + ["up", "--force-recreate", "--no-deps", "-d", node.name])
node.ip_address = self.get_instance_ip(node.name)
node.client = Client(node.ip_address, command=self.client_bin_path)
start_deadline = time.time() + 20.0 # seconds
start_deadline = time.time() + 20.0 # seconds
node.wait_for_start(start_deadline)
return node
@ -327,19 +356,39 @@ class ClickHouseCluster:
print "Can't connect to Mongo " + str(ex)
time.sleep(1)
def wait_minio_to_start(self, timeout=10):
minio_client = Minio('localhost:9001',
access_key='minio',
secret_key='minio123',
secure=False)
start = time.time()
while time.time() - start < timeout:
try:
buckets = minio_client.list_buckets()
self.minio_client = minio_client
logging.info("Connected to Minio %s", buckets)
return
except Exception as ex:
logging.warning("Can't connect to Minio: %s", str(ex))
time.sleep(1)
def start(self, destroy_dirs=True):
if self.is_up:
return
# Just in case kill unstopped containers from previous launch
try:
logging.info("Trying to kill unstopped containers...")
if not subprocess_call(['docker-compose', 'kill']):
subprocess_call(['docker-compose', 'down', '--volumes'])
except:
pass
logging.info("Unstopped containers killed")
if destroy_dirs and p.exists(self.instances_dir):
print "Removing instances dir", self.instances_dir
logging.info("Removing instances dir %s", self.instances_dir)
shutil.rmtree(self.instances_dir)
for instance in self.instances.values():
@ -379,25 +428,45 @@ class ClickHouseCluster:
subprocess_check_call(self.base_redis_cmd + ['up', '-d', '--force-recreate'])
time.sleep(10)
if self.with_minio and self.base_minio_cmd:
minio_start_cmd = self.base_minio_cmd + common_opts
logging.info("Trying to create Minio instance by command %s", ' '.join(map(str, minio_start_cmd)))
subprocess_check_call(minio_start_cmd)
logging.info("Trying to connect to Minio...")
self.wait_minio_to_start()
subprocess_check_call(self.base_cmd + ['up', '-d', '--no-recreate'])
clickhouse_start_cmd = self.base_cmd + ['up', '-d', '--no-recreate']
logging.info("Trying to create ClickHouse instance by command %s", ' '.join(map(str, clickhouse_start_cmd)))
subprocess_check_call(clickhouse_start_cmd)
logging.info("ClickHouse instance created")
start_deadline = time.time() + 20.0 # seconds
start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues():
instance.docker_client = self.docker_client
instance.ip_address = self.get_instance_ip(instance.name)
logging.info("Waiting for ClickHouse start...")
instance.wait_for_start(start_deadline)
logging.info("ClickHouse started")
instance.client = Client(instance.ip_address, command=self.client_bin_path)
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'])
self.is_up = False
self.docker_client = None
@ -407,18 +476,19 @@ 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']))
def get_kazoo_client(self, zoo_instance_name):
zk = KazooClient(hosts=self.get_instance_ip(zoo_instance_name))
zk.start()
return zk
def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name = 'zoo1', repeats=1, sleep_for=1):
def run_kazoo_commands_with_retries(self, kazoo_callback, zoo_instance_name='zoo1', repeats=1, sleep_for=1):
for i in range(repeats - 1):
try:
kazoo_callback(self.get_kazoo_client(zoo_instance_name))
@ -429,7 +499,6 @@ class ClickHouseCluster:
kazoo_callback(self.get_kazoo_client(zoo_instance_name))
def add_zookeeper_startup_command(self, command):
self.pre_zookeeper_commands.append(command)
@ -472,8 +541,10 @@ class ClickHouseInstance:
def __init__(
self, cluster, base_path, name, custom_config_dir, custom_main_configs, custom_user_configs, macros,
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, base_configs_dir, server_bin_path, odbc_bridge_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={}, image="yandex/clickhouse-integration-test",
with_zookeeper, zookeeper_config_path, with_mysql, with_kafka, with_mongo, with_redis, with_minio,
base_configs_dir, server_bin_path, odbc_bridge_bin_path,
clickhouse_path_dir, with_odbc_drivers, hostname=None, env_variables={},
image="yandex/clickhouse-integration-test",
stay_alive=False, ipv4_address=None, ipv6_address=None, with_installed_binary=False, tmpfs=[]):
self.name = name
@ -499,6 +570,7 @@ class ClickHouseInstance:
self.with_kafka = with_kafka
self.with_mongo = with_mongo
self.with_redis = with_redis
self.with_minio = with_minio
self.path = p.join(self.cluster.instances_dir, name)
self.docker_compose_path = p.join(self.path, 'docker_compose.yml')
@ -512,7 +584,7 @@ class ClickHouseInstance:
self.docker_client = None
self.ip_address = None
self.client = None
self.default_timeout = 20.0 # 20 sec
self.default_timeout = 20.0 # 20 sec
self.image = image
self.stay_alive = stay_alive
self.ipv4_address = ipv4_address
@ -523,7 +595,8 @@ class ClickHouseInstance:
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False):
return self.client.query(sql, stdin, timeout, settings, user, ignore_error)
def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False, retry_count=20, sleep_time=0.5, check_callback=lambda x: True):
def query_with_retry(self, sql, stdin=None, timeout=None, settings=None, user=None, ignore_error=False,
retry_count=20, sleep_time=0.5, check_callback=lambda x: True):
result = None
for i in range(retry_count):
try:
@ -553,7 +626,7 @@ class ClickHouseInstance:
# Connects to the instance via HTTP interface, sends a query and returns the answer
def http_query(self, sql, data=None):
return urllib.urlopen("http://"+self.ip_address+":8123/?query="+urllib.quote(sql,safe=''), data).read()
return urllib.urlopen("http://" + self.ip_address + ":8123/?query=" + urllib.quote(sql, safe=''), data).read()
def restart_clickhouse(self, stop_start_wait_sec=5):
if not self.stay_alive:
@ -575,17 +648,21 @@ class ClickHouseInstance:
return output
def contains_in_log(self, substring):
result = self.exec_in_container(["bash", "-c", "grep '{}' /var/log/clickhouse-server/clickhouse-server.log || true".format(substring)])
result = self.exec_in_container(
["bash", "-c", "grep '{}' /var/log/clickhouse-server/clickhouse-server.log || true".format(substring)])
return len(result) > 0
def copy_file_to_container(self, local_path, dest_path):
with open(local_path, 'r') as fdata:
data = fdata.read()
encoded_data = base64.b64encode(data)
self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)], user='root')
self.exec_in_container(["bash", "-c", "echo {} | base64 --decode > {}".format(encoded_data, dest_path)],
user='root')
def get_process_pid(self, process_name):
output = self.exec_in_container(["bash", "-c", "ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format(process_name)])
output = self.exec_in_container(["bash", "-c",
"ps ax | grep '{}' | grep -v 'grep' | grep -v 'bash -c' | awk '{{print $1}}'".format(
process_name)])
if output:
try:
pid = int(output.split('\n')[0].strip())
@ -594,7 +671,6 @@ class ClickHouseInstance:
return None
return None
def restart_with_latest_version(self, stop_start_wait_sec=10, callback_onstop=None, signal=15):
if not self.stay_alive:
raise Exception("Cannot restart not stay alive container")
@ -610,8 +686,12 @@ class ClickHouseInstance:
if callback_onstop:
callback_onstop(self)
self.exec_in_container(["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"], user='root')
self.exec_in_container(["bash", "-c", "cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"], user='root')
self.exec_in_container(
["bash", "-c", "cp /usr/share/clickhouse_fresh /usr/bin/clickhouse && chmod 777 /usr/bin/clickhouse"],
user='root')
self.exec_in_container(["bash", "-c",
"cp /usr/share/clickhouse-odbc-bridge_fresh /usr/bin/clickhouse-odbc-bridge && chmod 777 /usr/bin/clickhouse"],
user='root')
self.exec_in_container(["bash", "-c", "{} --daemon".format(CLICKHOUSE_START_COMMAND)], user=str(os.getuid()))
from helpers.test_tools import assert_eq_with_retry
# wait start
@ -620,15 +700,12 @@ class ClickHouseInstance:
def get_docker_handle(self):
return self.docker_client.containers.get(self.docker_id)
def stop(self):
self.get_docker_handle().stop()
def start(self):
self.get_docker_handle().start()
def wait_for_start(self, deadline=None, timeout=None):
start_time = time.time()
@ -637,9 +714,11 @@ class ClickHouseInstance:
while True:
handle = self.get_docker_handle()
status = handle.status;
status = handle.status
if status == 'exited':
raise Exception("Instance `{}' failed to start. Container status: {}, logs: {}".format(self.name, status, handle.logs()))
raise Exception(
"Instance `{}' failed to start. Container status: {}, logs: {}".format(self.name, status,
handle.logs()))
current_time = time.time()
time_left = deadline - current_time
@ -664,7 +743,6 @@ class ClickHouseInstance:
finally:
sock.close()
@staticmethod
def dict_to_xml(dictionary):
xml_str = dicttoxml(dictionary, custom_root="yandex", attr_type=False)
@ -676,7 +754,7 @@ class ClickHouseInstance:
return {
"SQLite3": {
"DSN": "sqlite3_odbc",
"Database" : "/tmp/sqliteodbc",
"Database": "/tmp/sqliteodbc",
"Driver": "/usr/lib/x86_64-linux-gnu/odbc/libsqlite3odbc.so",
"Setup": "/usr/lib/x86_64-linux-gnu/odbc/libsqlite3odbc.so",
},
@ -750,7 +828,7 @@ class ClickHouseInstance:
macros = self.macros.copy()
macros['instance'] = self.name
with open(p.join(config_d_dir, 'macros.xml'), 'w') as macros_config:
macros_config.write(self.dict_to_xml({"macros" : macros}))
macros_config.write(self.dict_to_xml({"macros": macros}))
# Put ZooKeeper config
if self.with_zookeeper:
@ -789,6 +867,10 @@ class ClickHouseInstance:
depends_on.append("zoo2")
depends_on.append("zoo3")
if self.with_minio:
depends_on.append("minio1")
depends_on.append("redirect")
env_file = _create_env_file(os.path.dirname(self.docker_compose_path), self.env_variables)
odbc_ini_path = ""
@ -820,7 +902,6 @@ class ClickHouseInstance:
binary_volume = "- " + self.server_bin_path + ":/usr/share/clickhouse_fresh"
odbc_bridge_volume = "- " + self.odbc_bridge_bin_path + ":/usr/share/clickhouse-odbc-bridge_fresh"
with open(self.docker_compose_path, 'w') as docker_compose:
docker_compose.write(DOCKER_COMPOSE_TEMPLATE.format(
image=self.image,
@ -844,7 +925,6 @@ class ClickHouseInstance:
ipv6_address=ipv6_address,
))
def destroy_dir(self):
if p.exists(self.path):
shutil.rmtree(self.path)

View File

@ -0,0 +1,31 @@
version: '2.2'
services:
minio1:
image: minio/minio
volumes:
- data1-1:/data1
ports:
- "9001:9001"
environment:
MINIO_ACCESS_KEY: minio
MINIO_SECRET_KEY: minio123
command: server --address :9001 /data1-1
healthcheck:
test: ["CMD", "curl", "-f", "http://localhost:9001/minio/health/live"]
interval: 30s
timeout: 20s
retries: 3
# Redirects all requests to origin Minio.
redirect:
image: schmunk42/nginx-redirect
volumes:
- /nginx:/nginx
environment:
- SERVER_REDIRECT=minio1:9001
- SERVER_REDIRECT_CODE=307
- SERVER_ACCESS_LOG=/nginx/access.log
volumes:
data1-1:

View File

@ -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()

View File

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

View File

@ -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'

View File

@ -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'

View File

@ -108,10 +108,28 @@ def test_insert_select_with_mysql_style_table(started_cluster):
assert node1.query("SELECT sum(`float`) FROM `clickhouse_mysql`.`{}`".format('test_mysql\`_style_table')).rstrip() == '30000'
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(

View File

@ -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"

View File

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

View File

@ -31,7 +31,7 @@ import kafka_pb2
cluster = ClickHouseCluster(__file__)
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;

View File

@ -1,368 +0,0 @@
try:
from BaseHTTPServer import BaseHTTPRequestHandler
except ImportError:
from http.server import BaseHTTPRequestHandler
try:
from BaseHTTPServer import HTTPServer
except ImportError:
from http.server import HTTPServer
try:
import urllib.parse as urlparse
except ImportError:
import urlparse
import json
import logging
import os
import socket
import sys
import threading
import time
import uuid
import xml.etree.ElementTree
BASE_DIR = os.path.dirname(__file__)
logging.getLogger().setLevel(logging.INFO)
file_handler = logging.FileHandler(os.path.join(BASE_DIR, "test-server.log"), "a", encoding="utf-8")
file_handler.setFormatter(logging.Formatter("%(asctime)s %(message)s"))
logging.getLogger().addHandler(file_handler)
logging.getLogger().addHandler(logging.StreamHandler())
communication_port = int(sys.argv[1])
bucket = sys.argv[2]
def GetFreeTCPPortsAndIP(n):
result = []
sockets = []
for i in range(n):
tcp = socket.socket(socket.AF_INET, socket.SOCK_STREAM)
tcp.bind((socket.gethostname(), 0))
addr, port = tcp.getsockname()
result.append(port)
sockets.append(tcp)
[ s.close() for s in sockets ]
return result, addr
(
redirecting_to_http_port,
simple_server_port,
preserving_data_port,
multipart_preserving_data_port,
redirecting_preserving_data_port
), localhost = GetFreeTCPPortsAndIP(5)
data = {
"redirecting_to_http_port": redirecting_to_http_port,
"preserving_data_port": preserving_data_port,
"multipart_preserving_data_port": multipart_preserving_data_port,
"redirecting_preserving_data_port": redirecting_preserving_data_port,
}
class SimpleHTTPServerHandler(BaseHTTPRequestHandler):
def do_GET(self):
logging.info("GET {}".format(self.path))
if self.path == "/milovidov/test.csv":
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.end_headers()
data["redirect_csv_data"] = [[42, 87, 44], [55, 33, 81], [1, 0, 9]]
self.wfile.write("".join([ "{},{},{}\n".format(*row) for row in data["redirect_csv_data"]]))
else:
self.send_response(404)
self.end_headers()
self.finish()
class RedirectingToHTTPHandler(BaseHTTPRequestHandler):
def do_GET(self):
self.send_response(307)
self.send_header("Content-type", "text/xml")
self.send_header("Location", "http://{}:{}/milovidov/test.csv".format(localhost, simple_server_port))
self.end_headers()
self.wfile.write(r"""<?xml version="1.0" encoding="UTF-8"?>
<Error>
<Code>TemporaryRedirect</Code>
<Message>Please re-send this request to the specified temporary endpoint.
Continue to use the original request endpoint for future requests.</Message>
<Endpoint>storage.yandexcloud.net</Endpoint>
</Error>""".encode())
self.finish()
class PreservingDataHandler(BaseHTTPRequestHandler):
protocol_version = "HTTP/1.1"
def parse_request(self):
result = BaseHTTPRequestHandler.parse_request(self)
# Adaptation to Python 3.
if sys.version_info.major == 2 and result == True:
expect = self.headers.get("Expect", "")
if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"):
if not self.handle_expect_100():
return False
return result
def send_response_only(self, code, message=None):
if message is None:
if code in self.responses:
message = self.responses[code][0]
else:
message = ""
if self.request_version != "HTTP/0.9":
self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message))
def handle_expect_100(self):
logging.info("Received Expect-100")
self.send_response_only(100)
self.end_headers()
return True
def do_POST(self):
self.send_response(200)
query = urlparse.urlparse(self.path).query
logging.info("PreservingDataHandler POST ?" + query)
if query == "uploads":
post_data = r"""<?xml version="1.0" encoding="UTF-8"?>
<hi><UploadId>TEST</UploadId></hi>""".encode()
self.send_header("Content-length", str(len(post_data)))
self.send_header("Content-type", "text/plain")
self.end_headers()
self.wfile.write(post_data)
else:
post_data = self.rfile.read(int(self.headers.get("Content-Length")))
self.send_header("Content-type", "text/plain")
self.end_headers()
data["received_data_completed"] = True
data["finalize_data"] = post_data
data["finalize_data_query"] = query
self.finish()
def do_PUT(self):
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("ETag", "hello-etag")
self.end_headers()
query = urlparse.urlparse(self.path).query
path = urlparse.urlparse(self.path).path
logging.info("Content-Length = " + self.headers.get("Content-Length"))
logging.info("PUT " + query)
assert self.headers.get("Content-Length")
assert self.headers["Expect"] == "100-continue"
put_data = self.rfile.read()
data.setdefault("received_data", []).append(put_data)
logging.info("PUT to {}".format(path))
self.server.storage[path] = put_data
self.finish()
def do_GET(self):
path = urlparse.urlparse(self.path).path
if path in self.server.storage:
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("Content-length", str(len(self.server.storage[path])))
self.end_headers()
self.wfile.write(self.server.storage[path])
else:
self.send_response(404)
self.end_headers()
self.finish()
class MultipartPreservingDataHandler(BaseHTTPRequestHandler):
protocol_version = "HTTP/1.1"
def parse_request(self):
result = BaseHTTPRequestHandler.parse_request(self)
# Adaptation to Python 3.
if sys.version_info.major == 2 and result == True:
expect = self.headers.get("Expect", "")
if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"):
if not self.handle_expect_100():
return False
return result
def send_response_only(self, code, message=None):
if message is None:
if code in self.responses:
message = self.responses[code][0]
else:
message = ""
if self.request_version != "HTTP/0.9":
self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message))
def handle_expect_100(self):
logging.info("Received Expect-100")
self.send_response_only(100)
self.end_headers()
return True
def do_POST(self):
query = urlparse.urlparse(self.path).query
logging.info("MultipartPreservingDataHandler POST ?" + query)
if query == "uploads":
self.send_response(200)
post_data = r"""<?xml version="1.0" encoding="UTF-8"?>
<hi><UploadId>TEST</UploadId></hi>""".encode()
self.send_header("Content-length", str(len(post_data)))
self.send_header("Content-type", "text/plain")
self.end_headers()
self.wfile.write(post_data)
else:
try:
assert query == "uploadId=TEST"
logging.info("Content-Length = " + self.headers.get("Content-Length"))
post_data = self.rfile.read(int(self.headers.get("Content-Length")))
root = xml.etree.ElementTree.fromstring(post_data)
assert root.tag == "CompleteMultipartUpload"
assert len(root) > 1
content = ""
for i, part in enumerate(root):
assert part.tag == "Part"
assert len(part) == 2
assert part[0].tag == "PartNumber"
assert part[1].tag == "ETag"
assert int(part[0].text) == i + 1
content += self.server.storage["@"+part[1].text]
data.setdefault("multipart_received_data", []).append(content)
data["multipart_parts"] = len(root)
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.end_headers()
logging.info("Sending 200")
except:
logging.error("Sending 500")
self.send_response(500)
self.finish()
def do_PUT(self):
uid = uuid.uuid4()
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("ETag", str(uid))
self.end_headers()
query = urlparse.urlparse(self.path).query
path = urlparse.urlparse(self.path).path
logging.info("Content-Length = " + self.headers.get("Content-Length"))
logging.info("PUT " + query)
assert self.headers.get("Content-Length")
assert self.headers["Expect"] == "100-continue"
put_data = self.rfile.read()
data.setdefault("received_data", []).append(put_data)
logging.info("PUT to {}".format(path))
self.server.storage["@"+str(uid)] = put_data
self.finish()
def do_GET(self):
path = urlparse.urlparse(self.path).path
if path in self.server.storage:
self.send_response(200)
self.send_header("Content-type", "text/plain")
self.send_header("Content-length", str(len(self.server.storage[path])))
self.end_headers()
self.wfile.write(self.server.storage[path])
else:
self.send_response(404)
self.end_headers()
self.finish()
class RedirectingPreservingDataHandler(BaseHTTPRequestHandler):
protocol_version = "HTTP/1.1"
def parse_request(self):
result = BaseHTTPRequestHandler.parse_request(self)
# Adaptation to Python 3.
if sys.version_info.major == 2 and result == True:
expect = self.headers.get("Expect", "")
if (expect.lower() == "100-continue" and self.protocol_version >= "HTTP/1.1" and self.request_version >= "HTTP/1.1"):
if not self.handle_expect_100():
return False
return result
def send_response_only(self, code, message=None):
if message is None:
if code in self.responses:
message = self.responses[code][0]
else:
message = ""
if self.request_version != "HTTP/0.9":
self.wfile.write("%s %d %s\r\n" % (self.protocol_version, code, message))
def handle_expect_100(self):
logging.info("Received Expect-100")
return True
def do_POST(self):
query = urlparse.urlparse(self.path).query
if query:
query = "?{}".format(query)
self.send_response(307)
self.send_header("Content-type", "text/xml")
self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query))
self.end_headers()
self.wfile.write(r"""<?xml version="1.0" encoding="UTF-8"?>
<Error>
<Code>TemporaryRedirect</Code>
<Message>Please re-send this request to the specified temporary endpoint.
Continue to use the original request endpoint for future requests.</Message>
<Endpoint>{host}:{port}</Endpoint>
</Error>""".format(host=localhost, port=preserving_data_port).encode())
self.finish()
def do_PUT(self):
query = urlparse.urlparse(self.path).query
if query:
query = "?{}".format(query)
self.send_response(307)
self.send_header("Content-type", "text/xml")
self.send_header("Location", "http://{host}:{port}/{bucket}/test.csv{query}".format(host=localhost, port=preserving_data_port, bucket=bucket, query=query))
self.end_headers()
self.wfile.write(r"""<?xml version="1.0" encoding="UTF-8"?>
<Error>
<Code>TemporaryRedirect</Code>
<Message>Please re-send this request to the specified temporary endpoint.
Continue to use the original request endpoint for future requests.</Message>
<Endpoint>{host}:{port}</Endpoint>
</Error>""".format(host=localhost, port=preserving_data_port).encode())
self.finish()
class CommunicationServerHandler(BaseHTTPRequestHandler):
def do_GET(self):
self.send_response(200)
self.end_headers()
self.wfile.write(json.dumps(data))
self.finish()
def do_PUT(self):
self.send_response(200)
self.end_headers()
logging.info(self.rfile.read())
self.finish()
servers = []
servers.append(HTTPServer((localhost, communication_port), CommunicationServerHandler))
servers.append(HTTPServer((localhost, redirecting_to_http_port), RedirectingToHTTPHandler))
servers.append(HTTPServer((localhost, preserving_data_port), PreservingDataHandler))
servers[-1].storage = {}
servers.append(HTTPServer((localhost, multipart_preserving_data_port), MultipartPreservingDataHandler))
servers[-1].storage = {}
servers.append(HTTPServer((localhost, simple_server_port), SimpleHTTPServerHandler))
servers.append(HTTPServer((localhost, redirecting_preserving_data_port), RedirectingPreservingDataHandler))
jobs = [ threading.Thread(target=server.serve_forever) for server in servers ]
[ job.start() for job in jobs ]
time.sleep(60) # Timeout
logging.info("Shutting down")
[ server.shutdown() for server in servers ]
logging.info("Joining threads")
[ job.join() for job in jobs ]
logging.info("Done")

View File

@ -1,159 +1,191 @@
import httplib
import json
import logging
import os
import time
import traceback
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.cluster import ClickHouseCluster, ClickHouseInstance
logging.getLogger().setLevel(logging.INFO)
logging.getLogger().addHandler(logging.StreamHandler())
def get_communication_data(started_cluster):
conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port)
conn.request("GET", "/")
r = conn.getresponse()
raw_data = r.read()
conn.close()
return json.loads(raw_data)
# Creates S3 bucket for tests and allows anonymous read-write access to it.
def prepare_s3_bucket(cluster):
minio_client = cluster.minio_client
if minio_client.bucket_exists(cluster.minio_bucket):
minio_client.remove_bucket(cluster.minio_bucket)
minio_client.make_bucket(cluster.minio_bucket)
# Allows read-write access for bucket without authorization.
bucket_read_write_policy = {"Version": "2012-10-17",
"Statement": [
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetBucketLocation",
"Resource": "arn:aws:s3:::root"
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:ListBucket",
"Resource": "arn:aws:s3:::root"
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:GetObject",
"Resource": "arn:aws:s3:::root/*"
},
{
"Sid": "",
"Effect": "Allow",
"Principal": {"AWS": "*"},
"Action": "s3:PutObject",
"Resource": "arn:aws:s3:::root/*"
}
]}
minio_client.set_bucket_policy(cluster.minio_bucket, json.dumps(bucket_read_write_policy))
def put_communication_data(started_cluster, body):
conn = httplib.HTTPConnection(started_cluster.instances["dummy"].ip_address, started_cluster.communication_port)
conn.request("PUT", "/", body)
r = conn.getresponse()
conn.close()
# Returns content of given S3 file as string.
def get_s3_file_content(cluster, filename):
# type: (ClickHouseCluster, str) -> str
data = cluster.minio_client.get_object(cluster.minio_bucket, filename)
data_str = ""
for chunk in data.stream():
data_str += chunk
return data_str
# Returns nginx access log lines.
def get_nginx_access_logs():
handle = open("/nginx/access.log", "r")
data = handle.readlines()
handle.close()
return data
@pytest.fixture(scope="module")
def started_cluster():
def cluster():
try:
cluster = ClickHouseCluster(__file__)
instance = cluster.add_instance("dummy")
cluster.add_instance("dummy", with_minio=True)
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
cluster.communication_port = 10000
instance.copy_file_to_container(os.path.join(os.path.dirname(__file__), "server.py"), "test_server.py")
cluster.bucket = "abc"
instance.exec_in_container(["python", "test_server.py", str(cluster.communication_port), cluster.bucket], detach=True)
cluster.mock_host = instance.ip_address
for i in range(10):
try:
data = get_communication_data(cluster)
cluster.redirecting_to_http_port = data["redirecting_to_http_port"]
cluster.preserving_data_port = data["preserving_data_port"]
cluster.multipart_preserving_data_port = data["multipart_preserving_data_port"]
cluster.redirecting_preserving_data_port = data["redirecting_preserving_data_port"]
except:
logging.error(traceback.format_exc())
time.sleep(0.5)
else:
break
else:
assert False, "Could not initialize mock server"
prepare_s3_bucket(cluster)
logging.info("S3 bucket created")
yield cluster
finally:
cluster.shutdown()
def run_query(instance, query, stdin=None, settings=None):
# type: (ClickHouseInstance, str, object, dict) -> str
logging.info("Running query '{}'...".format(query))
result = instance.query(query, stdin=stdin, settings=settings)
logging.info("Query finished")
return result
def test_get_with_redirect(started_cluster):
instance = started_cluster.instances["dummy"]
format = "column1 UInt32, column2 UInt32, column3 UInt32"
# Test simple put.
def test_put(cluster):
# type: (ClickHouseCluster) -> None
put_communication_data(started_cluster, "=== Get with redirect test ===")
query = "select *, column1*column2*column3 from s3('http://{}:{}/', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.redirecting_to_http_port, format)
stdout = run_query(instance, query)
data = get_communication_data(started_cluster)
expected = [ [str(row[0]), str(row[1]), str(row[2]), str(row[0]*row[1]*row[2])] for row in data["redirect_csv_data"] ]
assert list(map(str.split, stdout.splitlines())) == expected
def test_put(started_cluster):
instance = started_cluster.instances["dummy"]
format = "column1 UInt32, column2 UInt32, column3 UInt32"
logging.info("Phase 3")
put_communication_data(started_cluster, "=== Put test ===")
instance = cluster.instances["dummy"] # type: ClickHouseInstance
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
values = "(1, 2, 3), (3, 2, 1), (78, 43, 45)"
put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format, values)
values_csv = "1,2,3\n3,2,1\n78,43,45\n"
filename = "test.csv"
put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format(
cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format, values)
run_query(instance, put_query)
data = get_communication_data(started_cluster)
received_data_completed = data["received_data_completed"]
received_data = data["received_data"]
finalize_data = data["finalize_data"]
finalize_data_query = data["finalize_data_query"]
assert received_data[-1].decode() == "1,2,3\n3,2,1\n78,43,45\n"
assert received_data_completed
assert finalize_data == "<CompleteMultipartUpload><Part><PartNumber>1</PartNumber><ETag>hello-etag</ETag></Part></CompleteMultipartUpload>"
assert finalize_data_query == "uploadId=TEST"
def test_put_csv(started_cluster):
instance = started_cluster.instances["dummy"]
format = "column1 UInt32, column2 UInt32, column3 UInt32"
assert values_csv == get_s3_file_content(cluster, filename)
put_communication_data(started_cluster, "=== Put test CSV ===")
put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format)
# Test put values in CSV format.
def test_put_csv(cluster):
# type: (ClickHouseCluster) -> None
instance = cluster.instances["dummy"] # type: ClickHouseInstance
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
filename = "test.csv"
put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format(
cluster.minio_host, cluster.minio_port, cluster.minio_bucket, filename, table_format)
csv_data = "8,9,16\n11,18,13\n22,14,2\n"
run_query(instance, put_query, stdin=csv_data)
data = get_communication_data(started_cluster)
received_data_completed = data["received_data_completed"]
received_data = data["received_data"]
finalize_data = data["finalize_data"]
finalize_data_query = data["finalize_data_query"]
assert received_data[-1].decode() == csv_data
assert received_data_completed
assert finalize_data == "<CompleteMultipartUpload><Part><PartNumber>1</PartNumber><ETag>hello-etag</ETag></Part></CompleteMultipartUpload>"
assert finalize_data_query == "uploadId=TEST"
def test_put_with_redirect(started_cluster):
instance = started_cluster.instances["dummy"]
format = "column1 UInt32, column2 UInt32, column3 UInt32"
assert csv_data == get_s3_file_content(cluster, filename)
put_communication_data(started_cluster, "=== Put with redirect test ===")
other_values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)"
query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') values {}".format(started_cluster.mock_host, started_cluster.redirecting_preserving_data_port, started_cluster.bucket, format, other_values)
# Test put and get with S3 server redirect.
def test_put_get_with_redirect(cluster):
# type: (ClickHouseCluster) -> None
instance = cluster.instances["dummy"] # type: ClickHouseInstance
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
values = "(1, 1, 1), (1, 1, 1), (11, 11, 11)"
values_csv = "1,1,1\n1,1,1\n11,11,11\n"
filename = "test.csv"
query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') values {}".format(
cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format, values)
run_query(instance, query)
query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/test.csv', 'CSV', '{}')".format(started_cluster.mock_host, started_cluster.preserving_data_port, started_cluster.bucket, format)
assert values_csv == get_s3_file_content(cluster, filename)
query = "select *, column1*column2*column3 from s3('http://{}:{}/{}/{}', 'CSV', '{}')".format(
cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format)
stdout = run_query(instance, query)
assert list(map(str.split, stdout.splitlines())) == [
["1", "1", "1", "1"],
["1", "1", "1", "1"],
["11", "11", "11", "1331"],
]
data = get_communication_data(started_cluster)
received_data = data["received_data"]
assert received_data[-1].decode() == "1,1,1\n1,1,1\n11,11,11\n"
def test_multipart_put(started_cluster):
instance = started_cluster.instances["dummy"]
format = "column1 UInt32, column2 UInt32, column3 UInt32"
# Test multipart put.
def test_multipart_put(cluster):
# type: (ClickHouseCluster) -> None
put_communication_data(started_cluster, "=== Multipart test ===")
long_data = [[i, i+1, i+2] for i in range(100000)]
long_values = "".join([ "{},{},{}\n".format(x,y,z) for x, y, z in long_data ])
put_query = "insert into table function s3('http://{}:{}/{}/test.csv', 'CSV', '{}') format CSV".format(started_cluster.mock_host, started_cluster.multipart_preserving_data_port, started_cluster.bucket, format)
run_query(instance, put_query, stdin=long_values, settings={'s3_min_upload_part_size': 1000000})
data = get_communication_data(started_cluster)
assert "multipart_received_data" in data
received_data = data["multipart_received_data"]
assert received_data[-1].decode() == "".join([ "{},{},{}\n".format(x, y, z) for x, y, z in long_data ])
assert 1 < data["multipart_parts"] < 10000
instance = cluster.instances["dummy"] # type: ClickHouseInstance
table_format = "column1 UInt32, column2 UInt32, column3 UInt32"
# Minimum size of part is 5 Mb for Minio.
# See: https://github.com/minio/minio/blob/master/docs/minio-limits.md
min_part_size_bytes = 5 * 1024 * 1024
csv_size_bytes = int(min_part_size_bytes * 1.5) # To have 2 parts.
one_line_length = 6 # 3 digits, 2 commas, 1 line separator.
# Generate data having size more than one part
int_data = [[1, 2, 3] for i in range(csv_size_bytes / one_line_length)]
csv_data = "".join(["{},{},{}\n".format(x, y, z) for x, y, z in int_data])
assert len(csv_data) > min_part_size_bytes
filename = "test_multipart.csv"
put_query = "insert into table function s3('http://{}:{}/{}/{}', 'CSV', '{}') format CSV".format(
cluster.minio_redirect_host, cluster.minio_redirect_port, cluster.minio_bucket, filename, table_format)
run_query(instance, put_query, stdin=csv_data, settings={'s3_min_upload_part_size': min_part_size_bytes})
# Use Nginx access logs to count number of parts uploaded to Minio.
nginx_logs = get_nginx_access_logs()
uploaded_parts = filter(lambda log_line: log_line.find(filename) >= 0 and log_line.find("PUT") >= 0, nginx_logs)
assert uploaded_parts > 1
assert csv_data == get_s3_file_content(cluster, filename)

View File

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

View File

@ -2,7 +2,7 @@ drop table if exists test_table_hdfs_syntax
;
create table test_table_hdfs_syntax (id UInt32) ENGINE = HDFS('')
; -- { 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
;

View File

@ -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
;

View File

@ -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
;

View File

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

View File

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

View File

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

View File

@ -753,12 +753,12 @@ This section contains the following parameters:
For example:
```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.

View File

@ -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 -->

View File

@ -20,12 +20,183 @@ The result type is an integer with bits equal to the maximum bits of its argumen
## bitRotateRight(a, b)
## bitTest(a, b)
## bitTest {#bittest}
## bitTestAll(a, b)
Takes any integer and converts it into [binary form](https://en.wikipedia.org/wiki/Binary_number), returns the value of a bit at specified position. The countdown starts from 0 from the right to the left.
## bitTestAny(a, b)
**Syntax**
```sql
SELECT bitTest(number, index)
```
**Parameters**
- `number` integer number.
- `index` position of bit.
**Returned values**
Returns a value of bit at specified position.
Type: `UInt8`.
**Example**
For example, the number 43 in base-2 (binary) numeral system is 101011.
Query:
```sql
SELECT bitTest(43, 1)
```
Result:
```text
┌─bitTest(43, 1)─┐
│ 1 │
└────────────────┘
```
Another example:
Query:
```sql
SELECT bitTest(43, 2)
```
Result:
```text
┌─bitTest(43, 2)─┐
│ 0 │
└────────────────┘
```
## bitTestAll {#bittestall}
Returns result of [logical conjuction](https://en.wikipedia.org/wiki/Logical_conjunction) (AND operator) of all bits at given positions. The countdown starts from 0 from the right to the left.
The conjuction for bitwise operations:
0 AND 0 = 0
0 AND 1 = 0
1 AND 0 = 0
1 AND 1 = 1
**Syntax**
```sql
SELECT bitTestAll(number, index1, index2, index3, index4, ...)
```
**Parameters**
- `number` integer number.
- `index1`, `index2`, `index3`, `index4` positions of bit. For example, for set of positions (`index1`, `index2`, `index3`, `index4`) is true if and only if all of its positions are true (`index1` ⋀ `index2`, ⋀ `index3``index4`).
**Returned values**
Returns result of logical conjuction.
Type: `UInt8`.
**Example**
For example, the number 43 in base-2 (binary) numeral system is 101011.
Query:
```sql
SELECT bitTestAll(43, 0, 1, 3, 5)
```
Result:
```text
┌─bitTestAll(43, 0, 1, 3, 5)─┐
│ 1 │
└────────────────────────────┘
```
Another example:
Query:
```sql
SELECT bitTestAll(43, 0, 1, 3, 5, 2)
```
Result:
```text
┌─bitTestAll(43, 0, 1, 3, 5, 2)─┐
│ 0 │
└───────────────────────────────┘
```
## bitTestAny {#bittestany}
Returns result of [logical disjunction](https://en.wikipedia.org/wiki/Logical_disjunction) (OR operator) of all bits at given positions. The countdown starts from 0 from the right to the left.
The disjunction for bitwise operations:
0 OR 0 = 0
0 OR 1 = 1
1 OR 0 = 1
1 OR 1 = 1
**Syntax**
```sql
SELECT bitTestAny(number, index1, index2, index3, index4, ...)
```
**Parameters**
- `number` integer number.
- `index1`, `index2`, `index3`, `index4` positions of bit.
**Returned values**
Returns result of logical disjuction.
Type: `UInt8`.
**Example**
For example, the number 43 in base-2 (binary) numeral system is 101011.
Query:
```sql
SELECT bitTestAny(43, 0, 2)
```
Result:
```text
┌─bitTestAny(43, 0, 2)─┐
│ 1 │
└──────────────────────┘
```
Another example:
Query:
```sql
SELECT bitTestAny(43, 4, 2)
```
Result:
```text
┌─bitTestAny(43, 4, 2)─┐
│ 0 │
└──────────────────────┘
```
[Original article](https://clickhouse.yandex/docs/en/query_language/functions/bit_functions/) <!--hide-->

View File

@ -40,8 +40,36 @@ SELECT toInt64(nan), toInt32(32), toInt16('16'), toInt8(8.8)
## toInt(8|16|32|64)OrZero
It takes an argument of type String and tries to parse it into Int (8 | 16 | 32 | 64). If failed, returns 0.
**Example**
```sql
select toInt64OrZero('123123'), toInt8OrZero('123qwe123')
```
```text
┌─toInt64OrZero('123123')─┬─toInt8OrZero('123qwe123')─┐
│ 123123 │ 0 │
└─────────────────────────┴───────────────────────────┘
```
## toInt(8|16|32|64)OrNull
It takes an argument of type String and tries to parse it into Int (8 | 16 | 32 | 64). If failed, returns NULL.
**Example**
```sql
select toInt64OrNull('123123'), toInt8OrNull('123qwe123')
```
```text
┌─toInt64OrNull('123123')─┬─toInt8OrNull('123qwe123')─┐
│ 123123 │ ᴺᵁᴸᴸ │
└─────────────────────────┴───────────────────────────┘
```
## toUInt(8|16|32|64)
Converts an input value to the [UInt](../../data_types/int_uint.md) data type. This function family includes:

View File

@ -95,7 +95,7 @@ Provides possibility to stop background merges for tables in the MergeTree famil
```sql
SYSTEM STOP MERGES [[db.]merge_tree_family_table_name]
```
!!! note "Note":
!!! note "Note"
`DETACH / ATTACH` table will start background merges for the table even in case when merges have been stopped for all MergeTree tables before.

View File

@ -180,7 +180,7 @@ ClickHouse использует небольшое подмножество фу
### 2.14. Все функции с состоянием переделать на FunctionBuilder.
Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлимо.
Долг [Николай Кочетов](https://github.com/KochetovNicolai). Сейчас код находится в переходном состоянии, что неприемлемо.
### 2.15. Функция subscribe для IStorage.
@ -199,7 +199,7 @@ ClickHouse использует небольшое подмножество фу
Требует 3.1.
### 3.3. Исправить катастрофически отвратительно неприемлимый поиск по документации.
### 3.3. Исправить катастрофически отвратительно неприемлемый поиск по документации.
[Иван Блинков](https://github.com/blinkov/) - очень хороший человек. Сам сайт документации основан на технологиях, не удовлетворяющих требованиям задачи, и эти технологии трудно исправить.

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