diff --git a/.gitignore b/.gitignore index 5738aec9dca..ac01f15002f 100644 --- a/.gitignore +++ b/.gitignore @@ -12,6 +12,7 @@ build /docs/en_single_page/ /docs/ru_single_page/ +/docs/venv/ # callgrind files callgrind.out.* diff --git a/dbms/cmake/version.cmake b/dbms/cmake/version.cmake index 2eedb20c4c6..eb14b0de818 100644 --- a/dbms/cmake/version.cmake +++ b/dbms/cmake/version.cmake @@ -1,6 +1,6 @@ # This strings autochanged from release_lib.sh: -set(VERSION_DESCRIBE v1.1.54354-testing) -set(VERSION_REVISION 54354) +set(VERSION_DESCRIBE v1.1.54355-testing) +set(VERSION_REVISION 54355) # end of autochange set (VERSION_MAJOR 1) diff --git a/dbms/src/Client/Connection.cpp b/dbms/src/Client/Connection.cpp index 4d0b20f0168..3c6c8d75d0e 100644 --- a/dbms/src/Client/Connection.cpp +++ b/dbms/src/Client/Connection.cpp @@ -393,7 +393,7 @@ void Connection::sendData(const Block & block, const String & name) else maybe_compressed_out = out; - block_out = std::make_shared(*maybe_compressed_out, server_revision); + block_out = std::make_shared(*maybe_compressed_out, server_revision, block.cloneEmpty()); } writeVarUInt(Protocol::Client::Data, *out); diff --git a/dbms/src/Common/ConfigProcessor/ConfigProcessor.cpp b/dbms/src/Common/ConfigProcessor/ConfigProcessor.cpp index aa51203dd9a..0b4572c4997 100644 --- a/dbms/src/Common/ConfigProcessor/ConfigProcessor.cpp +++ b/dbms/src/Common/ConfigProcessor/ConfigProcessor.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include #include @@ -356,7 +357,7 @@ void ConfigProcessor::doIncludesRecursive( ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & config_path) { - Files res; + Files files; Poco::Path merge_dir_path(config_path); merge_dir_path.setExtension("d"); @@ -378,12 +379,14 @@ ConfigProcessor::Files ConfigProcessor::getConfigMergeFiles(const std::string & Poco::File & file = *it; if (file.isFile() && (endsWith(file.path(), ".xml") || endsWith(file.path(), ".conf"))) { - res.push_back(file.path()); + files.push_back(file.path()); } } } - return res; + std::sort(files.begin(), files.end()); + + return files; } XMLDocumentPtr ConfigProcessor::processConfig( diff --git a/dbms/src/Common/ConfigProcessor/ConfigProcessor.h b/dbms/src/Common/ConfigProcessor/ConfigProcessor.h index a5e58b21b24..a10f0ffe967 100644 --- a/dbms/src/Common/ConfigProcessor/ConfigProcessor.h +++ b/dbms/src/Common/ConfigProcessor/ConfigProcessor.h @@ -2,6 +2,7 @@ #include #include +#include #include #include @@ -87,7 +88,7 @@ public: void savePreprocessedConfig(const LoadedConfig & loaded_config); public: - using Files = std::list; + using Files = std::vector; static Files getConfigMergeFiles(const std::string & config_path); diff --git a/dbms/src/Core/Block.cpp b/dbms/src/Core/Block.cpp index 328aa6074f0..4d3889da8b9 100644 --- a/dbms/src/Core/Block.cpp +++ b/dbms/src/Core/Block.cpp @@ -6,6 +6,8 @@ #include #include +#include + #include #include @@ -18,6 +20,7 @@ namespace ErrorCodes extern const int POSITION_OUT_OF_BOUND; extern const int NOT_FOUND_COLUMN_IN_BLOCK; extern const int SIZES_OF_COLUMNS_DOESNT_MATCH; + extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; } @@ -276,13 +279,7 @@ std::string Block::dumpStructure() const { if (it != data.begin()) out << ", "; - - out << it->name << ' ' << it->type->getName(); - - if (it->column) - out << ' ' << it->column->dumpStructure(); - else - out << " nullptr"; + it->dumpStructure(out); } return out.str(); } @@ -379,22 +376,58 @@ Names Block::getNames() const } -bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs) +template +static ReturnType checkBlockStructure(const Block & lhs, const Block & rhs, const std::string & context_description) { - size_t columns = lhs.columns(); - if (rhs.columns() != columns) - return false; + auto on_error = [](const std::string & message [[maybe_unused]], int code [[maybe_unused]]) + { + if constexpr (std::is_same_v) + throw Exception(message, code); + else + return false; + }; + + size_t columns = rhs.columns(); + if (lhs.columns() != columns) + return on_error("Block structure mismatch in " + context_description + " stream: different number of columns:\n" + + lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); for (size_t i = 0; i < columns; ++i) { - const IDataType & lhs_type = *lhs.safeGetByPosition(i).type; - const IDataType & rhs_type = *rhs.safeGetByPosition(i).type; + const auto & expected = rhs.getByPosition(i); + const auto & actual = lhs.getByPosition(i); - if (!lhs_type.equals(rhs_type)) - return false; + if (actual.name != expected.name) + return on_error("Block structure mismatch in " + context_description + " stream: different names of columns:\n" + + lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); + + if (!actual.type->equals(*expected.type)) + return on_error("Block structure mismatch in " + context_description + " stream: different types:\n" + + lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); + + if (actual.column->getName() != expected.column->getName()) + return on_error("Block structure mismatch in " + context_description + " stream: different columns:\n" + + lhs.dumpStructure() + "\n" + rhs.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); + + if (actual.column->isColumnConst() && expected.column->isColumnConst() + && static_cast(*actual.column).getField() != static_cast(*expected.column).getField()) + return on_error("Block structure mismatch in " + context_description + " stream: different values of constants", + ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); } - return true; + return ReturnType(true); +} + + +bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs) +{ + return checkBlockStructure(lhs, rhs, {}); +} + + +void assertBlocksHaveEqualStructure(const Block & lhs, const Block & rhs, const std::string & context_description) +{ + checkBlockStructure(lhs, rhs, context_description); } @@ -453,12 +486,12 @@ void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out for (auto it = left_columns.rbegin(); it != left_columns.rend(); ++it) { - lhs_diff_writer << it->prettyPrint(); + lhs_diff_writer << it->dumpStructure(); lhs_diff_writer << ", position: " << lhs.getPositionByName(it->name) << '\n'; } for (auto it = right_columns.rbegin(); it != right_columns.rend(); ++it) { - rhs_diff_writer << it->prettyPrint(); + rhs_diff_writer << it->dumpStructure(); rhs_diff_writer << ", position: " << rhs.getPositionByName(it->name) << '\n'; } } diff --git a/dbms/src/Core/Block.h b/dbms/src/Core/Block.h index 0513f50456e..7c836e49532 100644 --- a/dbms/src/Core/Block.h +++ b/dbms/src/Core/Block.h @@ -137,10 +137,13 @@ using Blocks = std::vector; using BlocksList = std::list; -/// Compare column types for blocks. The order of the columns matters. Names do not matter. +/// Compare number of columns, data types, column types, column names, and values of constant columns. bool blocksHaveEqualStructure(const Block & lhs, const Block & rhs); -/// Calculate difference in structure of blocks and write description into output strings. +/// Throw exception when blocks are different. +void assertBlocksHaveEqualStructure(const Block & lhs, const Block & rhs, const std::string & context_description); + +/// Calculate difference in structure of blocks and write description into output strings. NOTE It doesn't compare values of constant columns. void getBlocksDifference(const Block & lhs, const Block & rhs, std::string & out_lhs_diff, std::string & out_rhs_diff); diff --git a/dbms/src/Core/ColumnWithTypeAndName.cpp b/dbms/src/Core/ColumnWithTypeAndName.cpp index 37afe8a4641..9acc2d56408 100644 --- a/dbms/src/Core/ColumnWithTypeAndName.cpp +++ b/dbms/src/Core/ColumnWithTypeAndName.cpp @@ -1,6 +1,7 @@ #include #include #include +#include namespace DB @@ -19,7 +20,7 @@ ColumnWithTypeAndName ColumnWithTypeAndName::cloneEmpty() const } -bool ColumnWithTypeAndName::operator== (const ColumnWithTypeAndName & other) const +bool ColumnWithTypeAndName::operator==(const ColumnWithTypeAndName & other) const { return name == other.name && ((!type && !other.type) || (type && other.type && type->equals(*other.type))) @@ -27,20 +28,25 @@ bool ColumnWithTypeAndName::operator== (const ColumnWithTypeAndName & other) con } -String ColumnWithTypeAndName::prettyPrint() const +void ColumnWithTypeAndName::dumpStructure(WriteBuffer & out) const +{ + out << name; + + if (type) + out << ' ' << type->getName(); + else + out << " nullptr"; + + if (column) + out << ' ' << column->dumpStructure(); + else + out << " nullptr"; +} + +String ColumnWithTypeAndName::dumpStructure() const { WriteBufferFromOwnString out; - writeString(name, out); - if (type) - { - writeChar(' ', out); - writeString(type->getName(), out); - } - if (column) - { - writeChar(' ', out); - writeString(column->getName(), out); - } + dumpStructure(out); return out.str(); } diff --git a/dbms/src/Core/ColumnWithTypeAndName.h b/dbms/src/Core/ColumnWithTypeAndName.h index edf61430abf..9c52145f581 100644 --- a/dbms/src/Core/ColumnWithTypeAndName.h +++ b/dbms/src/Core/ColumnWithTypeAndName.h @@ -7,6 +7,9 @@ namespace DB { +class WriteBuffer; + + /** Column data along with its data type and name. * Column data could be nullptr - to represent just 'header' of column. * Name could be either name from a table or some temporary generated name during expression evaluation. @@ -28,7 +31,9 @@ struct ColumnWithTypeAndName ColumnWithTypeAndName cloneEmpty() const; bool operator==(const ColumnWithTypeAndName & other) const; - String prettyPrint() const; + + void dumpStructure(WriteBuffer & out) const; + String dumpStructure() const; }; } diff --git a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp index a52c35da8ad..205b665c712 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp +++ b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.cpp @@ -10,7 +10,7 @@ namespace DB { -void AddingDefaultBlockOutputStream::write(const DB::Block & block) +void AddingDefaultBlockOutputStream::write(const Block & block) { Block res = block; @@ -71,9 +71,6 @@ void AddingDefaultBlockOutputStream::write(const DB::Block & block) } /// Computes explicitly specified values (in column_defaults) by default. - /** @todo if somehow block does not contain values for implicitly-defaulted columns that are prerequisites - * for explicitly-defaulted ones, exception will be thrown during evaluating such columns - * (implicitly-defaulted columns are evaluated on the line after following one. */ evaluateMissingDefaults(res, required_columns, column_defaults, context); output->write(res); diff --git a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h index 9ac92e74ac9..b54fb475a4b 100644 --- a/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h +++ b/dbms/src/DataStreams/AddingDefaultBlockOutputStream.h @@ -19,16 +19,18 @@ class AddingDefaultBlockOutputStream : public IBlockOutputStream public: AddingDefaultBlockOutputStream( const BlockOutputStreamPtr & output_, + const Block & header_, NamesAndTypesList required_columns_, const ColumnDefaults & column_defaults_, const Context & context_, bool only_explicit_column_defaults_) - : output(output_), required_columns(required_columns_), + : output(output_), header(header_), required_columns(required_columns_), column_defaults(column_defaults_), context(context_), only_explicit_column_defaults(only_explicit_column_defaults_) { } + Block getHeader() const override { return header; } void write(const Block & block) override; void flush() override; @@ -38,6 +40,7 @@ public: private: BlockOutputStreamPtr output; + Block header; NamesAndTypesList required_columns; const ColumnDefaults column_defaults; const Context & context; diff --git a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp index 8896c40e511..ce91333bfe0 100644 --- a/dbms/src/DataStreams/AggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/AggregatingBlockInputStream.cpp @@ -76,7 +76,7 @@ Block AggregatingBlockInputStream::readImpl() AggregatingBlockInputStream::TemporaryFileStream::TemporaryFileStream(const std::string & path) - : file_in(path), compressed_in(file_in), block_in(std::make_shared(compressed_in, ClickHouseRevision::get())) {} - + : file_in(path), compressed_in(file_in), + block_in(std::make_shared(compressed_in, ClickHouseRevision::get())) {} } diff --git a/dbms/src/DataStreams/BlockIO.h b/dbms/src/DataStreams/BlockIO.h index 8cd19db9154..6d97e30e510 100644 --- a/dbms/src/DataStreams/BlockIO.h +++ b/dbms/src/DataStreams/BlockIO.h @@ -21,8 +21,6 @@ struct BlockIO BlockInputStreamPtr in; BlockOutputStreamPtr out; - Block out_sample; /// Example of a block to be written to `out`. - /// Callbacks for query logging could be set here. std::function finish_callback; std::function exception_callback; @@ -50,7 +48,6 @@ struct BlockIO process_list_entry = rhs.process_list_entry; in = rhs.in; out = rhs.out; - out_sample = rhs.out_sample; finish_callback = rhs.finish_callback; exception_callback = rhs.exception_callback; diff --git a/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.cpp b/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.cpp index 8c466bc3c00..3206e918232 100644 --- a/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.cpp +++ b/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.cpp @@ -5,8 +5,8 @@ namespace DB { -BlockOutputStreamFromRowOutputStream::BlockOutputStreamFromRowOutputStream(RowOutputStreamPtr row_output_) - : row_output(row_output_), first_row(true) {} +BlockOutputStreamFromRowOutputStream::BlockOutputStreamFromRowOutputStream(RowOutputStreamPtr row_output_, const Block & header_) + : row_output(row_output_), header(header_) {} void BlockOutputStreamFromRowOutputStream::write(const Block & block) diff --git a/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.h b/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.h index 63743f7827a..dfb6f49ecec 100644 --- a/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.h +++ b/dbms/src/DataStreams/BlockOutputStreamFromRowOutputStream.h @@ -13,7 +13,9 @@ namespace DB class BlockOutputStreamFromRowOutputStream : public IBlockOutputStream { public: - BlockOutputStreamFromRowOutputStream(RowOutputStreamPtr row_output_); + BlockOutputStreamFromRowOutputStream(RowOutputStreamPtr row_output_, const Block & header_); + + Block getHeader() const override { return header; } void write(const Block & block) override; void writePrefix() override { row_output->writePrefix(); } void writeSuffix() override { row_output->writeSuffix(); } @@ -29,7 +31,8 @@ public: private: RowOutputStreamPtr row_output; - bool first_row; + Block header; + bool first_row = true; }; } diff --git a/dbms/src/DataStreams/CastTypeBlockInputStream.cpp b/dbms/src/DataStreams/CastTypeBlockInputStream.cpp index 9dbd0962c93..5028799d41d 100644 --- a/dbms/src/DataStreams/CastTypeBlockInputStream.cpp +++ b/dbms/src/DataStreams/CastTypeBlockInputStream.cpp @@ -8,11 +8,37 @@ namespace DB CastTypeBlockInputStream::CastTypeBlockInputStream( const Context & context_, - const BlockInputStreamPtr & input_, - const Block & reference_definition_) - : context(context_), ref_definition(reference_definition_) + const BlockInputStreamPtr & input, + const Block & reference_definition) + : context(context_) { - children.emplace_back(input_); + children.emplace_back(input); + + Block input_header = input->getHeader(); + + for (size_t col_num = 0, num_columns = input_header.columns(); col_num < num_columns; ++col_num) + { + const auto & elem = input_header.getByPosition(col_num); + + if (!reference_definition.has(elem.name)) + { + header.insert(elem); + continue; + } + + const auto & ref_column = reference_definition.getByName(elem.name); + + /// Force conversion if source and destination types is different. + if (ref_column.type->equals(*elem.type)) + { + header.insert(elem); + } + else + { + header.insert({ castColumn(elem, ref_column.type, context), ref_column.type, elem.name }); + cast_description.emplace(col_num, ref_column.type); + } + } } String CastTypeBlockInputStream::getName() const @@ -27,12 +53,6 @@ Block CastTypeBlockInputStream::readImpl() if (!block) return block; - if (!initialized) - { - initialized = true; - initialize(block); - } - if (cast_description.empty()) return block; @@ -53,23 +73,4 @@ Block CastTypeBlockInputStream::readImpl() return res; } - -void CastTypeBlockInputStream::initialize(const Block & src_block) -{ - for (size_t src_col = 0, num_columns = src_block.columns(); src_col < num_columns; ++src_col) - { - const auto & src_column = src_block.getByPosition(src_col); - - /// Skip, if it is a problem, it will be detected on the next pipeline stage - if (!ref_definition.has(src_column.name)) - continue; - - const auto & ref_column = ref_definition.getByName(src_column.name); - - /// Force conversion if source and destination types is different. - if (!ref_column.type->equals(*src_column.type)) - cast_description.emplace(src_col, ref_column.type); - } -} - } diff --git a/dbms/src/DataStreams/CastTypeBlockInputStream.h b/dbms/src/DataStreams/CastTypeBlockInputStream.h index b92a7ffa31f..f84f6dacf0e 100644 --- a/dbms/src/DataStreams/CastTypeBlockInputStream.h +++ b/dbms/src/DataStreams/CastTypeBlockInputStream.h @@ -7,7 +7,7 @@ namespace DB { -/// Implicitly converts string and numeric values to Enum, numeric types to other numeric types. +/// Implicitly converts types. class CastTypeBlockInputStream : public IProfilingBlockInputStream { public: @@ -17,18 +17,13 @@ public: String getName() const override; - Block getHeader() const override { return ref_definition; } - -protected: - Block readImpl() override; + Block getHeader() const override { return header; } private: - const Context & context; - Block ref_definition; + Block readImpl() override; - /// Initializes cast_description and prepares tmp_conversion_block - void initialize(const Block & src_block); - bool initialized = false; + const Context & context; + Block header; /// Describes required conversions on source block /// Contains column numbers in source block that should be converted diff --git a/dbms/src/DataStreams/CountingBlockOutputStream.h b/dbms/src/DataStreams/CountingBlockOutputStream.h index 63ece36c2b0..0918d33f113 100644 --- a/dbms/src/DataStreams/CountingBlockOutputStream.h +++ b/dbms/src/DataStreams/CountingBlockOutputStream.h @@ -12,7 +12,6 @@ namespace DB class CountingBlockOutputStream : public IBlockOutputStream { public: - CountingBlockOutputStream(const BlockOutputStreamPtr & stream_) : stream(stream_) {} @@ -31,6 +30,7 @@ public: return progress; } + Block getHeader() const override { return stream->getHeader(); } void write(const Block & block) override; void writePrefix() override { stream->writePrefix(); } @@ -40,7 +40,6 @@ public: String getContentType() const override { return stream->getContentType(); } protected: - BlockOutputStreamPtr stream; Progress progress; ProgressCallback progress_callback; diff --git a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp index 455a05e3ad5..5e1d3835d71 100644 --- a/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp +++ b/dbms/src/DataStreams/CreatingSetsBlockInputStream.cpp @@ -1,5 +1,6 @@ #include #include +#include #include #include #include @@ -108,6 +109,7 @@ void CreatingSetsBlockInputStream::createOne(SubqueryForSet & subquery) if (!done_with_table) { + block = materializeBlock(block); table_out->write(block); rows_to_transfer += block.rows(); diff --git a/dbms/src/DataStreams/FilterColumnsBlockInputStream.h b/dbms/src/DataStreams/FilterColumnsBlockInputStream.h index b2ac83c8fdf..dc63ec2823f 100644 --- a/dbms/src/DataStreams/FilterColumnsBlockInputStream.h +++ b/dbms/src/DataStreams/FilterColumnsBlockInputStream.h @@ -21,7 +21,7 @@ public: String getName() const override { - return "FilterColumnsBlockInputStream"; + return "FilterColumns"; } Block getHeader() const override; diff --git a/dbms/src/DataStreams/FormatFactory.cpp b/dbms/src/DataStreams/FormatFactory.cpp index d871a4b23a5..a985c9f3dc0 100644 --- a/dbms/src/DataStreams/FormatFactory.cpp +++ b/dbms/src/DataStreams/FormatFactory.cpp @@ -141,66 +141,66 @@ static BlockOutputStreamPtr getOutputImpl(const String & name, WriteBuffer & buf FormatSettingsJSON json_settings(settings.output_format_json_quote_64bit_integers, settings.output_format_json_quote_denormals); if (name == "Native") - return std::make_shared(buf); + return std::make_shared(buf, 0, sample); else if (name == "RowBinary") - return std::make_shared(std::make_shared(buf)); + return std::make_shared(std::make_shared(buf), sample); else if (name == "TabSeparated" || name == "TSV") - return std::make_shared(std::make_shared(buf, sample)); + return std::make_shared(std::make_shared(buf, sample), sample); else if (name == "TabSeparatedWithNames" || name == "TSVWithNames") - return std::make_shared(std::make_shared(buf, sample, true)); + return std::make_shared(std::make_shared(buf, sample, true), sample); else if (name == "TabSeparatedWithNamesAndTypes" || name == "TSVWithNamesAndTypes") - return std::make_shared(std::make_shared(buf, sample, true, true)); + return std::make_shared(std::make_shared(buf, sample, true, true), sample); else if (name == "TabSeparatedRaw" || name == "TSVRaw") - return std::make_shared(std::make_shared(buf, sample)); + return std::make_shared(std::make_shared(buf, sample), sample); else if (name == "CSV") - return std::make_shared(std::make_shared(buf, sample)); + return std::make_shared(std::make_shared(buf, sample), sample); else if (name == "CSVWithNames") - return std::make_shared(std::make_shared(buf, sample, true)); + return std::make_shared(std::make_shared(buf, sample, true), sample); else if (name == "Pretty") - return std::make_shared(buf, false, settings.output_format_pretty_max_rows, context); + return std::make_shared(buf, sample, false, settings.output_format_pretty_max_rows, context); else if (name == "PrettyCompact") - return std::make_shared(buf, false, settings.output_format_pretty_max_rows, context); + return std::make_shared(buf, sample, false, settings.output_format_pretty_max_rows, context); else if (name == "PrettyCompactMonoBlock") { - BlockOutputStreamPtr dst = std::make_shared(buf, false, settings.output_format_pretty_max_rows, context); + BlockOutputStreamPtr dst = std::make_shared(buf, sample, false, settings.output_format_pretty_max_rows, context); auto res = std::make_shared(dst, settings.output_format_pretty_max_rows, 0); res->disableFlush(); return res; } else if (name == "PrettySpace") - return std::make_shared(buf, false, settings.output_format_pretty_max_rows, context); + return std::make_shared(buf, sample, false, settings.output_format_pretty_max_rows, context); else if (name == "PrettyNoEscapes") - return std::make_shared(buf, true, settings.output_format_pretty_max_rows, context); + return std::make_shared(buf, sample, true, settings.output_format_pretty_max_rows, context); else if (name == "PrettyCompactNoEscapes") - return std::make_shared(buf, true, settings.output_format_pretty_max_rows, context); + return std::make_shared(buf, sample, true, settings.output_format_pretty_max_rows, context); else if (name == "PrettySpaceNoEscapes") - return std::make_shared(buf, true, settings.output_format_pretty_max_rows, context); + return std::make_shared(buf, sample, true, settings.output_format_pretty_max_rows, context); else if (name == "Vertical") return std::make_shared(std::make_shared( - buf, sample, settings.output_format_pretty_max_rows)); + buf, sample, settings.output_format_pretty_max_rows), sample); else if (name == "VerticalRaw") return std::make_shared(std::make_shared( - buf, sample, settings.output_format_pretty_max_rows)); + buf, sample, settings.output_format_pretty_max_rows), sample); else if (name == "Values") - return std::make_shared(std::make_shared(buf)); + return std::make_shared(std::make_shared(buf), sample); else if (name == "JSON") return std::make_shared(std::make_shared( - buf, sample, settings.output_format_write_statistics, json_settings)); + buf, sample, settings.output_format_write_statistics, json_settings), sample); else if (name == "JSONCompact") return std::make_shared(std::make_shared( - buf, sample, settings.output_format_write_statistics, json_settings)); + buf, sample, settings.output_format_write_statistics, json_settings), sample); else if (name == "JSONEachRow") return std::make_shared(std::make_shared( - buf, sample, json_settings)); + buf, sample, json_settings), sample); else if (name == "XML") return std::make_shared(std::make_shared(buf, sample, - settings.output_format_write_statistics)); + settings.output_format_write_statistics), sample); else if (name == "TSKV") - return std::make_shared(std::make_shared(buf, sample)); + return std::make_shared(std::make_shared(buf, sample), sample); else if (name == "ODBCDriver") return std::make_shared(buf, sample); else if (name == "Null") - return std::make_shared(); + return std::make_shared(sample); else throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT); } @@ -211,7 +211,7 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer & /** Materialization is needed, because formats can use the functions `IDataType`, * which only work with full columns. */ - return std::make_shared(getOutputImpl(name, buf, sample, context)); + return std::make_shared(getOutputImpl(name, buf, materializeBlock(sample), context), sample); } } diff --git a/dbms/src/DataStreams/IBlockInputStream.h b/dbms/src/DataStreams/IBlockInputStream.h index 77ee6b94157..cf158f9b46c 100644 --- a/dbms/src/DataStreams/IBlockInputStream.h +++ b/dbms/src/DataStreams/IBlockInputStream.h @@ -118,9 +118,7 @@ private: size_t checkDepthImpl(size_t max_depth, size_t level) const; - /** Get text that identifies this source and the entire subtree. - * Unlike getID - without taking into account the parameters. - */ + /// Get text with names of this source and the entire subtree. String getTreeID() const; }; diff --git a/dbms/src/DataStreams/IBlockOutputStream.h b/dbms/src/DataStreams/IBlockOutputStream.h index 58e6607f383..e33fced86a3 100644 --- a/dbms/src/DataStreams/IBlockOutputStream.h +++ b/dbms/src/DataStreams/IBlockOutputStream.h @@ -4,12 +4,12 @@ #include #include #include +#include namespace DB { -class Block; struct Progress; class TableStructureReadLock; @@ -26,6 +26,12 @@ class IBlockOutputStream : private boost::noncopyable public: IBlockOutputStream() {} + /** Get data structure of the stream in a form of "header" block (it is also called "sample block"). + * Header block contains column names, data types, columns of size 0. Constant columns must have corresponding values. + * You must pass blocks of exactly this structure to the 'write' method. + */ + virtual Block getHeader() const = 0; + /** Write block. */ virtual void write(const Block & block) = 0; diff --git a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp index eb3715044e1..ec8eb407f9c 100644 --- a/dbms/src/DataStreams/IProfilingBlockInputStream.cpp +++ b/dbms/src/DataStreams/IProfilingBlockInputStream.cpp @@ -5,6 +5,7 @@ #include #include + namespace DB { @@ -15,6 +16,7 @@ namespace ErrorCodes extern const int TIMEOUT_EXCEEDED; extern const int TOO_SLOW; extern const int LOGICAL_ERROR; + extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; } @@ -70,6 +72,15 @@ Block IProfilingBlockInputStream::read() progress(Progress(res.rows(), res.bytes())); +#ifndef NDEBUG + if (res) + { + Block header = getHeader(); + if (header) + assertBlocksHaveEqualStructure(res, header, getName()); + } +#endif + return res; } diff --git a/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp b/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp index 9f5f10d19bf..0e4f876925d 100644 --- a/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp +++ b/dbms/src/DataStreams/InputStreamFromASTInsertQuery.cpp @@ -43,7 +43,7 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery( input_buffer_contacenated = std::make_unique(buffers); - res_stream = context.getInputFormat(format, *input_buffer_contacenated, streams.out_sample, context.getSettings().max_insert_block_size); + res_stream = context.getInputFormat(format, *input_buffer_contacenated, streams.out->getHeader(), context.getSettings().max_insert_block_size); } } diff --git a/dbms/src/DataStreams/MaterializingBlockOutputStream.h b/dbms/src/DataStreams/MaterializingBlockOutputStream.h index 2d8489156f4..9e1efeb29d3 100644 --- a/dbms/src/DataStreams/MaterializingBlockOutputStream.h +++ b/dbms/src/DataStreams/MaterializingBlockOutputStream.h @@ -12,9 +12,10 @@ namespace DB class MaterializingBlockOutputStream : public IBlockOutputStream { public: - MaterializingBlockOutputStream(const BlockOutputStreamPtr & output) - : output{output} {} + MaterializingBlockOutputStream(const BlockOutputStreamPtr & output, const Block & header) + : output{output}, header(header) {} + Block getHeader() const override { return header; } void write(const Block & block) override { output->write(materializeBlock(block)); } void flush() override { output->flush(); } void writePrefix() override { output->writePrefix(); } @@ -27,6 +28,7 @@ public: private: BlockOutputStreamPtr output; + Block header; }; } diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp index d5b48f0b803..ed165fc0e82 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.cpp @@ -34,29 +34,29 @@ static void removeConstantsFromBlock(Block & block) } } -static void removeConstantsFromSortDescription(const Block & sample_block, SortDescription & description) +static void removeConstantsFromSortDescription(const Block & header, SortDescription & description) { description.erase(std::remove_if(description.begin(), description.end(), [&](const SortColumnDescription & elem) { if (!elem.column_name.empty()) - return sample_block.getByName(elem.column_name).column->isColumnConst(); + return header.getByName(elem.column_name).column->isColumnConst(); else - return sample_block.safeGetByPosition(elem.column_number).column->isColumnConst(); + return header.safeGetByPosition(elem.column_number).column->isColumnConst(); }), description.end()); } /** Add into block, whose constant columns was removed by previous function, - * constant columns from sample_block (which must have structure as before removal of constants from block). + * constant columns from header (which must have structure as before removal of constants from block). */ -static void enrichBlockWithConstants(Block & block, const Block & sample_block) +static void enrichBlockWithConstants(Block & block, const Block & header) { size_t rows = block.rows(); - size_t columns = sample_block.columns(); + size_t columns = header.columns(); for (size_t i = 0; i < columns; ++i) { - const auto & col_type_name = sample_block.getByPosition(i); + const auto & col_type_name = header.getByPosition(i); if (col_type_name.column->isColumnConst()) block.insert(i, {col_type_name.column->cloneResized(rows), col_type_name.type, col_type_name.name}); } @@ -65,6 +65,12 @@ static void enrichBlockWithConstants(Block & block, const Block & sample_block) Block MergeSortingBlockInputStream::readImpl() { + if (!header) + { + header = getHeader(); + removeConstantsFromSortDescription(header, description); + } + /** Algorithm: * - read to memory blocks from source stream; * - if too much of them and if external sorting is enabled, @@ -77,12 +83,6 @@ Block MergeSortingBlockInputStream::readImpl() { while (Block block = children.back()->read()) { - if (!sample_block) - { - sample_block = block.cloneEmpty(); - removeConstantsFromSortDescription(sample_block, description); - } - /// If there were only const columns in sort description, then there is no need to sort. /// Return the blocks as is. if (description.empty()) @@ -103,7 +103,7 @@ Block MergeSortingBlockInputStream::readImpl() const std::string & path = temporary_files.back()->path(); WriteBufferFromFile file_buf(path); CompressedWriteBuffer compressed_buf(file_buf); - NativeBlockOutputStream block_out(compressed_buf); + NativeBlockOutputStream block_out(compressed_buf, 0, block.cloneEmpty()); MergeSortingBlocksBlockInputStream block_in(blocks, description, max_merged_block_size, limit); LOG_INFO(log, "Sorting and writing part of data into temporary file " + path); @@ -148,7 +148,7 @@ Block MergeSortingBlockInputStream::readImpl() Block res = impl->read(); if (res) - enrichBlockWithConstants(res, sample_block); + enrichBlockWithConstants(res, header); return res; } diff --git a/dbms/src/DataStreams/MergeSortingBlockInputStream.h b/dbms/src/DataStreams/MergeSortingBlockInputStream.h index cb1101c0843..416dc0ecce7 100644 --- a/dbms/src/DataStreams/MergeSortingBlockInputStream.h +++ b/dbms/src/DataStreams/MergeSortingBlockInputStream.h @@ -108,7 +108,7 @@ private: /// Before operation, will remove constant columns from blocks. And after, place constant columns back. /// (to avoid excessive virtual function calls and because constants cannot be serialized in Native format for temporary files) /// Save original block structure here. - Block sample_block; + Block header; /// Everything below is for external sorting. std::vector> temporary_files; diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp index 56e816cb05a..3b38dae42a9 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.cpp @@ -219,10 +219,10 @@ Block MergingAggregatedMemoryEfficientBlockInputStream::readImpl() parallel_merge_data->merged_blocks_changed.wait(lock, [this] { - return parallel_merge_data->finish /// Requested to finish early. - || parallel_merge_data->exception /// An error in merging thread. - || parallel_merge_data->exhausted /// No more data in sources. - || !parallel_merge_data->merged_blocks.empty(); /// Have another merged block. + return parallel_merge_data->finish /// Requested to finish early. + || parallel_merge_data->exception /// An error in merging thread. + || parallel_merge_data->exhausted /// No more data in sources. + || !parallel_merge_data->merged_blocks.empty(); /// Have another merged block. }); if (parallel_merge_data->exception) @@ -493,7 +493,7 @@ MergingAggregatedMemoryEfficientBlockInputStream::BlocksToMerge MergingAggregate while (true) { - if (current_bucket_num == NUM_BUCKETS) + if (current_bucket_num >= NUM_BUCKETS) { /// All ordinary data was processed. Maybe, there are also 'overflows'-blocks. // std::cerr << "at end\n"; diff --git a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h index f7b978e9c66..69af976c52c 100644 --- a/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h +++ b/dbms/src/DataStreams/MergingAggregatedMemoryEfficientBlockInputStream.h @@ -84,7 +84,7 @@ protected: Block readImpl() override; private: - static constexpr size_t NUM_BUCKETS = 256; + static constexpr int NUM_BUCKETS = 256; Aggregator aggregator; bool final; diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp index aee5b34899f..c256e49e60e 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.cpp @@ -11,7 +11,6 @@ namespace ErrorCodes { extern const int LOGICAL_ERROR; extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; - extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; } @@ -92,19 +91,7 @@ void MergingSortedBlockInputStream::init(Block & header, MutableColumns & merged if (!*shared_block_ptr) continue; - size_t src_columns = shared_block_ptr->columns(); - size_t dst_columns = header.columns(); - - if (src_columns != dst_columns) - throw Exception("Merging blocks have different number of columns (" - + toString(src_columns) + " and " + toString(dst_columns) + ")", - ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); - - for (size_t i = 0; i < src_columns; ++i) - if (!blocksHaveEqualStructure(*shared_block_ptr, header)) - throw Exception("Merging blocks have different names or types of columns:\n" - + shared_block_ptr->dumpStructure() + "\nand\n" + header.dumpStructure(), - ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); + assertBlocksHaveEqualStructure(*shared_block_ptr, header, getName()); } merged_columns.resize(num_columns); diff --git a/dbms/src/DataStreams/NativeBlockInputStream.cpp b/dbms/src/DataStreams/NativeBlockInputStream.cpp index 1ec7c902065..787ba974546 100644 --- a/dbms/src/DataStreams/NativeBlockInputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockInputStream.cpp @@ -52,7 +52,7 @@ NativeBlockInputStream::NativeBlockInputStream(ReadBuffer & istr_, UInt64 server for (const auto & column : index_block_it->columns) { auto type = DataTypeFactory::instance().get(column.type); - header.insert({ type->createColumn(), type, column.name }); + header.insert(ColumnWithTypeAndName{ type, column.name }); } } diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.cpp b/dbms/src/DataStreams/NativeBlockOutputStream.cpp index 0e38a3e5bd7..b02d435b39f 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.cpp +++ b/dbms/src/DataStreams/NativeBlockOutputStream.cpp @@ -20,9 +20,9 @@ namespace ErrorCodes NativeBlockOutputStream::NativeBlockOutputStream( - WriteBuffer & ostr_, UInt64 client_revision_, + WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, WriteBuffer * index_ostr_, size_t initial_size_of_file_) - : ostr(ostr_), client_revision(client_revision_), + : ostr(ostr_), client_revision(client_revision_), header(header_), index_ostr(index_ostr_), initial_size_of_file(initial_size_of_file_) { if (index_ostr) diff --git a/dbms/src/DataStreams/NativeBlockOutputStream.h b/dbms/src/DataStreams/NativeBlockOutputStream.h index d76cb827863..7e3f14e06ea 100644 --- a/dbms/src/DataStreams/NativeBlockOutputStream.h +++ b/dbms/src/DataStreams/NativeBlockOutputStream.h @@ -23,9 +23,10 @@ public: /** If non-zero client_revision is specified, additional block information can be written. */ NativeBlockOutputStream( - WriteBuffer & ostr_, UInt64 client_revision_ = 0, + WriteBuffer & ostr_, UInt64 client_revision_, const Block & header_, WriteBuffer * index_ostr_ = nullptr, size_t initial_size_of_file_ = 0); + Block getHeader() const override { return header; } void write(const Block & block) override; void flush() override; @@ -36,7 +37,7 @@ public: private: WriteBuffer & ostr; UInt64 client_revision; - + Block header; WriteBuffer * index_ostr; size_t initial_size_of_file; /// The initial size of the data file, if `append` done. Used for the index. /// If you need to write index, then `ostr` must be a CompressedWriteBuffer. diff --git a/dbms/src/DataStreams/NullBlockOutputStream.h b/dbms/src/DataStreams/NullBlockOutputStream.h index ad0c398629a..3d437527960 100644 --- a/dbms/src/DataStreams/NullBlockOutputStream.h +++ b/dbms/src/DataStreams/NullBlockOutputStream.h @@ -11,7 +11,12 @@ namespace DB class NullBlockOutputStream : public IBlockOutputStream { public: + NullBlockOutputStream(const Block & header) : header(header) {} + Block getHeader() const override { return header; } void write(const Block &) override {} + +private: + Block header; }; } diff --git a/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp b/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp index 60fe4013595..d7d23633b72 100644 --- a/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp +++ b/dbms/src/DataStreams/NullableAdapterBlockInputStream.cpp @@ -14,32 +14,19 @@ namespace ErrorCodes extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH; } -NullableAdapterBlockInputStream::NullableAdapterBlockInputStream( - const BlockInputStreamPtr & input, - const Block & in_sample_, const Block & out_sample_) - : header(out_sample_) -{ - buildActions(in_sample_, out_sample_); - children.push_back(input); -} -Block NullableAdapterBlockInputStream::readImpl() +static Block transform(const Block & block, const NullableAdapterBlockInputStream::Actions & actions, const std::vector> & rename) { - Block block = children.back()->read(); - - if (!block && !must_transform) - return block; + size_t num_columns = block.columns(); Block res; - size_t s = block.columns(); - - for (size_t i = 0; i < s; ++i) + for (size_t i = 0; i < num_columns; ++i) { const auto & elem = block.getByPosition(i); switch (actions[i]) { - case TO_ORDINARY: + case NullableAdapterBlockInputStream::TO_ORDINARY: { const auto & nullable_col = static_cast(*elem.column); const auto & nullable_type = static_cast(*elem.type); @@ -54,11 +41,10 @@ Block NullableAdapterBlockInputStream::readImpl() res.insert({ nullable_col.getNestedColumnPtr(), nullable_type.getNestedType(), - rename[i].value_or(elem.name) - }); + rename[i].value_or(elem.name)}); break; } - case TO_NULLABLE: + case NullableAdapterBlockInputStream::TO_NULLABLE: { ColumnPtr null_map = ColumnUInt8::create(elem.column->size(), 0); @@ -68,12 +54,9 @@ Block NullableAdapterBlockInputStream::readImpl() rename[i].value_or(elem.name)}); break; } - case NONE: + case NullableAdapterBlockInputStream::NONE: { - if (rename[i]) - res.insert({elem.column, elem.type, *rename[i]}); - else - res.insert(elem); + res.insert({elem.column, elem.type, rename[i].value_or(elem.name)}); break; } } @@ -82,13 +65,34 @@ Block NullableAdapterBlockInputStream::readImpl() return res; } -void NullableAdapterBlockInputStream::buildActions( - const Block & in_sample, - const Block & out_sample) -{ - size_t in_size = in_sample.columns(); - if (out_sample.columns() != in_size) +NullableAdapterBlockInputStream::NullableAdapterBlockInputStream( + const BlockInputStreamPtr & input, + const Block & src_header, const Block & res_header) +{ + buildActions(src_header, res_header); + children.push_back(input); + header = transform(src_header, actions, rename); +} + + +Block NullableAdapterBlockInputStream::readImpl() +{ + Block block = children.back()->read(); + + if (!block) + return block; + + return transform(block, actions, rename); +} + +void NullableAdapterBlockInputStream::buildActions( + const Block & src_header, + const Block & res_header) +{ + size_t in_size = src_header.columns(); + + if (res_header.columns() != in_size) throw Exception("Number of columns in INSERT SELECT doesn't match", ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH); actions.reserve(in_size); @@ -96,8 +100,8 @@ void NullableAdapterBlockInputStream::buildActions( for (size_t i = 0; i < in_size; ++i) { - const auto & in_elem = in_sample.getByPosition(i); - const auto & out_elem = out_sample.getByPosition(i); + const auto & in_elem = src_header.getByPosition(i); + const auto & out_elem = res_header.getByPosition(i); bool is_in_nullable = in_elem.type->isNullable(); bool is_out_nullable = out_elem.type->isNullable(); @@ -113,9 +117,6 @@ void NullableAdapterBlockInputStream::buildActions( rename.emplace_back(std::make_optional(out_elem.name)); else rename.emplace_back(); - - if (actions.back() != NONE || rename.back()) - must_transform = true; } } diff --git a/dbms/src/DataStreams/NullableAdapterBlockInputStream.h b/dbms/src/DataStreams/NullableAdapterBlockInputStream.h index 47e064ecdf2..60c2b2ec16e 100644 --- a/dbms/src/DataStreams/NullableAdapterBlockInputStream.h +++ b/dbms/src/DataStreams/NullableAdapterBlockInputStream.h @@ -18,16 +18,13 @@ namespace DB class NullableAdapterBlockInputStream : public IProfilingBlockInputStream { public: - NullableAdapterBlockInputStream(const BlockInputStreamPtr & input, const Block & in_sample_, const Block & out_sample_); + NullableAdapterBlockInputStream(const BlockInputStreamPtr & input, const Block & src_header, const Block & res_header); - String getName() const override { return "NullableAdapterBlockInputStream"; } + String getName() const override { return "NullableAdapter"; } Block getHeader() const override { return header; } -protected: - Block readImpl() override; -private: /// Given a column of a block we have just read, /// how must we process it? enum Action @@ -44,17 +41,17 @@ private: using Actions = std::vector; private: + Block readImpl() override; + /// Determine the actions to be taken using the source sample block, /// which describes the columns from which we fetch data inside an INSERT /// query, and the target sample block which contains the columns /// we insert data into. - void buildActions(const Block & in_sample, const Block & out_sample); + void buildActions(const Block & src_header, const Block & res_header); -private: Block header; Actions actions; std::vector> rename; - bool must_transform = false; }; } diff --git a/dbms/src/DataStreams/ODBCDriverBlockOutputStream.cpp b/dbms/src/DataStreams/ODBCDriverBlockOutputStream.cpp index 71161eeb117..da961948907 100644 --- a/dbms/src/DataStreams/ODBCDriverBlockOutputStream.cpp +++ b/dbms/src/DataStreams/ODBCDriverBlockOutputStream.cpp @@ -7,9 +7,8 @@ namespace DB { -ODBCDriverBlockOutputStream::ODBCDriverBlockOutputStream(WriteBuffer & out_, const Block & sample_) - : out(out_) - , sample(sample_) +ODBCDriverBlockOutputStream::ODBCDriverBlockOutputStream(WriteBuffer & out_, const Block & header_) + : out(out_), header(header_) { } @@ -43,7 +42,7 @@ void ODBCDriverBlockOutputStream::write(const Block & block) void ODBCDriverBlockOutputStream::writePrefix() { - const size_t columns = sample.columns(); + const size_t columns = header.columns(); /// Number of columns. writeVarUInt(columns, out); @@ -51,7 +50,7 @@ void ODBCDriverBlockOutputStream::writePrefix() /// Names and types of columns. for (size_t i = 0; i < columns; ++i) { - const ColumnWithTypeAndName & col = sample.getByPosition(i); + const ColumnWithTypeAndName & col = header.getByPosition(i); writeStringBinary(col.name, out); writeStringBinary(col.type->getName(), out); diff --git a/dbms/src/DataStreams/ODBCDriverBlockOutputStream.h b/dbms/src/DataStreams/ODBCDriverBlockOutputStream.h index 09795b72a3a..a40603c356e 100644 --- a/dbms/src/DataStreams/ODBCDriverBlockOutputStream.h +++ b/dbms/src/DataStreams/ODBCDriverBlockOutputStream.h @@ -19,8 +19,9 @@ class WriteBuffer; class ODBCDriverBlockOutputStream : public IBlockOutputStream { public: - ODBCDriverBlockOutputStream(WriteBuffer & out_, const Block & sample_); + ODBCDriverBlockOutputStream(WriteBuffer & out_, const Block & header_); + Block getHeader() const override { return header; } void write(const Block & block) override; void writePrefix() override; @@ -29,7 +30,7 @@ public: private: WriteBuffer & out; - const Block sample; + const Block header; }; } diff --git a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp index 8476f3020af..9405cbfd389 100644 --- a/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp +++ b/dbms/src/DataStreams/ParallelAggregatingBlockInputStream.cpp @@ -100,7 +100,8 @@ Block ParallelAggregatingBlockInputStream::readImpl() ParallelAggregatingBlockInputStream::TemporaryFileStream::TemporaryFileStream(const std::string & path) - : file_in(path), compressed_in(file_in), block_in(std::make_shared(compressed_in, ClickHouseRevision::get())) {} + : file_in(path), compressed_in(file_in), + block_in(std::make_shared(compressed_in, ClickHouseRevision::get())) {} diff --git a/dbms/src/DataStreams/PrettyBlockOutputStream.cpp b/dbms/src/DataStreams/PrettyBlockOutputStream.cpp index 9c556ed22c5..58b61664b7c 100644 --- a/dbms/src/DataStreams/PrettyBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PrettyBlockOutputStream.cpp @@ -17,8 +17,9 @@ namespace ErrorCodes } -PrettyBlockOutputStream::PrettyBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_) - : ostr(ostr_), max_rows(max_rows_), no_escapes(no_escapes_), context(context_) +PrettyBlockOutputStream::PrettyBlockOutputStream( + WriteBuffer & ostr_, const Block & header_, bool no_escapes_, size_t max_rows_, const Context & context_) + : ostr(ostr_), header(header_), max_rows(max_rows_), no_escapes(no_escapes_), context(context_) { struct winsize w; if (0 == ioctl(STDOUT_FILENO, TIOCGWINSZ, &w)) diff --git a/dbms/src/DataStreams/PrettyBlockOutputStream.h b/dbms/src/DataStreams/PrettyBlockOutputStream.h index 9c6eef51705..7702cd46435 100644 --- a/dbms/src/DataStreams/PrettyBlockOutputStream.h +++ b/dbms/src/DataStreams/PrettyBlockOutputStream.h @@ -17,8 +17,9 @@ class PrettyBlockOutputStream : public IBlockOutputStream { public: /// no_escapes - do not use ANSI escape sequences - to display in the browser, not in the console. - PrettyBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_); + PrettyBlockOutputStream(WriteBuffer & ostr_, const Block & header_, bool no_escapes_, size_t max_rows_, const Context & context_); + Block getHeader() const override { return header; } void write(const Block & block) override; void writeSuffix() override; @@ -32,6 +33,7 @@ protected: void writeExtremes(); WriteBuffer & ostr; + const Block header; size_t max_rows; size_t total_rows = 0; size_t terminal_width = 0; diff --git a/dbms/src/DataStreams/PrettyCompactBlockOutputStream.h b/dbms/src/DataStreams/PrettyCompactBlockOutputStream.h index 38e6fae71cf..82a3a44f720 100644 --- a/dbms/src/DataStreams/PrettyCompactBlockOutputStream.h +++ b/dbms/src/DataStreams/PrettyCompactBlockOutputStream.h @@ -11,8 +11,8 @@ namespace DB class PrettyCompactBlockOutputStream : public PrettyBlockOutputStream { public: - PrettyCompactBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_) - : PrettyBlockOutputStream(ostr_, no_escapes_, max_rows_, context_) {} + PrettyCompactBlockOutputStream(WriteBuffer & ostr_, const Block & header_, bool no_escapes_, size_t max_rows_, const Context & context_) + : PrettyBlockOutputStream(ostr_, header_, no_escapes_, max_rows_, context_) {} void write(const Block & block) override; diff --git a/dbms/src/DataStreams/PrettySpaceBlockOutputStream.h b/dbms/src/DataStreams/PrettySpaceBlockOutputStream.h index 2fd78fa883f..6dbd2c748c4 100644 --- a/dbms/src/DataStreams/PrettySpaceBlockOutputStream.h +++ b/dbms/src/DataStreams/PrettySpaceBlockOutputStream.h @@ -11,8 +11,8 @@ namespace DB class PrettySpaceBlockOutputStream : public PrettyBlockOutputStream { public: - PrettySpaceBlockOutputStream(WriteBuffer & ostr_, bool no_escapes_, size_t max_rows_, const Context & context_) - : PrettyBlockOutputStream(ostr_, no_escapes_, max_rows_, context_) {} + PrettySpaceBlockOutputStream(WriteBuffer & ostr_, const Block & header_, bool no_escapes_, size_t max_rows_, const Context & context_) + : PrettyBlockOutputStream(ostr_, header_, no_escapes_, max_rows_, context_) {} void write(const Block & block) override; void writeSuffix() override; diff --git a/dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.cpp b/dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.cpp deleted file mode 100644 index 0ef5d981da1..00000000000 --- a/dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.cpp +++ /dev/null @@ -1,23 +0,0 @@ -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int ILLEGAL_COLUMN; -} - - -void ProhibitColumnsBlockOutputStream::write(const Block & block) -{ - for (const auto & column : columns) - if (block.has(column.name)) - throw Exception{"Cannot insert column " + column.name, ErrorCodes::ILLEGAL_COLUMN}; - - output->write(block); -} - -} diff --git a/dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.h b/dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.h deleted file mode 100644 index 732eece7904..00000000000 --- a/dbms/src/DataStreams/ProhibitColumnsBlockOutputStream.h +++ /dev/null @@ -1,31 +0,0 @@ -#pragma once - -#include -#include - - -namespace DB -{ - -/// Throws exception on encountering prohibited column in block -class ProhibitColumnsBlockOutputStream : public IBlockOutputStream -{ -public: - ProhibitColumnsBlockOutputStream(const BlockOutputStreamPtr & output, const NamesAndTypesList & columns) - : output{output}, columns{columns} - { - } - -private: - void write(const Block & block) override; - - void flush() override { output->flush(); } - - void writePrefix() override { output->writePrefix(); } - void writeSuffix() override { output->writeSuffix(); } - - BlockOutputStreamPtr output; - NamesAndTypesList columns; -}; - -} diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp index 99711fc31e3..06296b8b34b 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.cpp @@ -1,4 +1,5 @@ -#include "PushingToViewsBlockOutputStream.h" +#include +#include #include @@ -6,8 +7,8 @@ namespace DB { PushingToViewsBlockOutputStream::PushingToViewsBlockOutputStream( - String database, String table, StoragePtr storage, - const Context & context_, const ASTPtr & query_ptr_, bool no_destination) + const String & database, const String & table, const StoragePtr & storage, + const Context & context_, const ASTPtr & query_ptr_, bool no_destination) : context(context_), query_ptr(query_ptr_) { /** TODO This is a very important line. At any insertion into the table one of streams should own lock. diff --git a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h index 0fe47f677fd..7703b81dc4e 100644 --- a/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h +++ b/dbms/src/DataStreams/PushingToViewsBlockOutputStream.h @@ -4,7 +4,6 @@ #include #include #include -#include #include @@ -19,9 +18,11 @@ class ReplicatedMergeTreeBlockOutputStream; class PushingToViewsBlockOutputStream : public IBlockOutputStream { public: - PushingToViewsBlockOutputStream(String database, String table, StoragePtr storage, + PushingToViewsBlockOutputStream( + const String & database, const String & table, const StoragePtr & storage, const Context & context_, const ASTPtr & query_ptr_, bool no_destination = false); + Block getHeader() const override { return storage->getSampleBlock(); } void write(const Block & block) override; void flush() override diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp index d6284670127..7464e94e4be 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.cpp +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.cpp @@ -19,25 +19,18 @@ namespace ErrorCodes RemoteBlockOutputStream::RemoteBlockOutputStream(Connection & connection_, const String & query_, const Settings * settings_) : connection(connection_), query(query_), settings(settings_) { -} - - -void RemoteBlockOutputStream::writePrefix() -{ - /** Send query and receive "sample block", that describe table structure. - * Sample block is needed to know, what structure is required for blocks to be passed to 'write' method. + /** Send query and receive "header", that describe table structure. + * Header is needed to know, what structure is required for blocks to be passed to 'write' method. */ - - query_sent = true; connection.sendQuery(query, "", QueryProcessingStage::Complete, settings, nullptr); Connection::Packet packet = connection.receivePacket(); if (Protocol::Server::Data == packet.type) { - sample_block = packet.block; + header = packet.block; - if (!sample_block) + if (!header) throw Exception("Logical error: empty block received as table structure", ErrorCodes::LOGICAL_ERROR); } else if (Protocol::Server::Exception == packet.type) @@ -47,32 +40,20 @@ void RemoteBlockOutputStream::writePrefix() } else throw NetException("Unexpected packet from server (expected Data or Exception, got " - + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); + + String(Protocol::Server::toString(packet.type)) + ")", ErrorCodes::UNEXPECTED_PACKET_FROM_SERVER); } void RemoteBlockOutputStream::write(const Block & block) { - if (!sample_block) - throw Exception("You must call IBlockOutputStream::writePrefix before IBlockOutputStream::write", ErrorCodes::LOGICAL_ERROR); - - if (!blocksHaveEqualStructure(block, sample_block)) - { - std::stringstream message; - message << "Block structure is different from table structure.\n" - << "\nTable structure:\n(" << sample_block.dumpStructure() << ")\nBlock structure:\n(" << block.dumpStructure() << ")\n"; - - LOG_ERROR(&Logger::get("RemoteBlockOutputStream"), message.str()); - throw DB::Exception(message.str()); - } - + assertBlocksHaveEqualStructure(block, header, "RemoteBlockOutputStream"); connection.sendData(block); } void RemoteBlockOutputStream::writePrepared(ReadBuffer & input, size_t size) { - /// We cannot use 'sample_block'. Input must contain block with proper structure. + /// We cannot use 'header'. Input must contain block with proper structure. connection.sendPreparedData(input, size); } @@ -100,9 +81,19 @@ void RemoteBlockOutputStream::writeSuffix() RemoteBlockOutputStream::~RemoteBlockOutputStream() { - /// If interrupted in the middle of the loop of communication with the server, then interrupt the connection - if (query_sent && !finished) - connection.disconnect(); + /// If interrupted in the middle of the loop of communication with the server, then interrupt the connection, + /// to not leave the connection in unsynchronized state. + if (!finished) + { + try + { + connection.disconnect(); + } + catch (...) + { + tryLogCurrentException(__PRETTY_FUNCTION__); + } + } } } diff --git a/dbms/src/DataStreams/RemoteBlockOutputStream.h b/dbms/src/DataStreams/RemoteBlockOutputStream.h index 7c917393e4e..41740c39837 100644 --- a/dbms/src/DataStreams/RemoteBlockOutputStream.h +++ b/dbms/src/DataStreams/RemoteBlockOutputStream.h @@ -20,14 +20,8 @@ class RemoteBlockOutputStream : public IBlockOutputStream public: RemoteBlockOutputStream(Connection & connection_, const String & query_, const Settings * settings_ = nullptr); + Block getHeader() const override { return header; } - /// You can call this method after 'writePrefix', to get table required structure. (You must send data with that structure). - Block getSampleBlock() const - { - return sample_block; - } - - void writePrefix() override; void write(const Block & block) override; void writeSuffix() override; @@ -40,8 +34,7 @@ private: Connection & connection; String query; const Settings * settings; - Block sample_block; - bool query_sent = false; + Block header; bool finished = false; }; diff --git a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp index 687d5ae19f4..96ac8a98355 100644 --- a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.cpp @@ -17,7 +17,7 @@ void ReplacingSortedBlockInputStream::insertRow(MutableColumns & merged_columns, if (out_row_sources_buf) { /// true flag value means "skip row" - current_row_sources.back().setSkipFlag(false); + current_row_sources[max_pos].setSkipFlag(false); out_row_sources_buf->write(reinterpret_cast(current_row_sources.data()), current_row_sources.size() * sizeof(RowSourcePart)); @@ -96,6 +96,7 @@ void ReplacingSortedBlockInputStream::merge(MutableColumns & merged_columns, std } /// Initially, skip all rows. Unskip last on insert. + size_t current_pos = current_row_sources.size(); if (out_row_sources_buf) current_row_sources.emplace_back(current.impl->order, true); @@ -103,6 +104,7 @@ void ReplacingSortedBlockInputStream::merge(MutableColumns & merged_columns, std if (version >= max_version) { max_version = version; + max_pos = current_pos; setRowRef(selected_row, current); } diff --git a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h index 0ab6b185833..dabc1c7e3af 100644 --- a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h @@ -43,6 +43,7 @@ private: RowRef selected_row; /// Last row with maximum version for current primary key. UInt64 max_version = 0; /// Max version for current primary key. + size_t max_pos = 0; PODArray current_row_sources; /// Sources of rows with the current primary key diff --git a/dbms/src/DataStreams/SquashingBlockOutputStream.h b/dbms/src/DataStreams/SquashingBlockOutputStream.h index df3cf262fa6..9e660de59f1 100644 --- a/dbms/src/DataStreams/SquashingBlockOutputStream.h +++ b/dbms/src/DataStreams/SquashingBlockOutputStream.h @@ -14,6 +14,7 @@ class SquashingBlockOutputStream : public IBlockOutputStream public: SquashingBlockOutputStream(BlockOutputStreamPtr & dst, size_t min_block_size_rows, size_t min_block_size_bytes); + Block getHeader() const override { return output->getHeader(); } void write(const Block & block) override; void flush() override; diff --git a/dbms/src/DataStreams/UnionBlockInputStream.h b/dbms/src/DataStreams/UnionBlockInputStream.h index 176cb472a61..f45ff396cb6 100644 --- a/dbms/src/DataStreams/UnionBlockInputStream.h +++ b/dbms/src/DataStreams/UnionBlockInputStream.h @@ -82,6 +82,14 @@ public: children = inputs; if (additional_input_at_end) children.push_back(additional_input_at_end); + + size_t num_children = children.size(); + if (num_children > 1) + { + Block header = children.at(0)->getHeader(); + for (size_t i = 1; i < num_children; ++i) + assertBlocksHaveEqualStructure(children[i]->getHeader(), header, "UNION"); + } } String getName() const override { return "Union"; } diff --git a/dbms/src/DataStreams/tests/block_row_transforms.cpp b/dbms/src/DataStreams/tests/block_row_transforms.cpp index 6e216c59158..5f826542271 100644 --- a/dbms/src/DataStreams/tests/block_row_transforms.cpp +++ b/dbms/src/DataStreams/tests/block_row_transforms.cpp @@ -44,7 +44,7 @@ try RowInputStreamPtr row_input = std::make_shared(in_buf, sample); BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, 0); RowOutputStreamPtr row_output = std::make_shared(out_buf, sample); - BlockOutputStreamFromRowOutputStream block_output(row_output); + BlockOutputStreamFromRowOutputStream block_output(row_output, sample); copyData(block_input, block_output); } diff --git a/dbms/src/DataStreams/tests/expression_stream.cpp b/dbms/src/DataStreams/tests/expression_stream.cpp index f52a8f9c105..53d79634c80 100644 --- a/dbms/src/DataStreams/tests/expression_stream.cpp +++ b/dbms/src/DataStreams/tests/expression_stream.cpp @@ -56,7 +56,7 @@ try WriteBufferFromOStream out1(std::cout); RowOutputStreamPtr out2 = std::make_shared(out1, expression->getSampleBlock()); - BlockOutputStreamFromRowOutputStream out(out2); + BlockOutputStreamFromRowOutputStream out(out2, expression->getSampleBlock()); { Stopwatch stopwatch; diff --git a/dbms/src/DataStreams/tests/filter_stream.cpp b/dbms/src/DataStreams/tests/filter_stream.cpp index 1f5549204b0..da3b2f7d5ad 100644 --- a/dbms/src/DataStreams/tests/filter_stream.cpp +++ b/dbms/src/DataStreams/tests/filter_stream.cpp @@ -61,7 +61,7 @@ try WriteBufferFromOStream ob(std::cout); RowOutputStreamPtr out_ = std::make_shared(ob, expression->getSampleBlock()); - BlockOutputStreamFromRowOutputStream out(out_); + BlockOutputStreamFromRowOutputStream out(out_, expression->getSampleBlock()); { diff --git a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp index 4561ffde4be..bbfd65b1f93 100644 --- a/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp +++ b/dbms/src/DataStreams/tests/filter_stream_hitlog.cpp @@ -134,7 +134,7 @@ int main(int, char **) WriteBufferFromOStream ob(std::cout); RowOutputStreamPtr out_ = std::make_shared(ob, expression->getSampleBlock()); - BlockOutputStreamFromRowOutputStream out(out_); + BlockOutputStreamFromRowOutputStream out(out_, in->getHeader()); copyData(*in, out); } diff --git a/dbms/src/DataStreams/tests/native_streams.cpp b/dbms/src/DataStreams/tests/native_streams.cpp index d06d8aa3714..bd0a998f88e 100644 --- a/dbms/src/DataStreams/tests/native_streams.cpp +++ b/dbms/src/DataStreams/tests/native_streams.cpp @@ -106,7 +106,7 @@ try BlockInputStreamPtr in = table->read(column_names, {}, Context::createGlobal(), stage, 8192, 1)[0]; WriteBufferFromFileDescriptor out1(STDOUT_FILENO); CompressedWriteBuffer out2(out1); - NativeBlockOutputStream out3(out2, ClickHouseRevision::get()); + NativeBlockOutputStream out3(out2, ClickHouseRevision::get(), in->getHeader()); copyData(*in, out3); } diff --git a/dbms/src/DataStreams/tests/sorting_stream.cpp b/dbms/src/DataStreams/tests/sorting_stream.cpp index d3a32f4adf1..cd6fe515a53 100644 --- a/dbms/src/DataStreams/tests/sorting_stream.cpp +++ b/dbms/src/DataStreams/tests/sorting_stream.cpp @@ -152,7 +152,7 @@ try WriteBufferFromOStream ob(std::cout); RowOutputStreamPtr out_ = std::make_shared(ob, sample); - BlockOutputStreamFromRowOutputStream out(out_); + BlockOutputStreamFromRowOutputStream out(out_, sample); copyData(*in, out); diff --git a/dbms/src/DataStreams/tests/tab_separated_streams.cpp b/dbms/src/DataStreams/tests/tab_separated_streams.cpp index 2cc6abf9835..c765135484b 100644 --- a/dbms/src/DataStreams/tests/tab_separated_streams.cpp +++ b/dbms/src/DataStreams/tests/tab_separated_streams.cpp @@ -38,7 +38,7 @@ try RowOutputStreamPtr row_output = std::make_shared(out_buf, sample); BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, 0); - BlockOutputStreamFromRowOutputStream block_output(row_output); + BlockOutputStreamFromRowOutputStream block_output(row_output, sample); copyData(block_input, block_output); return 0; diff --git a/dbms/src/Dictionaries/DictionaryBlockInputStream.h b/dbms/src/Dictionaries/DictionaryBlockInputStream.h index 9c961b0fb9a..1b7b536a15c 100644 --- a/dbms/src/Dictionaries/DictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/DictionaryBlockInputStream.h @@ -43,7 +43,7 @@ public: String getName() const override { - return "DictionaryBlockInputStream"; + return "Dictionary"; } protected: diff --git a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h index 43c2fcb53c7..857c9b58cfb 100644 --- a/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/dbms/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -30,7 +30,7 @@ public: String getName() const override { - return "RangeDictionaryBlockInputStream"; + return "RangeDictionary"; } protected: diff --git a/dbms/src/Functions/FunctionsCoding.h b/dbms/src/Functions/FunctionsCoding.h index b91d60acb97..be9ffb68d66 100644 --- a/dbms/src/Functions/FunctionsCoding.h +++ b/dbms/src/Functions/FunctionsCoding.h @@ -1111,6 +1111,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { auto col_res = ColumnVector::create(); diff --git a/dbms/src/Functions/FunctionsDateTime.h b/dbms/src/Functions/FunctionsDateTime.h index c9f3335ebfa..6644c5aa94f 100644 --- a/dbms/src/Functions/FunctionsDateTime.h +++ b/dbms/src/Functions/FunctionsDateTime.h @@ -1208,6 +1208,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeUInt32().createColumnConst( @@ -1235,6 +1237,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeUInt16().createColumnConst( @@ -1262,6 +1266,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeUInt16().createColumnConst( diff --git a/dbms/src/Functions/FunctionsEmbeddedDictionaries.h b/dbms/src/Functions/FunctionsEmbeddedDictionaries.h index cb7ded5d174..7d39ad8d543 100644 --- a/dbms/src/Functions/FunctionsEmbeddedDictionaries.h +++ b/dbms/src/Functions/FunctionsEmbeddedDictionaries.h @@ -218,6 +218,8 @@ public: bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { /// The dictionary key that defines the "point of view". @@ -312,6 +314,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { /// The dictionary key that defines the "point of view". @@ -446,6 +450,8 @@ public: bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { /// The dictionary key that defines the "point of view". @@ -720,6 +726,8 @@ public: bool useDefaultImplementationForConstants() const override { return true; } ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override { RegionsNames::Language language = RegionsNames::Language::RU; diff --git a/dbms/src/Functions/FunctionsExternalDictionaries.h b/dbms/src/Functions/FunctionsExternalDictionaries.h index af5e2b751f9..cb77d84b751 100644 --- a/dbms/src/Functions/FunctionsExternalDictionaries.h +++ b/dbms/src/Functions/FunctionsExternalDictionaries.h @@ -94,6 +94,8 @@ private: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto dict_name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); @@ -274,6 +276,8 @@ private: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto dict_name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); @@ -535,6 +539,8 @@ private: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto dict_name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); @@ -821,6 +827,8 @@ private: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto dict_name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); @@ -1134,6 +1142,8 @@ private: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto dict_name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); @@ -1379,6 +1389,8 @@ private: return std::make_shared(std::make_shared()); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto dict_name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); @@ -1549,6 +1561,8 @@ private: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, const size_t result) override { const auto dict_name_col = checkAndGetColumnConst(block.getByPosition(arguments[0]).column.get()); diff --git a/dbms/src/Functions/FunctionsExternalModels.h b/dbms/src/Functions/FunctionsExternalModels.h index 74149920e6f..74822db9962 100644 --- a/dbms/src/Functions/FunctionsExternalModels.h +++ b/dbms/src/Functions/FunctionsExternalModels.h @@ -23,6 +23,8 @@ public: bool isVariadic() const override { return true; } + bool isDeterministic() override { return false; } + size_t getNumberOfArguments() const override { return 0; } DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override; diff --git a/dbms/src/Functions/FunctionsMiscellaneous.cpp b/dbms/src/Functions/FunctionsMiscellaneous.cpp index 58ca5ce1d2d..33abc2dfd41 100644 --- a/dbms/src/Functions/FunctionsMiscellaneous.cpp +++ b/dbms/src/Functions/FunctionsMiscellaneous.cpp @@ -104,6 +104,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, const size_t result) override { block.getByPosition(result).column = DataTypeString().createColumnConst(block.rows(), db_name); @@ -126,6 +128,8 @@ public: return name; } + bool isDeterministic() override { return false; } + bool isDeterministicInScopeOfQuery() override { return false; @@ -391,6 +395,8 @@ public: return name; } + bool isDeterministic() override { return false; } + bool isDeterministicInScopeOfQuery() override { return false; @@ -434,6 +440,8 @@ public: return 0; } + bool isDeterministic() override { return false; } + bool isDeterministicInScopeOfQuery() override { return false; @@ -482,6 +490,8 @@ public: return 0; } + bool isDeterministic() override { return false; } + bool isDeterministicInScopeOfQuery() override { return false; @@ -524,6 +534,8 @@ public: return 0; } + bool isDeterministic() override { return false; } + bool isDeterministicInScopeOfQuery() override { return false; @@ -889,6 +901,8 @@ public: } /** It could return many different values for single argument. */ + bool isDeterministic() override { return false; } + bool isDeterministicInScopeOfQuery() override { return false; @@ -1288,6 +1302,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeUInt32().createColumnConst(block.rows(), static_cast(uptime)); @@ -1323,6 +1339,8 @@ public: return std::make_shared(); } + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & /*arguments*/, size_t result) override { block.getByPosition(result).column = DataTypeString().createColumnConst(block.rows(), DateLUT::instance().getTimeZone()); @@ -1355,6 +1373,8 @@ public: return 1; } + bool isDeterministic() override { return false; } + bool isDeterministicInScopeOfQuery() override { return false; @@ -1632,6 +1652,8 @@ public: DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override; + bool isDeterministic() override { return false; } + void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result) override; private: diff --git a/dbms/src/Functions/IFunction.h b/dbms/src/Functions/IFunction.h index bb9aeffc71d..b7791268c79 100644 --- a/dbms/src/Functions/IFunction.h +++ b/dbms/src/Functions/IFunction.h @@ -126,6 +126,9 @@ public: * (even for distributed query), but not deterministic it general. * Example: now(). Another example: functions that work with periodically updated dictionaries. */ + + virtual bool isDeterministic() { return true; } + virtual bool isDeterministicInScopeOfQuery() { return true; } /** Lets you know if the function is monotonic in a range of values. @@ -320,6 +323,8 @@ public: bool isInjective(const Block & sample_block) override { return function->isInjective(sample_block); } + bool isDeterministic() override { return function->isDeterministic(); } + bool isDeterministicInScopeOfQuery() override { return function->isDeterministicInScopeOfQuery(); } bool hasInformationAboutMonotonicity() const override { return function->hasInformationAboutMonotonicity(); } diff --git a/dbms/src/IO/readFloatText.h b/dbms/src/IO/readFloatText.h index 82a6ea29eec..503203cba21 100644 --- a/dbms/src/IO/readFloatText.h +++ b/dbms/src/IO/readFloatText.h @@ -5,8 +5,6 @@ #include #include -#include - /** Methods for reading floating point numbers from text with decimal representation. * There are "precise", "fast" and "simple" implementations. diff --git a/dbms/src/Interpreters/Aggregator.cpp b/dbms/src/Interpreters/Aggregator.cpp index df1fb72584c..c0a998a3081 100644 --- a/dbms/src/Interpreters/Aggregator.cpp +++ b/dbms/src/Interpreters/Aggregator.cpp @@ -14,6 +14,7 @@ #include #include #include +#include #include #include @@ -130,7 +131,7 @@ Block Aggregator::getHeader(bool final) const } } - return res; + return materializeBlock(res); } @@ -840,7 +841,7 @@ void Aggregator::writeToTemporaryFile(AggregatedDataVariants & data_variants) const std::string & path = file->path(); WriteBufferFromFile file_buf(path); CompressedWriteBuffer compressed_buf(file_buf); - NativeBlockOutputStream block_out(compressed_buf, ClickHouseRevision::get()); + NativeBlockOutputStream block_out(compressed_buf, ClickHouseRevision::get(), getHeader(false)); LOG_DEBUG(log, "Writing part of aggregation data into temporary file " << path << "."); ProfileEvents::increment(ProfileEvents::ExternalAggregationWritePart); diff --git a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h index 53066a946e9..5325e5d463c 100644 --- a/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h +++ b/dbms/src/Interpreters/ClusterProxy/SelectStreamFactory.h @@ -29,7 +29,7 @@ private: const Block header; QueryProcessingStage::Enum processed_stage; QualifiedTableName main_table; - const Tables & external_tables; + Tables external_tables; }; } diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 1f8a5bb8f0d..d8bcb2936fd 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -142,39 +142,11 @@ ExpressionAction ExpressionAction::ordinaryJoin(std::shared_ptr join } -ExpressionActions::Actions ExpressionAction::getPrerequisites(Block & sample_block) -{ - ExpressionActions::Actions res; - - if (type == APPLY_FUNCTION) - { - if (sample_block.has(result_name)) - throw Exception("Column '" + result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); - - ColumnsWithTypeAndName arguments(argument_names.size()); - for (size_t i = 0; i < argument_names.size(); ++i) - { - if (!sample_block.has(argument_names[i])) - throw Exception("Unknown identifier: '" + argument_names[i] + "'", ErrorCodes::UNKNOWN_IDENTIFIER); - arguments[i] = sample_block.getByName(argument_names[i]); - } - - function = function_builder->build(arguments); - result_type = function->getReturnType(); - } - - return res; -} - void ExpressionAction::prepare(Block & sample_block) { // std::cerr << "preparing: " << toString() << std::endl; /** Constant expressions should be evaluated, and put the result in sample_block. - * For non-constant columns, put the nullptr as the column in sample_block. - * - * The fact that only for constant expressions column != nullptr, - * can be used later when optimizing the query. */ switch (type) @@ -567,40 +539,43 @@ void ExpressionActions::addInput(const NameAndTypePair & column) void ExpressionActions::add(const ExpressionAction & action, Names & out_new_columns) { - NameSet temp_names; - addImpl(action, temp_names, out_new_columns); + addImpl(action, out_new_columns); } void ExpressionActions::add(const ExpressionAction & action) { - NameSet temp_names; Names new_names; - addImpl(action, temp_names, new_names); + addImpl(action, new_names); } -void ExpressionActions::addImpl(ExpressionAction action, NameSet & current_names, Names & new_names) +void ExpressionActions::addImpl(ExpressionAction action, Names & new_names) { if (sample_block.has(action.result_name)) return; - if (current_names.count(action.result_name)) - throw Exception("Cyclic function prerequisites: " + action.result_name, ErrorCodes::LOGICAL_ERROR); - - current_names.insert(action.result_name); - if (action.result_name != "") new_names.push_back(action.result_name); new_names.insert(new_names.end(), action.array_joined_columns.begin(), action.array_joined_columns.end()); - Actions prerequisites = action.getPrerequisites(sample_block); + if (action.type == ExpressionAction::APPLY_FUNCTION) + { + if (sample_block.has(action.result_name)) + throw Exception("Column '" + action.result_name + "' already exists", ErrorCodes::DUPLICATE_COLUMN); - for (size_t i = 0; i < prerequisites.size(); ++i) - addImpl(prerequisites[i], current_names, new_names); + ColumnsWithTypeAndName arguments(action.argument_names.size()); + for (size_t i = 0; i < action.argument_names.size(); ++i) + { + if (!sample_block.has(action.argument_names[i])) + throw Exception("Unknown identifier: '" + action.argument_names[i] + "'", ErrorCodes::UNKNOWN_IDENTIFIER); + arguments[i] = sample_block.getByName(action.argument_names[i]); + } + + action.function = action.function_builder->build(arguments); + action.result_type = action.function->getReturnType(); + } action.prepare(sample_block); actions.push_back(action); - - current_names.erase(action.result_name); } void ExpressionActions::prependProjectInput() @@ -1017,19 +992,11 @@ void ExpressionActions::optimizeArrayJoin() } -BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRightJoin(size_t max_block_size) const +BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRightJoin(const Block & source_header, size_t max_block_size) const { for (const auto & action : actions) - { if (action.join && (action.join->getKind() == ASTTableJoin::Kind::Full || action.join->getKind() == ASTTableJoin::Kind::Right)) - { - Block left_sample_block; - for (const auto & input_elem : input_columns) - left_sample_block.insert(ColumnWithTypeAndName{ input_elem.type, input_elem.name }); - - return action.join->createStreamWithNonJoinedRows(left_sample_block, max_block_size); - } - } + return action.join->createStreamWithNonJoinedRows(source_header, max_block_size); return {}; } diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 59434c741ac..f29e53a1d7e 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -99,7 +99,6 @@ public: static ExpressionAction ordinaryJoin(std::shared_ptr join_, const NamesAndTypesList & columns_added_by_join_); /// Which columns necessary to perform this action. - /// If this `Action` is not already added to `ExpressionActions`, the returned list may be incomplete, because `prerequisites` are not taken into account. Names getNeededColumns() const; std::string toString() const; @@ -107,7 +106,6 @@ public: private: friend class ExpressionActions; - std::vector getPrerequisites(Block & sample_block); void prepare(Block & sample_block); void execute(Block & block) const; void executeOnTotals(Block & block) const; @@ -147,8 +145,7 @@ public: void add(const ExpressionAction & action); - /// Adds new column names to out_new_columns - /// (formed as a result of the added action and its prerequisites). + /// Adds new column names to out_new_columns (formed as a result of the added action). void add(const ExpressionAction & action, Names & out_new_columns); /// Adds to the beginning the removal of all extra columns. @@ -198,7 +195,7 @@ public: static std::string getSmallestColumn(const NamesAndTypesList & columns); - BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin(size_t max_block_size) const; + BlockInputStreamPtr createStreamWithNonJoinedDataIfFullOrRightJoin(const Block & source_header, size_t max_block_size) const; private: NamesAndTypesList input_columns; @@ -208,9 +205,7 @@ private: void checkLimits(Block & block) const; - /// Adds all `prerequisites` first, then the action itself. - /// current_names - columns whose `prerequisites` are currently being processed. - void addImpl(ExpressionAction action, NameSet & current_names, Names & new_names); + void addImpl(ExpressionAction action, Names & new_names); /// Try to improve something without changing the lists of input and output columns. void optimize(); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 259cd8f3503..cfcd7989df5 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -802,84 +802,24 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t StoragePtr external_storage = StorageMemory::create(external_table_name, columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}); external_storage->startup(); - /** There are two ways to perform distributed GLOBAL subqueries. - * - * "push" method: - * Subquery data is sent to all remote servers, where they are then used. - * For this method, the data is sent in the form of "external tables" and will be available on each remote server by the name of the type _data1. - * Replace in the query a subquery for this name. - * - * "pull" method: - * Remote servers download the subquery data from the request initiating server. - * For this method, replace the subquery with another subquery of the form (SELECT * FROM remote ('host: port', _query_QUERY_ID, _data1)) - * This subquery, in fact, says - "you need to download data from there." - * - * The "pull" method takes precedence, because in it a remote server can decide that it does not need data and does not download it in such cases. - */ + /** We replace the subquery with the name of the temporary table. + * It is in this form, the request will go to the remote server. + * This temporary table will go to the remote server, and on its side, + * instead of doing a subquery, you just need to read it. + */ - if (settings.global_subqueries_method == GlobalSubqueriesMethod::PUSH) + auto database_and_table_name = std::make_shared(StringRange(), external_table_name, ASTIdentifier::Table); + + if (auto ast_table_expr = typeid_cast(subquery_or_table_name_or_table_expression.get())) { - /** We replace the subquery with the name of the temporary table. - * It is in this form, the request will go to the remote server. - * This temporary table will go to the remote server, and on its side, - * instead of doing a subquery, you just need to read it. - */ + ast_table_expr->subquery.reset(); + ast_table_expr->database_and_table_name = database_and_table_name; - auto database_and_table_name = std::make_shared(StringRange(), external_table_name, ASTIdentifier::Table); - - if (auto ast_table_expr = typeid_cast(subquery_or_table_name_or_table_expression.get())) - { - ast_table_expr->subquery.reset(); - ast_table_expr->database_and_table_name = database_and_table_name; - - ast_table_expr->children.clear(); - ast_table_expr->children.emplace_back(database_and_table_name); - } - else - subquery_or_table_name_or_table_expression = database_and_table_name; - } - else if (settings.global_subqueries_method == GlobalSubqueriesMethod::PULL) - { - throw Exception("Support for 'pull' method of execution of global subqueries is disabled.", ErrorCodes::SUPPORT_IS_DISABLED); - - /// TODO -/* String host_port = getFQDNOrHostName() + ":" + toString(context.getTCPPort()); - String database = "_query_" + context.getCurrentQueryId(); - - auto subquery = std::make_shared(); - subquery_or_table_name = subquery; - - auto select = std::make_shared(); - subquery->children.push_back(select); - - auto exp_list = std::make_shared(); - select->select_expression_list = exp_list; - select->children.push_back(select->select_expression_list); - - Names column_names = external_storage->getColumnNamesList(); - for (const auto & name : column_names) - exp_list->children.push_back(std::make_shared(StringRange(), name)); - - auto table_func = std::make_shared(); - select->table = table_func; - select->children.push_back(select->table); - - table_func->name = "remote"; - auto args = std::make_shared(); - table_func->arguments = args; - table_func->children.push_back(table_func->arguments); - - auto address_lit = std::make_shared(StringRange(), host_port); - args->children.push_back(address_lit); - - auto database_lit = std::make_shared(StringRange(), database); - args->children.push_back(database_lit); - - auto table_lit = std::make_shared(StringRange(), external_table_name); - args->children.push_back(table_lit);*/ + ast_table_expr->children.clear(); + ast_table_expr->children.emplace_back(database_and_table_name); } else - throw Exception("Unknown global subqueries execution method", ErrorCodes::UNKNOWN_GLOBAL_SUBQUERIES_METHOD); + subquery_or_table_name_or_table_expression = database_and_table_name; external_tables[external_table_name] = external_storage; subqueries_for_sets[external_table_name].source = interpreter->execute().in; @@ -2637,25 +2577,6 @@ void ExpressionAnalyzer::appendProjectResult(ExpressionActionsChain & chain) con } -Block ExpressionAnalyzer::getSelectSampleBlock() -{ - assertSelect(); - - ExpressionActionsPtr temp_actions = std::make_shared(aggregated_columns, settings); - NamesWithAliases result_columns; - - ASTs asts = select_query->select_expression_list->children; - for (size_t i = 0; i < asts.size(); ++i) - { - result_columns.emplace_back(asts[i]->getColumnName(), asts[i]->getAliasOrColumnName()); - getRootActions(asts[i], true, false, temp_actions); - } - - temp_actions->add(ExpressionAction::project(result_columns)); - - return temp_actions->getSampleBlock(); -} - void ExpressionAnalyzer::getActionsBeforeAggregation(const ASTPtr & ast, ExpressionActionsPtr & actions, bool no_subqueries) { ASTFunction * node = typeid_cast(ast.get()); diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index ccd60b296cb..41e56c691a5 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -135,9 +135,6 @@ public: */ const Tables & getExternalTables() const { return external_tables; } - /// If ast is a SELECT query, it gets the aliases and column types from the SELECT section. - Block getSelectSampleBlock(); - /// Create Set-s that we can from IN section to use the index on them. void makeSetsForIndex(); diff --git a/dbms/src/Interpreters/InterpreterCheckQuery.cpp b/dbms/src/Interpreters/InterpreterCheckQuery.cpp index d96344190b5..068b7897105 100644 --- a/dbms/src/Interpreters/InterpreterCheckQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCheckQuery.cpp @@ -92,49 +92,6 @@ InterpreterCheckQuery::InterpreterCheckQuery(const ASTPtr & query_ptr_, const Co { } -Block InterpreterCheckQuery::getSampleBlock() const -{ - Block block; - ColumnWithTypeAndName col; - - col.name = "status"; - col.type = std::make_shared(); - col.column = col.type->createColumn(); - block.insert(col); - - col.name = "host_name"; - col.type = std::make_shared(); - col.column = col.type->createColumn(); - block.insert(col); - - col.name = "host_address"; - col.type = std::make_shared(); - col.column = col.type->createColumn(); - block.insert(col); - - col.name = "port"; - col.type = std::make_shared(); - col.column = col.type->createColumn(); - block.insert(col); - - col.name = "user"; - col.type = std::make_shared(); - col.column = col.type->createColumn(); - block.insert(col); - - col.name = "structure_class"; - col.type = std::make_shared(); - col.column = col.type->createColumn(); - block.insert(col); - - col.name = "structure"; - col.type = std::make_shared(); - col.column = col.type->createColumn(); - block.insert(col); - - return block; -} - BlockIO InterpreterCheckQuery::execute() { diff --git a/dbms/src/Interpreters/InterpreterCheckQuery.h b/dbms/src/Interpreters/InterpreterCheckQuery.h index dc2e9cc8bb1..a0a0d677235 100644 --- a/dbms/src/Interpreters/InterpreterCheckQuery.h +++ b/dbms/src/Interpreters/InterpreterCheckQuery.h @@ -16,9 +16,6 @@ public: BlockIO execute() override; -private: - Block getSampleBlock() const; - private: ASTPtr query_ptr; diff --git a/dbms/src/Interpreters/InterpreterCreateQuery.cpp b/dbms/src/Interpreters/InterpreterCreateQuery.cpp index aa4bf23fc20..58060b6d2c6 100644 --- a/dbms/src/Interpreters/InterpreterCreateQuery.cpp +++ b/dbms/src/Interpreters/InterpreterCreateQuery.cpp @@ -10,17 +10,12 @@ #include #include -#include -#include -#include -#include -#include - #include #include #include #include #include +#include #include #include #include @@ -33,6 +28,7 @@ #include #include #include +#include #include #include @@ -43,8 +39,10 @@ #include + namespace DB { + namespace ErrorCodes { extern const int DIRECTORY_DOESNT_EXIST; @@ -474,13 +472,9 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.select && (create.is_view || create.is_materialized_view)) create.select->setDatabaseIfNeeded(current_database); - std::unique_ptr interpreter_select; Block as_select_sample; if (create.select && (!create.attach || !create.columns)) - { - interpreter_select = std::make_unique(create.select->clone(), context); - as_select_sample = interpreter_select->getSampleBlock(); - } + as_select_sample = InterpreterSelectQuery::getSampleBlock(create.select->clone(), context); String as_database_name = create.as_database.empty() ? current_database : create.as_database; String as_table_name = create.as_table; @@ -554,28 +548,15 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create) if (create.select && !create.attach && !create.is_view && (!create.is_materialized_view || create.is_populate)) { - auto table_lock = res->lockStructure(true, __PRETTY_FUNCTION__); + auto insert = std::make_shared(); - /// Also see InterpreterInsertQuery. - BlockOutputStreamPtr out; + if (!create.is_temporary) + insert->database = database_name; - out = std::make_shared( - create.database, create.table, res, create.is_temporary ? context.getSessionContext() : context, query_ptr); + insert->table = table_name; + insert->select = create.select->clone(); - out = std::make_shared(out); - - /// @note shouldn't these two contexts be session contexts in case of temporary table? - bool strict_insert_defaults = static_cast(context.getSettingsRef().strict_insert_defaults); - out = std::make_shared( - out, columns.columns, columns.column_defaults, context, strict_insert_defaults); - - if (!context.getSettingsRef().insert_allow_materialized_columns) - out = std::make_shared(out, columns.materialized_columns); - - BlockIO io; - io.in = std::make_shared(interpreter_select->execute().in, out); - - return io; + return InterpreterInsertQuery(insert, context.getSessionContext(), context.getSettingsRef().insert_allow_materialized_columns).execute(); } return {}; diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.cpp b/dbms/src/Interpreters/InterpreterInsertQuery.cpp index 63eea9542b8..60cd5ea70cb 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.cpp +++ b/dbms/src/Interpreters/InterpreterInsertQuery.cpp @@ -8,7 +8,6 @@ #include #include #include -#include #include #include #include @@ -23,17 +22,20 @@ #include #include + namespace ProfileEvents { -extern const Event InsertQuery; + extern const Event InsertQuery; } namespace DB { + namespace ErrorCodes { extern const int NO_SUCH_COLUMN_IN_TABLE; extern const int READONLY; + extern const int ILLEGAL_COLUMN; } @@ -45,10 +47,8 @@ InterpreterInsertQuery::InterpreterInsertQuery( } -StoragePtr InterpreterInsertQuery::loadTable() +StoragePtr InterpreterInsertQuery::getTable(const ASTInsertQuery & query) { - ASTInsertQuery & query = typeid_cast(*query_ptr); - if (query.table_function) { auto table_function = typeid_cast(query.table_function.get()); @@ -56,27 +56,19 @@ StoragePtr InterpreterInsertQuery::loadTable() return factory.get(table_function->name, context)->execute(query.table_function, context); } - /// In what table to write. + /// Into what table to write. return context.getTable(query.database, query.table); } -StoragePtr InterpreterInsertQuery::getTable() +Block InterpreterInsertQuery::getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table) { - if (!cached_table) - cached_table = loadTable(); - - return cached_table; -} - -Block InterpreterInsertQuery::getSampleBlock() -{ - ASTInsertQuery & query = typeid_cast(*query_ptr); + Block table_sample_non_materialized = table->getSampleBlockNonMaterialized(); /// If the query does not include information about columns if (!query.columns) - return getTable()->getSampleBlockNonMaterialized(); + return table_sample_non_materialized; - Block table_sample = getTable()->getSampleBlock(); + Block table_sample = table->getSampleBlock(); /// Form the block based on the column names from the query Block res; @@ -88,13 +80,11 @@ Block InterpreterInsertQuery::getSampleBlock() if (!table_sample.has(current_name)) throw Exception("No such column " + current_name + " in table " + query.table, ErrorCodes::NO_SUCH_COLUMN_IN_TABLE); - ColumnWithTypeAndName col; - col.name = current_name; - col.type = table_sample.getByName(current_name).type; - col.column = col.type->createColumn(); - res.insert(std::move(col)); - } + if (!allow_materialized && !table_sample_non_materialized.has(current_name)) + throw Exception("Cannot insert column " + current_name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); + res.insert(ColumnWithTypeAndName(table_sample.getByName(current_name).type, current_name)); + } return res; } @@ -103,7 +93,7 @@ BlockIO InterpreterInsertQuery::execute() { ASTInsertQuery & query = typeid_cast(*query_ptr); checkAccess(query); - StoragePtr table = getTable(); + StoragePtr table = getTable(query); auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__); @@ -114,13 +104,11 @@ BlockIO InterpreterInsertQuery::execute() out = std::make_shared(query.database, query.table, table, context, query_ptr, query.no_destination); - out = std::make_shared(out); + out = std::make_shared(out, table->getSampleBlock()); out = std::make_shared( - out, required_columns, table->column_defaults, context, static_cast(context.getSettingsRef().strict_insert_defaults)); - - if (!allow_materialized) - out = std::make_shared(out, table->materialized_columns); + out, getSampleBlock(query, table), required_columns, table->column_defaults, context, + static_cast(context.getSettingsRef().strict_insert_defaults)); out = std::make_shared( out, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes); @@ -130,27 +118,34 @@ BlockIO InterpreterInsertQuery::execute() out = std::move(out_wrapper); BlockIO res; - res.out_sample = getSampleBlock(); + res.out = std::move(out); /// What type of query: INSERT or INSERT SELECT? - if (!query.select) - { - res.out = out; - } - else + if (query.select) { InterpreterSelectQuery interpreter_select{query.select, context}; res.in = interpreter_select.execute().in; - res.in = std::make_shared(res.in, res.in->getHeader(), res.out_sample); - res.in = std::make_shared(context, res.in, res.out_sample); - res.in = std::make_shared(res.in, out); + res.in = std::make_shared(res.in, res.in->getHeader(), res.out->getHeader()); + res.in = std::make_shared(context, res.in, res.out->getHeader()); + res.in = std::make_shared(res.in, res.out); + + res.out = nullptr; + + if (!allow_materialized) + { + Block in_header = res.in->getHeader(); + for (const auto & name_type : table->materialized_columns) + if (in_header.has(name_type.name)) + throw Exception("Cannot insert column " + name_type.name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN); + } } return res; } + void InterpreterInsertQuery::checkAccess(const ASTInsertQuery & query) { const Settings & settings = context.getSettingsRef(); @@ -163,4 +158,5 @@ void InterpreterInsertQuery::checkAccess(const ASTInsertQuery & query) throw Exception("Cannot insert into table in readonly mode", ErrorCodes::READONLY); } + } diff --git a/dbms/src/Interpreters/InterpreterInsertQuery.h b/dbms/src/Interpreters/InterpreterInsertQuery.h index 9bdc5cfcaba..2180ebe0550 100644 --- a/dbms/src/Interpreters/InterpreterInsertQuery.h +++ b/dbms/src/Interpreters/InterpreterInsertQuery.h @@ -25,14 +25,8 @@ public: BlockIO execute() override; private: - /// Cache storage to avoid double table function call. - StoragePtr cached_table; - StoragePtr loadTable(); - - StoragePtr getTable(); - - Block getSampleBlock(); - + StoragePtr getTable(const ASTInsertQuery & query); + Block getSampleBlock(const ASTInsertQuery & query, const StoragePtr & table); void checkAccess(const ASTInsertQuery & query); ASTPtr query_ptr; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index be916e0b7b0..da60ef7ada4 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -21,6 +21,7 @@ #include #include #include +#include #include #include @@ -63,13 +64,14 @@ namespace ErrorCodes extern const int ILLEGAL_FINAL; extern const int ILLEGAL_PREWHERE; extern const int TOO_MUCH_COLUMNS; + extern const int LOGICAL_ERROR; } InterpreterSelectQuery::~InterpreterSelectQuery() = default; -void InterpreterSelectQuery::init(const BlockInputStreamPtr & input, const Names & required_column_names) +void InterpreterSelectQuery::init(const Names & required_column_names) { ProfileEvents::increment(ProfileEvents::SelectQuery); @@ -95,22 +97,21 @@ void InterpreterSelectQuery::init(const BlockInputStreamPtr & input, const Names ASTSelectQuery & head_query = static_cast(*head); tail = head_query.next_union_all; - interpreter->next_select_in_union_all = - std::make_unique(head, context, to_stage, subquery_depth); + interpreter->next_select_in_union_all = std::make_unique(head, context, to_stage, subquery_depth); interpreter = interpreter->next_select_in_union_all.get(); } } if (is_first_select_inside_union_all && hasAsterisk()) { - basicInit(input); + basicInit(); // We execute this code here, because otherwise the following kind of query would not work // SELECT X FROM (SELECT * FROM (SELECT 1 AS X, 2 AS Y) UNION ALL SELECT 3, 4) // because the asterisk is replaced with columns only when query_analyzer objects are created in basicInit(). renameColumns(); - if (!required_column_names.empty() && (table_column_names.size() != required_column_names.size())) + if (!required_column_names.empty() && (source_header.columns() != required_column_names.size())) { rewriteExpressionList(required_column_names); /// Now there is obsolete information to execute the query. We update this information. @@ -128,12 +129,12 @@ void InterpreterSelectQuery::init(const BlockInputStreamPtr & input, const Names { for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) p->query_analyzer = std::make_unique( - p->query_ptr, p->context, p->storage, p->table_column_names, p->subquery_depth, + p->query_ptr, p->context, p->storage, p->source_header.getNamesAndTypesList(), p->subquery_depth, false, p->query_analyzer->getSubqueriesForSets()); } } - basicInit(input); + basicInit(); } } @@ -146,13 +147,12 @@ bool InterpreterSelectQuery::hasAggregation(const ASTSelectQuery & query_ptr) return false; } -void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input) +void InterpreterSelectQuery::basicInit() { /// Read from prepared input. if (input) { - if (table_column_names.empty()) - table_column_names = input->getHeader().getNamesAndTypesList(); + source_header = input->getHeader(); } else { @@ -161,8 +161,7 @@ void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input) /// Read from subquery. if (table_expression && typeid_cast(table_expression.get())) { - if (table_column_names.empty()) - table_column_names = InterpreterSelectQuery::getSampleBlock(table_expression, context).getNamesAndTypesList(); + source_header = InterpreterSelectQuery::getSampleBlock(table_expression, context); } else { @@ -187,15 +186,14 @@ void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input) } table_lock = storage->lockStructure(false, __PRETTY_FUNCTION__); - if (table_column_names.empty()) - table_column_names = storage->getColumnsListNonMaterialized(); + source_header = storage->getSampleBlockNonMaterialized(); } } - if (table_column_names.empty()) + if (!source_header) throw Exception("There are no available columns", ErrorCodes::THERE_IS_NO_COLUMN); - query_analyzer = std::make_unique(query_ptr, context, storage, table_column_names, subquery_depth, !only_analyze); + query_analyzer = std::make_unique(query_ptr, context, storage, source_header.getNamesAndTypesList(), subquery_depth, !only_analyze); if (query.sample_size() && (input || !storage || !storage->supportsSampling())) throw Exception("Illegal SAMPLE: table doesn't support sampling", ErrorCodes::SAMPLING_NOT_SUPPORTED); @@ -210,46 +208,46 @@ void InterpreterSelectQuery::basicInit(const BlockInputStreamPtr & input) for (const auto & it : query_analyzer->getExternalTables()) if (!context.tryGetExternalTable(it.first)) context.addExternalTable(it.first, it.second); - - if (input) - streams.push_back(input); - - if (is_first_select_inside_union_all) - { - /// We check that the results of all SELECT queries are compatible. - Block first = getSampleBlock(); - for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) - { - Block current = p->getSampleBlock(); - if (!blocksHaveEqualStructure(first, current)) - throw Exception("Result structures mismatch in the SELECT queries of the UNION ALL chain. Found result structure:\n\n" + current.dumpStructure() - + "\n\nwhile expecting:\n\n" + first.dumpStructure() + "\n\ninstead", - ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH); - } - } } void InterpreterSelectQuery::initQueryAnalyzer() { - query_analyzer = std::make_unique(query_ptr, context, storage, table_column_names, subquery_depth, !only_analyze); + query_analyzer = std::make_unique(query_ptr, context, storage, source_header.getNamesAndTypesList(), subquery_depth, !only_analyze); for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) - p->query_analyzer = std::make_unique(p->query_ptr, p->context, p->storage, p->table_column_names, p->subquery_depth, !only_analyze); + p->query_analyzer = std::make_unique(p->query_ptr, p->context, p->storage, p->source_header.getNamesAndTypesList(), p->subquery_depth, !only_analyze); } -InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, QueryProcessingStage::Enum to_stage_, - size_t subquery_depth_, BlockInputStreamPtr input) +InterpreterSelectQuery::InterpreterSelectQuery( + const ASTPtr & query_ptr_, + const Context & context_, + QueryProcessingStage::Enum to_stage_, + size_t subquery_depth_, + const BlockInputStreamPtr & input) + : InterpreterSelectQuery(query_ptr_, context_, {}, to_stage_, subquery_depth_, input) +{ +} + +InterpreterSelectQuery::InterpreterSelectQuery( + const ASTPtr & query_ptr_, + const Context & context_, + const Names & required_column_names_, + QueryProcessingStage::Enum to_stage_, + size_t subquery_depth_, + const BlockInputStreamPtr & input) : query_ptr(query_ptr_) , query(typeid_cast(*query_ptr)) , context(context_) , to_stage(to_stage_) , subquery_depth(subquery_depth_) , is_first_select_inside_union_all(query.isUnionAllHead()) + , input(input) , log(&Logger::get("InterpreterSelectQuery")) { - init(input); + init(required_column_names_); } + InterpreterSelectQuery::InterpreterSelectQuery(OnlyAnalyzeTag, const ASTPtr & query_ptr_, const Context & context_) : query_ptr(query_ptr_) , query(typeid_cast(*query_ptr)) @@ -262,27 +260,6 @@ InterpreterSelectQuery::InterpreterSelectQuery(OnlyAnalyzeTag, const ASTPtr & qu init({}); } -InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, - const Names & required_column_names_, - QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input) - : InterpreterSelectQuery(query_ptr_, context_, required_column_names_, {}, to_stage_, subquery_depth_, input) -{ -} - -InterpreterSelectQuery::InterpreterSelectQuery(const ASTPtr & query_ptr_, const Context & context_, - const Names & required_column_names_, - const NamesAndTypesList & table_column_names_, QueryProcessingStage::Enum to_stage_, size_t subquery_depth_, BlockInputStreamPtr input) - : query_ptr(query_ptr_) - , query(typeid_cast(*query_ptr)) - , context(context_) - , to_stage(to_stage_) - , subquery_depth(subquery_depth_) - , table_column_names(table_column_names_) - , is_first_select_inside_union_all(query.isUnionAllHead()) - , log(&Logger::get("InterpreterSelectQuery")) -{ - init(input, required_column_names_); -} bool InterpreterSelectQuery::hasAsterisk() const { @@ -290,13 +267,9 @@ bool InterpreterSelectQuery::hasAsterisk() const return true; if (is_first_select_inside_union_all) - { for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) - { if (p->query.hasAsterisk()) return true; - } - } return false; } @@ -304,10 +277,8 @@ bool InterpreterSelectQuery::hasAsterisk() const void InterpreterSelectQuery::renameColumns() { if (is_first_select_inside_union_all) - { for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) p->query.renameColumns(query); - } } void InterpreterSelectQuery::rewriteExpressionList(const Names & required_column_names) @@ -316,21 +287,15 @@ void InterpreterSelectQuery::rewriteExpressionList(const Names & required_column return; if (is_first_select_inside_union_all) - { for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) - { if (p->query.distinct) return; - } - } query.rewriteSelectExpressionList(required_column_names); if (is_first_select_inside_union_all) - { for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) p->query.rewriteSelectExpressionList(required_column_names); - } } void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, String & table_name) @@ -361,28 +326,12 @@ void InterpreterSelectQuery::getDatabaseAndTableNames(String & database_name, St } -DataTypes InterpreterSelectQuery::getReturnTypes() -{ - DataTypes res; - const NamesAndTypesList & columns = query_analyzer->getSelectSampleBlock().getNamesAndTypesList(); - for (auto & column : columns) - res.push_back(column.type); - - return res; -} - - Block InterpreterSelectQuery::getSampleBlock() { - Block block = query_analyzer->getSelectSampleBlock(); - /// create non-zero columns so that SampleBlock can be - /// written (read) with BlockOut(In)putStreams - for (size_t i = 0; i < block.columns(); ++i) - { - ColumnWithTypeAndName & col = block.safeGetByPosition(i); - col.column = col.type->createColumn(); - } - return block; + Pipeline pipeline; + executeWithoutUnionImpl(pipeline, std::make_shared(source_header)); + auto res = pipeline.firstStream()->getHeader(); + return res; } @@ -394,12 +343,12 @@ Block InterpreterSelectQuery::getSampleBlock(const ASTPtr & query_ptr_, const Co BlockIO InterpreterSelectQuery::execute() { - (void) executeWithoutUnion(); - - executeUnion(); + Pipeline pipeline; + executeWithoutUnionImpl(pipeline, input); + executeUnion(pipeline); /// Constraints on the result, the quota on the result, and also callback for progress. - if (IProfilingBlockInputStream * stream = dynamic_cast(streams[0].get())) + if (IProfilingBlockInputStream * stream = dynamic_cast(pipeline.firstStream().get())) { /// Constraints apply only to the final result. if (to_stage == QueryProcessingStage::Complete) @@ -418,34 +367,42 @@ BlockIO InterpreterSelectQuery::execute() } BlockIO res; - res.in = streams[0]; + res.in = pipeline.firstStream(); return res; } -const BlockInputStreams & InterpreterSelectQuery::executeWithoutUnion() +BlockInputStreams InterpreterSelectQuery::executeWithoutUnion() { + Pipeline pipeline; + executeWithoutUnionImpl(pipeline, input); + return pipeline.streams; +} + +void InterpreterSelectQuery::executeWithoutUnionImpl(Pipeline & pipeline, const BlockInputStreamPtr & input) +{ + if (input) + pipeline.streams.push_back(input); + if (is_first_select_inside_union_all) { - executeSingleQuery(); + executeSingleQuery(pipeline); for (auto p = next_select_in_union_all.get(); p != nullptr; p = p->next_select_in_union_all.get()) { - p->executeSingleQuery(); - const auto & others = p->streams; - streams.insert(streams.end(), others.begin(), others.end()); + Pipeline other_pipeline; + p->executeSingleQuery(other_pipeline); + pipeline.streams.insert(pipeline.streams.end(), other_pipeline.streams.begin(), other_pipeline.streams.end()); } - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream); }); } else - executeSingleQuery(); - - return streams; + executeSingleQuery(pipeline); } -void InterpreterSelectQuery::executeSingleQuery() +void InterpreterSelectQuery::executeSingleQuery(Pipeline & pipeline) { /** 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 @@ -462,7 +419,7 @@ void InterpreterSelectQuery::executeSingleQuery() union_within_single_query = false; /** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */ - QueryProcessingStage::Enum from_stage = executeFetchColumns(); + QueryProcessingStage::Enum from_stage = executeFetchColumns(pipeline); LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(to_stage)); @@ -510,10 +467,6 @@ void InterpreterSelectQuery::executeSingleQuery() has_join = true; before_join = chain.getLastActions(); chain.addStep(); - - const ASTTableJoin & join = static_cast(*query.join()->table_join); - if (join.kind == ASTTableJoin::Kind::Full || join.kind == ASTTableJoin::Kind::Right) - stream_with_non_joined_data = before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(settings.max_block_size); } if (query_analyzer->appendWhere(chain, !first_stage)) @@ -579,18 +532,25 @@ void InterpreterSelectQuery::executeSingleQuery() if (first_stage) { if (has_join) - for (auto & stream : streams) /// Applies to all sources except stream_with_non_joined_data. + { + const ASTTableJoin & join = static_cast(*query.join()->table_join); + if (join.kind == ASTTableJoin::Kind::Full || join.kind == ASTTableJoin::Kind::Right) + pipeline.stream_with_non_joined_data = before_join->createStreamWithNonJoinedDataIfFullOrRightJoin( + pipeline.firstStream()->getHeader(), settings.max_block_size); + + for (auto & stream : pipeline.streams) /// Applies to all sources except stream_with_non_joined_data. stream = std::make_shared(stream, before_join); + } if (has_where) - executeWhere(before_where); + executeWhere(pipeline, before_where); if (need_aggregate) - executeAggregation(before_aggregation, aggregate_overflow_row, aggregate_final); + executeAggregation(pipeline, before_aggregation, aggregate_overflow_row, aggregate_final); else { - executeExpression(before_order_and_select); - executeDistinct(true, selected_columns); + executeExpression(pipeline, before_order_and_select); + executeDistinct(pipeline, true, selected_columns); } /** For distributed query processing, @@ -601,13 +561,13 @@ void InterpreterSelectQuery::executeSingleQuery() if (!second_stage && !need_aggregate && !has_having) { if (has_order_by) - executeOrder(); + executeOrder(pipeline); if (has_order_by && query.limit_length) - executeDistinct(false, selected_columns); + executeDistinct(pipeline, false, selected_columns); if (query.limit_length) - executePreLimit(); + executePreLimit(pipeline); } } @@ -619,24 +579,24 @@ void InterpreterSelectQuery::executeSingleQuery() { /// If you need to combine aggregated results from multiple servers if (!first_stage) - executeMergeAggregated(aggregate_overflow_row, aggregate_final); + executeMergeAggregated(pipeline, aggregate_overflow_row, aggregate_final); if (!aggregate_final) - executeTotalsAndHaving(has_having, before_having, aggregate_overflow_row); + executeTotalsAndHaving(pipeline, has_having, before_having, aggregate_overflow_row); else if (has_having) - executeHaving(before_having); + executeHaving(pipeline, before_having); - executeExpression(before_order_and_select); - executeDistinct(true, selected_columns); + executeExpression(pipeline, before_order_and_select); + executeDistinct(pipeline, true, selected_columns); - need_second_distinct_pass = query.distinct && hasMoreThanOneStream(); + need_second_distinct_pass = query.distinct && pipeline.hasMoreThanOneStream(); } else { - need_second_distinct_pass = query.distinct && hasMoreThanOneStream(); + need_second_distinct_pass = query.distinct && pipeline.hasMoreThanOneStream(); if (query.group_by_with_totals && !aggregate_final) - executeTotalsAndHaving(false, nullptr, aggregate_overflow_row); + executeTotalsAndHaving(pipeline, false, nullptr, aggregate_overflow_row); } if (has_order_by) @@ -646,17 +606,17 @@ void InterpreterSelectQuery::executeSingleQuery() * - therefore, we merge the sorted streams from remote servers. */ if (!first_stage && !need_aggregate && !(query.group_by_with_totals && !aggregate_final)) - executeMergeSorted(); + executeMergeSorted(pipeline); else /// Otherwise, just sort. - executeOrder(); + executeOrder(pipeline); } - executeProjection(final_projection); + executeProjection(pipeline, final_projection); /// At this stage, we can calculate the minimums and maximums, if necessary. if (settings.extremes) { - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) p_stream->enableExtremes(); @@ -666,36 +626,36 @@ void InterpreterSelectQuery::executeSingleQuery() /** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT, * limiting the number of entries in each up to `offset + limit`. */ - if (query.limit_length && hasMoreThanOneStream() && !query.distinct && !query.limit_by_expression_list) - executePreLimit(); + if (query.limit_length && pipeline.hasMoreThanOneStream() && !query.distinct && !query.limit_by_expression_list) + executePreLimit(pipeline); - if (stream_with_non_joined_data || need_second_distinct_pass) + if (pipeline.stream_with_non_joined_data || need_second_distinct_pass) union_within_single_query = true; /// To execute LIMIT BY we should merge all streams together. - if (query.limit_by_expression_list && hasMoreThanOneStream()) + if (query.limit_by_expression_list && pipeline.hasMoreThanOneStream()) union_within_single_query = true; if (union_within_single_query) - executeUnion(); + executeUnion(pipeline); - if (streams.size() == 1) + if (pipeline.streams.size() == 1) { /** If there was more than one stream, * then DISTINCT needs to be performed once again after merging all streams. */ if (need_second_distinct_pass) - executeDistinct(false, Names()); + executeDistinct(pipeline, false, Names()); - executeLimitBy(); - executeLimit(); + executeLimitBy(pipeline); + executeLimit(pipeline); } } } SubqueriesForSets subqueries_for_sets = query_analyzer->getSubqueriesForSets(); if (!subqueries_for_sets.empty()) - executeSubqueriesInSetsAndJoins(subqueries_for_sets); + executeSubqueriesInSetsAndJoins(pipeline, subqueries_for_sets); } @@ -711,7 +671,7 @@ static void getLimitLengthAndOffset(ASTSelectQuery & query, size_t & length, siz } } -QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns() +QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline & pipeline) { /// The subquery interpreter, if the subquery std::optional interpreter_subquery; @@ -749,7 +709,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns() required_columns_expr_list->children.emplace_back(std::make_shared(StringRange(), column)); } - alias_actions = ExpressionAnalyzer{required_columns_expr_list, context, storage, table_column_names}.getActions(true); + alias_actions = ExpressionAnalyzer{required_columns_expr_list, context, storage, source_header.getNamesAndTypesList()}.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(); @@ -830,9 +790,20 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns() query_analyzer->makeSetsForIndex(); - /// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery? - if (!interpreter_subquery) + /// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery or prepared input? + if (!pipeline.streams.empty()) { + /// Prepared input. + } + else if (interpreter_subquery) + { + /// Subquery. + interpreter_subquery->executeWithoutUnionImpl(pipeline, {}); + } + else if (storage) + { + /// Table. + if (max_streams == 0) throw Exception("Logical error: zero number of streams requested", ErrorCodes::LOGICAL_ERROR); @@ -860,76 +831,73 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns() } /// If there was no already prepared input. - if (streams.empty()) - streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); + if (pipeline.streams.empty()) + pipeline.streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams); - if (streams.empty()) - streams.emplace_back(std::make_shared(storage->getSampleBlockForColumns(required_columns))); + if (pipeline.streams.empty()) + pipeline.streams.emplace_back(std::make_shared(storage->getSampleBlockForColumns(required_columns))); if (alias_actions) { /// Wrap each stream returned from the table to calculate and add ALIAS columns - transformStreams([&] (auto & stream) + pipeline.transform([&] (auto & stream) { stream = std::make_shared(stream, alias_actions); }); } - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream->addTableLock(table_lock); }); + + /** Set the limits and quota for reading data, the speed and time of the query. + * Such restrictions are checked on the initiating server of the request, and not on remote servers. + * Because the initiating server has a summary of the execution of the request on all servers. + */ + if (to_stage == QueryProcessingStage::Complete) + { + IProfilingBlockInputStream::LocalLimits limits; + limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; + limits.max_rows_to_read = settings.limits.max_rows_to_read; + limits.max_bytes_to_read = settings.limits.max_bytes_to_read; + limits.read_overflow_mode = settings.limits.read_overflow_mode; + limits.max_execution_time = settings.limits.max_execution_time; + limits.timeout_overflow_mode = settings.limits.timeout_overflow_mode; + limits.min_execution_speed = settings.limits.min_execution_speed; + limits.timeout_before_checking_execution_speed = settings.limits.timeout_before_checking_execution_speed; + + QuotaForIntervals & quota = context.getQuota(); + + pipeline.transform([&](auto & stream) + { + if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) + { + p_stream->setLimits(limits); + p_stream->setQuota(quota); + } + }); + } } else - { - const auto & subquery_streams = interpreter_subquery->executeWithoutUnion(); - streams.insert(streams.end(), subquery_streams.begin(), subquery_streams.end()); - } - - /** Set the limits and quota for reading data, the speed and time of the query. - * Such restrictions are checked on the initiating server of the request, and not on remote servers. - * Because the initiating server has a summary of the execution of the request on all servers. - */ - if (storage && to_stage == QueryProcessingStage::Complete) - { - IProfilingBlockInputStream::LocalLimits limits; - limits.mode = IProfilingBlockInputStream::LIMITS_TOTAL; - limits.max_rows_to_read = settings.limits.max_rows_to_read; - limits.max_bytes_to_read = settings.limits.max_bytes_to_read; - limits.read_overflow_mode = settings.limits.read_overflow_mode; - limits.max_execution_time = settings.limits.max_execution_time; - limits.timeout_overflow_mode = settings.limits.timeout_overflow_mode; - limits.min_execution_speed = settings.limits.min_execution_speed; - limits.timeout_before_checking_execution_speed = settings.limits.timeout_before_checking_execution_speed; - - QuotaForIntervals & quota = context.getQuota(); - - transformStreams([&](auto & stream) - { - if (IProfilingBlockInputStream * p_stream = dynamic_cast(stream.get())) - { - p_stream->setLimits(limits); - p_stream->setQuota(quota); - } - }); - } + throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR); return from_stage; } -void InterpreterSelectQuery::executeWhere(const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression) { - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expression, query.where_expression->getColumnName()); }); } -void InterpreterSelectQuery::executeAggregation(const ExpressionActionsPtr & expression, bool overflow_row, bool final) +void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final) { - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expression); }); @@ -938,7 +906,7 @@ void InterpreterSelectQuery::executeAggregation(const ExpressionActionsPtr & exp AggregateDescriptions aggregates; query_analyzer->getAggregateInfo(key_names, aggregates); - Block header = streams[0]->getHeader(); + Block header = pipeline.firstStream()->getHeader(); ColumnNumbers keys; for (const auto & name : key_names) keys.push_back(header.getPositionByName(name)); @@ -953,7 +921,7 @@ void InterpreterSelectQuery::executeAggregation(const ExpressionActionsPtr & exp * 1. Parallel aggregation is done, and the results should be merged in parallel. * 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way. */ - bool allow_to_use_two_level_group_by = streams.size() > 1 || settings.limits.max_bytes_before_external_group_by != 0; + bool allow_to_use_two_level_group_by = pipeline.streams.size() > 1 || settings.limits.max_bytes_before_external_group_by != 0; Aggregator::Params params(header, keys, aggregates, overflow_row, settings.limits.max_rows_to_group_by, settings.limits.group_by_overflow_mode, @@ -964,43 +932,43 @@ void InterpreterSelectQuery::executeAggregation(const ExpressionActionsPtr & exp context.getTemporaryPath()); /// If there are several sources, then we perform parallel aggregation - if (streams.size() > 1) + if (pipeline.streams.size() > 1) { - streams[0] = std::make_shared( - streams, stream_with_non_joined_data, params, final, + pipeline.firstStream() = std::make_shared( + pipeline.streams, pipeline.stream_with_non_joined_data, params, final, max_streams, settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads)); - stream_with_non_joined_data = nullptr; - streams.resize(1); + pipeline.stream_with_non_joined_data = nullptr; + pipeline.streams.resize(1); } else { BlockInputStreams inputs; - if (!streams.empty()) - inputs.push_back(streams[0]); + if (!pipeline.streams.empty()) + inputs.push_back(pipeline.firstStream()); else - streams.resize(1); + pipeline.streams.resize(1); - if (stream_with_non_joined_data) - inputs.push_back(stream_with_non_joined_data); + if (pipeline.stream_with_non_joined_data) + inputs.push_back(pipeline.stream_with_non_joined_data); - streams[0] = std::make_shared(std::make_shared(inputs), params, final); + pipeline.firstStream() = std::make_shared(std::make_shared(inputs), params, final); - stream_with_non_joined_data = nullptr; + pipeline.stream_with_non_joined_data = nullptr; } } -void InterpreterSelectQuery::executeMergeAggregated(bool overflow_row, bool final) +void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final) { Names key_names; AggregateDescriptions aggregates; query_analyzer->getAggregateInfo(key_names, aggregates); - Block header = streams[0]->getHeader(); + Block header = pipeline.firstStream()->getHeader(); ColumnNumbers keys; for (const auto & name : key_names) @@ -1028,48 +996,48 @@ void InterpreterSelectQuery::executeMergeAggregated(bool overflow_row, bool fina if (!settings.distributed_aggregation_memory_efficient) { /// We union several sources into one, parallelizing the work. - executeUnion(); + executeUnion(pipeline); /// Now merge the aggregated blocks - streams[0] = std::make_shared(streams[0], params, final, settings.max_threads); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), params, final, settings.max_threads); } else { - streams[0] = std::make_shared(streams, params, final, + pipeline.firstStream() = std::make_shared(pipeline.streams, params, final, max_streams, settings.aggregation_memory_efficient_merge_threads ? static_cast(settings.aggregation_memory_efficient_merge_threads) : static_cast(settings.max_threads)); - streams.resize(1); + pipeline.streams.resize(1); } } -void InterpreterSelectQuery::executeHaving(const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression) { - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expression, query.having_expression->getColumnName()); }); } -void InterpreterSelectQuery::executeTotalsAndHaving(bool has_having, const ExpressionActionsPtr & expression, bool overflow_row) +void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row) { - executeUnion(); + executeUnion(pipeline); const Settings & settings = context.getSettingsRef(); - streams[0] = std::make_shared( - streams[0], overflow_row, expression, + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), overflow_row, expression, has_having ? query.having_expression->getColumnName() : "", settings.totals_mode, settings.totals_auto_threshold); } -void InterpreterSelectQuery::executeExpression(const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression) { - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expression); }); @@ -1111,14 +1079,14 @@ static size_t getLimitForSorting(ASTSelectQuery & query) } -void InterpreterSelectQuery::executeOrder() +void InterpreterSelectQuery::executeOrder(Pipeline & pipeline) { SortDescription order_descr = getSortDescription(query); size_t limit = getLimitForSorting(query); const Settings & settings = context.getSettingsRef(); - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { auto sorting_stream = std::make_shared(stream, order_descr, limit); @@ -1134,16 +1102,16 @@ void InterpreterSelectQuery::executeOrder() }); /// If there are several streams, we merge them into one - executeUnion(); + executeUnion(pipeline); /// Merge the sorted blocks. - streams[0] = std::make_shared( - streams[0], order_descr, settings.max_block_size, limit, + pipeline.firstStream() = std::make_shared( + pipeline.firstStream(), order_descr, settings.max_block_size, limit, settings.limits.max_bytes_before_external_sort, context.getTemporaryPath()); } -void InterpreterSelectQuery::executeMergeSorted() +void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) { SortDescription order_descr = getSortDescription(query); size_t limit = getLimitForSorting(query); @@ -1151,33 +1119,33 @@ void InterpreterSelectQuery::executeMergeSorted() const Settings & settings = context.getSettingsRef(); /// If there are several streams, then we merge them into one - if (hasMoreThanOneStream()) + if (pipeline.hasMoreThanOneStream()) { /** MergingSortedBlockInputStream reads the sources sequentially. * To make the data on the remote servers prepared in parallel, we wrap it in AsynchronousBlockInputStream. */ - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream); }); /// Merge the sorted sources into one sorted source. - streams[0] = std::make_shared(streams, order_descr, settings.max_block_size, limit); - streams.resize(1); + pipeline.firstStream() = std::make_shared(pipeline.streams, order_descr, settings.max_block_size, limit); + pipeline.streams.resize(1); } } -void InterpreterSelectQuery::executeProjection(const ExpressionActionsPtr & expression) +void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression) { - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, expression); }); } -void InterpreterSelectQuery::executeDistinct(bool before_order, Names columns) +void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_order, Names columns) { if (query.distinct) { @@ -1193,7 +1161,7 @@ void InterpreterSelectQuery::executeDistinct(bool before_order, Names columns) if (!query.order_expression_list || !before_order) limit_for_distinct = limit_length + limit_offset; - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { if (stream->isGroupedOutput()) stream = std::make_shared(stream, settings.limits, limit_for_distinct, columns); @@ -1201,33 +1169,33 @@ void InterpreterSelectQuery::executeDistinct(bool before_order, Names columns) stream = std::make_shared(stream, settings.limits, limit_for_distinct, columns); }); - if (hasMoreThanOneStream()) + if (pipeline.hasMoreThanOneStream()) union_within_single_query = true; } } -void InterpreterSelectQuery::executeUnion() +void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) { /// If there are still several streams, then we combine them into one - if (hasMoreThanOneStream()) + if (pipeline.hasMoreThanOneStream()) { - streams[0] = std::make_shared>(streams, stream_with_non_joined_data, max_streams); - stream_with_non_joined_data = nullptr; - streams.resize(1); + pipeline.firstStream() = std::make_shared>(pipeline.streams, pipeline.stream_with_non_joined_data, max_streams); + pipeline.stream_with_non_joined_data = nullptr; + pipeline.streams.resize(1); union_within_single_query = false; } - else if (stream_with_non_joined_data) + else if (pipeline.stream_with_non_joined_data) { - streams.push_back(stream_with_non_joined_data); - stream_with_non_joined_data = nullptr; + pipeline.streams.push_back(pipeline.stream_with_non_joined_data); + pipeline.stream_with_non_joined_data = nullptr; union_within_single_query = false; } } /// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined. -void InterpreterSelectQuery::executePreLimit() +void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) { size_t limit_length = 0; size_t limit_offset = 0; @@ -1236,18 +1204,18 @@ void InterpreterSelectQuery::executePreLimit() /// If there is LIMIT if (query.limit_length) { - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit_length + limit_offset, 0, false); }); - if (hasMoreThanOneStream()) + if (pipeline.hasMoreThanOneStream()) union_within_single_query = true; } } -void InterpreterSelectQuery::executeLimitBy() +void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline) { if (!query.limit_by_value || !query.limit_by_expression_list) return; @@ -1260,7 +1228,7 @@ void InterpreterSelectQuery::executeLimitBy() columns.emplace_back(elem->getAliasOrColumnName()); } - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared( stream, value, columns @@ -1269,7 +1237,7 @@ void InterpreterSelectQuery::executeLimitBy() } -void InterpreterSelectQuery::executeLimit() +void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) { size_t limit_length = 0; size_t limit_offset = 0; @@ -1319,7 +1287,7 @@ void InterpreterSelectQuery::executeLimit() } } - transformStreams([&](auto & stream) + pipeline.transform([&](auto & stream) { stream = std::make_shared(stream, limit_length, limit_offset, always_read_till_end); }); @@ -1327,31 +1295,16 @@ void InterpreterSelectQuery::executeLimit() } -void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(SubqueriesForSets & subqueries_for_sets) +void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, SubqueriesForSets & subqueries_for_sets) { const Settings & settings = context.getSettingsRef(); - executeUnion(); - streams[0] = std::make_shared(streams[0], subqueries_for_sets, settings.limits); -} - -template -void InterpreterSelectQuery::transformStreams(Transform && transform) -{ - for (auto & stream : streams) - transform(stream); - - if (stream_with_non_joined_data) - transform(stream_with_non_joined_data); -} - - -bool InterpreterSelectQuery::hasMoreThanOneStream() const -{ - return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1; + executeUnion(pipeline); + pipeline.firstStream() = std::make_shared(pipeline.firstStream(), subqueries_for_sets, settings.limits); } +/// TODO This is trash. void InterpreterSelectQuery::ignoreWithTotals() { query.group_by_with_totals = false; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index cfc61e35f2c..14416e5fd46 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -50,7 +50,7 @@ public: const Context & context_, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, - BlockInputStreamPtr input = nullptr); + const BlockInputStreamPtr & input = nullptr); InterpreterSelectQuery( const ASTPtr & query_ptr_, @@ -58,16 +58,7 @@ public: const Names & required_column_names, QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, size_t subquery_depth_ = 0, - BlockInputStreamPtr input = nullptr); - - InterpreterSelectQuery( - const ASTPtr & query_ptr_, - const Context & context_, - const Names & required_column_names, - const NamesAndTypesList & table_column_names_, - QueryProcessingStage::Enum to_stage_ = QueryProcessingStage::Complete, - size_t subquery_depth_ = 0, - BlockInputStreamPtr input = nullptr); + const BlockInputStreamPtr & input = nullptr); ~InterpreterSelectQuery(); @@ -78,10 +69,8 @@ public: /** Execute the query without union of threads, if it is possible. */ - const BlockInputStreams & executeWithoutUnion(); + BlockInputStreams executeWithoutUnion(); - /// TODO It's confusing that these methods return result structure for the case of QueryProcessingStage::Complete regardless to the actual 'to_stage'. - DataTypes getReturnTypes(); Block getSampleBlock(); static Block getSampleBlock( @@ -89,23 +78,55 @@ public: const Context & context_); private: - /** - * - Optimization if an object is created only to call getSampleBlock(): consider only the first SELECT of the UNION ALL chain, because - * the first SELECT is sufficient to determine the required columns. - */ + struct Pipeline + { + /** Streams of data. + * The source data streams are produced in the executeFetchColumns function. + * Then they are converted (wrapped in other streams) using the `execute*` functions, + * to get the whole pipeline running the query. + */ + BlockInputStreams streams; + + /** When executing FULL or RIGHT JOIN, there will be a data stream from which you can read "not joined" rows. + * It has a special meaning, since reading from it should be done after reading from the main streams. + * It is joined to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream. + */ + BlockInputStreamPtr stream_with_non_joined_data; + + BlockInputStreamPtr & firstStream() { return streams.at(0); } + + template + void transform(Transform && transform) + { + for (auto & stream : streams) + transform(stream); + + if (stream_with_non_joined_data) + transform(stream_with_non_joined_data); + } + + bool hasMoreThanOneStream() const + { + return streams.size() + (stream_with_non_joined_data ? 1 : 0) > 1; + } + }; + + /** - Optimization if an object is created only to call getSampleBlock(): consider only the first SELECT of the UNION ALL chain, because + * the first SELECT is sufficient to determine the required columns. + */ struct OnlyAnalyzeTag {}; InterpreterSelectQuery( OnlyAnalyzeTag, const ASTPtr & query_ptr_, const Context & context_); - void init(const BlockInputStreamPtr & input, const Names & required_column_names = Names{}); - void basicInit(const BlockInputStreamPtr & input); + void init(const Names & required_column_names); + void basicInit(); void initQueryAnalyzer(); bool hasAggregation(const ASTSelectQuery & query_ptr); /// Execute one SELECT query from the UNION ALL chain. - void executeSingleQuery(); + void executeSingleQuery(Pipeline & pipeline); /** Leave only the necessary columns of the SELECT section in each query of the UNION ALL chain. * However, if you use at least one DISTINCT in the chain, then all the columns are considered necessary, @@ -136,30 +157,24 @@ private: /// Different stages of query execution. /// Fetch data from the table. Returns the stage to which the query was processed in Storage. - QueryProcessingStage::Enum executeFetchColumns(); + QueryProcessingStage::Enum executeFetchColumns(Pipeline & pipeline); - void executeWhere(const ExpressionActionsPtr & expression); - void executeAggregation(const ExpressionActionsPtr & expression, bool overflow_row, bool final); - void executeMergeAggregated(bool overflow_row, bool final); - void executeTotalsAndHaving(bool has_having, const ExpressionActionsPtr & expression, bool overflow_row); - void executeHaving(const ExpressionActionsPtr & expression); - void executeExpression(const ExpressionActionsPtr & expression); - void executeOrder(); - void executeMergeSorted(); - void executePreLimit(); - void executeUnion(); - void executeLimitBy(); - void executeLimit(); - void executeProjection(const ExpressionActionsPtr & expression); - void executeDistinct(bool before_order, Names columns); - void executeSubqueriesInSetsAndJoins(std::unordered_map & subqueries_for_sets); - - template - void transformStreams(Transform && transform); - - bool hasNoData() const; - - bool hasMoreThanOneStream() const; + void executeWithoutUnionImpl(Pipeline & pipeline, const BlockInputStreamPtr & input); + void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression); + void executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final); + void executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final); + void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row); + void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); + void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); + void executeOrder(Pipeline & pipeline); + void executeMergeSorted(Pipeline & pipeline); + void executePreLimit(Pipeline & pipeline); + void executeUnion(Pipeline & pipeline); + void executeLimitBy(Pipeline & pipeline); + void executeLimit(Pipeline & pipeline); + void executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression); + void executeDistinct(Pipeline & pipeline, bool before_order, Names columns); + void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map & subqueries_for_sets); void ignoreWithTotals(); @@ -177,24 +192,11 @@ private: QueryProcessingStage::Enum to_stage; size_t subquery_depth; std::unique_ptr query_analyzer; - NamesAndTypesList table_column_names; + Block source_header; /// How many streams we ask for storage to produce, and in how many threads we will do further processing. size_t max_streams = 1; - /** Streams of data. - * The source data streams are produced in the executeFetchColumns function. - * Then they are converted (wrapped in other streams) using the `execute*` functions, - * to get the whole pipeline running the query. - */ - BlockInputStreams streams; - - /** When executing FULL or RIGHT JOIN, there will be a data stream from which you can read "not joined" rows. - * It has a special meaning, since reading from it should be done after reading from the main streams. - * It is joined to the main streams in UnionBlockInputStream or ParallelAggregatingBlockInputStream. - */ - BlockInputStreamPtr stream_with_non_joined_data; - /// Is it the first SELECT query of the UNION ALL chain? bool is_first_select_inside_union_all; @@ -208,6 +210,9 @@ private: StoragePtr storage; TableStructureReadLockPtr table_lock; + /// Used when we read from prepared input, not table or subquery. + BlockInputStreamPtr input; + /// Do union of streams within a SELECT query? bool union_within_single_query = false; diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index c469fc733a8..18e1e056a03 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -11,6 +11,8 @@ #include #include +#include + #include #include @@ -281,7 +283,7 @@ void Join::setSampleBlock(const Block & block) /// Choose data structure to use for JOIN. init(chooseMethod(key_columns, key_sizes)); - sample_block_with_columns_to_add = block; + sample_block_with_columns_to_add = materializeBlock(block); /// Move from `sample_block_with_columns_to_add` key columns to `sample_block_with_keys`, keeping the order. size_t pos = 0; @@ -462,8 +464,8 @@ bool Join::insertFromBlock(const Block & block) if (getFullness(kind)) { - /** Transfer the key columns to the beginning of the block. - * This is where NonJoinedBlockInputStream will wait for them. + /** Move the key columns to the beginning of the block. + * This is where NonJoinedBlockInputStream will expect. */ size_t key_num = 0; for (const auto & name : key_names_right) @@ -990,7 +992,7 @@ public: size_t num_columns_left = left_sample_block.columns() - num_keys; size_t num_columns_right = parent.sample_block_with_columns_to_add.columns(); - result_sample_block = left_sample_block; + result_sample_block = materializeBlock(left_sample_block); /// Add columns from the right-side table to the block. for (size_t i = 0; i < num_columns_right; ++i) @@ -1154,7 +1156,7 @@ private: }; -BlockInputStreamPtr Join::createStreamWithNonJoinedRows(Block & left_sample_block, size_t max_block_size) const +BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, size_t max_block_size) const { return std::make_shared(*this, left_sample_block, max_block_size); } diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 5e7ad9728c1..20df3829af9 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -253,7 +253,7 @@ public: * Use only after all calls to joinBlock was done. * left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside). */ - BlockInputStreamPtr createStreamWithNonJoinedRows(Block & left_sample_block, size_t max_block_size) const; + BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, size_t max_block_size) const; /// Number of keys in all built JOIN maps. size_t getTotalRowCount() const; diff --git a/dbms/src/Interpreters/Set.cpp b/dbms/src/Interpreters/Set.cpp index 2c06149aa44..63b7024cb73 100644 --- a/dbms/src/Interpreters/Set.cpp +++ b/dbms/src/Interpreters/Set.cpp @@ -540,7 +540,7 @@ MergeTreeSetIndex::MergeTreeSetIndex(const SetElements & set_elements, std::vect * 1: the intersection of the set and the range is non-empty * 2: the range contains elements not in the set */ -BoolMask MergeTreeSetIndex::mayBeTrueInRange(const std::vector & key_ranges) +BoolMask MergeTreeSetIndex::mayBeTrueInRange(const std::vector & key_ranges, const DataTypes & data_types) { std::vector left_point; std::vector right_point; @@ -555,7 +555,7 @@ BoolMask MergeTreeSetIndex::mayBeTrueInRange(const std::vector & key_rang std::optional new_range = PKCondition::applyMonotonicFunctionsChainToRange( key_ranges[indexes_mapping[i].pk_index], indexes_mapping[i].functions, - indexes_mapping[i].data_type); + data_types[indexes_mapping[i].pk_index]); if (!new_range) return {true, true}; diff --git a/dbms/src/Interpreters/Set.h b/dbms/src/Interpreters/Set.h index 94d0d11f338..bd2a7363ad4 100644 --- a/dbms/src/Interpreters/Set.h +++ b/dbms/src/Interpreters/Set.h @@ -179,12 +179,11 @@ public: size_t tuple_index; size_t pk_index; std::vector functions; - DataTypePtr data_type; }; MergeTreeSetIndex(const SetElements & set_elements, std::vector && indexes_mapping_); - BoolMask mayBeTrueInRange(const std::vector & key_ranges); + BoolMask mayBeTrueInRange(const std::vector & key_ranges, const DataTypes & data_types); private: using OrderedTuples = std::vector>; OrderedTuples ordered_set; diff --git a/dbms/src/Interpreters/Settings.h b/dbms/src/Interpreters/Settings.h index 4946be8acb6..0a725509186 100644 --- a/dbms/src/Interpreters/Settings.h +++ b/dbms/src/Interpreters/Settings.h @@ -113,8 +113,6 @@ struct Settings \ M(SettingDistributedProductMode, distributed_product_mode, DistributedProductMode::DENY, "How are distributed subqueries performed inside IN or JOIN sections?") \ \ - M(SettingGlobalSubqueriesMethod, global_subqueries_method, GlobalSubqueriesMethod::PUSH, "The method for executing GLOBAL subqueries.") \ - \ M(SettingUInt64, max_concurrent_queries_for_user, 0, "The maximum number of concurrent requests per user.") \ \ M(SettingBool, insert_deduplicate, true, "For INSERT queries in the replicated table, specifies that deduplication of insertings blocks should be preformed") \ diff --git a/dbms/src/Interpreters/SettingsCommon.h b/dbms/src/Interpreters/SettingsCommon.h index 55ffc48dff1..59c1c0dac28 100644 --- a/dbms/src/Interpreters/SettingsCommon.h +++ b/dbms/src/Interpreters/SettingsCommon.h @@ -672,73 +672,6 @@ struct SettingDistributedProductMode } }; -/// Method for executing global distributed subqueries. -enum class GlobalSubqueriesMethod -{ - PUSH = 0, /// Send the subquery data to all remote servers. - PULL = 1, /// Remote servers will download the subquery data from the initiating server. -}; - -struct SettingGlobalSubqueriesMethod -{ - GlobalSubqueriesMethod value; - bool changed = false; - - SettingGlobalSubqueriesMethod(GlobalSubqueriesMethod x = GlobalSubqueriesMethod::PUSH) : value(x) {} - - operator GlobalSubqueriesMethod() const { return value; } - SettingGlobalSubqueriesMethod & operator= (GlobalSubqueriesMethod x) { set(x); return *this; } - - static GlobalSubqueriesMethod getGlobalSubqueriesMethod(const String & s) - { - if (s == "push") - return GlobalSubqueriesMethod::PUSH; - if (s == "pull") - return GlobalSubqueriesMethod::PULL; - - throw Exception("Unknown global subqueries execution method: '" + s + "', must be one of 'push', 'pull'", - ErrorCodes::UNKNOWN_GLOBAL_SUBQUERIES_METHOD); - } - - String toString() const - { - const char * strings[] = { "push", "pull" }; - - if (value < GlobalSubqueriesMethod::PUSH || value > GlobalSubqueriesMethod::PULL) - throw Exception("Unknown global subqueries execution method", ErrorCodes::UNKNOWN_GLOBAL_SUBQUERIES_METHOD); - - return strings[static_cast(value)]; - } - - void set(GlobalSubqueriesMethod x) - { - value = x; - changed = true; - } - - void set(const Field & x) - { - set(safeGet(x)); - } - - void set(const String & x) - { - set(getGlobalSubqueriesMethod(x)); - } - - void set(ReadBuffer & buf) - { - String x; - readBinary(x, buf); - set(x); - } - - void write(WriteBuffer & buf) const - { - writeBinary(toString(), buf); - } -}; - struct SettingString { diff --git a/dbms/src/Interpreters/tests/expression.cpp b/dbms/src/Interpreters/tests/expression.cpp index 0b2ead8bd52..734f89623ea 100644 --- a/dbms/src/Interpreters/tests/expression.cpp +++ b/dbms/src/Interpreters/tests/expression.cpp @@ -125,7 +125,7 @@ int main(int argc, char ** argv) LimitBlockInputStream lis(is, 20, std::max(0, static_cast(n) - 20)); WriteBufferFromOStream out_buf(std::cout); RowOutputStreamPtr os_ = std::make_shared(out_buf, block); - BlockOutputStreamFromRowOutputStream os(os_); + BlockOutputStreamFromRowOutputStream os(os_, is->getHeader()); copyData(lis, os); } diff --git a/dbms/src/Server/ClusterCopier.cpp b/dbms/src/Server/ClusterCopier.cpp index ed06710c19e..0f2af436be6 100644 --- a/dbms/src/Server/ClusterCopier.cpp +++ b/dbms/src/Server/ClusterCopier.cpp @@ -1758,7 +1758,7 @@ protected: { /// CREATE TABLE and DROP PARTITION return empty block RemoteBlockInputStream stream(*connection, query, Block(), context, ¤t_settings); - NullBlockOutputStream output; + NullBlockOutputStream output(Block()); copyData(stream, output); if (increment_and_check_exit()) diff --git a/dbms/src/Server/TCPHandler.cpp b/dbms/src/Server/TCPHandler.cpp index e27d4f088e0..345cf118a5b 100644 --- a/dbms/src/Server/TCPHandler.cpp +++ b/dbms/src/Server/TCPHandler.cpp @@ -288,7 +288,7 @@ void TCPHandler::processInsertQuery(const Settings & global_settings) state.io.out->writePrefix(); /// Send block to the client - table structure. - Block block = state.io.out_sample; + Block block = state.io.out->getHeader(); sendData(block); readData(global_settings); @@ -417,7 +417,7 @@ void TCPHandler::sendTotals() if (totals) { - initBlockOutput(); + initBlockOutput(totals); writeVarUInt(Protocol::Server::Totals, *out); writeStringBinary("", *out); @@ -438,7 +438,7 @@ void TCPHandler::sendExtremes() if (extremes) { - initBlockOutput(); + initBlockOutput(extremes); writeVarUInt(Protocol::Server::Extremes, *out); writeStringBinary("", *out); @@ -662,7 +662,7 @@ void TCPHandler::initBlockInput() } -void TCPHandler::initBlockOutput() +void TCPHandler::initBlockOutput(const Block & block) { if (!state.block_out) { @@ -674,7 +674,8 @@ void TCPHandler::initBlockOutput() state.block_out = std::make_shared( *state.maybe_compressed_out, - client_revision); + client_revision, + block.cloneEmpty()); } } @@ -715,7 +716,7 @@ bool TCPHandler::isQueryCancelled() void TCPHandler::sendData(const Block & block) { - initBlockOutput(); + initBlockOutput(block); writeVarUInt(Protocol::Server::Data, *out); writeStringBinary("", *out); diff --git a/dbms/src/Server/TCPHandler.h b/dbms/src/Server/TCPHandler.h index 93b82acd7ea..444d8eb4990 100644 --- a/dbms/src/Server/TCPHandler.h +++ b/dbms/src/Server/TCPHandler.h @@ -140,7 +140,7 @@ private: /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. void initBlockInput(); - void initBlockOutput(); + void initBlockOutput(const Block & block); bool isQueryCancelled(); diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp index fe8344da118..1155d4c6362 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.cpp @@ -33,6 +33,7 @@ #include #include + namespace CurrentMetrics { extern const Metric DistributedSend; @@ -53,14 +54,20 @@ namespace ErrorCodes } -DistributedBlockOutputStream::DistributedBlockOutputStream(StorageDistributed & storage, const ASTPtr & query_ast, const ClusterPtr & cluster_, - const Settings & settings_, bool insert_sync_, UInt64 insert_timeout_) - : storage(storage), query_ast(query_ast), cluster(cluster_), settings(settings_), insert_sync(insert_sync_), - insert_timeout(insert_timeout_) +DistributedBlockOutputStream::DistributedBlockOutputStream( + StorageDistributed & storage, const ASTPtr & query_ast, const ClusterPtr & cluster_, + const Settings & settings_, bool insert_sync_, UInt64 insert_timeout_) + : storage(storage), query_ast(query_ast), cluster(cluster_), settings(settings_), insert_sync(insert_sync_), insert_timeout(insert_timeout_) { } +Block DistributedBlockOutputStream::getHeader() const +{ + return storage.getSampleBlock(); +} + + void DistributedBlockOutputStream::writePrefix() { deadline = std::chrono::steady_clock::now() + std::chrono::seconds(insert_timeout); @@ -472,7 +479,7 @@ void DistributedBlockOutputStream::writeToShard(const Block & block, const std:: WriteBufferFromFile out{block_file_tmp_path}; CompressedWriteBuffer compress{out}; - NativeBlockOutputStream stream{compress, ClickHouseRevision::get()}; + NativeBlockOutputStream stream{compress, ClickHouseRevision::get(), block.cloneEmpty()}; writeStringBinary(query_string, out); diff --git a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h index cd9abaa0b89..b7fa9a3aa25 100644 --- a/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h +++ b/dbms/src/Storages/Distributed/DistributedBlockOutputStream.h @@ -36,8 +36,8 @@ public: DistributedBlockOutputStream(StorageDistributed & storage, const ASTPtr & query_ast, const ClusterPtr & cluster_, const Settings & settings_, bool insert_sync_, UInt64 insert_timeout_); + Block getHeader() const override; void write(const Block & block) override; - void writePrefix() override; void writeSuffix() override; diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp index bf1e9a6399d..b5095e79118 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.cpp @@ -3,6 +3,7 @@ #include #include #include +#include #include #include @@ -337,11 +338,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() const size_t limit = std::min(pre_filter.size(), pre_filter_pos + unread_rows_in_current_granule); bool will_read_until_mark = unread_rows_in_current_granule == limit - pre_filter_pos; - UInt8 nonzero = 0; - for (size_t row = pre_filter_pos; row < limit; ++row) - nonzero |= pre_filter[row]; - - if (!nonzero) + if (memoryIsZero(&pre_filter[pre_filter_pos], (limit - pre_filter_pos) * sizeof(pre_filter[0]))) { /// Zero! Prewhere condition is false for all (limit - pre_filter_pos) rows. readRows(); @@ -395,16 +392,21 @@ Block MergeTreeBaseBlockInputStream::readFromPart() post_filter.resize(post_filter_pos); + /// At this point we may have arrays with non-zero offsets but with empty data, + /// as a result of reading components of Nested data structures with no data in filesystem. + /// We must fill these arrays to filter them correctly. + + reader->fillMissingColumns(res, task->ordered_names, task->should_reorder); + /// Filter the columns related to PREWHERE using pre_filter, /// other columns - using post_filter. size_t rows = 0; for (const auto i : ext::range(0, res.columns())) { - auto & col = res.safeGetByPosition(i); + auto & col = res.getByPosition(i); if (col.name == prewhere_column_name && res.columns() > 1) continue; - col.column = - col.column->filter(task->column_name_set.count(col.name) ? post_filter : pre_filter, -1); + col.column = col.column->filter(task->column_name_set.count(col.name) ? post_filter : pre_filter, -1); rows = col.column->size(); } if (task->size_predictor) @@ -412,8 +414,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() /// Replace column with condition value from PREWHERE to a constant. if (!task->remove_prewhere_column) - res.getByName(prewhere_column_name).column = DataTypeUInt8().createColumnConst(rows, UInt64(1)); - + res.getByName(prewhere_column_name).column = DataTypeUInt8().createColumnConst(rows, UInt64(1))->convertToFullColumnIfConst(); } if (res) @@ -470,7 +471,7 @@ Block MergeTreeBaseBlockInputStream::readFromPart() } -void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) +void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) const { const auto rows = block.rows(); @@ -482,17 +483,23 @@ void MergeTreeBaseBlockInputStream::injectVirtualColumns(Block & block) { if (virt_column_name == "_part") { - block.insert(ColumnWithTypeAndName{ - DataTypeString().createColumnConst(rows, task->data_part->name)->convertToFullColumnIfConst(), - std::make_shared(), - virt_column_name}); + ColumnPtr column; + if (rows) + column = DataTypeString().createColumnConst(rows, task->data_part->name)->convertToFullColumnIfConst(); + else + column = DataTypeString().createColumn(); + + block.insert({ column, std::make_shared(), virt_column_name}); } else if (virt_column_name == "_part_index") { - block.insert(ColumnWithTypeAndName{ - DataTypeUInt64().createColumnConst(rows, static_cast(task->part_index_in_query))->convertToFullColumnIfConst(), - std::make_shared(), - virt_column_name}); + ColumnPtr column; + if (rows) + column = DataTypeUInt64().createColumnConst(rows, static_cast(task->part_index_in_query))->convertToFullColumnIfConst(); + else + column = DataTypeUInt64().createColumn(); + + block.insert({ column, std::make_shared(), virt_column_name}); } } } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h index 024f39a619c..6586027f9b6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBaseBlockInputStream.h @@ -42,7 +42,7 @@ protected: Block readFromPart(); - void injectVirtualColumns(Block & block); + void injectVirtualColumns(Block & block) const; protected: MergeTreeData & storage; diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp index fcfb00084cf..13fa4100a70 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.cpp @@ -60,12 +60,31 @@ MergeTreeBlockInputStream::MergeTreeBlockInputStream( << " rows starting from " << all_mark_ranges.front().begin * storage.index_granularity); setTotalRowsApprox(total_rows); + + header = storage.getSampleBlockForColumns(ordered_names); + + /// Types may be different during ALTER (when this stream is used to perform an ALTER). + /// NOTE: We may use similar code to implement non blocking ALTERs. + for (const auto & name_type : data_part->columns) + { + if (header.has(name_type.name)) + { + auto & elem = header.getByName(name_type.name); + if (!elem.type->equals(*name_type.type)) + { + elem.type = name_type.type; + elem.column = elem.type->createColumn(); + } + } + } + + injectVirtualColumns(header); } Block MergeTreeBlockInputStream::getHeader() const { - return storage.getSampleBlockForColumns(ordered_names); + return header; } diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h index 9739cfd49fc..13bd55ba7eb 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockInputStream.h @@ -48,6 +48,7 @@ protected: bool getNewTask() override; private: + Block header; /// Used by Task Names ordered_names; diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp index 46b5f470439..af0a207bafc 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.cpp @@ -6,6 +6,12 @@ namespace DB { +Block MergeTreeBlockOutputStream::getHeader() const +{ + return storage.getSampleBlock(); +} + + void MergeTreeBlockOutputStream::write(const Block & block) { storage.data.delayInsertIfNeeded(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.h index 1ecf621f0f9..64243b6e7bf 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergeTreeBlockOutputStream.h @@ -16,6 +16,7 @@ public: MergeTreeBlockOutputStream(StorageMergeTree & storage_) : storage(storage_) {} + Block getHeader() const override; void write(const Block & block) override; private: diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.cpp b/dbms/src/Storages/MergeTree/MergeTreeData.cpp index 3d95b9408a0..c8c1079f631 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeData.cpp @@ -113,14 +113,12 @@ MergeTreeData::MergeTreeData( { merging_params.check(columns); - if (primary_expr_ast && merging_params.mode == MergingParams::Unsorted) - throw Exception("Primary key cannot be set for UnsortedMergeTree", ErrorCodes::BAD_ARGUMENTS); - if (!primary_expr_ast && merging_params.mode != MergingParams::Unsorted) - throw Exception("Primary key can be empty only for UnsortedMergeTree", ErrorCodes::BAD_ARGUMENTS); + if (!primary_expr_ast) + throw Exception("Primary key cannot be empty", ErrorCodes::BAD_ARGUMENTS); initPrimaryKey(); - if (sampling_expression && (!primary_expr_ast || !primary_key_sample.has(sampling_expression->getColumnName())) + if (sampling_expression && (!primary_key_sample.has(sampling_expression->getColumnName())) && !attach && !settings.compatibility_allow_sampling_expression_not_in_primary_key) /// This is for backward compatibility. throw Exception("Sampling expression must be present in the primary key", ErrorCodes::BAD_ARGUMENTS); @@ -180,22 +178,37 @@ MergeTreeData::MergeTreeData( } -static void checkForAllowedKeyColumns(const ColumnWithTypeAndName & element, const std::string & key_name) +static void checkKeyExpression(const ExpressionActions & expr, const Block & sample_block, const String & key_name) { - const ColumnPtr & column = element.column; - if (column && (column->isColumnConst() || column->isDummy())) - throw Exception{key_name + " key cannot contain constants", ErrorCodes::ILLEGAL_COLUMN}; + for (const ExpressionAction & action : expr.getActions()) + { + if (action.type == ExpressionAction::ARRAY_JOIN) + throw Exception(key_name + " key cannot contain array joins"); - if (element.type->isNullable()) - throw Exception{key_name + " key cannot contain nullable columns", ErrorCodes::ILLEGAL_COLUMN}; + if (action.type == ExpressionAction::APPLY_FUNCTION) + { + IFunctionBase & func = *action.function; + if (!func.isDeterministic()) + throw Exception(key_name + " key cannot contain non-deterministic functions, " + "but contains function " + func.getName(), + ErrorCodes::BAD_ARGUMENTS); + } + } + + for (const ColumnWithTypeAndName & element : sample_block) + { + const ColumnPtr & column = element.column; + if (column && (column->isColumnConst() || column->isDummy())) + throw Exception{key_name + " key cannot contain constants", ErrorCodes::ILLEGAL_COLUMN}; + + if (element.type->isNullable()) + throw Exception{key_name + " key cannot contain nullable columns", ErrorCodes::ILLEGAL_COLUMN}; + } } void MergeTreeData::initPrimaryKey() { - if (!primary_expr_ast) - return; - auto addSortDescription = [](SortDescription & descr, const ASTPtr & expr_ast) { descr.reserve(descr.size() + expr_ast->children.size()); @@ -218,14 +231,9 @@ void MergeTreeData::initPrimaryKey() primary_key_sample = projected_expr->getSampleBlock(); } + checkKeyExpression(*primary_expr, primary_key_sample, "Primary"); + size_t primary_key_size = primary_key_sample.columns(); - - /// A primary key cannot contain constants. It is meaningless. - /// (And also couldn't work because primary key is serialized with method of IDataType that doesn't support constants). - /// Also a primary key must not contain any nullable column. - for (size_t i = 0; i < primary_key_size; ++i) - checkForAllowedKeyColumns(primary_key_sample.getByPosition(i), "Primary"); - primary_key_data_types.resize(primary_key_size); for (size_t i = 0; i < primary_key_size; ++i) primary_key_data_types[i] = primary_key_sample.getByPosition(i).type; @@ -240,8 +248,7 @@ void MergeTreeData::initPrimaryKey() ExpressionAnalyzer(secondary_sort_expr_ast, context, nullptr, getColumnsList()).getActions(true); auto secondary_key_sample = projected_expr->getSampleBlock(); - for (size_t i = 0; i < secondary_key_sample.columns(); ++i) - checkForAllowedKeyColumns(secondary_key_sample.getByPosition(i), "Secondary"); + checkKeyExpression(*secondary_sort_expr, secondary_key_sample, "Secondary"); } } @@ -255,14 +262,11 @@ void MergeTreeData::initPartitionKey() for (const ASTPtr & ast : partition_expr_ast->children) { String col_name = ast->getColumnName(); - partition_expr_columns.emplace_back(col_name); - - const ColumnWithTypeAndName & element = partition_expr->getSampleBlock().getByName(col_name); - checkForAllowedKeyColumns(element, "Partition"); - - partition_expr_column_types.emplace_back(element.type); + partition_key_sample.insert(partition_expr->getSampleBlock().getByName(col_name)); } + checkKeyExpression(*partition_expr, partition_key_sample, "Partition"); + /// Add all columns used in the partition key to the min-max index. const NamesAndTypesList & minmax_idx_columns_with_types = partition_expr->getRequiredColumnsWithTypes(); minmax_idx_expr = std::make_shared(minmax_idx_columns_with_types, context.getSettingsRef()); @@ -402,7 +406,6 @@ String MergeTreeData::MergingParams::getModeName() const case Collapsing: return "Collapsing"; case Summing: return "Summing"; case Aggregating: return "Aggregating"; - case Unsorted: return "Unsorted"; case Replacing: return "Replacing"; case Graphite: return "Graphite"; case VersionedCollapsing: return "VersionedCollapsing"; @@ -1061,14 +1064,17 @@ void MergeTreeData::createConvertExpression(const DataPartPtr & part, const Name if (part && !out_rename_map.empty()) { WriteBufferFromOwnString out; - out << "Will rename "; + out << "Will "; bool first = true; for (const auto & from_to : out_rename_map) { if (!first) out << ", "; first = false; - out << from_to.first << " to " << from_to.second; + if (from_to.second.empty()) + out << "remove " << from_to.first; + else + out << "rename " << from_to.first << " to " << from_to.second; } out << " in part " << part->name; LOG_DEBUG(log, out.str()); @@ -1221,7 +1227,7 @@ MergeTreeData::AlterDataPartTransactionPtr MergeTreeData::alterDataPart( * temporary column name ('converting_column_name') created in 'createConvertExpression' method * will have old name of shared offsets for arrays. */ - MergedColumnOnlyOutputStream out(*this, full_path + part->name + '/', true /* sync */, compression_settings, true /* skip_offsets */); + MergedColumnOnlyOutputStream out(*this, in.getHeader(), full_path + part->name + '/', true /* sync */, compression_settings, true /* skip_offsets */); in.readPrefix(); out.writePrefix(); @@ -2031,7 +2037,7 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context /// Re-parse partition key fields using the information about expected field types. - size_t fields_count = partition_expr_column_types.size(); + size_t fields_count = partition_key_sample.columns(); if (partition_ast.fields_count != fields_count) throw Exception( "Wrong number of fields in the partition expression: " + toString(partition_ast.fields_count) + @@ -2047,12 +2053,8 @@ String MergeTreeData::getPartitionIDFromQuery(const ASTPtr & ast, const Context ReadBufferFromMemory right_paren_buf(")", 1); ConcatReadBuffer buf({&left_paren_buf, &fields_buf, &right_paren_buf}); - Block header; - for (size_t i = 0; i < fields_count; ++i) - header.insert(ColumnWithTypeAndName(partition_expr_column_types[i], partition_expr_columns[i])); - - ValuesRowInputStream input_stream(buf, header, context, /* interpret_expressions = */true); - MutableColumns columns = header.cloneEmptyColumns(); + ValuesRowInputStream input_stream(buf, partition_key_sample, context, /* interpret_expressions = */true); + MutableColumns columns = partition_key_sample.cloneEmptyColumns(); if (!input_stream.read(columns)) throw Exception( diff --git a/dbms/src/Storages/MergeTree/MergeTreeData.h b/dbms/src/Storages/MergeTree/MergeTreeData.h index 87164d687d4..ff2c3987357 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeData.h +++ b/dbms/src/Storages/MergeTree/MergeTreeData.h @@ -78,8 +78,6 @@ namespace ErrorCodes /// column is set, keep the latest row with the maximal version. /// - Summing - sum all numeric columns not contained in the primary key for all rows with the same primary key. /// - Aggregating - merge columns containing aggregate function states for all rows with the same primary key. -/// - Unsorted - during the merge the data is not sorted but merely concatenated; this allows reading the data -/// in the same batches as they were written. /// - Graphite - performs coarsening of historical data for Graphite (a system for quantitative monitoring). /// The MergeTreeData class contains a list of parts and the data structure parameters. @@ -239,7 +237,6 @@ public: Collapsing = 1, Summing = 2, Aggregating = 3, - Unsorted = 4, Replacing = 5, Graphite = 6, VersionedCollapsing = 7, @@ -269,7 +266,7 @@ public: /// Attach the table corresponding to the directory in full_path (must end with /), with the given columns. /// Correctness of names and paths is not checked. /// - /// primary_expr_ast - expression used for sorting; empty for UnsortedMergeTree. + /// primary_expr_ast - expression used for sorting; /// date_column_name - if not empty, the name of the Date column used for partitioning by month. /// Otherwise, partition_expr_ast is used for partitioning. /// require_part_metadata - should checksums.txt and columns.txt exist in the part directory. @@ -442,6 +439,7 @@ public: broken_part_callback(name); } + bool hasPrimaryKey() const { return !primary_sort_descr.empty(); } ExpressionActionsPtr getPrimaryExpression() const { return primary_expr; } ExpressionActionsPtr getSecondarySortExpression() const { return secondary_sort_expr; } /// may return nullptr SortDescription getPrimarySortDescription() const { return primary_sort_descr; } @@ -526,8 +524,7 @@ public: ASTPtr partition_expr_ast; ExpressionActionsPtr partition_expr; - Names partition_expr_columns; - DataTypes partition_expr_column_types; + Block partition_key_sample; ExpressionActionsPtr minmax_idx_expr; Names minmax_idx_columns; diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp index 0bc73afc690..11b43e69ee6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataMerger.cpp @@ -592,7 +592,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart input->setProgressCallback(MergeProgressCallback( merge_entry, sum_input_rows_upper_bound, column_sizes, watch_prev_elapsed, merge_alg)); - if (data.merging_params.mode != MergeTreeData::MergingParams::Unsorted) + if (data.hasPrimaryKey()) src_streams.emplace_back(std::make_shared( std::make_shared(BlockInputStreamPtr(std::move(input)), data.getPrimaryExpression()))); else @@ -642,10 +642,6 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart src_streams, sort_desc, data.merging_params.sign_column, DEFAULT_MERGE_BLOCK_SIZE, false, rows_sources_write_buf.get()); break; - case MergeTreeData::MergingParams::Unsorted: - merged_stream = std::make_unique(src_streams); - break; - default: throw Exception("Unknown mode of operation for MergeTreeData: " + toString(data.merging_params.mode), ErrorCodes::LOGICAL_ERROR); } @@ -736,7 +732,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataMerger::mergePartsToTemporaryPart rows_sources_read_buf.seek(0, 0); ColumnGathererStream column_gathered_stream(column_name, column_part_streams, rows_sources_read_buf); - MergedColumnOnlyOutputStream column_to(data, new_part_tmp_path, false, compression_settings, offset_written); + MergedColumnOnlyOutputStream column_to(data, column_gathered_stream.getHeader(), new_part_tmp_path, false, compression_settings, offset_written); size_t column_elems_written = 0; column_to.writePrefix(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp index 79d5d226f31..dc7018bb4d6 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp @@ -520,7 +520,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read( { RangesInDataPart ranges(part, part_index++); - if (data.merging_params.mode != MergeTreeData::MergingParams::Unsorted) + if (data.hasPrimaryKey()) ranges.ranges = markRangesFromPKRange(part->index, key_condition, settings); else ranges.ranges = MarkRanges{MarkRange{0, part->marks_count}}; @@ -830,9 +830,6 @@ BlockInputStreams MergeTreeDataSelectExecutor::spreadMarkRangesAmongStreamsFinal to_merge, data.getSortDescription(), data.merging_params.sign_column, max_block_size, true); break; - case MergeTreeData::MergingParams::Unsorted: - throw Exception("UnsortedMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); - case MergeTreeData::MergingParams::Graphite: throw Exception("GraphiteMergeTree doesn't support FINAL", ErrorCodes::LOGICAL_ERROR); } diff --git a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp index 3d63966484a..fabbca3a473 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeDataWriter.cpp @@ -81,9 +81,9 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block data.partition_expr->execute(block_copy); ColumnRawPtrs partition_columns; - partition_columns.reserve(data.partition_expr_columns.size()); - for (const String & name : data.partition_expr_columns) - partition_columns.emplace_back(block_copy.getByName(name).column.get()); + partition_columns.reserve(data.partition_key_sample.columns()); + for (const ColumnWithTypeAndName & element : data.partition_key_sample) + partition_columns.emplace_back(block_copy.getByName(element.name).column.get()); PODArray partition_num_to_first_row; IColumn::Selector selector; @@ -103,7 +103,9 @@ BlocksWithPartition MergeTreeDataWriter::splitBlockIntoParts(const Block & block if (partitions_count == 1) { /// A typical case is when there is one partition (you do not need to split anything). - result.emplace_back(std::move(block_copy), get_partition(0)); + /// NOTE: returning a copy of the original block so that calculated partition key columns + /// do not interfere with possible calculated primary key columns of the same name. + result.emplace_back(Block(block), get_partition(0)); return result; } @@ -172,8 +174,8 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa dir.createDirectories(); - /// If you need to calculate some columns to sort, we do it. - if (data.merging_params.mode != MergeTreeData::MergingParams::Unsorted) + /// If we need to calculate some columns to sort. + if (data.hasPrimaryKey()) { data.getPrimaryExpression()->execute(block); auto secondary_sort_expr = data.getSecondarySortExpression(); @@ -188,7 +190,7 @@ MergeTreeData::MutableDataPartPtr MergeTreeDataWriter::writeTempPart(BlockWithPa /// Sort. IColumn::Permutation * perm_ptr = nullptr; IColumn::Permutation perm; - if (data.merging_params.mode != MergeTreeData::MergingParams::Unsorted) + if (data.hasPrimaryKey()) { if (!isAlreadySorted(block, sort_descr)) { diff --git a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h index 4bc660c84f1..a5be3010f95 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartInfo.h +++ b/dbms/src/Storages/MergeTree/MergeTreePartInfo.h @@ -40,6 +40,10 @@ struct MergeTreePartInfo String getPartName() const; String getPartNameV0(DayNum_t left_date, DayNum_t right_date) const; + UInt64 getBlocksCount() const + { + return static_cast(max_block - min_block + 1); + } static MergeTreePartInfo fromPartName(const String & part_name, MergeTreeDataFormatVersion format_version); diff --git a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp index 56de34f9d84..b95916b2164 100644 --- a/dbms/src/Storages/MergeTree/MergeTreePartition.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreePartition.cpp @@ -23,7 +23,7 @@ static ReadBufferFromFile openForReading(const String & path) /// So if you want to change this method, be sure to guarantee compatibility with existing table data. String MergeTreePartition::getID(const MergeTreeData & storage) const { - if (value.size() != storage.partition_expr_columns.size()) + if (value.size() != storage.partition_key_sample.columns()) throw Exception("Invalid partition key size: " + toString(value.size()), ErrorCodes::LOGICAL_ERROR); if (value.empty()) @@ -51,7 +51,7 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const if (i > 0) result += '-'; - if (typeid_cast(storage.partition_expr_column_types[i].get())) + if (typeid_cast(storage.partition_key_sample.getByPosition(i).type.get())) result += toString(DateLUT::instance().toNumYYYYMMDD(DayNum_t(value[i].safeGet()))); else result += applyVisitor(to_string_visitor, value[i]); @@ -79,7 +79,7 @@ String MergeTreePartition::getID(const MergeTreeData & storage) const void MergeTreePartition::serializeTextQuoted(const MergeTreeData & storage, WriteBuffer & out) const { - size_t key_size = storage.partition_expr_column_types.size(); + size_t key_size = storage.partition_key_sample.columns(); if (key_size == 0) { @@ -95,7 +95,7 @@ void MergeTreePartition::serializeTextQuoted(const MergeTreeData & storage, Writ if (i > 0) writeCString(", ", out); - const DataTypePtr & type = storage.partition_expr_column_types[i]; + const DataTypePtr & type = storage.partition_key_sample.getByPosition(i).type; auto column = type->createColumn(); column->insert(value[i]); type->serializeTextQuoted(*column, 0, out); @@ -111,9 +111,9 @@ void MergeTreePartition::load(const MergeTreeData & storage, const String & part return; ReadBufferFromFile file = openForReading(part_path + "partition.dat"); - value.resize(storage.partition_expr_column_types.size()); - for (size_t i = 0; i < storage.partition_expr_column_types.size(); ++i) - storage.partition_expr_column_types[i]->deserializeBinary(value[i], file); + value.resize(storage.partition_key_sample.columns()); + for (size_t i = 0; i < storage.partition_key_sample.columns(); ++i) + storage.partition_key_sample.getByPosition(i).type->deserializeBinary(value[i], file); } void MergeTreePartition::store(const MergeTreeData & storage, const String & part_path, MergeTreeDataPartChecksums & checksums) const @@ -124,7 +124,7 @@ void MergeTreePartition::store(const MergeTreeData & storage, const String & par WriteBufferFromFile out(part_path + "partition.dat"); HashingWriteBuffer out_hashing(out); for (size_t i = 0; i < value.size(); ++i) - storage.partition_expr_column_types[i]->serializeBinary(value[i], out_hashing); + storage.partition_key_sample.getByPosition(i).type->serializeBinary(value[i], out_hashing); out_hashing.next(); checksums.files["partition.dat"].file_size = out_hashing.count(); checksums.files["partition.dat"].file_hash = out_hashing.getHash(); diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp index 4ae50b00b2e..63d763d5939 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.cpp @@ -422,7 +422,6 @@ void MergeTreeReader::fillMissingColumns(Block & res, const Names & ordered_name { /// For a missing column of a nested data structure we must create not a column of empty /// arrays, but a column of arrays of correct length. - /// NOTE: Similar, but slightly different code is present in Block::addDefaults. /// First, collect offset columns for all arrays in the block. OffsetColumns offset_columns; diff --git a/dbms/src/Storages/MergeTree/MergeTreeReader.h b/dbms/src/Storages/MergeTree/MergeTreeReader.h index 6f4c1f60dc2..c2cd086d343 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeReader.h +++ b/dbms/src/Storages/MergeTree/MergeTreeReader.h @@ -44,7 +44,7 @@ public: /// Add columns from ordered_names that are not present in the block. /// Missing columns are added in the order specified by ordered_names. /// If at least one column was added, reorders all columns in the block according to ordered_names. - void fillMissingColumns(Block & res, const Names & ordered_names, const bool always_reorder = false); + void fillMissingColumns(Block & res, const Names & ordered_names, const bool always_reorder); private: class Stream diff --git a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp index ec961f28a99..2a749513154 100644 --- a/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergeTreeThreadBlockInputStream.cpp @@ -40,7 +40,9 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream( Block MergeTreeThreadBlockInputStream::getHeader() const { - return pool->getHeader(); + auto res = pool->getHeader(); + injectVirtualColumns(res); + return res; }; diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp index 9c680b814d5..d3162810a1c 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.cpp @@ -293,7 +293,7 @@ void MergedBlockOutputStream::writeSuffixAndFinalizePart( if (additional_column_checksums) checksums = std::move(*additional_column_checksums); - if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted) + if (index_stream) { index_stream->next(); checksums.files["primary.idx"].file_size = index_stream->count(); @@ -354,7 +354,7 @@ void MergedBlockOutputStream::init() { Poco::File(part_path).createDirectories(); - if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted) + if (storage.hasPrimaryKey()) { index_file_stream = std::make_unique( part_path + "primary.idx", DBMS_DEFAULT_BUFFER_SIZE, O_TRUNC | O_CREAT | O_WRONLY); @@ -443,7 +443,7 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm /// Write index. The index contains Primary Key value for each `index_granularity` row. for (size_t i = index_offset; i < rows; i += storage.index_granularity) { - if (storage.merging_params.mode != MergeTreeData::MergingParams::Unsorted) + if (storage.hasPrimaryKey()) { for (size_t j = 0, size = primary_columns.size(); j < size; ++j) { @@ -465,12 +465,12 @@ void MergedBlockOutputStream::writeImpl(const Block & block, const IColumn::Perm /// Implementation of MergedColumnOnlyOutputStream. MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( - MergeTreeData & storage_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_) + MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_) : IMergedBlockOutputStream( storage_, storage_.context.getSettings().min_compress_block_size, storage_.context.getSettings().max_compress_block_size, compression_settings, storage_.context.getSettings().min_bytes_to_use_direct_io), - part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_) + header(header_), part_path(part_path_), sync(sync_), skip_offsets(skip_offsets_) { } diff --git a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h index 60196c3ecdd..4b83f959991 100644 --- a/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedBlockOutputStream.h @@ -105,6 +105,8 @@ public: std::string getPartPath() const; + Block getHeader() const override { return storage.getSampleBlock(); } + /// If the data is pre-sorted. void write(const Block & block) override; @@ -149,13 +151,15 @@ class MergedColumnOnlyOutputStream final : public IMergedBlockOutputStream { public: MergedColumnOnlyOutputStream( - MergeTreeData & storage_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_); + MergeTreeData & storage_, const Block & header_, String part_path_, bool sync_, CompressionSettings compression_settings, bool skip_offsets_); + Block getHeader() const override { return header; } void write(const Block & block) override; void writeSuffix() override; MergeTreeData::DataPart::Checksums writeSuffixAndGetChecksums(); private: + Block header; String part_path; bool initialized = false; diff --git a/dbms/src/Storages/MergeTree/PKCondition.cpp b/dbms/src/Storages/MergeTree/PKCondition.cpp index 2852e9f5551..a7198d5625d 100644 --- a/dbms/src/Storages/MergeTree/PKCondition.cpp +++ b/dbms/src/Storages/MergeTree/PKCondition.cpp @@ -462,9 +462,10 @@ void PKCondition::getPKTuplePositionMapping( { MergeTreeSetIndex::PKTuplePositionMapping index_mapping; index_mapping.tuple_index = tuple_index; + DataTypePtr data_type; if (isPrimaryKeyPossiblyWrappedByMonotonicFunctions( node, context, index_mapping.pk_index, - index_mapping.data_type, index_mapping.functions)) + data_type, index_mapping.functions)) { indexes_mapping.push_back(index_mapping); if (out_primary_key_column_num < index_mapping.pk_index) @@ -999,7 +1000,7 @@ bool PKCondition::mayBeTrueInRangeImpl(const std::vector & key_ranges, co PreparedSets::const_iterator it = prepared_sets.find(args[1].get()); if (in_func && it != prepared_sets.end()) { - rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(key_ranges)); + rpn_stack.emplace_back(element.set_index->mayBeTrueInRange(key_ranges, data_types)); if (element.function == RPNElement::FUNCTION_NOT_IN_SET) { rpn_stack.back() = !rpn_stack.back(); diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp index a36a400aea5..e34f0f68b6d 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.cpp @@ -39,6 +39,12 @@ ReplicatedMergeTreeBlockOutputStream::ReplicatedMergeTreeBlockOutputStream( } +Block ReplicatedMergeTreeBlockOutputStream::getHeader() const +{ + return storage.getSampleBlock(); +} + + /// Allow to verify that the session in ZooKeeper is still alive. static void assertSessionIsNotExpired(zkutil::ZooKeeperPtr & zookeeper) { diff --git a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h index 12d8f97a1d7..29ca8657038 100644 --- a/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h +++ b/dbms/src/Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h @@ -25,6 +25,7 @@ public: ReplicatedMergeTreeBlockOutputStream(StorageReplicatedMergeTree & storage_, size_t quorum_, size_t quorum_timeout_ms_, bool deduplicate_); + Block getHeader() const override; void write(const Block & block) override; /// For ATTACHing existing data on filesystem. diff --git a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp index 2c1f6ddf270..89787fca4f0 100644 --- a/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp +++ b/dbms/src/Storages/MergeTree/registerStorageMergeTree.cpp @@ -586,6 +586,10 @@ static StoragePtr create(const StorageFactory::Arguments & args) if (args.storage_def->order_by) primary_expr_list = extractKeyExpressionList(*args.storage_def->order_by); + else + throw Exception("You must provide an ORDER BY expression in the table definition. " + "If you don't want this table to be sorted, use ORDER BY tuple()", + ErrorCodes::BAD_ARGUMENTS); if (args.storage_def->sample_by) sampling_expression = args.storage_def->sample_by->ptr(); diff --git a/dbms/src/Storages/StorageBuffer.cpp b/dbms/src/Storages/StorageBuffer.cpp index ec4f7b498da..1b32ba197ff 100644 --- a/dbms/src/Storages/StorageBuffer.cpp +++ b/dbms/src/Storages/StorageBuffer.cpp @@ -44,7 +44,6 @@ namespace DB namespace ErrorCodes { extern const int INFINITE_LOOP; - extern const int BLOCKS_HAVE_DIFFERENT_STRUCTURE; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; } @@ -150,9 +149,7 @@ static void appendBlock(const Block & from, Block & to) if (!to) throw Exception("Cannot append to empty block", ErrorCodes::LOGICAL_ERROR); - if (!blocksHaveEqualStructure(from, to)) - throw Exception("Cannot append block to buffer: block has different structure. " - "Block: " + from.dumpStructure() + ", Buffer: " + to.dumpStructure(), ErrorCodes::BLOCKS_HAVE_DIFFERENT_STRUCTURE); + assertBlocksHaveEqualStructure(from, to, "Buffer"); from.checkNumberOfRows(); to.checkNumberOfRows(); @@ -208,6 +205,8 @@ class BufferBlockOutputStream : public IBlockOutputStream public: explicit BufferBlockOutputStream(StorageBuffer & storage_) : storage(storage_) {} + Block getHeader() const override { return storage.getSampleBlock(); } + void write(const Block & block) override { if (!block) diff --git a/dbms/src/Storages/StorageCatBoostPool.cpp b/dbms/src/Storages/StorageCatBoostPool.cpp index d484158b018..74a40372b79 100644 --- a/dbms/src/Storages/StorageCatBoostPool.cpp +++ b/dbms/src/Storages/StorageCatBoostPool.cpp @@ -36,7 +36,7 @@ public: String getName() const override { - return "CatBoostDatasetBlockInputStream"; + return "CatBoostDataset"; } Block readImpl() override diff --git a/dbms/src/Storages/StorageDistributed.cpp b/dbms/src/Storages/StorageDistributed.cpp index e6683d6218b..b191f598a0e 100644 --- a/dbms/src/Storages/StorageDistributed.cpp +++ b/dbms/src/Storages/StorageDistributed.cpp @@ -1,4 +1,5 @@ #include +#include #include @@ -190,16 +191,11 @@ BlockInputStreams StorageDistributed::read( const auto & modified_query_ast = rewriteSelectQuery( query_info.query, remote_database, remote_table); - Tables external_tables; - - if (settings.global_subqueries_method == GlobalSubqueriesMethod::PUSH) - external_tables = context.getExternalTables(); - - Block header = InterpreterSelectQuery(query_info.query, context, processed_stage, 0, - std::make_shared(getSampleBlockForColumns(column_names))).execute().in->getHeader(); + Block header = materializeBlock(InterpreterSelectQuery(query_info.query, context, processed_stage, 0, + std::make_shared(getSampleBlockForColumns(column_names))).execute().in->getHeader()); ClusterProxy::SelectStreamFactory select_stream_factory( - header, processed_stage, QualifiedTableName{remote_database, remote_table}, external_tables); + header, processed_stage, QualifiedTableName{remote_database, remote_table}, context.getExternalTables()); return ClusterProxy::executeQuery( select_stream_factory, cluster, modified_query_ast, context, settings); diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 1bbe9d6d00c..78fb3990978 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -184,7 +184,6 @@ BlockInputStreams StorageFile::read( class StorageFileBlockOutputStream : public IBlockOutputStream { public: - explicit StorageFileBlockOutputStream(StorageFile & storage_) : storage(storage_), lock(storage.rwlock) { @@ -205,6 +204,8 @@ public: writer = FormatFactory().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); } + Block getHeader() const override { return storage.getSampleBlock(); } + void write(const Block & block) override { writer->write(block); diff --git a/dbms/src/Storages/StorageLog.cpp b/dbms/src/Storages/StorageLog.cpp index 443510fea42..711d48621f0 100644 --- a/dbms/src/Storages/StorageLog.cpp +++ b/dbms/src/Storages/StorageLog.cpp @@ -127,6 +127,7 @@ public: } } + Block getHeader() const override { return storage.getSampleBlock(); } void write(const Block & block) override; void writeSuffix() override; diff --git a/dbms/src/Storages/StorageMemory.cpp b/dbms/src/Storages/StorageMemory.cpp index 89ce474b065..96af6a9a138 100644 --- a/dbms/src/Storages/StorageMemory.cpp +++ b/dbms/src/Storages/StorageMemory.cpp @@ -61,6 +61,8 @@ class MemoryBlockOutputStream : public IBlockOutputStream public: explicit MemoryBlockOutputStream(StorageMemory & storage_) : storage(storage_) {} + Block getHeader() const override { return storage.getSampleBlock(); } + void write(const Block & block) override { storage.check(block, true); diff --git a/dbms/src/Storages/StorageMerge.cpp b/dbms/src/Storages/StorageMerge.cpp index 39be47eab9b..f503606a742 100644 --- a/dbms/src/Storages/StorageMerge.cpp +++ b/dbms/src/Storages/StorageMerge.cpp @@ -137,7 +137,6 @@ BlockInputStreams StorageMerge::read( const unsigned num_streams) { BlockInputStreams res; - Block header = getSampleBlockForColumns(column_names); Names virt_column_names, real_column_names; for (const auto & it : column_names) @@ -167,7 +166,6 @@ BlockInputStreams StorageMerge::read( if (!virt_column_names.empty()) { VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context); - auto values = VirtualColumnUtils::extractSingleValueFromBlock(virtual_columns_block, "_table"); /// Remove unused tables from the list @@ -180,7 +178,10 @@ BlockInputStreams StorageMerge::read( Context modified_context = context; modified_context.getSettingsRef().optimize_move_to_prewhere = false; + Block header = getSampleBlockForColumns(real_column_names); + size_t tables_count = selected_tables.size(); + size_t curr_table_number = 0; for (auto it = selected_tables.begin(); it != selected_tables.end(); ++it, ++curr_table_number) { @@ -191,7 +192,7 @@ BlockInputStreams StorageMerge::read( if (real_column_names.size() == 0) real_column_names.push_back(ExpressionActions::getSmallestColumn(table->getColumnsList())); - /// Substitute virtual column for its value + /// Substitute virtual column for its value when querying tables. ASTPtr modified_query_ast = query->clone(); VirtualColumnUtils::rewriteEntityInAst(modified_query_ast, "_table", table->getTableName()); @@ -219,6 +220,12 @@ BlockInputStreams StorageMerge::read( throw Exception("Source tables for Merge table are processing data up to different stages", ErrorCodes::INCOMPATIBLE_SOURCE_TABLES); + /// The table may return excessive columns if we query only its virtual column. + /// We filter excessive columns. This is done only if query was not processed more than FetchColumns. + if (processed_stage_in_source_table == QueryProcessingStage::FetchColumns) + for (auto & stream : source_streams) + stream = std::make_shared(stream, real_column_names, true); + /// Subordinary tables could have different but convertible types, like numeric types of different width. /// We must return streams with structure equals to structure of Merge table. for (auto & stream : source_streams) @@ -248,6 +255,10 @@ BlockInputStreams StorageMerge::read( throw Exception("Source tables for Merge table are processing data up to different stages", ErrorCodes::INCOMPATIBLE_SOURCE_TABLES); + if (processed_stage_in_source_table == QueryProcessingStage::FetchColumns) + for (auto & stream : streams) + stream = std::make_shared(stream, real_column_names, true); + auto stream = streams.empty() ? std::make_shared(header) : streams.front(); if (!streams.empty()) { @@ -262,14 +273,10 @@ BlockInputStreams StorageMerge::read( stream->addTableLock(table_lock); for (auto & virtual_column : virt_column_names) - { if (virtual_column == "_table") - { for (auto & stream : source_streams) stream = std::make_shared>( stream, std::make_shared(), table->getTableName(), "_table"); - } - } res.insert(res.end(), source_streams.begin(), source_streams.end()); } diff --git a/dbms/src/Storages/StorageNull.h b/dbms/src/Storages/StorageNull.h index e413ff2d930..ef3f0f6fcd1 100644 --- a/dbms/src/Storages/StorageNull.h +++ b/dbms/src/Storages/StorageNull.h @@ -33,7 +33,7 @@ public: BlockOutputStreamPtr write(const ASTPtr &, const Settings &) override { - return std::make_shared(); + return std::make_shared(getSampleBlock()); } void rename(const String & /*new_path_to_db*/, const String & /*new_database_name*/, const String & new_table_name) override diff --git a/dbms/src/Storages/StorageReplicatedMergeTree.cpp b/dbms/src/Storages/StorageReplicatedMergeTree.cpp index 14cc102f609..2715eb8dae0 100644 --- a/dbms/src/Storages/StorageReplicatedMergeTree.cpp +++ b/dbms/src/Storages/StorageReplicatedMergeTree.cpp @@ -741,6 +741,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) /// Which local parts to added into ZK. MergeTreeData::DataPartsVector parts_to_add; + UInt64 parts_to_add_rows = 0; /// Which parts should be taken from other replicas. Strings parts_to_fetch; @@ -756,6 +757,7 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) { parts_to_add.push_back(containing); unexpected_parts.erase(containing); + parts_to_add_rows += containing->rows_count; } } else @@ -768,21 +770,53 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) for (const String & name : parts_to_fetch) expected_parts.erase(name); + /** To check the adequacy, for the parts that are in the FS, but not in ZK, we will only consider not the most recent parts. * Because unexpected new parts usually arise only because they did not have time to enroll in ZK with a rough restart of the server. * It also occurs from deduplicated parts that did not have time to retire. */ size_t unexpected_parts_nonnew = 0; + UInt64 unexpected_parts_nonnew_rows = 0; + UInt64 unexpected_parts_rows = 0; for (const auto & part : unexpected_parts) + { if (part->info.level > 0) + { ++unexpected_parts_nonnew; + unexpected_parts_nonnew_rows += part->rows_count; + } - String sanity_report = "There are " - + toString(unexpected_parts.size()) + " unexpected parts (" - + toString(unexpected_parts_nonnew) + " of them is not just-written), " - + toString(parts_to_add.size()) + " unexpectedly merged parts, " - + toString(expected_parts.size()) + " missing obsolete parts, " - + toString(parts_to_fetch.size()) + " missing parts"; + unexpected_parts_rows += part->rows_count; + } + + + /// Additional helpful statistics + auto get_blocks_count_in_data_part = [&] (const String & part_name) -> UInt64 + { + MergeTreePartInfo part_info; + if (MergeTreePartInfo::tryParsePartName(part_name, &part_info, data.format_version)) + return part_info.getBlocksCount(); + + LOG_ERROR(log, "Unexpected part name: " << part_name); + return 0; + }; + + UInt64 parts_to_fetch_blocks = 0; + for (const String & name : parts_to_fetch) + parts_to_fetch_blocks += get_blocks_count_in_data_part(name); + + UInt64 expected_parts_blocks = 0; + for (const String & name : expected_parts) + expected_parts_blocks += get_blocks_count_in_data_part(name); + + + std::stringstream sanity_report; + sanity_report << "There are " + << unexpected_parts.size() << " unexpected parts with " << unexpected_parts_rows << " rows (" + << unexpected_parts_nonnew << " of them is not just-written with " << unexpected_parts_rows << " rows), " + << parts_to_add.size() << " unexpectedly merged parts with " << parts_to_add_rows << " rows, " + << expected_parts.size() << " missing obsolete parts (with " << expected_parts_blocks << " blocks), " + << parts_to_fetch.size() << " missing parts (with " << parts_to_fetch_blocks << " blocks)."; /** We can automatically synchronize data, * if the ratio of the total number of errors to the total number of parts (minimum - on the local filesystem or in ZK) @@ -793,17 +827,28 @@ void StorageReplicatedMergeTree::checkParts(bool skip_sanity_checks) * In this case, the protection mechanism does not allow the server to start. */ - size_t min_parts_local_or_expected = std::min(expected_parts_vec.size(), parts.size()); - size_t total_difference = parts_to_add.size() + unexpected_parts_nonnew + parts_to_fetch.size(); + UInt64 total_rows_on_filesystem = 0; + for (const auto & part : parts) + total_rows_on_filesystem += part->rows_count; - bool insane = total_difference > min_parts_local_or_expected * data.settings.replicated_max_ratio_of_wrong_parts; + UInt64 total_suspicious_rows = parts_to_add_rows + unexpected_parts_rows; + UInt64 total_suspicious_rows_no_new = parts_to_add_rows + unexpected_parts_nonnew_rows; + + bool insane = total_suspicious_rows > total_rows_on_filesystem * data.settings.replicated_max_ratio_of_wrong_parts; if (insane && !skip_sanity_checks) - throw Exception("The local set of parts of table " + getTableName() + " doesn't look like the set of parts in ZooKeeper. " - + sanity_report, ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS); + { + std::stringstream why; + why << "The local set of parts of table " << database_name << "." << table_name << " doesn't look like the set of parts " + << "in ZooKeeper: " + << formatReadableQuantity(total_suspicious_rows) << " rows of " << formatReadableQuantity(total_rows_on_filesystem) + << " total rows in filesystem are suspicious."; - if (total_difference > 0) - LOG_WARNING(log, sanity_report); + throw Exception(why.str() + " " + sanity_report.str(), ErrorCodes::TOO_MANY_UNEXPECTED_DATA_PARTS); + } + + if (total_suspicious_rows_no_new > 0) + LOG_WARNING(log, sanity_report.str()); /// Add information to the ZK about the parts that cover the missing parts. for (const MergeTreeData::DataPartPtr & part : parts_to_add) @@ -3214,7 +3259,7 @@ void StorageReplicatedMergeTree::sendRequestToLeaderReplica(const ASTPtr & query "", "", timeouts, "ClickHouse replica"); RemoteBlockInputStream stream(connection, formattedAST(new_query), {}, context, &settings); - NullBlockOutputStream output; + NullBlockOutputStream output({}); copyData(stream, output); return; diff --git a/dbms/src/Storages/StorageSet.cpp b/dbms/src/Storages/StorageSet.cpp index 498e475a465..bdbfca46d57 100644 --- a/dbms/src/Storages/StorageSet.cpp +++ b/dbms/src/Storages/StorageSet.cpp @@ -33,6 +33,7 @@ public: SetOrJoinBlockOutputStream(StorageSetOrJoinBase & table_, const String & backup_path_, const String & backup_tmp_path_, const String & backup_file_name_); + Block getHeader() const override { return table.getSampleBlock(); } void write(const Block & block) override; void writeSuffix() override; @@ -54,7 +55,7 @@ SetOrJoinBlockOutputStream::SetOrJoinBlockOutputStream(StorageSetOrJoinBase & ta backup_file_name(backup_file_name_), backup_buf(backup_tmp_path + backup_file_name), compressed_backup_buf(backup_buf), - backup_stream(compressed_backup_buf) + backup_stream(compressed_backup_buf, 0, table.getSampleBlock()) { } diff --git a/dbms/src/Storages/StorageStripeLog.cpp b/dbms/src/Storages/StorageStripeLog.cpp index 1d2d31e27bb..5f746dc246c 100644 --- a/dbms/src/Storages/StorageStripeLog.cpp +++ b/dbms/src/Storages/StorageStripeLog.cpp @@ -55,21 +55,20 @@ public: : storage(storage_), max_read_buffer_size(max_read_buffer_size_), index(index_), index_begin(index_begin_), index_end(index_end_) { + if (index_begin != index_end) + { + for (const auto & column : index_begin->columns) + { + auto type = DataTypeFactory::instance().get(column.type); + header.insert(ColumnWithTypeAndName{ type, column.name }); + } + } } String getName() const override { return "StripeLog"; } Block getHeader() const override { - if (index_begin == index_end) - return {}; - - Block header; - for (const auto & column : index_begin->columns) - { - auto type = DataTypeFactory::instance().get(column.type); - header.insert({ type->createColumn(), type, column.name }); - } return header; }; @@ -102,6 +101,7 @@ private: std::shared_ptr index; IndexForNativeFormat::Blocks::const_iterator index_begin; IndexForNativeFormat::Blocks::const_iterator index_end; + Block header; /** optional - to create objects only on first reading * and delete objects (release buffers) after the source is exhausted @@ -136,7 +136,7 @@ public: data_out(data_out_compressed, CompressionSettings(CompressionMethod::LZ4), storage.max_compress_block_size), index_out_compressed(storage.full_path() + "index.mrk", INDEX_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT), index_out(index_out_compressed), - block_out(data_out, 0, &index_out, Poco::File(storage.full_path() + "data.bin").getSize()) + block_out(data_out, 0, storage.getSampleBlock(), &index_out, Poco::File(storage.full_path() + "data.bin").getSize()) { } @@ -152,6 +152,8 @@ public: } } + Block getHeader() const override { return storage.getSampleBlock(); } + void write(const Block & block) override { block_out.write(block); diff --git a/dbms/src/Storages/StorageTinyLog.cpp b/dbms/src/Storages/StorageTinyLog.cpp index 957d1ca9aff..4ed7d6dfff2 100644 --- a/dbms/src/Storages/StorageTinyLog.cpp +++ b/dbms/src/Storages/StorageTinyLog.cpp @@ -118,6 +118,8 @@ public: } } + Block getHeader() const override { return storage.getSampleBlock(); } + void write(const Block & block) override; void writeSuffix() override; diff --git a/dbms/src/Storages/VirtualColumnUtils.cpp b/dbms/src/Storages/VirtualColumnUtils.cpp index 5864d35da48..29409ce715e 100644 --- a/dbms/src/Storages/VirtualColumnUtils.cpp +++ b/dbms/src/Storages/VirtualColumnUtils.cpp @@ -133,11 +133,11 @@ static ASTPtr buildWhereExpression(const ASTs & functions) return new_query; } -bool filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context) +void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context) { const ASTSelectQuery & select = typeid_cast(*query); if (!select.where_expression && !select.prewhere_expression) - return false; + return; NameSet columns; for (const auto & it : block.getNamesAndTypesList()) @@ -151,7 +151,7 @@ bool filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c extractFunctions(select.prewhere_expression, columns, functions); ASTPtr expression_ast = buildWhereExpression(functions); if (!expression_ast) - return false; + return; /// Let's analyze and calculate the expression. ExpressionAnalyzer analyzer(expression_ast, context, {}, block.getNamesAndTypesList()); @@ -165,16 +165,11 @@ bool filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c filter_column = converted; const IColumn::Filter & filter = typeid_cast(*filter_column).getData(); - if (countBytesInFilter(filter) == 0) - return false; - for (size_t i = 0; i < block.columns(); ++i) { ColumnPtr & column = block.safeGetByPosition(i).column; column = column->filter(filter, -1); } - - return true; } } diff --git a/dbms/src/Storages/VirtualColumnUtils.h b/dbms/src/Storages/VirtualColumnUtils.h index bfbd44cf2c8..a1e1db4f04c 100644 --- a/dbms/src/Storages/VirtualColumnUtils.h +++ b/dbms/src/Storages/VirtualColumnUtils.h @@ -30,7 +30,7 @@ void rewriteEntityInAst(ASTPtr ast, const String & column_name, const Field & va /// Leave in the block only the rows that fit under the WHERE clause and the PREWHERE clause of the query. /// Only elements of the outer conjunction are considered, depending only on the columns present in the block. /// Returns true if at least one row is discarded. -bool filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context); +void filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & context); /// Extract from the input stream a set of `name` column values template diff --git a/dbms/src/Storages/tests/hit_log.cpp b/dbms/src/Storages/tests/hit_log.cpp index bd1777db18f..3dd75206a2b 100644 --- a/dbms/src/Storages/tests/hit_log.cpp +++ b/dbms/src/Storages/tests/hit_log.cpp @@ -134,7 +134,7 @@ try BlockInputStreamPtr in = table->read(column_names, {}, Context::createGlobal(), stage, 8192, 1)[0]; RowOutputStreamPtr out_ = std::make_shared(out_buf, sample); - BlockOutputStreamFromRowOutputStream out(out_); + BlockOutputStreamFromRowOutputStream out(out_, sample); copyData(*in, out); } diff --git a/dbms/src/Storages/tests/storage_log.cpp b/dbms/src/Storages/tests/storage_log.cpp index 70c73d8c0b5..6d9cb5d0def 100644 --- a/dbms/src/Storages/tests/storage_log.cpp +++ b/dbms/src/Storages/tests/storage_log.cpp @@ -93,7 +93,7 @@ try LimitBlockInputStream in_limit(in, 10, 0); RowOutputStreamPtr output_ = std::make_shared(out_buf, sample); - BlockOutputStreamFromRowOutputStream output(output_); + BlockOutputStreamFromRowOutputStream output(output_, sample); copyData(in_limit, output); } diff --git a/dbms/src/Storages/tests/system_numbers.cpp b/dbms/src/Storages/tests/system_numbers.cpp index d2d0f9785b2..93e31939555 100644 --- a/dbms/src/Storages/tests/system_numbers.cpp +++ b/dbms/src/Storages/tests/system_numbers.cpp @@ -31,7 +31,7 @@ try LimitBlockInputStream input(table->read(column_names, {}, Context::createGlobal(), stage, 10, 1)[0], 10, 96); RowOutputStreamPtr output_ = std::make_shared(out_buf, sample); - BlockOutputStreamFromRowOutputStream output(output_); + BlockOutputStreamFromRowOutputStream output(output_, sample); copyData(input, output); diff --git a/dbms/tests/integration/test_random_inserts/test.sh b/dbms/tests/integration/test_random_inserts/test.sh index 76f1b05e61e..9022f307d56 100755 --- a/dbms/tests/integration/test_random_inserts/test.sh +++ b/dbms/tests/integration/test_random_inserts/test.sh @@ -2,9 +2,9 @@ #set -e [[ -n "$1" ]] && host="$1" || host="localhost" -[[ -n "$2" ]] && min_timestamp="$2" || min_timestamp=$(( $(date +%s) - 10 )) -[[ -n "$3" ]] && max_timestamp="$3" || max_timestamp=$(( $(date +%s) + 10 )) -[[ -n "$4" ]] && iters_per_timestamp="$4" || iters_per_timestamp=1 +[[ -n "$2" ]] && min_timestamp="$2" || min_timestamp=$(( $(date +%s) - 60 )) +[[ -n "$3" ]] && max_timestamp="$3" || max_timestamp=$(( $(date +%s) + 60 )) +[[ -n "$4" ]] && iters_per_timestamp="$4" || iters_per_timestamp=5 timestamps=`seq $min_timestamp $max_timestamp` diff --git a/dbms/tests/queries/0_stateless/00294_enums.sql b/dbms/tests/queries/0_stateless/00294_enums.sql index 13c8302fbd4..24a75126990 100644 --- a/dbms/tests/queries/0_stateless/00294_enums.sql +++ b/dbms/tests/queries/0_stateless/00294_enums.sql @@ -2,9 +2,9 @@ set max_threads = 1; drop table if exists test.enums; create table test.enums ( - d Date default '2015-12-29', k default 0, - e Enum8('world' = 2, 'hello' = 1), sign Enum8('minus' = -1, 'plus' = 1), - letter Enum16('a' = 0, 'b' = 1, 'c' = 2, '*' = -256) + d Date default '2015-12-29', k default 0, + e Enum8('world' = 2, 'hello' = 1), sign Enum8('minus' = -1, 'plus' = 1), + letter Enum16('a' = 0, 'b' = 1, 'c' = 2, '*' = -256) ) engine = MergeTree(d, k, 1); desc table test.enums; @@ -21,16 +21,16 @@ select * from test.enums ORDER BY _part; -- expand `e` and `sign` from Enum8 to Enum16 without changing values, change values of `letter` without changing type alter table test.enums - modify column e Enum16('world' = 2, 'hello' = 1, '!' = 3), - modify column sign Enum16('minus' = -1, 'plus' = 1), - modify column letter Enum16('a' = 0, 'b' = 1, 'c' = 2, 'no letter' = -256); + modify column e Enum16('world' = 2, 'hello' = 1, '!' = 3), + modify column sign Enum16('minus' = -1, 'plus' = 1), + modify column letter Enum16('a' = 0, 'b' = 1, 'c' = 2, 'no letter' = -256); desc table test.enums; select * from test.enums ORDER BY _part; alter table test.enums - modify column e Enum8('world' = 2, 'hello' = 1, '!' = 3), - modify column sign Enum8('minus' = -1, 'plus' = 1); + modify column e Enum8('world' = 2, 'hello' = 1, '!' = 3), + modify column sign Enum8('minus' = -1, 'plus' = 1); desc table test.enums; diff --git a/dbms/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql b/dbms/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql index f29d897017a..d26e947b54f 100644 --- a/dbms/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql +++ b/dbms/tests/queries/0_stateless/00563_insert_into_remote_and_zookeeper.sql @@ -1,6 +1,6 @@ -- Check that settings are correctly passed through Distributed table DROP TABLE IF EXISTS test.simple; -CREATE TABLE test.simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse_test/tables/test/simple', '1') ORDER BY d; +CREATE TABLE test.simple (d Int8) ENGINE = ReplicatedMergeTree('/clickhouse/test/tables/test/simple', '1') ORDER BY d; -- TODO: replace '127.0.0.2' -> '127.0.0.1' after a fix INSERT INTO TABLE FUNCTION remote('127.0.0.2', 'test', 'simple') VALUES (1); diff --git a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh index 27db57e8469..7db122dcbb7 100755 --- a/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh +++ b/dbms/tests/queries/0_stateless/00568_compile_catch_throw.sh @@ -6,8 +6,8 @@ SETTINGS="--compile=1 --min_count_to_compile=0 --max_threads=1 --max_memory_usag output=$($CLICKHOUSE_CLIENT -q "SELECT length(groupArray(number)) FROM (SELECT * FROM system.numbers LIMIT 1000000)" $SETTINGS 2>&1) [[ $? -eq 0 ]] && echo "Expected non-zero RC" -if ! echo "$output" | grep -Fc 'Memory limit (for query) exceeded' ; then - echo -e 'There is no expected exception "Memory limit (for query) exceeded: would use...". Got:' "\n$output" +if ! echo "$output" | grep -Fc -e 'Memory limit (for query) exceeded' -e 'Cannot compile code' ; then + echo -e 'There is no expected exception "Memory limit (for query) exceeded: would use..." or "Cannot compile code..."' "Whereas got:\n$output" fi $CLICKHOUSE_CLIENT -q "SELECT 1" diff --git a/dbms/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.reference b/dbms/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.reference new file mode 100644 index 00000000000..d00491fd7e5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.reference @@ -0,0 +1 @@ +1 diff --git a/dbms/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.sql b/dbms/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.sql new file mode 100644 index 00000000000..8fc5b3faf16 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00575_merge_and_index_with_function_in_in.sql @@ -0,0 +1,9 @@ +DROP TABLE IF EXISTS test.t; + +create table test.t(d Date) engine MergeTree(d, d, 8192); + +insert into test.t values ('2018-02-20'); + +select count() from test.t where toDayOfWeek(d) in (2); + +DROP TABLE test.t; diff --git a/dbms/tests/queries/0_stateless/00576_nested_and_prewhere.reference b/dbms/tests/queries/0_stateless/00576_nested_and_prewhere.reference new file mode 100644 index 00000000000..abc233c46f4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00576_nested_and_prewhere.reference @@ -0,0 +1,15 @@ +[0] +[0,0,0] +[0,0,0,0,0] +[0,0,0,0,0,0,0] +[0,0,0,0,0,0,0,0,0] +[0] +[0,2,4] +[0,2,4,6,8] +[0,2,4,6,8,10,12] +[0,2,4,6,8,10,12,14,16] +[0] [0] +[0,1,2] [0,2,4] +[0,1,2,3,4] [0,2,4,6,8] +[0,1,2,3,4,5,6] [0,2,4,6,8,10,12] +[0,1,2,3,4,5,6,7,8] [0,2,4,6,8,10,12,14,16] diff --git a/dbms/tests/queries/0_stateless/00576_nested_and_prewhere.sql b/dbms/tests/queries/0_stateless/00576_nested_and_prewhere.sql new file mode 100644 index 00000000000..5483086e8f6 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00576_nested_and_prewhere.sql @@ -0,0 +1,13 @@ +DROP TABLE IF EXISTS test.nested; + +CREATE TABLE test.nested (x UInt64, filter UInt8, n Nested(a UInt64)) ENGINE = MergeTree ORDER BY x; +INSERT INTO test.nested SELECT number, number % 2, range(number % 10) FROM system.numbers LIMIT 100000; + +ALTER TABLE test.nested ADD COLUMN n.b Array(UInt64); +SELECT DISTINCT n.b FROM test.nested PREWHERE filter; + +ALTER TABLE test.nested ADD COLUMN n.c Array(UInt64) DEFAULT arrayMap(x -> x * 2, n.a); +SELECT DISTINCT n.c FROM test.nested PREWHERE filter; +SELECT DISTINCT n.a, n.c FROM test.nested PREWHERE filter; + +DROP TABLE test.nested; diff --git a/dbms/tests/queries/0_stateless/00577_full_join_segfault.reference b/dbms/tests/queries/0_stateless/00577_full_join_segfault.reference new file mode 100644 index 00000000000..e91c16f7902 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00577_full_join_segfault.reference @@ -0,0 +1,8 @@ +0 hello 123 0 +1 0 hello 456 +0 hello 123 +1 0 +0 hello 123 \N \N +1 \N \N hello 456 +0 hello 123 +1 \N \N diff --git a/dbms/tests/queries/0_stateless/00577_full_join_segfault.sql b/dbms/tests/queries/0_stateless/00577_full_join_segfault.sql new file mode 100644 index 00000000000..a53c9ffe4eb --- /dev/null +++ b/dbms/tests/queries/0_stateless/00577_full_join_segfault.sql @@ -0,0 +1,7 @@ +SELECT k, a1, b1, a2, b2 FROM (SELECT 0 AS k, 'hello' AS a1, 123 AS b1, a1) ANY FULL OUTER JOIN (SELECT 1 AS k, 'hello' AS a2, 456 AS b2, a2) USING (k) ORDER BY k; +SELECT k, a, b FROM (SELECT 0 AS k, 'hello' AS a, 123 AS b, a) ANY FULL OUTER JOIN (SELECT 1 AS k) USING (k) ORDER BY k; + +SET join_use_nulls = 1; + +SELECT k, a1, b1, a2, b2 FROM (SELECT 0 AS k, 'hello' AS a1, 123 AS b1, a1) ANY FULL OUTER JOIN (SELECT 1 AS k, 'hello' AS a2, 456 AS b2, a2) USING (k) ORDER BY k; +SELECT k, a, b FROM (SELECT 0 AS k, 'hello' AS a, 123 AS b, a) ANY FULL OUTER JOIN (SELECT 1 AS k) USING (k) ORDER BY k; diff --git a/dbms/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference b/dbms/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference new file mode 100644 index 00000000000..e0f8c3bae3f --- /dev/null +++ b/dbms/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.reference @@ -0,0 +1,4 @@ +2018-01-01 0 0 +2018-01-01 1 1 +2018-01-01 2 2 +2018-01-01 2 2 diff --git a/dbms/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql b/dbms/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql new file mode 100644 index 00000000000..25b53d9b169 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00577_replacing_merge_tree_vertical_merge.sql @@ -0,0 +1,8 @@ +drop table if exists test.tab; +create table test.tab (date Date, version UInt64, val UInt64) engine = ReplacingMergeTree(version) partition by date order by date settings enable_vertical_merge_algorithm = 1, vertical_merge_algorithm_min_rows_to_activate = 1, vertical_merge_algorithm_min_columns_to_activate = 0; +insert into test.tab values ('2018-01-01', 2, 2), ('2018-01-01', 1, 1); +insert into test.tab values ('2018-01-01', 0, 0); +select * from test.tab order by version; +OPTIMIZE TABLE test.tab; +select * from test.tab; + diff --git a/dbms/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.reference b/dbms/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.reference new file mode 100644 index 00000000000..943d6e5f7f4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.reference @@ -0,0 +1,8 @@ +5000 +5 +1000 +1000 +1000 +0 +0 +0 diff --git a/dbms/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.sql b/dbms/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.sql new file mode 100644 index 00000000000..66c740f8ee1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00578_merge_table_and_table_virtual_column.sql @@ -0,0 +1,42 @@ +DROP TABLE IF EXISTS test.numbers1; +DROP TABLE IF EXISTS test.numbers2; +DROP TABLE IF EXISTS test.numbers3; +DROP TABLE IF EXISTS test.numbers4; +DROP TABLE IF EXISTS test.numbers5; + +CREATE TABLE test.numbers1 ENGINE = StripeLog AS SELECT number FROM numbers(1000); +CREATE TABLE test.numbers2 ENGINE = TinyLog AS SELECT number FROM numbers(1000); +CREATE TABLE test.numbers3 ENGINE = Log AS SELECT number FROM numbers(1000); +CREATE TABLE test.numbers4 ENGINE = Memory AS SELECT number FROM numbers(1000); +CREATE TABLE test.numbers5 ENGINE = MergeTree ORDER BY number AS SELECT number FROM numbers(1000); + +SELECT count() FROM merge(test, '^numbers\\d+$'); +SELECT DISTINCT count() FROM merge(test, '^numbers\\d+$') GROUP BY number; + +SET max_rows_to_read = 1000; + +SET max_threads = 'auto'; +SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'numbers1'; + +SET max_threads = 1; +SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'numbers2'; + +SET max_threads = 10; +SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'numbers3'; + +SET max_rows_to_read = 1; + +SET max_threads = 'auto'; +SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'non_existing'; + +SET max_threads = 1; +SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'non_existing'; + +SET max_threads = 10; +SELECT count() FROM merge(test, '^numbers\\d+$') WHERE _table = 'non_existing'; + +DROP TABLE test.numbers1; +DROP TABLE test.numbers2; +DROP TABLE test.numbers3; +DROP TABLE test.numbers4; +DROP TABLE test.numbers5; diff --git a/dbms/tests/queries/0_stateless/00578_merge_trees_without_primary_key.reference b/dbms/tests/queries/0_stateless/00578_merge_trees_without_primary_key.reference new file mode 100644 index 00000000000..ccff1dac5cd --- /dev/null +++ b/dbms/tests/queries/0_stateless/00578_merge_trees_without_primary_key.reference @@ -0,0 +1,14 @@ +*** MergeTree *** +1 a +5 b +2 c +4 d +3 e +*** ReplacingMergeTree *** +1 a 5 +--- +1 a 5 +*** CollapsingMergeTree *** +3 c 1 +--- +3 c 1 diff --git a/dbms/tests/queries/0_stateless/00578_merge_trees_without_primary_key.sql b/dbms/tests/queries/0_stateless/00578_merge_trees_without_primary_key.sql new file mode 100644 index 00000000000..efd881ae006 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00578_merge_trees_without_primary_key.sql @@ -0,0 +1,56 @@ +SELECT '*** MergeTree ***'; + +DROP TABLE IF EXISTS test.unsorted; +CREATE TABLE test.unsorted (x UInt32, y String) ENGINE MergeTree ORDER BY tuple(); + +INSERT INTO test.unsorted VALUES (1, 'a'), (5, 'b'); +INSERT INTO test.unsorted VALUES (2, 'c'), (4, 'd'); +INSERT INTO test.unsorted VALUES (3, 'e'); + +OPTIMIZE TABLE test.unsorted PARTITION tuple() FINAL; + +SELECT * FROM test.unsorted; + +DROP TABLE test.unsorted; + + +SELECT '*** ReplacingMergeTree ***'; + +DROP TABLE IF EXISTS test.unsorted_replacing; + +CREATE TABLE test.unsorted_replacing (x UInt32, s String, v UInt32) ENGINE ReplacingMergeTree(v) ORDER BY tuple(); + +INSERT INTO test.unsorted_replacing VALUES (1, 'a', 5), (5, 'b', 4); +INSERT INTO test.unsorted_replacing VALUES (2, 'c', 3), (4, 'd', 2); +INSERT INTO test.unsorted_replacing VALUES (3, 'e', 1); + +SELECT * FROM test.unsorted_replacing FINAL; + +SELECT '---'; + +OPTIMIZE TABLE test.unsorted_replacing PARTITION tuple() FINAL; + +SELECT * FROM test.unsorted_replacing; + +DROP TABLE test.unsorted_replacing; + + +SELECT '*** CollapsingMergeTree ***'; + +DROP TABLE IF EXISTS test.unsorted_collapsing; + +CREATE TABLE test.unsorted_collapsing (x UInt32, s String, sign Int8) ENGINE CollapsingMergeTree(sign) ORDER BY tuple(); + +INSERT INTO test.unsorted_collapsing VALUES (1, 'a', 1); +INSERT INTO test.unsorted_collapsing VALUES (1, 'a', -1), (2, 'b', 1); +INSERT INTO test.unsorted_collapsing VALUES (2, 'b', -1), (3, 'c', 1); + +SELECT * FROM test.unsorted_collapsing FINAL; + +SELECT '---'; + +OPTIMIZE TABLE test.unsorted_collapsing PARTITION tuple() FINAL; + +SELECT * FROM test.unsorted_collapsing; + +DROP TABLE test.unsorted_collapsing; diff --git a/dbms/tests/queries/0_stateless/00579_merge_tree_partition_and_primary_keys_using_same_expression.reference b/dbms/tests/queries/0_stateless/00579_merge_tree_partition_and_primary_keys_using_same_expression.reference new file mode 100644 index 00000000000..74af60e7bb1 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00579_merge_tree_partition_and_primary_keys_using_same_expression.reference @@ -0,0 +1,6 @@ +2018-02-19 12:00:00 +2018-02-20 12:00:00 +2018-02-21 12:00:00 +--- +2018-02-19 12:00:00 +2018-02-21 12:00:00 diff --git a/dbms/tests/queries/0_stateless/00579_merge_tree_partition_and_primary_keys_using_same_expression.sql b/dbms/tests/queries/0_stateless/00579_merge_tree_partition_and_primary_keys_using_same_expression.sql new file mode 100644 index 00000000000..f897de5a645 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00579_merge_tree_partition_and_primary_keys_using_same_expression.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.partition_and_primary_keys_using_same_expression; + +CREATE TABLE test.partition_and_primary_keys_using_same_expression(dt DateTime) + ENGINE MergeTree PARTITION BY toDate(dt) ORDER BY toDayOfWeek(toDate(dt)); + +INSERT INTO test.partition_and_primary_keys_using_same_expression + VALUES ('2018-02-19 12:00:00'); +INSERT INTO test.partition_and_primary_keys_using_same_expression + VALUES ('2018-02-20 12:00:00'), ('2018-02-21 12:00:00'); + +SELECT * FROM test.partition_and_primary_keys_using_same_expression ORDER BY dt; + +SELECT '---'; + +ALTER TABLE test.partition_and_primary_keys_using_same_expression DROP PARTITION '2018-02-20'; +SELECT * FROM test.partition_and_primary_keys_using_same_expression ORDER BY dt; + +DROP TABLE test.partition_and_primary_keys_using_same_expression; diff --git a/dbms/tests/queries/0_stateless/00579_virtual_column_and_lazy.reference b/dbms/tests/queries/0_stateless/00579_virtual_column_and_lazy.reference new file mode 100644 index 00000000000..1e17df0ebb4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00579_virtual_column_and_lazy.reference @@ -0,0 +1,3000 @@ +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 +1 diff --git a/dbms/tests/queries/0_stateless/00579_virtual_column_and_lazy.sql b/dbms/tests/queries/0_stateless/00579_virtual_column_and_lazy.sql new file mode 100644 index 00000000000..c30133863b5 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00579_virtual_column_and_lazy.sql @@ -0,0 +1,18 @@ +DROP TABLE IF EXISTS test.sample1; +DROP TABLE IF EXISTS test.sample2; +DROP TABLE IF EXISTS test.sample_merge; + +CREATE TABLE test.sample1 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10); +CREATE TABLE test.sample2 (x UInt64, d Date DEFAULT today()) ENGINE = MergeTree(d, intHash64(x), intHash64(x), 10); + +INSERT INTO test.sample1 (x) SELECT number AS x FROM system.numbers LIMIT 1000; +INSERT INTO test.sample2 (x) SELECT number AS x FROM system.numbers LIMIT 2000; + +CREATE TABLE test.sample_merge AS test.sample1 ENGINE = Merge(test, '^sample\\d$'); + +SET max_threads = 1; +SELECT _sample_factor FROM merge(test, '^sample\\d$'); + +DROP TABLE test.sample1; +DROP TABLE test.sample2; +DROP TABLE test.sample_merge; diff --git a/debian/changelog b/debian/changelog index ecae07de998..4365c1943ec 100644 --- a/debian/changelog +++ b/debian/changelog @@ -1,5 +1,5 @@ -clickhouse (1.1.54354) unstable; urgency=low +clickhouse (1.1.54355) unstable; urgency=low * Modified source code - -- Fri, 16 Feb 2018 20:44:58 +0300 + -- Wed, 21 Feb 2018 10:35:08 +0300 diff --git a/docs/build.sh b/docs/build.sh index 5fbff59a744..17ebfbc32e3 100755 --- a/docs/build.sh +++ b/docs/build.sh @@ -8,7 +8,7 @@ fi for lang in $LANGS; do echo -e "\n\nLANG=$lang. Creating single page source" - mkdir $lang'_single_page' + mkdir $lang'_single_page' 2>/dev/null cp -r $lang/images $lang'_single_page' ./concatenate.py $lang echo -e "\n\nLANG=$lang. Building multipage..." diff --git a/docs/concatenate.py b/docs/concatenate.py index 0a964e8a54a..a2843fd79a3 100755 --- a/docs/concatenate.py +++ b/docs/concatenate.py @@ -14,7 +14,6 @@ # - For not http-links without anchor script logs an error and cuts them from the resulting single-page document. - import codecs import sys import re @@ -29,19 +28,19 @@ if not os.path.exists(sys.argv[1]): print "Pass language_dir correctly. For example, 'ru'." sys.exit(2) -#Configuration -PROJ_CONFIG = 'mkdocs_'+sys.argv[1]+'.yml' -SINGLE_PAGE = sys.argv[1]+'_single_page/index.md' -DOCS_DIR = sys.argv[1]+'/' +# Configuration +PROJ_CONFIG = 'mkdocs_' + sys.argv[1] + '.yml' +SINGLE_PAGE = sys.argv[1] + '_single_page/index.md' +DOCS_DIR = sys.argv[1] + '/' # 1. Open mkdocs.yml file and read `pages` configuration to get an ordered list of files cfg_file = open(PROJ_CONFIG) -files_to_concatenate=[] +files_to_concatenate = [] -for l in cfg_file : - if( '.md' in l ) and ('single_page' not in l): - path = (l[l.index(':')+1:]).strip(" '\n") +for l in cfg_file: + if('.md' in l) and ('single_page' not in l): + path = (l[l.index(':') + 1:]).strip(" '\n") files_to_concatenate.append(path) print str(len(files_to_concatenate)) + " files will be concatenated into single md-file.\nFiles:" @@ -57,30 +56,30 @@ for path in files_to_concatenate: single_page_file.write('\n\n') - file = open(DOCS_DIR+path) + file = open(DOCS_DIR + path) - #function is passed into re.sub() to process links - def link_proc( matchObj ): + # function is passed into re.sub() to process links + def link_proc(matchObj): text, link = matchObj.group().strip('[)').split('](') if link.startswith('http'): - return '['+text+']('+link+')' - else : + return '[' + text + '](' + link + ')' + else: sharp_pos = link.find('#') if sharp_pos > -1: - return '['+text+']('+link[sharp_pos:]+')' - else : - print 'ERROR: Link ['+text+']('+link+') in file '+path+' has no anchor. Please provide it.' - #return '['+text+'](#'+link.replace('/','-')+')' + return '[' + text + '](' + link[sharp_pos:] + ')' + else: + print 'ERROR: Link [' + text + '](' + link + ') in file ' + path + ' has no anchor. Please provide it.' + # return '['+text+'](#'+link.replace('/','-')+')' for l in file: - #Processing links in a string + # Processing links in a string l = re.sub(r'\[.+?\]\(.+?\)', link_proc, l) - #Correcting headers levels + # Correcting headers levels if not first_file: - if( l.startswith('#') ): - l='#'+l - else : + if(l.startswith('#')): + l = '#' + l + else: first_file = False single_page_file.write(l) diff --git a/docs/create_contents.py b/docs/create_contents.py index 46b822d6c51..c2f8ed58534 100644 --- a/docs/create_contents.py +++ b/docs/create_contents.py @@ -4,12 +4,13 @@ SOURCES_TREE = 'ru' from os import walk + def get_header(filepath): f = open(filepath) header = '' for line in f: - if line.startswith('#') : -# print line + if line.startswith('#'): + # print line header = line[1:].strip(' \n') break @@ -17,19 +18,22 @@ def get_header(filepath): return header -pages_file = open("strings_for_pages.txt","w") -md_links_file = open("links_for_md.txt","w") +pages_file = open("strings_for_pages.txt", "w") +md_links_file = open("links_for_md.txt", "w") for (dirpath, dirnames, filenames) in walk(SOURCES_TREE): - for filename in filenames : + for filename in filenames: - if '.md' not in filename: continue + if '.md' not in filename: + continue - header = get_header(dirpath+'/'+filename) - path = dirpath.replace('docs/','')+'/'+filename + header = get_header(dirpath + '/' + filename) + path = dirpath.replace('docs/', '') + '/' + filename - if filename == 'index.md': pages_file.write("- '" + header + "': " + "'" + path + "'\n") - else: pages_file.write(" - '" + header + "': " + "'" + path + "'\n") + if filename == 'index.md': + pages_file.write("- '" + header + "': " + "'" + path + "'\n") + else: + pages_file.write(" - '" + header + "': " + "'" + path + "'\n") md_links_file.write("[" + header + "](" + path + ")\n") diff --git a/docs/en/operations/access_rights.md b/docs/en/operations/access_rights.md index e2e79d7f2aa..9879dab9a99 100644 --- a/docs/en/operations/access_rights.md +++ b/docs/en/operations/access_rights.md @@ -58,7 +58,7 @@ Users are recorded in the 'users' section. We'll look at a fragment of the `user You can see a declaration from two users: `default`and`web`. We added the `web` user separately. -The `default` user is chosen in cases when the username is not passed. The `default` user is also used for distributed query processing, if the configuration of the server or cluster doesn't specify `user` and `password` (see the section on the [Distributed](../table_engines/distributed.html) engine). +The `default` user is chosen in cases when the username is not passed. The `default` user is also used for distributed query processing, if the configuration of the server or cluster doesn't specify `user` and `password` (see the section on the [Distributed](../table_engines/distributed.md#distributed_distributed) engine). The user that is used for exchanging information between servers combined in a cluster must not have substantial restrictions or quotas – otherwise, distributed queries will fail. diff --git a/docs/en/operations/server_settings/settings.md b/docs/en/operations/server_settings/settings.md index 08505556f11..fe24bab9d85 100644 --- a/docs/en/operations/server_settings/settings.md +++ b/docs/en/operations/server_settings/settings.md @@ -67,7 +67,7 @@ ClickHouse checks ` min_part_size` and ` min_part_size_ratio` and processes th The default database. -Use a [ SHOW DATABASES](../query_language/queries.md#query_language_queries_show_databases) query to get a list of databases. +Use a [ SHOW DATABASES](../../query_language/queries.md#query_language_queries_show_databases) query to get a list of databases. **Example** diff --git a/docs/en/table_engines/distributed.md b/docs/en/table_engines/distributed.md index bdf0ba51ef6..b8643461fbb 100644 --- a/docs/en/table_engines/distributed.md +++ b/docs/en/table_engines/distributed.md @@ -1,6 +1,6 @@ -# Distributed (Sharding sheme) +# Distributed **The Distributed engine does not store data itself**, but allows distributed query processing on multiple servers. Reading is automatically parallelized. During a read, the table indexes on remote servers are used, if there are any. diff --git a/docs/en/table_engines/replication.md b/docs/en/table_engines/replication.md index 0ebd5ccfc51..1e58878c34e 100644 --- a/docs/en/table_engines/replication.md +++ b/docs/en/table_engines/replication.md @@ -2,11 +2,15 @@ # Data replication -## ReplicatedMergeTree +## ReplicatedAggregatingMergeTree ## ReplicatedCollapsingMergeTree -## ReplicatedAggregatingMergeTree +## ReplicatedGraphiteMergeTree + +## ReplicatedMergeTree + +## ReplicatedReplacingMergeTree ## ReplicatedSummingMergeTree diff --git a/docs/mkdocs-material-theme/assets/stylesheets/application.ac64251e.css b/docs/mkdocs-material-theme/assets/stylesheets/application.ac64251e.css index da350a2b17d..1383191d726 100644 --- a/docs/mkdocs-material-theme/assets/stylesheets/application.ac64251e.css +++ b/docs/mkdocs-material-theme/assets/stylesheets/application.ac64251e.css @@ -1,2 +1,2 @@ -html{-webkit-box-sizing:border-box;box-sizing:border-box}*,:after,:before{-webkit-box-sizing:inherit;box-sizing:inherit}html{-webkit-text-size-adjust:none;-moz-text-size-adjust:none;-ms-text-size-adjust:none;text-size-adjust:none}body{margin:0}hr{overflow:visible;-webkit-box-sizing:content-box;box-sizing:content-box}a{-webkit-text-decoration-skip:objects}a,button,input,label{-webkit-tap-highlight-color:transparent}a{color:inherit;text-decoration:none}a:active,a:hover{outline-width:0}small,sub,sup{font-size:80%}sub,sup{position:relative;line-height:0;vertical-align:baseline}sub{bottom:-.25em}sup{top:-.5em}img{border-style:none}table{border-collapse:collapse;border-spacing:0}td,th{font-weight:400;vertical-align:top}button{padding:0;background:transparent;font-size:inherit}button,input{border:0;outline:0}.md-clipboard:before,.md-icon,.md-nav__button,.md-nav__link:after,.md-nav__title:before,.md-search-result__article--document:before,.md-source-file:before,.md-typeset .admonition>.admonition-title:before,.md-typeset .admonition>summary:before,.md-typeset .critic.comment:before,.md-typeset .footnote-backref,.md-typeset .task-list-control .task-list-indicator:before,.md-typeset details>.admonition-title:before,.md-typeset details>summary:before,.md-typeset summary:after{font-family:Material Icons;font-style:normal;font-variant:normal;font-weight:400;line-height:1;text-transform:none;white-space:nowrap;speak:none;word-wrap:normal;direction:ltr}.md-content__icon,.md-footer-nav__button,.md-header-nav__button,.md-nav__button,.md-nav__title:before,.md-search-result__article--document:before{display:inline-block;margin:.4rem;padding:.8rem;font-size:2.4rem;cursor:pointer}.md-icon--arrow-back:before{content:"\E5C4"}.md-icon--arrow-forward:before{content:"\E5C8"}.md-icon--menu:before{content:"\E5D2"}.md-icon--search:before{content:"\E8B6"}body{-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}body,input{color:rgba(0,0,0,.87);-webkit-font-feature-settings:"kern","liga";font-feature-settings:"kern","liga";font-family:Helvetica Neue,Helvetica,Arial,sans-serif}code,kbd,pre{color:rgba(0,0,0,.87);-webkit-font-feature-settings:"kern";font-feature-settings:"kern";font-family:Courier New,Courier,monospace}.md-typeset{font-size:1.6rem;line-height:1.6;-webkit-print-color-adjust:exact}.md-typeset blockquote,.md-typeset ol,.md-typeset p,.md-typeset ul{margin:1em 0}.md-typeset h1{margin:0 0 4rem;color:rgba(0,0,0,.54);font-size:3.125rem;line-height:1.3}.md-typeset h1,.md-typeset h2{font-weight:300;letter-spacing:-.01em}.md-typeset h2{margin:4rem 0 1.6rem;font-size:2.5rem;line-height:1.4}.md-typeset h3{margin:3.2rem 0 1.6rem;font-size:2rem;font-weight:400;letter-spacing:-.01em;line-height:1.5}.md-typeset h2+h3{margin-top:1.6rem}.md-typeset h4{font-size:1.6rem}.md-typeset h4,.md-typeset h5,.md-typeset h6{margin:1.6rem 0;font-weight:700;letter-spacing:-.01em}.md-typeset h5,.md-typeset h6{color:rgba(0,0,0,.54);font-size:1.28rem}.md-typeset h5{text-transform:uppercase}.md-typeset hr{margin:1.5em 0;border-bottom:.1rem dotted rgba(0,0,0,.26)}.md-typeset a{color:#3f51b5;word-break:break-word}.md-typeset a,.md-typeset a:before{-webkit-transition:color .125s;transition:color .125s}.md-typeset a:active,.md-typeset a:hover{color:#536dfe}.md-typeset code,.md-typeset pre{background-color:hsla(0,0%,93%,.5);color:#37474f;font-size:85%}.md-typeset code{margin:0 .29412em;padding:.07353em 0;border-radius:.2rem;-webkit-box-shadow:.29412em 0 0 hsla(0,0%,93%,.5),-.29412em 0 0 hsla(0,0%,93%,.5);box-shadow:.29412em 0 0 hsla(0,0%,93%,.5),-.29412em 0 0 hsla(0,0%,93%,.5);word-break:break-word;-webkit-box-decoration-break:clone;box-decoration-break:clone}.md-typeset h1 code,.md-typeset h2 code,.md-typeset h3 code,.md-typeset h4 code,.md-typeset h5 code,.md-typeset h6 code{margin:0;background-color:transparent;-webkit-box-shadow:none;box-shadow:none}.md-typeset a>code{margin:inherit;padding:inherit;border-radius:none;background-color:inherit;color:inherit;-webkit-box-shadow:none;box-shadow:none}.md-typeset pre{position:relative;margin:1em 0;border-radius:.2rem;line-height:1.4;-webkit-overflow-scrolling:touch}.md-typeset pre>code{display:block;margin:0;padding:1.05rem 1.2rem;background-color:transparent;font-size:inherit;-webkit-box-shadow:none;box-shadow:none;-webkit-box-decoration-break:none;box-decoration-break:none;overflow:auto}.md-typeset pre>code::-webkit-scrollbar{width:.4rem;height:.4rem}.md-typeset pre>code::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-typeset pre>code::-webkit-scrollbar-thumb:hover{background-color:#536dfe}.md-typeset kbd{padding:0 .29412em;border:.1rem solid #c9c9c9;border-radius:.2rem;border-bottom-color:#bcbcbc;background-color:#fcfcfc;color:#555;font-size:85%;-webkit-box-shadow:0 .1rem 0 #b0b0b0;box-shadow:0 .1rem 0 #b0b0b0;word-break:break-word}.md-typeset mark{margin:0 .25em;padding:.0625em 0;border-radius:.2rem;background-color:rgba(255,235,59,.5);-webkit-box-shadow:.25em 0 0 rgba(255,235,59,.5),-.25em 0 0 rgba(255,235,59,.5);box-shadow:.25em 0 0 rgba(255,235,59,.5),-.25em 0 0 rgba(255,235,59,.5);word-break:break-word;-webkit-box-decoration-break:clone;box-decoration-break:clone}.md-typeset abbr{border-bottom:.1rem dotted rgba(0,0,0,.54);text-decoration:none;cursor:help}.md-typeset small{opacity:.75}.md-typeset sub,.md-typeset sup{margin-left:.07812em}.md-typeset blockquote{padding-left:1.2rem;border-left:.4rem solid rgba(0,0,0,.26);color:rgba(0,0,0,.54)}.md-typeset ul{list-style-type:disc}.md-typeset ol,.md-typeset ul{margin-left:.625em;padding:0}.md-typeset ol ol,.md-typeset ul ol{list-style-type:lower-alpha}.md-typeset ol ol ol,.md-typeset ul ol ol{list-style-type:lower-roman}.md-typeset ol li,.md-typeset ul li{margin-bottom:.5em;margin-left:1.25em}.md-typeset ol li blockquote,.md-typeset ol li p,.md-typeset ul li blockquote,.md-typeset ul li p{margin:.5em 0}.md-typeset ol li:last-child,.md-typeset ul li:last-child{margin-bottom:0}.md-typeset ol li ol,.md-typeset ol li ul,.md-typeset ul li ol,.md-typeset ul li ul{margin:.5em 0 .5em .625em}.md-typeset dd{margin:1em 0 1em 1.875em}.md-typeset iframe,.md-typeset img,.md-typeset svg{max-width:100%}.md-typeset table:not([class]){-webkit-box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);display:inline-block;max-width:100%;border-radius:.2rem;font-size:1.28rem;overflow:auto;-webkit-overflow-scrolling:touch}.md-typeset table:not([class])+*{margin-top:1.5em}.md-typeset table:not([class]) td:not([align]),.md-typeset table:not([class]) th:not([align]){text-align:left}.md-typeset table:not([class]) th{min-width:10rem;padding:1.2rem 1.6rem;background-color:rgba(0,0,0,.54);color:#fff;vertical-align:top}.md-typeset table:not([class]) td{padding:1.2rem 1.6rem;border-top:.1rem solid rgba(0,0,0,.07);vertical-align:top}.md-typeset table:not([class]) tr:first-child td{border-top:0}.md-typeset table:not([class]) a{word-break:normal}.md-typeset__scrollwrap{margin:1em -1.6rem;overflow-x:auto;-webkit-overflow-scrolling:touch}.md-typeset .md-typeset__table{display:inline-block;margin-bottom:.5em;padding:0 1.6rem}.md-typeset .md-typeset__table table{display:table;width:100%;margin:0;overflow:hidden}html{font-size:62.5%;overflow-x:hidden}body,html{height:100%}body{position:relative}hr{display:block;height:.1rem;padding:0;border:0}.md-svg{display:none}.md-grid{max-width:122rem;margin-right:auto;margin-left:auto}.md-container,.md-main{overflow:auto}.md-container{display:table;width:100%;height:100%;padding-top:4.8rem;table-layout:fixed}.md-main{display:table-row;height:100%}.md-main__inner{height:100%;padding-top:3rem;padding-bottom:.1rem}.md-toggle{display:none}.md-overlay{position:fixed;top:0;width:0;height:0;-webkit-transition:width 0s .25s,height 0s .25s,opacity .25s;transition:width 0s .25s,height 0s .25s,opacity .25s;background-color:rgba(0,0,0,.54);opacity:0;z-index:3}.md-flex{display:table}.md-flex__cell{display:table-cell;position:relative;vertical-align:top}.md-flex__cell--shrink{width:0}.md-flex__cell--stretch{display:table;width:100%;table-layout:fixed}.md-flex__ellipsis{display:table-cell;text-overflow:ellipsis;white-space:nowrap;overflow:hidden}@page{margin:25mm}.md-clipboard{position:absolute;top:.6rem;right:.6rem;width:2.8rem;height:2.8rem;border-radius:.2rem;font-size:1.6rem;cursor:pointer;z-index:1;-webkit-backface-visibility:hidden;backface-visibility:hidden}.md-clipboard:before{-webkit-transition:color .25s,opacity .25s;transition:color .25s,opacity .25s;color:rgba(0,0,0,.54);content:"\E14D";opacity:.25}.codehilite:hover .md-clipboard:before,.md-typeset .highlight:hover .md-clipboard:before,pre:hover .md-clipboard:before{opacity:1}.md-clipboard:active:before,.md-clipboard:hover:before{color:#536dfe}.md-clipboard__message{display:block;position:absolute;top:0;right:3.4rem;padding:.6rem 1rem;-webkit-transform:translateX(.8rem);transform:translateX(.8rem);-webkit-transition:opacity .175s,-webkit-transform .25s cubic-bezier(.9,.1,.9,0);transition:opacity .175s,-webkit-transform .25s cubic-bezier(.9,.1,.9,0);transition:transform .25s cubic-bezier(.9,.1,.9,0),opacity .175s;transition:transform .25s cubic-bezier(.9,.1,.9,0),opacity .175s,-webkit-transform .25s cubic-bezier(.9,.1,.9,0);border-radius:.2rem;background-color:rgba(0,0,0,.54);color:#fff;font-size:1.28rem;white-space:nowrap;opacity:0;pointer-events:none}.md-clipboard__message--active{-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:opacity .175s 75ms,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:opacity .175s 75ms,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .175s 75ms;transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .175s 75ms,-webkit-transform .25s cubic-bezier(.4,0,.2,1);opacity:1;pointer-events:auto}.md-clipboard__message:before{content:attr(aria-label)}.md-clipboard__message:after{display:block;position:absolute;top:50%;right:-.4rem;width:0;margin-top:-.4rem;border-width:.4rem 0 .4rem .4rem;border-style:solid;border-color:transparent rgba(0,0,0,.54);content:""}.md-content__inner{margin:0 1.6rem 2.4rem;padding-top:1.2rem}.md-content__inner:before{display:block;height:.8rem;content:""}.md-content__inner>:last-child{margin-bottom:0}.md-content__icon{position:relative;margin:.8rem 0;padding:0;float:right}.md-typeset .md-content__icon{color:rgba(0,0,0,.26)}.md-header{position:fixed;top:0;right:0;left:0;height:4.8rem;-webkit-transition:background-color .25s,color .25s;transition:background-color .25s,color .25s;background-color:#3f51b5;color:#fff;z-index:2;-webkit-backface-visibility:hidden;backface-visibility:hidden}.md-header,.no-js .md-header{-webkit-box-shadow:none;box-shadow:none}.md-header[data-md-state=shadow]{-webkit-transition:background-color .25s,color .25s,-webkit-box-shadow .25s;transition:background-color .25s,color .25s,-webkit-box-shadow .25s;transition:background-color .25s,color .25s,box-shadow .25s;transition:background-color .25s,color .25s,box-shadow .25s,-webkit-box-shadow .25s;-webkit-box-shadow:0 0 .4rem rgba(0,0,0,.1),0 .4rem .8rem rgba(0,0,0,.2);box-shadow:0 0 .4rem rgba(0,0,0,.1),0 .4rem .8rem rgba(0,0,0,.2)}.md-header-nav{padding:0 .4rem}.md-header-nav__button{position:relative;-webkit-transition:opacity .25s;transition:opacity .25s;z-index:1}.md-header-nav__button:hover{opacity:.7}.md-header-nav__button.md-logo *{display:block}.no-js .md-header-nav__button.md-icon--search{display:none}.md-header-nav__topic{display:block;position:absolute;-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);text-overflow:ellipsis;white-space:nowrap;overflow:hidden}.md-header-nav__topic+.md-header-nav__topic{-webkit-transform:translateX(2.5rem);transform:translateX(2.5rem);-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s;transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);opacity:0;z-index:-1;pointer-events:none}.no-js .md-header-nav__topic{position:static}.md-header-nav__title{padding:0 2rem;font-size:1.8rem;line-height:4.8rem}.md-header-nav__title[data-md-state=active] .md-header-nav__topic{-webkit-transform:translateX(-2.5rem);transform:translateX(-2.5rem);-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s;transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);opacity:0;z-index:-1;pointer-events:none}.md-header-nav__title[data-md-state=active] .md-header-nav__topic+.md-header-nav__topic{-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);opacity:1;z-index:0;pointer-events:auto}.md-header-nav__source{display:none}.md-hero{-webkit-transition:background .25s;transition:background .25s;background-color:#3f51b5;color:#fff;font-size:2rem;overflow:hidden}.md-hero__inner{margin-top:2rem;padding:1.6rem 1.6rem .8rem;-webkit-transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);-webkit-transition-delay:.1s;transition-delay:.1s}[data-md-state=hidden] .md-hero__inner{pointer-events:none;-webkit-transform:translateY(1.25rem);transform:translateY(1.25rem);-webkit-transition:opacity .1s 0s,-webkit-transform 0s .4s;transition:opacity .1s 0s,-webkit-transform 0s .4s;transition:transform 0s .4s,opacity .1s 0s;transition:transform 0s .4s,opacity .1s 0s,-webkit-transform 0s .4s;opacity:0}.md-hero--expand .md-hero__inner{margin-bottom:2.4rem}.md-footer-nav{background-color:rgba(0,0,0,.87);color:#fff}.md-footer-nav__inner{padding:.4rem;overflow:auto}.md-footer-nav__link{padding-top:2.8rem;padding-bottom:.8rem;-webkit-transition:opacity .25s;transition:opacity .25s}.md-footer-nav__link:hover{opacity:.7}.md-footer-nav__link--prev{width:25%;float:left}.md-footer-nav__link--next{width:75%;float:right;text-align:right}.md-footer-nav__button{-webkit-transition:background .25s;transition:background .25s}.md-footer-nav__title{position:relative;padding:0 2rem;font-size:1.8rem;line-height:4.8rem}.md-footer-nav__direction{position:absolute;right:0;left:0;margin-top:-2rem;padding:0 2rem;color:hsla(0,0%,100%,.7);font-size:1.5rem}.md-footer-meta{background-color:rgba(0,0,0,.895)}.md-footer-meta__inner{padding:.4rem;overflow:auto}html .md-footer-meta.md-typeset a{color:hsla(0,0%,100%,.7)}html .md-footer-meta.md-typeset a:focus,html .md-footer-meta.md-typeset a:hover{color:#fff}.md-footer-copyright{margin:0 1.2rem;padding:.8rem 0;color:hsla(0,0%,100%,.3);font-size:1.28rem}.md-footer-copyright__highlight{color:hsla(0,0%,100%,.7)}.md-footer-social{margin:0 .8rem;padding:.4rem 0 1.2rem}.md-footer-social__link{display:inline-block;width:3.2rem;height:3.2rem;font-size:1.6rem;text-align:center}.md-footer-social__link:before{line-height:1.9}.md-nav{font-size:1.4rem;line-height:1.3}.md-nav--secondary .md-nav__link--active{color:#3f51b5}.md-nav__title{display:block;padding:0 1.2rem;font-weight:700;text-overflow:ellipsis;overflow:hidden}.md-nav__title:before{display:none;content:"\E5C4"}.md-nav__title .md-nav__button{display:none}.md-nav__list{margin:0;padding:0;list-style:none}.md-nav__item{padding:0 1.2rem}.md-nav__item:last-child{padding-bottom:1.2rem}.md-nav__item .md-nav__item{padding-right:0}.md-nav__item .md-nav__item:last-child{padding-bottom:0}.md-nav__button img{width:100%;height:auto}.md-nav__link{display:block;margin-top:.625em;-webkit-transition:color .125s;transition:color .125s;text-overflow:ellipsis;cursor:pointer;overflow:hidden}.md-nav__item--nested>.md-nav__link:after{content:"\E313"}html .md-nav__link[for=toc],html .md-nav__link[for=toc]+.md-nav__link:after,html .md-nav__link[for=toc]~.md-nav{display:none}.md-nav__link[data-md-state=blur]{color:rgba(0,0,0,.54)}.md-nav__link:active{color:#3f51b5}.md-nav__item--nested>.md-nav__link{color:inherit}.md-nav__link:focus,.md-nav__link:hover{color:#536dfe}.md-nav__source,.no-js .md-search{display:none}.md-search__overlay{opacity:0;z-index:1}.md-search__form{position:relative}.md-search__input{position:relative;padding:0 4.8rem 0 7.2rem;text-overflow:ellipsis;z-index:2}.md-search__input::-webkit-input-placeholder{-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1);transition:color .25s cubic-bezier(.1,.7,.1,1)}.md-search__input:-ms-input-placeholder,.md-search__input::-ms-input-placeholder{-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1);transition:color .25s cubic-bezier(.1,.7,.1,1)}.md-search__input::placeholder{-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1);transition:color .25s cubic-bezier(.1,.7,.1,1)}.md-search__input::-webkit-input-placeholder,.md-search__input~.md-search__icon{color:rgba(0,0,0,.54)}.md-search__input:-ms-input-placeholder,.md-search__input::-ms-input-placeholder,.md-search__input~.md-search__icon{color:rgba(0,0,0,.54)}.md-search__input::placeholder,.md-search__input~.md-search__icon{color:rgba(0,0,0,.54)}.md-search__input::-ms-clear{display:none}.md-search__icon{position:absolute;-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1),opacity .25s;transition:color .25s cubic-bezier(.1,.7,.1,1),opacity .25s;font-size:2.4rem;cursor:pointer;z-index:2}.md-search__icon:hover{opacity:.7}.md-search__icon[for=search]{top:.6rem;left:1rem}.md-search__icon[for=search]:before{content:"\E8B6"}.md-search__icon[type=reset]{top:.6rem;right:1rem;-webkit-transform:scale(.125);transform:scale(.125);-webkit-transition:opacity .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1);transition:opacity .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1);transition:transform .15s cubic-bezier(.1,.7,.1,1),opacity .15s;transition:transform .15s cubic-bezier(.1,.7,.1,1),opacity .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1);opacity:0}[data-md-toggle=search]:checked~.md-header .md-search__input:valid~.md-search__icon[type=reset]{-webkit-transform:scale(1);transform:scale(1);opacity:1}[data-md-toggle=search]:checked~.md-header .md-search__input:valid~.md-search__icon[type=reset]:hover{opacity:.7}.md-search__output{position:absolute;width:100%;border-radius:0 0 .2rem .2rem;overflow:hidden;z-index:1}.md-search__scrollwrap{height:100%;background-color:#fff;-webkit-box-shadow:0 .1rem 0 rgba(0,0,0,.07) inset;box-shadow:inset 0 .1rem 0 rgba(0,0,0,.07);overflow-y:auto;-webkit-overflow-scrolling:touch}.md-search-result{color:rgba(0,0,0,.87);word-break:break-word}.md-search-result__meta{padding:0 1.6rem;background-color:rgba(0,0,0,.07);color:rgba(0,0,0,.54);font-size:1.28rem;line-height:3.6rem}.md-search-result__list{margin:0;padding:0;border-top:.1rem solid rgba(0,0,0,.07);list-style:none}.md-search-result__item{-webkit-box-shadow:0 -.1rem 0 rgba(0,0,0,.07);box-shadow:0 -.1rem 0 rgba(0,0,0,.07)}.md-search-result__link{display:block;-webkit-transition:background .25s;transition:background .25s;outline:0;overflow:hidden}.md-search-result__link:hover,.md-search-result__link[data-md-state=active]{background-color:rgba(83,109,254,.1)}.md-search-result__link:hover .md-search-result__article:before,.md-search-result__link[data-md-state=active] .md-search-result__article:before{opacity:.7}.md-search-result__link:last-child .md-search-result__teaser{margin-bottom:1.2rem}.md-search-result__article{position:relative;padding:0 1.6rem;overflow:auto}.md-search-result__article--document:before{position:absolute;left:0;margin:.2rem;-webkit-transition:opacity .25s;transition:opacity .25s;color:rgba(0,0,0,.54);content:"\E880"}.md-search-result__article--document .md-search-result__title{margin:1.1rem 0;font-size:1.6rem;font-weight:400;line-height:1.4}.md-search-result__title{margin:.5em 0;font-size:1.28rem;font-weight:700;line-height:1.4}.md-search-result__teaser{display:-webkit-box;max-height:3.3rem;margin:.5em 0;color:rgba(0,0,0,.54);font-size:1.28rem;line-height:1.4;text-overflow:ellipsis;overflow:hidden;-webkit-box-orient:vertical;-webkit-line-clamp:2}.md-search-result em{font-style:normal;font-weight:700;text-decoration:underline}.md-sidebar{position:absolute;width:24.2rem;padding:2.4rem 0;overflow:hidden}.md-sidebar[data-md-state=lock]{position:fixed;top:4.8rem}.md-sidebar--secondary{display:none}.md-sidebar__scrollwrap{max-height:100%;margin:0 .4rem;overflow-y:auto;-webkit-backface-visibility:hidden;backface-visibility:hidden}.md-sidebar__scrollwrap::-webkit-scrollbar{width:.4rem;height:.4rem}.md-sidebar__scrollwrap::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-sidebar__scrollwrap::-webkit-scrollbar-thumb:hover{background-color:#536dfe}@-webkit-keyframes md-source__facts--done{0%{height:0}to{height:1.3rem}}@keyframes md-source__facts--done{0%{height:0}to{height:1.3rem}}@-webkit-keyframes md-source__fact--done{0%{-webkit-transform:translateY(100%);transform:translateY(100%);opacity:0}50%{opacity:0}to{-webkit-transform:translateY(0);transform:translateY(0);opacity:1}}@keyframes md-source__fact--done{0%{-webkit-transform:translateY(100%);transform:translateY(100%);opacity:0}50%{opacity:0}to{-webkit-transform:translateY(0);transform:translateY(0);opacity:1}}.md-source{display:block;padding-right:1.2rem;-webkit-transition:opacity .25s;transition:opacity .25s;font-size:1.3rem;line-height:1.2;white-space:nowrap}.md-source:hover{opacity:.7}.md-source:after,.md-source__icon{display:inline-block;height:4.8rem;content:"";vertical-align:middle}.md-source__icon{width:4.8rem}.md-source__icon svg{width:2.4rem;height:2.4rem;margin-top:1.2rem;margin-left:1.2rem}.md-source__icon+.md-source__repository{margin-left:-4.4rem;padding-left:4rem}.md-source__repository{display:inline-block;max-width:100%;margin-left:1.2rem;font-weight:700;text-overflow:ellipsis;overflow:hidden;vertical-align:middle}.md-source__facts{margin:0;padding:0;font-size:1.1rem;font-weight:700;list-style-type:none;opacity:.75;overflow:hidden}[data-md-state=done] .md-source__facts{-webkit-animation:md-source__facts--done .25s ease-in;animation:md-source__facts--done .25s ease-in}.md-source__fact{float:left}[data-md-state=done] .md-source__fact{-webkit-animation:md-source__fact--done .4s ease-out;animation:md-source__fact--done .4s ease-out}.md-source__fact:before{margin:0 .2rem;content:"\B7"}.md-source__fact:first-child:before{display:none}.md-source-file{display:inline-block;margin:1em .5em 1em 0;padding-right:.5rem;border-radius:.2rem;background-color:rgba(0,0,0,.07);font-size:1.28rem;list-style-type:none;cursor:pointer;overflow:hidden}.md-source-file:before{display:inline-block;margin-right:.5rem;padding:.5rem;background-color:rgba(0,0,0,.26);color:#fff;font-size:1.6rem;content:"\E86F";vertical-align:middle}html .md-source-file{-webkit-transition:background .4s,color .4s,-webkit-box-shadow .4s cubic-bezier(.4,0,.2,1);transition:background .4s,color .4s,-webkit-box-shadow .4s cubic-bezier(.4,0,.2,1);transition:background .4s,color .4s,box-shadow .4s cubic-bezier(.4,0,.2,1);transition:background .4s,color .4s,box-shadow .4s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .4s cubic-bezier(.4,0,.2,1)}html .md-source-file:before{-webkit-transition:inherit;transition:inherit}html body .md-typeset .md-source-file{color:rgba(0,0,0,.54)}.md-source-file:hover{-webkit-box-shadow:0 0 8px rgba(0,0,0,.18),0 8px 16px rgba(0,0,0,.36);box-shadow:0 0 8px rgba(0,0,0,.18),0 8px 16px rgba(0,0,0,.36)}.md-source-file:hover:before{background-color:#536dfe}.md-tabs{width:100%;-webkit-transition:background .25s;transition:background .25s;background-color:#3f51b5;color:#fff;overflow:auto}.md-tabs__list{margin:0;margin-left:.4rem;padding:0;list-style:none;white-space:nowrap}.md-tabs__item{display:inline-block;height:4.8rem;padding-right:1.2rem;padding-left:1.2rem}.md-tabs__link{display:block;margin-top:1.6rem;-webkit-transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);font-size:1.4rem;opacity:.7}.md-tabs__link--active,.md-tabs__link:hover{color:inherit;opacity:1}.md-tabs__item:nth-child(2) .md-tabs__link{-webkit-transition-delay:.02s;transition-delay:.02s}.md-tabs__item:nth-child(3) .md-tabs__link{-webkit-transition-delay:.04s;transition-delay:.04s}.md-tabs__item:nth-child(4) .md-tabs__link{-webkit-transition-delay:.06s;transition-delay:.06s}.md-tabs__item:nth-child(5) .md-tabs__link{-webkit-transition-delay:.08s;transition-delay:.08s}.md-tabs__item:nth-child(6) .md-tabs__link{-webkit-transition-delay:.1s;transition-delay:.1s}.md-tabs__item:nth-child(7) .md-tabs__link{-webkit-transition-delay:.12s;transition-delay:.12s}.md-tabs__item:nth-child(8) .md-tabs__link{-webkit-transition-delay:.14s;transition-delay:.14s}.md-tabs__item:nth-child(9) .md-tabs__link{-webkit-transition-delay:.16s;transition-delay:.16s}.md-tabs__item:nth-child(10) .md-tabs__link{-webkit-transition-delay:.18s;transition-delay:.18s}.md-tabs__item:nth-child(11) .md-tabs__link{-webkit-transition-delay:.2s;transition-delay:.2s}.md-tabs__item:nth-child(12) .md-tabs__link{-webkit-transition-delay:.22s;transition-delay:.22s}.md-tabs__item:nth-child(13) .md-tabs__link{-webkit-transition-delay:.24s;transition-delay:.24s}.md-tabs__item:nth-child(14) .md-tabs__link{-webkit-transition-delay:.26s;transition-delay:.26s}.md-tabs__item:nth-child(15) .md-tabs__link{-webkit-transition-delay:.28s;transition-delay:.28s}.md-tabs__item:nth-child(16) .md-tabs__link{-webkit-transition-delay:.3s;transition-delay:.3s}.md-tabs[data-md-state=hidden]{pointer-events:none}.md-tabs[data-md-state=hidden] .md-tabs__link{-webkit-transform:translateY(50%);transform:translateY(50%);-webkit-transition:color .25s,opacity .1s,-webkit-transform 0s .4s;transition:color .25s,opacity .1s,-webkit-transform 0s .4s;transition:color .25s,transform 0s .4s,opacity .1s;transition:color .25s,transform 0s .4s,opacity .1s,-webkit-transform 0s .4s;opacity:0}.md-typeset .admonition,.md-typeset details{-webkit-box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);position:relative;margin:1.5625em 0;padding:1.2rem 1.2rem 0;border-left:.4rem solid #448aff;border-radius:.2rem;font-size:1.28rem}.md-typeset .admonition :first-child,.md-typeset details :first-child{margin-top:0}html .md-typeset .admonition :last-child,html .md-typeset details :last-child{margin-bottom:0;padding-bottom:1.2rem}.md-typeset .admonition .admonition,.md-typeset .admonition details,.md-typeset details .admonition,.md-typeset details details{margin:1em 0}.md-typeset .admonition>.admonition-title,.md-typeset .admonition>summary,.md-typeset details>.admonition-title,.md-typeset details>summary{margin:-1.2rem -1.2rem 0;padding:.8rem 1.2rem .8rem 4rem;border-bottom:.1rem solid rgba(68,138,255,.1);background-color:rgba(68,138,255,.1);font-weight:700}html .md-typeset .admonition>.admonition-title,html .md-typeset .admonition>summary,html .md-typeset details>.admonition-title,html .md-typeset details>summary{padding-bottom:.8rem}.md-typeset .admonition>.admonition-title:before,.md-typeset .admonition>summary:before,.md-typeset details>.admonition-title:before,.md-typeset details>summary:before{position:absolute;left:1.2rem;color:#448aff;font-size:2rem;content:"\E3C9"}.md-typeset .admonition.summary,.md-typeset .admonition.tldr,.md-typeset details.summary,.md-typeset details.tldr{border-left:.4rem solid #00b0ff}.md-typeset .admonition.summary>.admonition-title,.md-typeset .admonition.summary>summary,.md-typeset .admonition.tldr>.admonition-title,.md-typeset .admonition.tldr>summary,.md-typeset details.summary>.admonition-title,.md-typeset details.summary>summary,.md-typeset details.tldr>.admonition-title,.md-typeset details.tldr>summary{border-bottom:.1rem solid rgba(0,176,255,.1);background-color:rgba(0,176,255,.1)}.md-typeset .admonition.summary>.admonition-title:before,.md-typeset .admonition.summary>summary:before,.md-typeset .admonition.tldr>.admonition-title:before,.md-typeset .admonition.tldr>summary:before,.md-typeset details.summary>.admonition-title:before,.md-typeset details.summary>summary:before,.md-typeset details.tldr>.admonition-title:before,.md-typeset details.tldr>summary:before{color:#00b0ff;content:"\E8D2"}.md-typeset .admonition.info,.md-typeset .admonition.todo,.md-typeset details.info,.md-typeset details.todo{border-left:.4rem solid #00b8d4}.md-typeset .admonition.info>.admonition-title,.md-typeset .admonition.info>summary,.md-typeset .admonition.todo>.admonition-title,.md-typeset .admonition.todo>summary,.md-typeset details.info>.admonition-title,.md-typeset details.info>summary,.md-typeset details.todo>.admonition-title,.md-typeset details.todo>summary{border-bottom:.1rem solid rgba(0,184,212,.1);background-color:rgba(0,184,212,.1)}.md-typeset .admonition.info>.admonition-title:before,.md-typeset .admonition.info>summary:before,.md-typeset .admonition.todo>.admonition-title:before,.md-typeset .admonition.todo>summary:before,.md-typeset details.info>.admonition-title:before,.md-typeset details.info>summary:before,.md-typeset details.todo>.admonition-title:before,.md-typeset details.todo>summary:before{color:#00b8d4;content:"\E88E"}.md-typeset .admonition.hint,.md-typeset .admonition.important,.md-typeset .admonition.tip,.md-typeset details.hint,.md-typeset details.important,.md-typeset details.tip{border-left:.4rem solid #00bfa5}.md-typeset .admonition.hint>.admonition-title,.md-typeset .admonition.hint>summary,.md-typeset .admonition.important>.admonition-title,.md-typeset .admonition.important>summary,.md-typeset .admonition.tip>.admonition-title,.md-typeset .admonition.tip>summary,.md-typeset details.hint>.admonition-title,.md-typeset details.hint>summary,.md-typeset details.important>.admonition-title,.md-typeset details.important>summary,.md-typeset details.tip>.admonition-title,.md-typeset details.tip>summary{border-bottom:.1rem solid rgba(0,191,165,.1);background-color:rgba(0,191,165,.1)}.md-typeset .admonition.hint>.admonition-title:before,.md-typeset .admonition.hint>summary:before,.md-typeset .admonition.important>.admonition-title:before,.md-typeset .admonition.important>summary:before,.md-typeset .admonition.tip>.admonition-title:before,.md-typeset .admonition.tip>summary:before,.md-typeset details.hint>.admonition-title:before,.md-typeset details.hint>summary:before,.md-typeset details.important>.admonition-title:before,.md-typeset details.important>summary:before,.md-typeset details.tip>.admonition-title:before,.md-typeset details.tip>summary:before{color:#00bfa5;content:"\E80E"}.md-typeset .admonition.check,.md-typeset .admonition.done,.md-typeset .admonition.success,.md-typeset details.check,.md-typeset details.done,.md-typeset details.success{border-left:.4rem solid #00c853}.md-typeset .admonition.check>.admonition-title,.md-typeset .admonition.check>summary,.md-typeset .admonition.done>.admonition-title,.md-typeset .admonition.done>summary,.md-typeset .admonition.success>.admonition-title,.md-typeset .admonition.success>summary,.md-typeset details.check>.admonition-title,.md-typeset details.check>summary,.md-typeset details.done>.admonition-title,.md-typeset details.done>summary,.md-typeset details.success>.admonition-title,.md-typeset details.success>summary{border-bottom:.1rem solid rgba(0,200,83,.1);background-color:rgba(0,200,83,.1)}.md-typeset .admonition.check>.admonition-title:before,.md-typeset .admonition.check>summary:before,.md-typeset .admonition.done>.admonition-title:before,.md-typeset .admonition.done>summary:before,.md-typeset .admonition.success>.admonition-title:before,.md-typeset .admonition.success>summary:before,.md-typeset details.check>.admonition-title:before,.md-typeset details.check>summary:before,.md-typeset details.done>.admonition-title:before,.md-typeset details.done>summary:before,.md-typeset details.success>.admonition-title:before,.md-typeset details.success>summary:before{color:#00c853;content:"\E876"}.md-typeset .admonition.faq,.md-typeset .admonition.help,.md-typeset .admonition.question,.md-typeset details.faq,.md-typeset details.help,.md-typeset details.question{border-left:.4rem solid #64dd17}.md-typeset .admonition.faq>.admonition-title,.md-typeset .admonition.faq>summary,.md-typeset .admonition.help>.admonition-title,.md-typeset .admonition.help>summary,.md-typeset .admonition.question>.admonition-title,.md-typeset .admonition.question>summary,.md-typeset details.faq>.admonition-title,.md-typeset details.faq>summary,.md-typeset details.help>.admonition-title,.md-typeset details.help>summary,.md-typeset details.question>.admonition-title,.md-typeset details.question>summary{border-bottom:.1rem solid rgba(100,221,23,.1);background-color:rgba(100,221,23,.1)}.md-typeset .admonition.faq>.admonition-title:before,.md-typeset .admonition.faq>summary:before,.md-typeset .admonition.help>.admonition-title:before,.md-typeset .admonition.help>summary:before,.md-typeset .admonition.question>.admonition-title:before,.md-typeset .admonition.question>summary:before,.md-typeset details.faq>.admonition-title:before,.md-typeset details.faq>summary:before,.md-typeset details.help>.admonition-title:before,.md-typeset details.help>summary:before,.md-typeset details.question>.admonition-title:before,.md-typeset details.question>summary:before{color:#64dd17;content:"\E887"}.md-typeset .admonition.attention,.md-typeset .admonition.caution,.md-typeset .admonition.warning,.md-typeset details.attention,.md-typeset details.caution,.md-typeset details.warning{border-left:.4rem solid #ff9100}.md-typeset .admonition.attention>.admonition-title,.md-typeset .admonition.attention>summary,.md-typeset .admonition.caution>.admonition-title,.md-typeset .admonition.caution>summary,.md-typeset .admonition.warning>.admonition-title,.md-typeset .admonition.warning>summary,.md-typeset details.attention>.admonition-title,.md-typeset details.attention>summary,.md-typeset details.caution>.admonition-title,.md-typeset details.caution>summary,.md-typeset details.warning>.admonition-title,.md-typeset details.warning>summary{border-bottom:.1rem solid rgba(255,145,0,.1);background-color:rgba(255,145,0,.1)}.md-typeset .admonition.attention>.admonition-title:before,.md-typeset .admonition.attention>summary:before,.md-typeset .admonition.caution>.admonition-title:before,.md-typeset .admonition.caution>summary:before,.md-typeset .admonition.warning>.admonition-title:before,.md-typeset .admonition.warning>summary:before,.md-typeset details.attention>.admonition-title:before,.md-typeset details.attention>summary:before,.md-typeset details.caution>.admonition-title:before,.md-typeset details.caution>summary:before,.md-typeset details.warning>.admonition-title:before,.md-typeset details.warning>summary:before{color:#ff9100;content:"\E002"}.md-typeset .admonition.fail,.md-typeset .admonition.failure,.md-typeset .admonition.missing,.md-typeset details.fail,.md-typeset details.failure,.md-typeset details.missing{border-left:.4rem solid #ff5252}.md-typeset .admonition.fail>.admonition-title,.md-typeset .admonition.fail>summary,.md-typeset .admonition.failure>.admonition-title,.md-typeset .admonition.failure>summary,.md-typeset .admonition.missing>.admonition-title,.md-typeset .admonition.missing>summary,.md-typeset details.fail>.admonition-title,.md-typeset details.fail>summary,.md-typeset details.failure>.admonition-title,.md-typeset details.failure>summary,.md-typeset details.missing>.admonition-title,.md-typeset details.missing>summary{border-bottom:.1rem solid rgba(255,82,82,.1);background-color:rgba(255,82,82,.1)}.md-typeset .admonition.fail>.admonition-title:before,.md-typeset .admonition.fail>summary:before,.md-typeset .admonition.failure>.admonition-title:before,.md-typeset .admonition.failure>summary:before,.md-typeset .admonition.missing>.admonition-title:before,.md-typeset .admonition.missing>summary:before,.md-typeset details.fail>.admonition-title:before,.md-typeset details.fail>summary:before,.md-typeset details.failure>.admonition-title:before,.md-typeset details.failure>summary:before,.md-typeset details.missing>.admonition-title:before,.md-typeset details.missing>summary:before{color:#ff5252;content:"\E14C"}.md-typeset .admonition.danger,.md-typeset .admonition.error,.md-typeset details.danger,.md-typeset details.error{border-left:.4rem solid #ff1744}.md-typeset .admonition.danger>.admonition-title,.md-typeset .admonition.danger>summary,.md-typeset .admonition.error>.admonition-title,.md-typeset .admonition.error>summary,.md-typeset details.danger>.admonition-title,.md-typeset details.danger>summary,.md-typeset details.error>.admonition-title,.md-typeset details.error>summary{border-bottom:.1rem solid rgba(255,23,68,.1);background-color:rgba(255,23,68,.1)}.md-typeset .admonition.danger>.admonition-title:before,.md-typeset .admonition.danger>summary:before,.md-typeset .admonition.error>.admonition-title:before,.md-typeset .admonition.error>summary:before,.md-typeset details.danger>.admonition-title:before,.md-typeset details.danger>summary:before,.md-typeset details.error>.admonition-title:before,.md-typeset details.error>summary:before{color:#ff1744;content:"\E3E7"}.md-typeset .admonition.bug,.md-typeset details.bug{border-left:.4rem solid #f50057}.md-typeset .admonition.bug>.admonition-title,.md-typeset .admonition.bug>summary,.md-typeset details.bug>.admonition-title,.md-typeset details.bug>summary{border-bottom:.1rem solid rgba(245,0,87,.1);background-color:rgba(245,0,87,.1)}.md-typeset .admonition.bug>.admonition-title:before,.md-typeset .admonition.bug>summary:before,.md-typeset details.bug>.admonition-title:before,.md-typeset details.bug>summary:before{color:#f50057;content:"\E868"}.md-typeset .admonition.cite,.md-typeset .admonition.quote,.md-typeset details.cite,.md-typeset details.quote{border-left:.4rem solid #9e9e9e}.md-typeset .admonition.cite>.admonition-title,.md-typeset .admonition.cite>summary,.md-typeset .admonition.quote>.admonition-title,.md-typeset .admonition.quote>summary,.md-typeset details.cite>.admonition-title,.md-typeset details.cite>summary,.md-typeset details.quote>.admonition-title,.md-typeset details.quote>summary{border-bottom:.1rem solid hsla(0,0%,62%,.1);background-color:hsla(0,0%,62%,.1)}.md-typeset .admonition.cite>.admonition-title:before,.md-typeset .admonition.cite>summary:before,.md-typeset .admonition.quote>.admonition-title:before,.md-typeset .admonition.quote>summary:before,.md-typeset details.cite>.admonition-title:before,.md-typeset details.cite>summary:before,.md-typeset details.quote>.admonition-title:before,.md-typeset details.quote>summary:before{color:#9e9e9e;content:"\E244"}.codehilite .o,.codehilite .ow,.md-typeset .highlight .o,.md-typeset .highlight .ow{color:inherit}.codehilite .ge,.md-typeset .highlight .ge{color:#000}.codehilite .gr,.md-typeset .highlight .gr{color:#a00}.codehilite .gh,.md-typeset .highlight .gh{color:#999}.codehilite .go,.md-typeset .highlight .go{color:#888}.codehilite .gp,.md-typeset .highlight .gp{color:#555}.codehilite .gs,.md-typeset .highlight .gs{color:inherit}.codehilite .gu,.md-typeset .highlight .gu{color:#aaa}.codehilite .gt,.md-typeset .highlight .gt{color:#a00}.codehilite .gd,.md-typeset .highlight .gd{background-color:#fdd}.codehilite .gi,.md-typeset .highlight .gi{background-color:#dfd}.codehilite .k,.md-typeset .highlight .k{color:#3b78e7}.codehilite .kc,.md-typeset .highlight .kc{color:#a71d5d}.codehilite .kd,.codehilite .kn,.md-typeset .highlight .kd,.md-typeset .highlight .kn{color:#3b78e7}.codehilite .kp,.md-typeset .highlight .kp{color:#a71d5d}.codehilite .kr,.codehilite .kt,.md-typeset .highlight .kr,.md-typeset .highlight .kt{color:#3e61a2}.codehilite .c,.codehilite .cm,.md-typeset .highlight .c,.md-typeset .highlight .cm{color:#999}.codehilite .cp,.md-typeset .highlight .cp{color:#666}.codehilite .c1,.codehilite .ch,.codehilite .cs,.md-typeset .highlight .c1,.md-typeset .highlight .ch,.md-typeset .highlight .cs{color:#999}.codehilite .na,.codehilite .nb,.md-typeset .highlight .na,.md-typeset .highlight .nb{color:#c2185b}.codehilite .bp,.md-typeset .highlight .bp{color:#3e61a2}.codehilite .nc,.md-typeset .highlight .nc{color:#c2185b}.codehilite .no,.md-typeset .highlight .no{color:#3e61a2}.codehilite .nd,.codehilite .ni,.md-typeset .highlight .nd,.md-typeset .highlight .ni{color:#666}.codehilite .ne,.codehilite .nf,.md-typeset .highlight .ne,.md-typeset .highlight .nf{color:#c2185b}.codehilite .nl,.md-typeset .highlight .nl{color:#3b5179}.codehilite .nn,.md-typeset .highlight .nn{color:#ec407a}.codehilite .nt,.md-typeset .highlight .nt{color:#3b78e7}.codehilite .nv,.codehilite .vc,.codehilite .vg,.codehilite .vi,.md-typeset .highlight .nv,.md-typeset .highlight .vc,.md-typeset .highlight .vg,.md-typeset .highlight .vi{color:#3e61a2}.codehilite .nx,.md-typeset .highlight .nx{color:#ec407a}.codehilite .il,.codehilite .m,.codehilite .mf,.codehilite .mh,.codehilite .mi,.codehilite .mo,.md-typeset .highlight .il,.md-typeset .highlight .m,.md-typeset .highlight .mf,.md-typeset .highlight .mh,.md-typeset .highlight .mi,.md-typeset .highlight .mo{color:#e74c3c}.codehilite .s,.codehilite .sb,.codehilite .sc,.md-typeset .highlight .s,.md-typeset .highlight .sb,.md-typeset .highlight .sc{color:#0d904f}.codehilite .sd,.md-typeset .highlight .sd{color:#999}.codehilite .s2,.md-typeset .highlight .s2{color:#0d904f}.codehilite .se,.codehilite .sh,.codehilite .si,.codehilite .sx,.md-typeset .highlight .se,.md-typeset .highlight .sh,.md-typeset .highlight .si,.md-typeset .highlight .sx{color:#183691}.codehilite .sr,.md-typeset .highlight .sr{color:#009926}.codehilite .s1,.codehilite .ss,.md-typeset .highlight .s1,.md-typeset .highlight .ss{color:#0d904f}.codehilite .err,.md-typeset .highlight .err{color:#a61717}.codehilite .w,.md-typeset .highlight .w{color:transparent}.codehilite .hll,.md-typeset .highlight .hll{display:block;margin:0 -1.2rem;padding:0 1.2rem;background-color:rgba(255,235,59,.5)}.md-typeset .codehilite,.md-typeset .highlight{position:relative;margin:1em 0;padding:0;border-radius:.2rem;background-color:hsla(0,0%,93%,.5);color:#37474f;line-height:1.4;-webkit-overflow-scrolling:touch}.md-typeset .codehilite code,.md-typeset .codehilite pre,.md-typeset .highlight code,.md-typeset .highlight pre{display:block;margin:0;padding:1.05rem 1.2rem;background-color:transparent;overflow:auto;vertical-align:top}.md-typeset .codehilite code::-webkit-scrollbar,.md-typeset .codehilite pre::-webkit-scrollbar,.md-typeset .highlight code::-webkit-scrollbar,.md-typeset .highlight pre::-webkit-scrollbar{width:.4rem;height:.4rem}.md-typeset .codehilite code::-webkit-scrollbar-thumb,.md-typeset .codehilite pre::-webkit-scrollbar-thumb,.md-typeset .highlight code::-webkit-scrollbar-thumb,.md-typeset .highlight pre::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-typeset .codehilite code::-webkit-scrollbar-thumb:hover,.md-typeset .codehilite pre::-webkit-scrollbar-thumb:hover,.md-typeset .highlight code::-webkit-scrollbar-thumb:hover,.md-typeset .highlight pre::-webkit-scrollbar-thumb:hover{background-color:#536dfe}.md-typeset pre.codehilite,.md-typeset pre.highlight{overflow:visible}.md-typeset pre.codehilite code,.md-typeset pre.highlight code{display:block;padding:1.05rem 1.2rem;overflow:auto}.md-typeset .codehilitetable{display:block;margin:1em 0;border-radius:.2em;font-size:1.6rem;overflow:hidden}.md-typeset .codehilitetable tbody,.md-typeset .codehilitetable td{display:block;padding:0}.md-typeset .codehilitetable tr{display:-webkit-box;display:-ms-flexbox;display:flex}.md-typeset .codehilitetable .codehilite,.md-typeset .codehilitetable .highlight,.md-typeset .codehilitetable .linenodiv{margin:0;border-radius:0}.md-typeset .codehilitetable .linenodiv{padding:1.05rem 1.2rem}.md-typeset .codehilitetable .linenos{background-color:rgba(0,0,0,.07);color:rgba(0,0,0,.26);-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}.md-typeset .codehilitetable .linenos pre{margin:0;padding:0;background-color:transparent;color:inherit;text-align:right}.md-typeset .codehilitetable .code{-webkit-box-flex:1;-ms-flex:1;flex:1;overflow:hidden}.md-typeset>.codehilitetable{-webkit-box-shadow:none;box-shadow:none}.md-typeset [id^="fnref:"]{display:inline-block}.md-typeset [id^="fnref:"]:target{margin-top:-7.6rem;padding-top:7.6rem;pointer-events:none}.md-typeset [id^="fn:"]:before{display:none;height:0;content:""}.md-typeset [id^="fn:"]:target:before{display:block;margin-top:-7rem;padding-top:7rem;pointer-events:none}.md-typeset .footnote{color:rgba(0,0,0,.54);font-size:1.28rem}.md-typeset .footnote ol{margin-left:0}.md-typeset .footnote li{-webkit-transition:color .25s;transition:color .25s}.md-typeset .footnote li:target{color:rgba(0,0,0,.87)}.md-typeset .footnote li :first-child{margin-top:0}.md-typeset .footnote li:hover .footnote-backref,.md-typeset .footnote li:target .footnote-backref{-webkit-transform:translateX(0);transform:translateX(0);opacity:1}.md-typeset .footnote li:hover .footnote-backref:hover,.md-typeset .footnote li:target .footnote-backref{color:#536dfe}.md-typeset .footnote-ref{display:inline-block;pointer-events:auto}.md-typeset .footnote-ref:before{display:inline;margin:0 .2em;border-left:.1rem solid rgba(0,0,0,.26);font-size:1.25em;content:"";vertical-align:-.5rem}.md-typeset .footnote-backref{display:inline-block;-webkit-transform:translateX(.5rem);transform:translateX(.5rem);-webkit-transition:color .25s,opacity .125s .125s,-webkit-transform .25s .125s;transition:color .25s,opacity .125s .125s,-webkit-transform .25s .125s;transition:transform .25s .125s,color .25s,opacity .125s .125s;transition:transform .25s .125s,color .25s,opacity .125s .125s,-webkit-transform .25s .125s;color:rgba(0,0,0,.26);font-size:0;opacity:0;vertical-align:text-bottom}.md-typeset .footnote-backref:before{font-size:1.6rem;content:"\E31B"}.md-typeset .headerlink{display:inline-block;margin-left:1rem;-webkit-transform:translateY(.5rem);transform:translateY(.5rem);-webkit-transition:color .25s,opacity .125s .25s,-webkit-transform .25s .25s;transition:color .25s,opacity .125s .25s,-webkit-transform .25s .25s;transition:transform .25s .25s,color .25s,opacity .125s .25s;transition:transform .25s .25s,color .25s,opacity .125s .25s,-webkit-transform .25s .25s;opacity:0}html body .md-typeset .headerlink{color:rgba(0,0,0,.26)}.md-typeset h1[id] .headerlink{display:none}.md-typeset h2[id]:before{display:block;margin-top:-.8rem;padding-top:.8rem;content:""}.md-typeset h2[id]:target:before{margin-top:-6.8rem;padding-top:6.8rem}.md-typeset h2[id] .headerlink:focus,.md-typeset h2[id]:hover .headerlink,.md-typeset h2[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h2[id] .headerlink:focus,.md-typeset h2[id]:hover .headerlink:hover,.md-typeset h2[id]:target .headerlink{color:#536dfe}.md-typeset h3[id]:before{display:block;margin-top:-.9rem;padding-top:.9rem;content:""}.md-typeset h3[id]:target:before{margin-top:-6.9rem;padding-top:6.9rem}.md-typeset h3[id] .headerlink:focus,.md-typeset h3[id]:hover .headerlink,.md-typeset h3[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h3[id] .headerlink:focus,.md-typeset h3[id]:hover .headerlink:hover,.md-typeset h3[id]:target .headerlink{color:#536dfe}.md-typeset h4[id]:before{display:block;margin-top:-.9rem;padding-top:.9rem;content:""}.md-typeset h4[id]:target:before{margin-top:-6.9rem;padding-top:6.9rem}.md-typeset h4[id] .headerlink:focus,.md-typeset h4[id]:hover .headerlink,.md-typeset h4[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h4[id] .headerlink:focus,.md-typeset h4[id]:hover .headerlink:hover,.md-typeset h4[id]:target .headerlink{color:#536dfe}.md-typeset h5[id]:before{display:block;margin-top:-1.1rem;padding-top:1.1rem;content:""}.md-typeset h5[id]:target:before{margin-top:-7.1rem;padding-top:7.1rem}.md-typeset h5[id] .headerlink:focus,.md-typeset h5[id]:hover .headerlink,.md-typeset h5[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h5[id] .headerlink:focus,.md-typeset h5[id]:hover .headerlink:hover,.md-typeset h5[id]:target .headerlink{color:#536dfe}.md-typeset h6[id]:before{display:block;margin-top:-1.1rem;padding-top:1.1rem;content:""}.md-typeset h6[id]:target:before{margin-top:-7.1rem;padding-top:7.1rem}.md-typeset h6[id] .headerlink:focus,.md-typeset h6[id]:hover .headerlink,.md-typeset h6[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h6[id] .headerlink:focus,.md-typeset h6[id]:hover .headerlink:hover,.md-typeset h6[id]:target .headerlink{color:#536dfe}.md-typeset .MJXc-display{margin:.75em 0;padding:.75em 0;overflow:auto;-webkit-overflow-scrolling:touch}.md-typeset .MathJax_CHTML{outline:0}.md-typeset .critic.comment,.md-typeset del.critic,.md-typeset ins.critic{margin:0 .25em;padding:.0625em 0;border-radius:.2rem;-webkit-box-decoration-break:clone;box-decoration-break:clone}.md-typeset del.critic{background-color:#fdd;-webkit-box-shadow:.25em 0 0 #fdd,-.25em 0 0 #fdd;box-shadow:.25em 0 0 #fdd,-.25em 0 0 #fdd}.md-typeset ins.critic{background-color:#dfd;-webkit-box-shadow:.25em 0 0 #dfd,-.25em 0 0 #dfd;box-shadow:.25em 0 0 #dfd,-.25em 0 0 #dfd}.md-typeset .critic.comment{background-color:hsla(0,0%,93%,.5);color:#37474f;-webkit-box-shadow:.25em 0 0 hsla(0,0%,93%,.5),-.25em 0 0 hsla(0,0%,93%,.5);box-shadow:.25em 0 0 hsla(0,0%,93%,.5),-.25em 0 0 hsla(0,0%,93%,.5)}.md-typeset .critic.comment:before{padding-right:.125em;color:rgba(0,0,0,.26);content:"\E0B7";vertical-align:-.125em}.md-typeset .critic.block{display:block;margin:1em 0;padding-right:1.6rem;padding-left:1.6rem;-webkit-box-shadow:none;box-shadow:none}.md-typeset .critic.block :first-child{margin-top:.5em}.md-typeset .critic.block :last-child{margin-bottom:.5em}.md-typeset details{padding-top:0}.md-typeset details[open]>summary:after{-webkit-transform:rotate(180deg);transform:rotate(180deg)}.md-typeset details:not([open]){padding-bottom:0}.md-typeset details:not([open])>summary{border-bottom:none}.md-typeset details summary{padding-right:4rem}.no-details .md-typeset details:not([open])>*{display:none}.no-details .md-typeset details:not([open]) summary{display:block}.md-typeset summary{display:block;outline:none;cursor:pointer}.md-typeset summary::-webkit-details-marker{display:none}.md-typeset summary:after{position:absolute;top:.8rem;right:1.2rem;color:rgba(0,0,0,.26);font-size:2rem;content:"\E313"}.md-typeset .emojione{width:2rem;vertical-align:text-top}.md-typeset code.codehilite,.md-typeset code.highlight{margin:0 .29412em;padding:.07353em 0}.md-typeset .task-list-item{position:relative;list-style-type:none}.md-typeset .task-list-item [type=checkbox]{position:absolute;top:.45em;left:-2em}.md-typeset .task-list-control .task-list-indicator:before{position:absolute;top:.15em;left:-1.25em;color:rgba(0,0,0,.26);font-size:1.25em;content:"\E835";vertical-align:-.25em}.md-typeset .task-list-control [type=checkbox]:checked+.task-list-indicator:before{content:"\E834"}.md-typeset .task-list-control [type=checkbox]{opacity:0;z-index:-1}@media print{.md-typeset a:after{color:rgba(0,0,0,.54);content:" [" attr(href) "]"}.md-typeset code,.md-typeset pre{white-space:pre-wrap}.md-typeset code{-webkit-box-shadow:none;box-shadow:none;-webkit-box-decoration-break:initial;box-decoration-break:slice}.md-clipboard,.md-content__icon,.md-footer,.md-header,.md-sidebar,.md-tabs,.md-typeset .headerlink{display:none}}@media only screen and (max-width:44.9375em){.md-typeset pre{margin:1em -1.6rem;border-radius:0}.md-typeset pre>code{padding:1.05rem 1.6rem}.md-footer-nav__link--prev .md-footer-nav__title{display:none}.md-search-result__teaser{max-height:5rem;-webkit-line-clamp:3}.codehilite .hll,.md-typeset .highlight .hll{margin:0 -1.6rem;padding:0 1.6rem}.md-typeset>.codehilite,.md-typeset>.highlight{margin:1em -1.6rem;border-radius:0}.md-typeset>.codehilite code,.md-typeset>.codehilite pre,.md-typeset>.highlight code,.md-typeset>.highlight pre{padding:1.05rem 1.6rem}.md-typeset>.codehilitetable{margin:1em -1.6rem;border-radius:0}.md-typeset>.codehilitetable .codehilite>code,.md-typeset>.codehilitetable .codehilite>pre,.md-typeset>.codehilitetable .highlight>code,.md-typeset>.codehilitetable .highlight>pre,.md-typeset>.codehilitetable .linenodiv{padding:1rem 1.6rem}.md-typeset>p>.MJXc-display{margin:.75em -1.6rem;padding:.25em 1.6rem}}@media only screen and (min-width:100em){html{font-size:68.75%}}@media only screen and (min-width:125em){html{font-size:75%}}@media only screen and (max-width:59.9375em){body[data-md-state=lock]{overflow:hidden}.ios body[data-md-state=lock] .md-container{display:none}html .md-nav__link[for=toc]{display:block;padding-right:4.8rem}html .md-nav__link[for=toc]:after{color:inherit;content:"\E8DE"}html .md-nav__link[for=toc]+.md-nav__link{display:none}html .md-nav__link[for=toc]~.md-nav{display:-webkit-box;display:-ms-flexbox;display:flex}.md-nav__source{display:block;padding:0 .4rem;background-color:rgba(50,64,144,.9675);color:#fff}.md-search__overlay{position:absolute;top:.4rem;left:.4rem;width:3.6rem;height:3.6rem;-webkit-transform-origin:center;transform-origin:center;-webkit-transition:opacity .2s .2s,-webkit-transform .3s .1s;transition:opacity .2s .2s,-webkit-transform .3s .1s;transition:transform .3s .1s,opacity .2s .2s;transition:transform .3s .1s,opacity .2s .2s,-webkit-transform .3s .1s;border-radius:2rem;background-color:#fff;overflow:hidden;pointer-events:none}[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transition:opacity .1s,-webkit-transform .4s;transition:opacity .1s,-webkit-transform .4s;transition:transform .4s,opacity .1s;transition:transform .4s,opacity .1s,-webkit-transform .4s;opacity:1}.md-search__inner{position:fixed;top:0;left:100%;width:100%;height:100%;-webkit-transform:translateX(5%);transform:translateX(5%);-webkit-transition:left 0s .3s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.4,0,.2,1) .15s;transition:left 0s .3s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.4,0,.2,1) .15s;transition:left 0s .3s,transform .15s cubic-bezier(.4,0,.2,1) .15s,opacity .15s .15s;transition:left 0s .3s,transform .15s cubic-bezier(.4,0,.2,1) .15s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.4,0,.2,1) .15s;opacity:0;z-index:2}[data-md-toggle=search]:checked~.md-header .md-search__inner{left:0;-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:left 0s 0s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1) .15s;transition:left 0s 0s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1) .15s;transition:left 0s 0s,transform .15s cubic-bezier(.1,.7,.1,1) .15s,opacity .15s .15s;transition:left 0s 0s,transform .15s cubic-bezier(.1,.7,.1,1) .15s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1) .15s;opacity:1}.md-search__input{width:100%;height:4.8rem;font-size:1.8rem}.md-search__icon[for=search]{top:1.2rem;left:1.6rem}.md-search__icon[for=search][for=search]:before{content:"\E5C4"}.md-search__icon[type=reset]{top:1.2rem;right:1.6rem}.md-search__output{top:4.8rem;bottom:0}.md-search-result__article--document:before{display:none}}@media only screen and (max-width:76.1875em){[data-md-toggle=drawer]:checked~.md-overlay{width:100%;height:100%;-webkit-transition:width 0s,height 0s,opacity .25s;transition:width 0s,height 0s,opacity .25s;opacity:1}.md-header-nav__button.md-icon--home,.md-header-nav__button.md-logo{display:none}.md-hero__inner{margin-top:4.8rem;margin-bottom:2.4rem}.md-nav{background-color:#fff}.md-nav--primary,.md-nav--primary .md-nav{display:-webkit-box;display:-ms-flexbox;display:flex;position:absolute;top:0;right:0;left:0;-webkit-box-orient:vertical;-webkit-box-direction:normal;-ms-flex-direction:column;flex-direction:column;height:100%;z-index:1}.md-nav--primary .md-nav__item,.md-nav--primary .md-nav__title{font-size:1.6rem;line-height:1.5}html .md-nav--primary .md-nav__title{position:relative;height:11.2rem;padding:6rem 1.6rem .4rem;background-color:rgba(0,0,0,.07);color:rgba(0,0,0,.54);font-weight:400;line-height:4.8rem;white-space:nowrap;cursor:pointer}html .md-nav--primary .md-nav__title:before{display:block;position:absolute;top:.4rem;left:.4rem;width:4rem;height:4rem;color:rgba(0,0,0,.54)}html .md-nav--primary .md-nav__title~.md-nav__list{background-color:#fff;-webkit-box-shadow:0 .1rem 0 rgba(0,0,0,.07) inset;box-shadow:inset 0 .1rem 0 rgba(0,0,0,.07)}html .md-nav--primary .md-nav__title~.md-nav__list>.md-nav__item:first-child{border-top:0}html .md-nav--primary .md-nav__title--site{position:relative;background-color:#3f51b5;color:#fff}html .md-nav--primary .md-nav__title--site .md-nav__button{display:block;position:absolute;top:.4rem;left:.4rem;width:6.4rem;height:6.4rem;font-size:4.8rem}html .md-nav--primary .md-nav__title--site:before{display:none}.md-nav--primary .md-nav__list{-webkit-box-flex:1;-ms-flex:1;flex:1;overflow-y:auto}.md-nav--primary .md-nav__item{padding:0;border-top:.1rem solid rgba(0,0,0,.07)}.md-nav--primary .md-nav__item--nested>.md-nav__link{padding-right:4.8rem}.md-nav--primary .md-nav__item--nested>.md-nav__link:after{content:"\E315"}.md-nav--primary .md-nav__link{position:relative;margin-top:0;padding:1.2rem 1.6rem}.md-nav--primary .md-nav__link:after{position:absolute;top:50%;right:1.2rem;margin-top:-1.2rem;color:inherit;font-size:2.4rem}.md-nav--primary .md-nav--secondary .md-nav__link{position:static}.md-nav--primary .md-nav--secondary .md-nav{position:static;background-color:transparent}.md-nav--primary .md-nav--secondary .md-nav .md-nav__link{padding-left:2.8rem}.md-nav--primary .md-nav--secondary .md-nav .md-nav .md-nav__link{padding-left:4rem}.md-nav--primary .md-nav--secondary .md-nav .md-nav .md-nav .md-nav__link{padding-left:5.2rem}.md-nav--primary .md-nav--secondary .md-nav .md-nav .md-nav .md-nav .md-nav__link{padding-left:6.4rem}.md-nav__toggle~.md-nav{display:-webkit-box;display:-ms-flexbox;display:flex;-webkit-transform:translateX(100%);transform:translateX(100%);-webkit-transition:opacity .125s .05s,-webkit-transform .25s cubic-bezier(.8,0,.6,1);transition:opacity .125s .05s,-webkit-transform .25s cubic-bezier(.8,0,.6,1);transition:transform .25s cubic-bezier(.8,0,.6,1),opacity .125s .05s;transition:transform .25s cubic-bezier(.8,0,.6,1),opacity .125s .05s,-webkit-transform .25s cubic-bezier(.8,0,.6,1);opacity:0}.no-csstransforms3d .md-nav__toggle~.md-nav{display:none}.md-nav__toggle:checked~.md-nav{-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:opacity .125s .125s,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:opacity .125s .125s,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .125s .125s;transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .125s .125s,-webkit-transform .25s cubic-bezier(.4,0,.2,1);opacity:1}.no-csstransforms3d .md-nav__toggle:checked~.md-nav{display:-webkit-box;display:-ms-flexbox;display:flex}.md-sidebar--primary{position:fixed;top:0;left:-24.2rem;width:24.2rem;height:100%;-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:-webkit-transform .25s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .25s;transition:-webkit-transform .25s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .25s;transition:transform .25s cubic-bezier(.4,0,.2,1),box-shadow .25s;transition:transform .25s cubic-bezier(.4,0,.2,1),box-shadow .25s,-webkit-transform .25s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .25s;background-color:#fff;z-index:3}.no-csstransforms3d .md-sidebar--primary{display:none}[data-md-toggle=drawer]:checked~.md-container .md-sidebar--primary{-webkit-box-shadow:0 8px 10px 1px rgba(0,0,0,.14),0 3px 14px 2px rgba(0,0,0,.12),0 5px 5px -3px rgba(0,0,0,.4);box-shadow:0 8px 10px 1px rgba(0,0,0,.14),0 3px 14px 2px rgba(0,0,0,.12),0 5px 5px -3px rgba(0,0,0,.4);-webkit-transform:translateX(24.2rem);transform:translateX(24.2rem)}.no-csstransforms3d [data-md-toggle=drawer]:checked~.md-container .md-sidebar--primary{display:block}.md-sidebar--primary .md-sidebar__scrollwrap{overflow:hidden;position:absolute;top:0;right:0;bottom:0;left:0;margin:0}.md-tabs{display:none}}@media only screen and (min-width:60em){.md-content{margin-right:24.2rem}.md-header-nav__button.md-icon--search{display:none}.md-header-nav__source{display:block;width:23rem;max-width:23rem;margin-left:2.8rem;padding-right:1.2rem}.md-search{padding:.4rem}.md-search__overlay{position:fixed;top:0;left:0;width:0;height:0;-webkit-transition:width 0s .25s,height 0s .25s,opacity .25s;transition:width 0s .25s,height 0s .25s,opacity .25s;background-color:rgba(0,0,0,.54);cursor:pointer}[data-md-toggle=search]:checked~.md-header .md-search__overlay{width:100%;height:100%;-webkit-transition:width 0s,height 0s,opacity .25s;transition:width 0s,height 0s,opacity .25s;opacity:1}.md-search__inner{position:relative;width:23rem;padding:.2rem 0;float:right;-webkit-transition:width .25s cubic-bezier(.1,.7,.1,1);transition:width .25s cubic-bezier(.1,.7,.1,1)}.md-search__form,.md-search__input{border-radius:.2rem}.md-search__input{width:100%;height:3.6rem;padding-left:4.4rem;-webkit-transition:background-color .25s cubic-bezier(.1,.7,.1,1),color .25s cubic-bezier(.1,.7,.1,1);transition:background-color .25s cubic-bezier(.1,.7,.1,1),color .25s cubic-bezier(.1,.7,.1,1);background-color:rgba(0,0,0,.26);color:inherit;font-size:1.6rem}.md-search__input+.md-search__icon{color:inherit}.md-search__input::-webkit-input-placeholder{color:hsla(0,0%,100%,.7)}.md-search__input:-ms-input-placeholder,.md-search__input::-ms-input-placeholder{color:hsla(0,0%,100%,.7)}.md-search__input::placeholder{color:hsla(0,0%,100%,.7)}.md-search__input:hover{background-color:hsla(0,0%,100%,.12)}[data-md-toggle=search]:checked~.md-header .md-search__input{border-radius:.2rem .2rem 0 0;background-color:#fff;color:rgba(0,0,0,.87);text-overflow:none}[data-md-toggle=search]:checked~.md-header .md-search__input+.md-search__icon,[data-md-toggle=search]:checked~.md-header .md-search__input::-webkit-input-placeholder{color:rgba(0,0,0,.54)}[data-md-toggle=search]:checked~.md-header .md-search__input+.md-search__icon,[data-md-toggle=search]:checked~.md-header .md-search__input:-ms-input-placeholder,[data-md-toggle=search]:checked~.md-header .md-search__input::-ms-input-placeholder{color:rgba(0,0,0,.54)}[data-md-toggle=search]:checked~.md-header .md-search__input+.md-search__icon,[data-md-toggle=search]:checked~.md-header .md-search__input::placeholder{color:rgba(0,0,0,.54)}.md-search__output{top:3.8rem;-webkit-transition:opacity .4s;transition:opacity .4s;opacity:0}[data-md-toggle=search]:checked~.md-header .md-search__output{-webkit-box-shadow:0 6px 10px 0 rgba(0,0,0,.14),0 1px 18px 0 rgba(0,0,0,.12),0 3px 5px -1px rgba(0,0,0,.4);box-shadow:0 6px 10px 0 rgba(0,0,0,.14),0 1px 18px 0 rgba(0,0,0,.12),0 3px 5px -1px rgba(0,0,0,.4);opacity:1}.md-search__scrollwrap{max-height:0}[data-md-toggle=search]:checked~.md-header .md-search__scrollwrap{max-height:75vh}.md-search__scrollwrap::-webkit-scrollbar{width:.4rem;height:.4rem}.md-search__scrollwrap::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-search__scrollwrap::-webkit-scrollbar-thumb:hover{background-color:#536dfe}.md-search-result__article,.md-search-result__meta{padding-left:4.4rem}.md-sidebar--secondary{display:block;margin-left:100%;-webkit-transform:translate(-100%);transform:translate(-100%)}}@media only screen and (min-width:76.25em){.md-content{margin-left:24.2rem}.md-content__inner{margin-right:2.4rem;margin-left:2.4rem}.md-header-nav__button.md-icon--menu{display:none}.md-nav[data-md-state=animate]{-webkit-transition:max-height .25s cubic-bezier(.86,0,.07,1);transition:max-height .25s cubic-bezier(.86,0,.07,1)}.md-nav__toggle~.md-nav{max-height:0;overflow:hidden}.md-nav[data-md-state=expand],.md-nav__toggle:checked~.md-nav{max-height:100%}.md-nav__item--nested>.md-nav>.md-nav__title{display:none}.md-nav__item--nested>.md-nav__link:after{display:inline-block;-webkit-transform-origin:.45em .45em;transform-origin:.45em .45em;-webkit-transform-style:preserve-3d;transform-style:preserve-3d;vertical-align:-.125em}.js .md-nav__item--nested>.md-nav__link:after{-webkit-transition:-webkit-transform .4s;transition:-webkit-transform .4s;transition:transform .4s;transition:transform .4s,-webkit-transform .4s}.md-nav__item--nested .md-nav__toggle:checked~.md-nav__link:after{-webkit-transform:rotateX(180deg);transform:rotateX(180deg)}.md-search__scrollwrap,[data-md-toggle=search]:checked~.md-header .md-search__inner{width:68.8rem}.md-sidebar--secondary{margin-left:122rem}.md-tabs~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested{font-size:0}.md-tabs--active~.md-main .md-nav--primary .md-nav__title--site{display:none}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item{font-size:0}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested{display:none;font-size:1.4rem;overflow:auto}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested>.md-nav__link{margin-top:0;font-weight:700;pointer-events:none}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested>.md-nav__link:after{display:none}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--active{display:block}.md-tabs--active~.md-main .md-nav[data-md-level="1"]{max-height:none}.md-tabs--active~.md-main .md-nav[data-md-level="1"]>.md-nav__list>.md-nav__item{padding-left:0}}@media only screen and (min-width:45em){.md-footer-nav__link{width:50%}.md-footer-copyright{max-width:75%;float:left}.md-footer-social{padding:1.2rem 0;float:right}}@media only screen and (max-width:29.9375em){[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transform:scale(45);transform:scale(45)}}@media only screen and (min-width:30em) and (max-width:44.9375em){[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transform:scale(60);transform:scale(60)}}@media only screen and (min-width:45em) and (max-width:59.9375em){[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transform:scale(75);transform:scale(75)}}@media only screen and (min-width:60em) and (max-width:76.1875em){.md-search__scrollwrap,[data-md-toggle=search]:checked~.md-header .md-search__inner{width:46.8rem}.md-search-result__teaser{max-height:5rem;-webkit-line-clamp:3}} -/*# sourceMappingURL=data:application/json;charset=utf-8;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbXSwibmFtZXMiOltdLCJtYXBwaW5ncyI6IiIsImZpbGUiOiJhc3NldHMvc3R5bGVzaGVldHMvYXBwbGljYXRpb24uYWM2NDI1MWUuY3NzIiwic291cmNlUm9vdCI6IiJ9*/ \ No newline at end of file +html{-webkit-box-sizing:border-box;box-sizing:border-box}*,:after,:before{-webkit-box-sizing:inherit;box-sizing:inherit}html{-webkit-text-size-adjust:none;-moz-text-size-adjust:none;-ms-text-size-adjust:none;text-size-adjust:none}body{margin:0}hr{overflow:visible;-webkit-box-sizing:content-box;box-sizing:content-box}a{-webkit-text-decoration-skip:objects}a,button,input,label{-webkit-tap-highlight-color:transparent}a{color:inherit;text-decoration:none}a:active,a:hover{outline-width:0}small,sub,sup{font-size:80%}sub,sup{position:relative;line-height:0;vertical-align:baseline}sub{bottom:-.25em}sup{top:-.5em}img{border-style:none}table{border-collapse:collapse;border-spacing:0}td,th{font-weight:400;vertical-align:top}button{padding:0;background:transparent;font-size:inherit}button,input{border:0;outline:0}.md-clipboard:before,.md-icon,.md-nav__button,.md-nav__link:after,.md-nav__title:before,.md-search-result__article--document:before,.md-source-file:before,.md-typeset .admonition>.admonition-title:before,.md-typeset .admonition>summary:before,.md-typeset .critic.comment:before,.md-typeset .footnote-backref,.md-typeset .task-list-control .task-list-indicator:before,.md-typeset details>.admonition-title:before,.md-typeset details>summary:before,.md-typeset summary:after{font-family:Material Icons;font-style:normal;font-variant:normal;font-weight:400;line-height:1;text-transform:none;white-space:nowrap;speak:none;word-wrap:normal;direction:ltr}.md-content__icon,.md-footer-nav__button,.md-header-nav__button,.md-nav__button,.md-nav__title:before,.md-search-result__article--document:before{display:inline-block;margin:.4rem;padding:.8rem;font-size:2.4rem;cursor:pointer}.md-icon--arrow-back:before{content:"\E5C4"}.md-icon--arrow-forward:before{content:"\E5C8"}.md-icon--menu:before{content:"\E5D2"}.md-icon--search:before{content:"\E8B6"}body{-webkit-font-smoothing:antialiased;-moz-osx-font-smoothing:grayscale}body,input{color:rgba(0,0,0,.87);-webkit-font-feature-settings:"kern","liga";font-feature-settings:"kern","liga";font-family:Helvetica Neue,Helvetica,Arial,sans-serif}code,kbd,pre{color:rgba(0,0,0,.87);-webkit-font-feature-settings:"kern";font-feature-settings:"kern";font-family:Courier New,Courier,monospace}.md-typeset{font-size:1.6rem;line-height:1.6;-webkit-print-color-adjust:exact}.md-typeset blockquote,.md-typeset ol,.md-typeset p,.md-typeset ul{margin:1em 0}.md-typeset h1{margin:0 0 4rem;color:rgba(0,0,0,.54);font-size:3.125rem;line-height:1.3}.md-typeset h1,.md-typeset h2{font-weight:300;letter-spacing:-.01em}.md-typeset h2{margin:4rem 0 1.6rem;font-size:2.5rem;line-height:1.4}.md-typeset h3{margin:3.2rem 0 1.6rem;font-size:2rem;font-weight:400;letter-spacing:-.01em;line-height:1.5}.md-typeset h2+h3{margin-top:1.6rem}.md-typeset h4{font-size:1.6rem}.md-typeset h4,.md-typeset h5,.md-typeset h6{margin:1.6rem 0;font-weight:700;letter-spacing:-.01em}.md-typeset h5,.md-typeset h6{color:rgba(0,0,0,.54);font-size:1.28rem}.md-typeset h5{text-transform:uppercase}.md-typeset hr{margin:1.5em 0;border-bottom:.1rem dotted rgba(0,0,0,.26)}.md-typeset a{color:#3f51b5;word-break:break-word}.md-typeset a,.md-typeset a:before{-webkit-transition:color .125s;transition:color .125s}.md-typeset a:active,.md-typeset a:hover{color:#536dfe}.md-typeset code,.md-typeset pre{background-color:hsla(0,0%,93%,.5);color:#37474f;font-size:85%}.md-typeset code{margin:0 .29412em;padding:.07353em 0;border-radius:.2rem;-webkit-box-shadow:.29412em 0 0 hsla(0,0%,93%,.5),-.29412em 0 0 hsla(0,0%,93%,.5);box-shadow:.29412em 0 0 hsla(0,0%,93%,.5),-.29412em 0 0 hsla(0,0%,93%,.5);word-break:break-word;-webkit-box-decoration-break:clone;box-decoration-break:clone}.md-typeset h1 code,.md-typeset h2 code,.md-typeset h3 code,.md-typeset h4 code,.md-typeset h5 code,.md-typeset h6 code{margin:0;background-color:transparent;-webkit-box-shadow:none;box-shadow:none}.md-typeset a>code{margin:inherit;padding:inherit;border-radius:none;background-color:inherit;color:inherit;-webkit-box-shadow:none;box-shadow:none}.md-typeset pre{position:relative;margin:1em 0;border-radius:.2rem;line-height:1.4;-webkit-overflow-scrolling:touch}.md-typeset pre>code{display:block;margin:0;padding:1.05rem 1.2rem;background-color:transparent;font-size:inherit;-webkit-box-shadow:none;box-shadow:none;-webkit-box-decoration-break:none;box-decoration-break:none;overflow:auto}.md-typeset pre>code::-webkit-scrollbar{width:.4rem;height:.4rem}.md-typeset pre>code::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-typeset pre>code::-webkit-scrollbar-thumb:hover{background-color:#536dfe}.md-typeset kbd{padding:0 .29412em;border:.1rem solid #c9c9c9;border-radius:.2rem;border-bottom-color:#bcbcbc;background-color:#fcfcfc;color:#555;font-size:85%;-webkit-box-shadow:0 .1rem 0 #b0b0b0;box-shadow:0 .1rem 0 #b0b0b0;word-break:break-word}.md-typeset mark{margin:0 .25em;padding:.0625em 0;border-radius:.2rem;background-color:rgba(255,235,59,.5);-webkit-box-shadow:.25em 0 0 rgba(255,235,59,.5),-.25em 0 0 rgba(255,235,59,.5);box-shadow:.25em 0 0 rgba(255,235,59,.5),-.25em 0 0 rgba(255,235,59,.5);word-break:break-word;-webkit-box-decoration-break:clone;box-decoration-break:clone}.md-typeset abbr{border-bottom:.1rem dotted rgba(0,0,0,.54);text-decoration:none;cursor:help}.md-typeset small{opacity:.75}.md-typeset sub,.md-typeset sup{margin-left:.07812em}.md-typeset blockquote{padding-left:1.2rem;border-left:.4rem solid rgba(0,0,0,.26);color:rgba(0,0,0,.54)}.md-typeset ul{list-style-type:disc}.md-typeset ol,.md-typeset ul{margin-left:.625em;padding:0}.md-typeset ol ol,.md-typeset ul ol{list-style-type:lower-alpha}.md-typeset ol ol ol,.md-typeset ul ol ol{list-style-type:lower-roman}.md-typeset ol li,.md-typeset ul li{margin-bottom:.5em;margin-left:1.25em}.md-typeset ol li blockquote,.md-typeset ol li p,.md-typeset ul li blockquote,.md-typeset ul li p{margin:.5em 0}.md-typeset ol li:last-child,.md-typeset ul li:last-child{margin-bottom:0}.md-typeset ol li ol,.md-typeset ol li ul,.md-typeset ul li ol,.md-typeset ul li ul{margin:.5em 0 .5em .625em}.md-typeset dd{margin:1em 0 1em 1.875em}.md-typeset iframe,.md-typeset img,.md-typeset svg{max-width:100%}.md-typeset table:not([class]){-webkit-box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);display:inline-block;max-width:100%;border-radius:.2rem;font-size:1.28rem;overflow:auto;-webkit-overflow-scrolling:touch}.md-typeset table:not([class])+*{margin-top:1.5em}.md-typeset table:not([class]) td:not([align]),.md-typeset table:not([class]) th:not([align]){text-align:left}.md-typeset table:not([class]) th{min-width:10rem;padding:1.2rem 1.6rem;background-color:rgba(0,0,0,.54);color:#fff;vertical-align:top}.md-typeset table:not([class]) td{padding:1.2rem 1.6rem;border-top:.1rem solid rgba(0,0,0,.07);vertical-align:top}.md-typeset table:not([class]) tr:first-child td{border-top:0}.md-typeset table:not([class]) a{word-break:normal}.md-typeset__scrollwrap{margin:1em -1.6rem;overflow-x:auto;-webkit-overflow-scrolling:touch}.md-typeset .md-typeset__table{display:inline-block;margin-bottom:.5em;padding:0 1.6rem}.md-typeset .md-typeset__table table{display:table;width:100%;margin:0;overflow:hidden}html{font-size:62.5%;overflow-x:hidden}body,html{height:100%}body{position:relative}hr{display:block;height:.1rem;padding:0;border:0}.md-svg{display:none}.md-grid{max-width:122rem;margin-right:auto;margin-left:auto}.md-container,.md-main{overflow:auto}.md-container{display:table;width:100%;height:100%;padding-top:4.8rem;table-layout:fixed}.md-main{display:table-row;height:100%}.md-main__inner{height:100%;padding-top:3rem;padding-bottom:.1rem}.md-toggle{display:none}.md-overlay{position:fixed;top:0;width:0;height:0;-webkit-transition:width 0s .25s,height 0s .25s,opacity .25s;transition:width 0s .25s,height 0s .25s,opacity .25s;background-color:rgba(0,0,0,.54);opacity:0;z-index:3}.md-flex{display:table}.md-flex__cell{display:table-cell;position:relative;vertical-align:top}.md-flex__cell--shrink{width:0}.md-flex__cell--stretch{display:table;width:100%;table-layout:fixed}.md-flex__ellipsis{display:table-cell;text-overflow:ellipsis;white-space:nowrap;overflow:hidden}@page{margin:25mm}.md-clipboard{position:absolute;top:.6rem;right:.6rem;width:2.8rem;height:2.8rem;border-radius:.2rem;font-size:1.6rem;cursor:pointer;z-index:1;-webkit-backface-visibility:hidden;backface-visibility:hidden}.md-clipboard:before{-webkit-transition:color .25s,opacity .25s;transition:color .25s,opacity .25s;color:rgba(0,0,0,.54);content:"\E14D";opacity:.25}.codehilite:hover .md-clipboard:before,.md-typeset .highlight:hover .md-clipboard:before,pre:hover .md-clipboard:before{opacity:1}.md-clipboard:active:before,.md-clipboard:hover:before{color:#536dfe}.md-clipboard__message{display:block;position:absolute;top:0;right:3.4rem;padding:.6rem 1rem;-webkit-transform:translateX(.8rem);transform:translateX(.8rem);-webkit-transition:opacity .175s,-webkit-transform .25s cubic-bezier(.9,.1,.9,0);transition:opacity .175s,-webkit-transform .25s cubic-bezier(.9,.1,.9,0);transition:transform .25s cubic-bezier(.9,.1,.9,0),opacity .175s;transition:transform .25s cubic-bezier(.9,.1,.9,0),opacity .175s,-webkit-transform .25s cubic-bezier(.9,.1,.9,0);border-radius:.2rem;background-color:rgba(0,0,0,.54);color:#fff;font-size:1.28rem;white-space:nowrap;opacity:0;pointer-events:none}.md-clipboard__message--active{-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:opacity .175s 75ms,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:opacity .175s 75ms,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .175s 75ms;transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .175s 75ms,-webkit-transform .25s cubic-bezier(.4,0,.2,1);opacity:1;pointer-events:auto}.md-clipboard__message:before{content:attr(aria-label)}.md-clipboard__message:after{display:block;position:absolute;top:50%;right:-.4rem;width:0;margin-top:-.4rem;border-width:.4rem 0 .4rem .4rem;border-style:solid;border-color:transparent rgba(0,0,0,.54);content:""}.md-content__inner{margin:0 1.6rem 2.4rem;padding-top:1.2rem}.md-content__inner:before{display:block;height:.8rem;content:""}.md-content__inner>:last-child{margin-bottom:0}.md-content__icon{position:relative;margin:.8rem 0;padding:0;float:right}.md-typeset .md-content__icon{color:rgba(0,0,0,.26)}.md-header{position:fixed;top:0;right:0;left:0;height:4.8rem;-webkit-transition:background-color .25s,color .25s;transition:background-color .25s,color .25s;background-color:#3f51b5;color:#fff;z-index:2;-webkit-backface-visibility:hidden;backface-visibility:hidden}.md-header,.no-js .md-header{-webkit-box-shadow:none;box-shadow:none}.md-header-nav{padding:0 .4rem}.md-header-nav__button{position:relative;-webkit-transition:opacity .25s;transition:opacity .25s;z-index:1}.md-header-nav__button:hover{opacity:.7}.md-header-nav__button.md-logo *{display:block}.no-js .md-header-nav__button.md-icon--search{display:none}.md-header-nav__topic{display:block;position:absolute;-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);text-overflow:ellipsis;white-space:nowrap;overflow:hidden}.md-header-nav__topic+.md-header-nav__topic{-webkit-transform:translateX(2.5rem);transform:translateX(2.5rem);-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s;transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);opacity:0;z-index:-1;pointer-events:none}.no-js .md-header-nav__topic{position:static}.md-header-nav__title{padding:0 2rem;font-size:1.8rem;line-height:4.8rem}.md-header-nav__title[data-md-state=active] .md-header-nav__topic{-webkit-transform:translateX(-2.5rem);transform:translateX(-2.5rem);-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s;transition:transform .4s cubic-bezier(1,.7,.1,.1),opacity .15s,-webkit-transform .4s cubic-bezier(1,.7,.1,.1);opacity:0;z-index:-1;pointer-events:none}.md-header-nav__title[data-md-state=active] .md-header-nav__topic+.md-header-nav__topic{-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .15s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);opacity:1;z-index:0;pointer-events:auto}.md-header-nav__source{display:none}.md-hero{-webkit-transition:background .25s;transition:background .25s;background-color:#3f51b5;color:#fff;font-size:2rem;overflow:hidden}.md-hero__inner{margin-top:2rem;padding:1.6rem 1.6rem .8rem;-webkit-transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);-webkit-transition-delay:.1s;transition-delay:.1s}[data-md-state=hidden] .md-hero__inner{pointer-events:none;-webkit-transform:translateY(1.25rem);transform:translateY(1.25rem);-webkit-transition:opacity .1s 0s,-webkit-transform 0s .4s;transition:opacity .1s 0s,-webkit-transform 0s .4s;transition:transform 0s .4s,opacity .1s 0s;transition:transform 0s .4s,opacity .1s 0s,-webkit-transform 0s .4s;opacity:0}.md-hero--expand .md-hero__inner{margin-bottom:2.4rem}.md-footer-nav{background-color:rgba(0,0,0,.87);color:#fff}.md-footer-nav__inner{padding:.4rem;overflow:auto}.md-footer-nav__link{padding-top:2.8rem;padding-bottom:.8rem;-webkit-transition:opacity .25s;transition:opacity .25s}.md-footer-nav__link:hover{opacity:.7}.md-footer-nav__link--prev{width:25%;float:left}.md-footer-nav__link--next{width:75%;float:right;text-align:right}.md-footer-nav__button{-webkit-transition:background .25s;transition:background .25s}.md-footer-nav__title{position:relative;padding:0 2rem;font-size:1.8rem;line-height:4.8rem}.md-footer-nav__direction{position:absolute;right:0;left:0;margin-top:-2rem;padding:0 2rem;color:hsla(0,0%,100%,.7);font-size:1.5rem}.md-footer-meta{background-color:rgba(0,0,0,.895)}.md-footer-meta__inner{padding:.4rem;overflow:auto}html .md-footer-meta.md-typeset a{color:hsla(0,0%,100%,.7)}html .md-footer-meta.md-typeset a:focus,html .md-footer-meta.md-typeset a:hover{color:#fff}.md-footer-copyright{margin:0 1.2rem;padding:.8rem 0;color:hsla(0,0%,100%,.3);font-size:1.28rem}.md-footer-copyright__highlight{color:hsla(0,0%,100%,.7)}.md-footer-social{margin:0 .8rem;padding:.4rem 0 1.2rem}.md-footer-social__link{display:inline-block;width:3.2rem;height:3.2rem;font-size:1.6rem;text-align:center}.md-footer-social__link:before{line-height:1.9}.md-nav{font-size:1.4rem;line-height:1.3}.md-nav--secondary .md-nav__link--active{color:#3f51b5}.md-nav__title{display:block;padding:0 1.2rem;font-weight:700;text-overflow:ellipsis;overflow:hidden}.md-nav__title:before{display:none;content:"\E5C4"}.md-nav__title .md-nav__button{display:none}.md-nav__list{margin:0;padding:0;list-style:none}.md-nav__item{padding:0 1.2rem}.md-nav__item:last-child{padding-bottom:1.2rem}.md-nav__item .md-nav__item{padding-right:0}.md-nav__item .md-nav__item:last-child{padding-bottom:0}.md-nav__button img{width:100%;height:auto}.md-nav__link{display:block;margin-top:.625em;-webkit-transition:color .125s;transition:color .125s;text-overflow:ellipsis;cursor:pointer;overflow:hidden}.md-nav__item--nested>.md-nav__link:after{content:"\E313"}html .md-nav__link[for=toc],html .md-nav__link[for=toc]+.md-nav__link:after,html .md-nav__link[for=toc]~.md-nav{display:none}.md-nav__link[data-md-state=blur]{color:rgba(0,0,0,.54)}.md-nav__link:active{color:#3f51b5}.md-nav__item--nested>.md-nav__link{color:inherit}.md-nav__link:focus,.md-nav__link:hover{color:#536dfe}.md-nav__source,.no-js .md-search{display:none}.md-search__overlay{opacity:0;z-index:1}.md-search__form{position:relative}.md-search__input{position:relative;padding:0 4.8rem 0 7.2rem;text-overflow:ellipsis;z-index:2}.md-search__input::-webkit-input-placeholder{-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1);transition:color .25s cubic-bezier(.1,.7,.1,1)}.md-search__input:-ms-input-placeholder,.md-search__input::-ms-input-placeholder{-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1);transition:color .25s cubic-bezier(.1,.7,.1,1)}.md-search__input::placeholder{-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1);transition:color .25s cubic-bezier(.1,.7,.1,1)}.md-search__input::-webkit-input-placeholder,.md-search__input~.md-search__icon{color:rgba(0,0,0,.54)}.md-search__input:-ms-input-placeholder,.md-search__input::-ms-input-placeholder,.md-search__input~.md-search__icon{color:rgba(0,0,0,.54)}.md-search__input::placeholder,.md-search__input~.md-search__icon{color:rgba(0,0,0,.54)}.md-search__input::-ms-clear{display:none}.md-search__icon{position:absolute;-webkit-transition:color .25s cubic-bezier(.1,.7,.1,1),opacity .25s;transition:color .25s cubic-bezier(.1,.7,.1,1),opacity .25s;font-size:2.4rem;cursor:pointer;z-index:2}.md-search__icon:hover{opacity:.7}.md-search__icon[for=search]{top:.6rem;left:1rem}.md-search__icon[for=search]:before{content:"\E8B6"}.md-search__icon[type=reset]{top:.6rem;right:1rem;-webkit-transform:scale(.125);transform:scale(.125);-webkit-transition:opacity .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1);transition:opacity .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1);transition:transform .15s cubic-bezier(.1,.7,.1,1),opacity .15s;transition:transform .15s cubic-bezier(.1,.7,.1,1),opacity .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1);opacity:0}[data-md-toggle=search]:checked~.md-header .md-search__input:valid~.md-search__icon[type=reset]{-webkit-transform:scale(1);transform:scale(1);opacity:1}[data-md-toggle=search]:checked~.md-header .md-search__input:valid~.md-search__icon[type=reset]:hover{opacity:.7}.md-search__output{position:absolute;width:100%;border-radius:0 0 .2rem .2rem;overflow:hidden;z-index:1}.md-search__scrollwrap{height:100%;background-color:#fff;-webkit-box-shadow:0 .1rem 0 rgba(0,0,0,.07) inset;box-shadow:inset 0 .1rem 0 rgba(0,0,0,.07);overflow-y:auto;-webkit-overflow-scrolling:touch}.md-search-result{color:rgba(0,0,0,.87);word-break:break-word}.md-search-result__meta{padding:0 1.6rem;background-color:rgba(0,0,0,.07);color:rgba(0,0,0,.54);font-size:1.28rem;line-height:3.6rem}.md-search-result__list{margin:0;padding:0;border-top:.1rem solid rgba(0,0,0,.07);list-style:none}.md-search-result__item{-webkit-box-shadow:0 -.1rem 0 rgba(0,0,0,.07);box-shadow:0 -.1rem 0 rgba(0,0,0,.07)}.md-search-result__link{display:block;-webkit-transition:background .25s;transition:background .25s;outline:0;overflow:hidden}.md-search-result__link:hover,.md-search-result__link[data-md-state=active]{background-color:rgba(83,109,254,.1)}.md-search-result__link:hover .md-search-result__article:before,.md-search-result__link[data-md-state=active] .md-search-result__article:before{opacity:.7}.md-search-result__link:last-child .md-search-result__teaser{margin-bottom:1.2rem}.md-search-result__article{position:relative;padding:0 1.6rem;overflow:auto}.md-search-result__article--document:before{position:absolute;left:0;margin:.2rem;-webkit-transition:opacity .25s;transition:opacity .25s;color:rgba(0,0,0,.54);content:"\E880"}.md-search-result__article--document .md-search-result__title{margin:1.1rem 0;font-size:1.6rem;font-weight:400;line-height:1.4}.md-search-result__title{margin:.5em 0;font-size:1.28rem;font-weight:700;line-height:1.4}.md-search-result__teaser{display:-webkit-box;max-height:3.3rem;margin:.5em 0;color:rgba(0,0,0,.54);font-size:1.28rem;line-height:1.4;text-overflow:ellipsis;overflow:hidden;-webkit-box-orient:vertical;-webkit-line-clamp:2}.md-search-result em{font-style:normal;font-weight:700;text-decoration:underline}.md-sidebar{position:absolute;width:24.2rem;padding:2.4rem 0;overflow:hidden}.md-sidebar[data-md-state=lock]{position:fixed;top:4.8rem}.md-sidebar--secondary{display:none}.md-sidebar__scrollwrap{max-height:100%;margin:0 .4rem;overflow-y:auto;-webkit-backface-visibility:hidden;backface-visibility:hidden}.md-sidebar__scrollwrap::-webkit-scrollbar{width:.4rem;height:.4rem}.md-sidebar__scrollwrap::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-sidebar__scrollwrap::-webkit-scrollbar-thumb:hover{background-color:#536dfe}@-webkit-keyframes md-source__facts--done{0%{height:0}to{height:1.3rem}}@keyframes md-source__facts--done{0%{height:0}to{height:1.3rem}}@-webkit-keyframes md-source__fact--done{0%{-webkit-transform:translateY(100%);transform:translateY(100%);opacity:0}50%{opacity:0}to{-webkit-transform:translateY(0);transform:translateY(0);opacity:1}}@keyframes md-source__fact--done{0%{-webkit-transform:translateY(100%);transform:translateY(100%);opacity:0}50%{opacity:0}to{-webkit-transform:translateY(0);transform:translateY(0);opacity:1}}.md-source{display:block;padding-right:1.2rem;-webkit-transition:opacity .25s;transition:opacity .25s;font-size:1.3rem;line-height:1.2;white-space:nowrap}.md-source:hover{opacity:.7}.md-source:after,.md-source__icon{display:inline-block;height:4.8rem;content:"";vertical-align:middle}.md-source__icon{width:4.8rem}.md-source__icon svg{width:2.4rem;height:2.4rem;margin-top:1.2rem;margin-left:1.2rem}.md-source__icon+.md-source__repository{margin-left:-4.4rem;padding-left:4rem}.md-source__repository{display:inline-block;max-width:100%;margin-left:1.2rem;font-weight:700;text-overflow:ellipsis;overflow:hidden;vertical-align:middle}.md-source__facts{margin:0;padding:0;font-size:1.1rem;font-weight:700;list-style-type:none;opacity:.75;overflow:hidden}[data-md-state=done] .md-source__facts{-webkit-animation:md-source__facts--done .25s ease-in;animation:md-source__facts--done .25s ease-in}.md-source__fact{float:left}[data-md-state=done] .md-source__fact{-webkit-animation:md-source__fact--done .4s ease-out;animation:md-source__fact--done .4s ease-out}.md-source__fact:before{margin:0 .2rem;content:"\B7"}.md-source__fact:first-child:before{display:none}.md-source-file{display:inline-block;margin:1em .5em 1em 0;padding-right:.5rem;border-radius:.2rem;background-color:rgba(0,0,0,.07);font-size:1.28rem;list-style-type:none;cursor:pointer;overflow:hidden}.md-source-file:before{display:inline-block;margin-right:.5rem;padding:.5rem;background-color:rgba(0,0,0,.26);color:#fff;font-size:1.6rem;content:"\E86F";vertical-align:middle}html .md-source-file{-webkit-transition:background .4s,color .4s,-webkit-box-shadow .4s cubic-bezier(.4,0,.2,1);transition:background .4s,color .4s,-webkit-box-shadow .4s cubic-bezier(.4,0,.2,1);transition:background .4s,color .4s,box-shadow .4s cubic-bezier(.4,0,.2,1);transition:background .4s,color .4s,box-shadow .4s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .4s cubic-bezier(.4,0,.2,1)}html .md-source-file:before{-webkit-transition:inherit;transition:inherit}html body .md-typeset .md-source-file{color:rgba(0,0,0,.54)}.md-source-file:hover{-webkit-box-shadow:0 0 8px rgba(0,0,0,.18),0 8px 16px rgba(0,0,0,.36);box-shadow:0 0 8px rgba(0,0,0,.18),0 8px 16px rgba(0,0,0,.36)}.md-source-file:hover:before{background-color:#536dfe}.md-tabs{width:100%;-webkit-transition:background .25s;transition:background .25s;background-color:#3f51b5;color:#fff;overflow:auto}.md-tabs__list{margin:0;margin-left:.4rem;padding:0;list-style:none;white-space:nowrap}.md-tabs__item{display:inline-block;height:4.8rem;padding-right:1.2rem;padding-left:1.2rem}.md-tabs__link{display:block;margin-top:1.6rem;-webkit-transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s;transition:transform .4s cubic-bezier(.1,.7,.1,1),opacity .25s,-webkit-transform .4s cubic-bezier(.1,.7,.1,1);font-size:1.4rem;opacity:.7}.md-tabs__link--active,.md-tabs__link:hover{color:inherit;opacity:1}.md-tabs__item:nth-child(2) .md-tabs__link{-webkit-transition-delay:.02s;transition-delay:.02s}.md-tabs__item:nth-child(3) .md-tabs__link{-webkit-transition-delay:.04s;transition-delay:.04s}.md-tabs__item:nth-child(4) .md-tabs__link{-webkit-transition-delay:.06s;transition-delay:.06s}.md-tabs__item:nth-child(5) .md-tabs__link{-webkit-transition-delay:.08s;transition-delay:.08s}.md-tabs__item:nth-child(6) .md-tabs__link{-webkit-transition-delay:.1s;transition-delay:.1s}.md-tabs__item:nth-child(7) .md-tabs__link{-webkit-transition-delay:.12s;transition-delay:.12s}.md-tabs__item:nth-child(8) .md-tabs__link{-webkit-transition-delay:.14s;transition-delay:.14s}.md-tabs__item:nth-child(9) .md-tabs__link{-webkit-transition-delay:.16s;transition-delay:.16s}.md-tabs__item:nth-child(10) .md-tabs__link{-webkit-transition-delay:.18s;transition-delay:.18s}.md-tabs__item:nth-child(11) .md-tabs__link{-webkit-transition-delay:.2s;transition-delay:.2s}.md-tabs__item:nth-child(12) .md-tabs__link{-webkit-transition-delay:.22s;transition-delay:.22s}.md-tabs__item:nth-child(13) .md-tabs__link{-webkit-transition-delay:.24s;transition-delay:.24s}.md-tabs__item:nth-child(14) .md-tabs__link{-webkit-transition-delay:.26s;transition-delay:.26s}.md-tabs__item:nth-child(15) .md-tabs__link{-webkit-transition-delay:.28s;transition-delay:.28s}.md-tabs__item:nth-child(16) .md-tabs__link{-webkit-transition-delay:.3s;transition-delay:.3s}.md-tabs[data-md-state=hidden]{pointer-events:none}.md-tabs[data-md-state=hidden] .md-tabs__link{-webkit-transform:translateY(50%);transform:translateY(50%);-webkit-transition:color .25s,opacity .1s,-webkit-transform 0s .4s;transition:color .25s,opacity .1s,-webkit-transform 0s .4s;transition:color .25s,transform 0s .4s,opacity .1s;transition:color .25s,transform 0s .4s,opacity .1s,-webkit-transform 0s .4s;opacity:0}.md-typeset .admonition,.md-typeset details{-webkit-box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);box-shadow:0 2px 2px 0 rgba(0,0,0,.14),0 1px 5px 0 rgba(0,0,0,.12),0 3px 1px -2px rgba(0,0,0,.2);position:relative;margin:1.5625em 0;padding:1.2rem 1.2rem 0;border-left:.4rem solid #448aff;border-radius:.2rem;font-size:1.28rem}.md-typeset .admonition :first-child,.md-typeset details :first-child{margin-top:0}html .md-typeset .admonition :last-child,html .md-typeset details :last-child{margin-bottom:0;padding-bottom:1.2rem}.md-typeset .admonition .admonition,.md-typeset .admonition details,.md-typeset details .admonition,.md-typeset details details{margin:1em 0}.md-typeset .admonition>.admonition-title,.md-typeset .admonition>summary,.md-typeset details>.admonition-title,.md-typeset details>summary{margin:-1.2rem -1.2rem 0;padding:.8rem 1.2rem .8rem 4rem;border-bottom:.1rem solid rgba(68,138,255,.1);background-color:rgba(68,138,255,.1);font-weight:700}html .md-typeset .admonition>.admonition-title,html .md-typeset .admonition>summary,html .md-typeset details>.admonition-title,html .md-typeset details>summary{padding-bottom:.8rem}.md-typeset .admonition>.admonition-title:before,.md-typeset .admonition>summary:before,.md-typeset details>.admonition-title:before,.md-typeset details>summary:before{position:absolute;left:1.2rem;color:#448aff;font-size:2rem;content:"\E3C9"}.md-typeset .admonition.summary,.md-typeset .admonition.tldr,.md-typeset details.summary,.md-typeset details.tldr{border-left:.4rem solid #00b0ff}.md-typeset .admonition.summary>.admonition-title,.md-typeset .admonition.summary>summary,.md-typeset .admonition.tldr>.admonition-title,.md-typeset .admonition.tldr>summary,.md-typeset details.summary>.admonition-title,.md-typeset details.summary>summary,.md-typeset details.tldr>.admonition-title,.md-typeset details.tldr>summary{border-bottom:.1rem solid rgba(0,176,255,.1);background-color:rgba(0,176,255,.1)}.md-typeset .admonition.summary>.admonition-title:before,.md-typeset .admonition.summary>summary:before,.md-typeset .admonition.tldr>.admonition-title:before,.md-typeset .admonition.tldr>summary:before,.md-typeset details.summary>.admonition-title:before,.md-typeset details.summary>summary:before,.md-typeset details.tldr>.admonition-title:before,.md-typeset details.tldr>summary:before{color:#00b0ff;content:"\E8D2"}.md-typeset .admonition.info,.md-typeset .admonition.todo,.md-typeset details.info,.md-typeset details.todo{border-left:.4rem solid #00b8d4}.md-typeset .admonition.info>.admonition-title,.md-typeset .admonition.info>summary,.md-typeset .admonition.todo>.admonition-title,.md-typeset .admonition.todo>summary,.md-typeset details.info>.admonition-title,.md-typeset details.info>summary,.md-typeset details.todo>.admonition-title,.md-typeset details.todo>summary{border-bottom:.1rem solid rgba(0,184,212,.1);background-color:rgba(0,184,212,.1)}.md-typeset .admonition.info>.admonition-title:before,.md-typeset .admonition.info>summary:before,.md-typeset .admonition.todo>.admonition-title:before,.md-typeset .admonition.todo>summary:before,.md-typeset details.info>.admonition-title:before,.md-typeset details.info>summary:before,.md-typeset details.todo>.admonition-title:before,.md-typeset details.todo>summary:before{color:#00b8d4;content:"\E88E"}.md-typeset .admonition.hint,.md-typeset .admonition.important,.md-typeset .admonition.tip,.md-typeset details.hint,.md-typeset details.important,.md-typeset details.tip{border-left:.4rem solid #00bfa5}.md-typeset .admonition.hint>.admonition-title,.md-typeset .admonition.hint>summary,.md-typeset .admonition.important>.admonition-title,.md-typeset .admonition.important>summary,.md-typeset .admonition.tip>.admonition-title,.md-typeset .admonition.tip>summary,.md-typeset details.hint>.admonition-title,.md-typeset details.hint>summary,.md-typeset details.important>.admonition-title,.md-typeset details.important>summary,.md-typeset details.tip>.admonition-title,.md-typeset details.tip>summary{border-bottom:.1rem solid rgba(0,191,165,.1);background-color:rgba(0,191,165,.1)}.md-typeset .admonition.hint>.admonition-title:before,.md-typeset .admonition.hint>summary:before,.md-typeset .admonition.important>.admonition-title:before,.md-typeset .admonition.important>summary:before,.md-typeset .admonition.tip>.admonition-title:before,.md-typeset .admonition.tip>summary:before,.md-typeset details.hint>.admonition-title:before,.md-typeset details.hint>summary:before,.md-typeset details.important>.admonition-title:before,.md-typeset details.important>summary:before,.md-typeset details.tip>.admonition-title:before,.md-typeset details.tip>summary:before{color:#00bfa5;content:"\E80E"}.md-typeset .admonition.check,.md-typeset .admonition.done,.md-typeset .admonition.success,.md-typeset details.check,.md-typeset details.done,.md-typeset details.success{border-left:.4rem solid #00c853}.md-typeset .admonition.check>.admonition-title,.md-typeset .admonition.check>summary,.md-typeset .admonition.done>.admonition-title,.md-typeset .admonition.done>summary,.md-typeset .admonition.success>.admonition-title,.md-typeset .admonition.success>summary,.md-typeset details.check>.admonition-title,.md-typeset details.check>summary,.md-typeset details.done>.admonition-title,.md-typeset details.done>summary,.md-typeset details.success>.admonition-title,.md-typeset details.success>summary{border-bottom:.1rem solid rgba(0,200,83,.1);background-color:rgba(0,200,83,.1)}.md-typeset .admonition.check>.admonition-title:before,.md-typeset .admonition.check>summary:before,.md-typeset .admonition.done>.admonition-title:before,.md-typeset .admonition.done>summary:before,.md-typeset .admonition.success>.admonition-title:before,.md-typeset .admonition.success>summary:before,.md-typeset details.check>.admonition-title:before,.md-typeset details.check>summary:before,.md-typeset details.done>.admonition-title:before,.md-typeset details.done>summary:before,.md-typeset details.success>.admonition-title:before,.md-typeset details.success>summary:before{color:#00c853;content:"\E876"}.md-typeset .admonition.faq,.md-typeset .admonition.help,.md-typeset .admonition.question,.md-typeset details.faq,.md-typeset details.help,.md-typeset details.question{border-left:.4rem solid #64dd17}.md-typeset .admonition.faq>.admonition-title,.md-typeset .admonition.faq>summary,.md-typeset .admonition.help>.admonition-title,.md-typeset .admonition.help>summary,.md-typeset .admonition.question>.admonition-title,.md-typeset .admonition.question>summary,.md-typeset details.faq>.admonition-title,.md-typeset details.faq>summary,.md-typeset details.help>.admonition-title,.md-typeset details.help>summary,.md-typeset details.question>.admonition-title,.md-typeset details.question>summary{border-bottom:.1rem solid rgba(100,221,23,.1);background-color:rgba(100,221,23,.1)}.md-typeset .admonition.faq>.admonition-title:before,.md-typeset .admonition.faq>summary:before,.md-typeset .admonition.help>.admonition-title:before,.md-typeset .admonition.help>summary:before,.md-typeset .admonition.question>.admonition-title:before,.md-typeset .admonition.question>summary:before,.md-typeset details.faq>.admonition-title:before,.md-typeset details.faq>summary:before,.md-typeset details.help>.admonition-title:before,.md-typeset details.help>summary:before,.md-typeset details.question>.admonition-title:before,.md-typeset details.question>summary:before{color:#64dd17;content:"\E887"}.md-typeset .admonition.attention,.md-typeset .admonition.caution,.md-typeset .admonition.warning,.md-typeset details.attention,.md-typeset details.caution,.md-typeset details.warning{border-left:.4rem solid #ff9100}.md-typeset .admonition.attention>.admonition-title,.md-typeset .admonition.attention>summary,.md-typeset .admonition.caution>.admonition-title,.md-typeset .admonition.caution>summary,.md-typeset .admonition.warning>.admonition-title,.md-typeset .admonition.warning>summary,.md-typeset details.attention>.admonition-title,.md-typeset details.attention>summary,.md-typeset details.caution>.admonition-title,.md-typeset details.caution>summary,.md-typeset details.warning>.admonition-title,.md-typeset details.warning>summary{border-bottom:.1rem solid rgba(255,145,0,.1);background-color:rgba(255,145,0,.1)}.md-typeset .admonition.attention>.admonition-title:before,.md-typeset .admonition.attention>summary:before,.md-typeset .admonition.caution>.admonition-title:before,.md-typeset .admonition.caution>summary:before,.md-typeset .admonition.warning>.admonition-title:before,.md-typeset .admonition.warning>summary:before,.md-typeset details.attention>.admonition-title:before,.md-typeset details.attention>summary:before,.md-typeset details.caution>.admonition-title:before,.md-typeset details.caution>summary:before,.md-typeset details.warning>.admonition-title:before,.md-typeset details.warning>summary:before{color:#ff9100;content:"\E002"}.md-typeset .admonition.fail,.md-typeset .admonition.failure,.md-typeset .admonition.missing,.md-typeset details.fail,.md-typeset details.failure,.md-typeset details.missing{border-left:.4rem solid #ff5252}.md-typeset .admonition.fail>.admonition-title,.md-typeset .admonition.fail>summary,.md-typeset .admonition.failure>.admonition-title,.md-typeset .admonition.failure>summary,.md-typeset .admonition.missing>.admonition-title,.md-typeset .admonition.missing>summary,.md-typeset details.fail>.admonition-title,.md-typeset details.fail>summary,.md-typeset details.failure>.admonition-title,.md-typeset details.failure>summary,.md-typeset details.missing>.admonition-title,.md-typeset details.missing>summary{border-bottom:.1rem solid rgba(255,82,82,.1);background-color:rgba(255,82,82,.1)}.md-typeset .admonition.fail>.admonition-title:before,.md-typeset .admonition.fail>summary:before,.md-typeset .admonition.failure>.admonition-title:before,.md-typeset .admonition.failure>summary:before,.md-typeset .admonition.missing>.admonition-title:before,.md-typeset .admonition.missing>summary:before,.md-typeset details.fail>.admonition-title:before,.md-typeset details.fail>summary:before,.md-typeset details.failure>.admonition-title:before,.md-typeset details.failure>summary:before,.md-typeset details.missing>.admonition-title:before,.md-typeset details.missing>summary:before{color:#ff5252;content:"\E14C"}.md-typeset .admonition.danger,.md-typeset .admonition.error,.md-typeset details.danger,.md-typeset details.error{border-left:.4rem solid #ff1744}.md-typeset .admonition.danger>.admonition-title,.md-typeset .admonition.danger>summary,.md-typeset .admonition.error>.admonition-title,.md-typeset .admonition.error>summary,.md-typeset details.danger>.admonition-title,.md-typeset details.danger>summary,.md-typeset details.error>.admonition-title,.md-typeset details.error>summary{border-bottom:.1rem solid rgba(255,23,68,.1);background-color:rgba(255,23,68,.1)}.md-typeset .admonition.danger>.admonition-title:before,.md-typeset .admonition.danger>summary:before,.md-typeset .admonition.error>.admonition-title:before,.md-typeset .admonition.error>summary:before,.md-typeset details.danger>.admonition-title:before,.md-typeset details.danger>summary:before,.md-typeset details.error>.admonition-title:before,.md-typeset details.error>summary:before{color:#ff1744;content:"\E3E7"}.md-typeset .admonition.bug,.md-typeset details.bug{border-left:.4rem solid #f50057}.md-typeset .admonition.bug>.admonition-title,.md-typeset .admonition.bug>summary,.md-typeset details.bug>.admonition-title,.md-typeset details.bug>summary{border-bottom:.1rem solid rgba(245,0,87,.1);background-color:rgba(245,0,87,.1)}.md-typeset .admonition.bug>.admonition-title:before,.md-typeset .admonition.bug>summary:before,.md-typeset details.bug>.admonition-title:before,.md-typeset details.bug>summary:before{color:#f50057;content:"\E868"}.md-typeset .admonition.cite,.md-typeset .admonition.quote,.md-typeset details.cite,.md-typeset details.quote{border-left:.4rem solid #9e9e9e}.md-typeset .admonition.cite>.admonition-title,.md-typeset .admonition.cite>summary,.md-typeset .admonition.quote>.admonition-title,.md-typeset .admonition.quote>summary,.md-typeset details.cite>.admonition-title,.md-typeset details.cite>summary,.md-typeset details.quote>.admonition-title,.md-typeset details.quote>summary{border-bottom:.1rem solid hsla(0,0%,62%,.1);background-color:hsla(0,0%,62%,.1)}.md-typeset .admonition.cite>.admonition-title:before,.md-typeset .admonition.cite>summary:before,.md-typeset .admonition.quote>.admonition-title:before,.md-typeset .admonition.quote>summary:before,.md-typeset details.cite>.admonition-title:before,.md-typeset details.cite>summary:before,.md-typeset details.quote>.admonition-title:before,.md-typeset details.quote>summary:before{color:#9e9e9e;content:"\E244"}.codehilite .o,.codehilite .ow,.md-typeset .highlight .o,.md-typeset .highlight .ow{color:inherit}.codehilite .ge,.md-typeset .highlight .ge{color:#000}.codehilite .gr,.md-typeset .highlight .gr{color:#a00}.codehilite .gh,.md-typeset .highlight .gh{color:#999}.codehilite .go,.md-typeset .highlight .go{color:#888}.codehilite .gp,.md-typeset .highlight .gp{color:#555}.codehilite .gs,.md-typeset .highlight .gs{color:inherit}.codehilite .gu,.md-typeset .highlight .gu{color:#aaa}.codehilite .gt,.md-typeset .highlight .gt{color:#a00}.codehilite .gd,.md-typeset .highlight .gd{background-color:#fdd}.codehilite .gi,.md-typeset .highlight .gi{background-color:#dfd}.codehilite .k,.md-typeset .highlight .k{color:#3b78e7}.codehilite .kc,.md-typeset .highlight .kc{color:#a71d5d}.codehilite .kd,.codehilite .kn,.md-typeset .highlight .kd,.md-typeset .highlight .kn{color:#3b78e7}.codehilite .kp,.md-typeset .highlight .kp{color:#a71d5d}.codehilite .kr,.codehilite .kt,.md-typeset .highlight .kr,.md-typeset .highlight .kt{color:#3e61a2}.codehilite .c,.codehilite .cm,.md-typeset .highlight .c,.md-typeset .highlight .cm{color:#999}.codehilite .cp,.md-typeset .highlight .cp{color:#666}.codehilite .c1,.codehilite .ch,.codehilite .cs,.md-typeset .highlight .c1,.md-typeset .highlight .ch,.md-typeset .highlight .cs{color:#999}.codehilite .na,.codehilite .nb,.md-typeset .highlight .na,.md-typeset .highlight .nb{color:#c2185b}.codehilite .bp,.md-typeset .highlight .bp{color:#3e61a2}.codehilite .nc,.md-typeset .highlight .nc{color:#c2185b}.codehilite .no,.md-typeset .highlight .no{color:#3e61a2}.codehilite .nd,.codehilite .ni,.md-typeset .highlight .nd,.md-typeset .highlight .ni{color:#666}.codehilite .ne,.codehilite .nf,.md-typeset .highlight .ne,.md-typeset .highlight .nf{color:#c2185b}.codehilite .nl,.md-typeset .highlight .nl{color:#3b5179}.codehilite .nn,.md-typeset .highlight .nn{color:#ec407a}.codehilite .nt,.md-typeset .highlight .nt{color:#3b78e7}.codehilite .nv,.codehilite .vc,.codehilite .vg,.codehilite .vi,.md-typeset .highlight .nv,.md-typeset .highlight .vc,.md-typeset .highlight .vg,.md-typeset .highlight .vi{color:#3e61a2}.codehilite .nx,.md-typeset .highlight .nx{color:#ec407a}.codehilite .il,.codehilite .m,.codehilite .mf,.codehilite .mh,.codehilite .mi,.codehilite .mo,.md-typeset .highlight .il,.md-typeset .highlight .m,.md-typeset .highlight .mf,.md-typeset .highlight .mh,.md-typeset .highlight .mi,.md-typeset .highlight .mo{color:#e74c3c}.codehilite .s,.codehilite .sb,.codehilite .sc,.md-typeset .highlight .s,.md-typeset .highlight .sb,.md-typeset .highlight .sc{color:#0d904f}.codehilite .sd,.md-typeset .highlight .sd{color:#999}.codehilite .s2,.md-typeset .highlight .s2{color:#0d904f}.codehilite .se,.codehilite .sh,.codehilite .si,.codehilite .sx,.md-typeset .highlight .se,.md-typeset .highlight .sh,.md-typeset .highlight .si,.md-typeset .highlight .sx{color:#183691}.codehilite .sr,.md-typeset .highlight .sr{color:#009926}.codehilite .s1,.codehilite .ss,.md-typeset .highlight .s1,.md-typeset .highlight .ss{color:#0d904f}.codehilite .err,.md-typeset .highlight .err{color:#a61717}.codehilite .w,.md-typeset .highlight .w{color:transparent}.codehilite .hll,.md-typeset .highlight .hll{display:block;margin:0 -1.2rem;padding:0 1.2rem;background-color:rgba(255,235,59,.5)}.md-typeset .codehilite,.md-typeset .highlight{position:relative;margin:1em 0;padding:0;border-radius:.2rem;background-color:hsla(0,0%,93%,.5);color:#37474f;line-height:1.4;-webkit-overflow-scrolling:touch}.md-typeset .codehilite code,.md-typeset .codehilite pre,.md-typeset .highlight code,.md-typeset .highlight pre{display:block;margin:0;padding:1.05rem 1.2rem;background-color:transparent;overflow:auto;vertical-align:top}.md-typeset .codehilite code::-webkit-scrollbar,.md-typeset .codehilite pre::-webkit-scrollbar,.md-typeset .highlight code::-webkit-scrollbar,.md-typeset .highlight pre::-webkit-scrollbar{width:.4rem;height:.4rem}.md-typeset .codehilite code::-webkit-scrollbar-thumb,.md-typeset .codehilite pre::-webkit-scrollbar-thumb,.md-typeset .highlight code::-webkit-scrollbar-thumb,.md-typeset .highlight pre::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-typeset .codehilite code::-webkit-scrollbar-thumb:hover,.md-typeset .codehilite pre::-webkit-scrollbar-thumb:hover,.md-typeset .highlight code::-webkit-scrollbar-thumb:hover,.md-typeset .highlight pre::-webkit-scrollbar-thumb:hover{background-color:#536dfe}.md-typeset pre.codehilite,.md-typeset pre.highlight{overflow:visible}.md-typeset pre.codehilite code,.md-typeset pre.highlight code{display:block;padding:1.05rem 1.2rem;overflow:auto}.md-typeset .codehilitetable{display:block;margin:1em 0;border-radius:.2em;font-size:1.6rem;overflow:hidden}.md-typeset .codehilitetable tbody,.md-typeset .codehilitetable td{display:block;padding:0}.md-typeset .codehilitetable tr{display:-webkit-box;display:-ms-flexbox;display:flex}.md-typeset .codehilitetable .codehilite,.md-typeset .codehilitetable .highlight,.md-typeset .codehilitetable .linenodiv{margin:0;border-radius:0}.md-typeset .codehilitetable .linenodiv{padding:1.05rem 1.2rem}.md-typeset .codehilitetable .linenos{background-color:rgba(0,0,0,.07);color:rgba(0,0,0,.26);-webkit-user-select:none;-moz-user-select:none;-ms-user-select:none;user-select:none}.md-typeset .codehilitetable .linenos pre{margin:0;padding:0;background-color:transparent;color:inherit;text-align:right}.md-typeset .codehilitetable .code{-webkit-box-flex:1;-ms-flex:1;flex:1;overflow:hidden}.md-typeset>.codehilitetable{-webkit-box-shadow:none;box-shadow:none}.md-typeset [id^="fnref:"]{display:inline-block}.md-typeset [id^="fnref:"]:target{margin-top:-7.6rem;padding-top:7.6rem;pointer-events:none}.md-typeset [id^="fn:"]:before{display:none;height:0;content:""}.md-typeset [id^="fn:"]:target:before{display:block;margin-top:-7rem;padding-top:7rem;pointer-events:none}.md-typeset .footnote{color:rgba(0,0,0,.54);font-size:1.28rem}.md-typeset .footnote ol{margin-left:0}.md-typeset .footnote li{-webkit-transition:color .25s;transition:color .25s}.md-typeset .footnote li:target{color:rgba(0,0,0,.87)}.md-typeset .footnote li :first-child{margin-top:0}.md-typeset .footnote li:hover .footnote-backref,.md-typeset .footnote li:target .footnote-backref{-webkit-transform:translateX(0);transform:translateX(0);opacity:1}.md-typeset .footnote li:hover .footnote-backref:hover,.md-typeset .footnote li:target .footnote-backref{color:#536dfe}.md-typeset .footnote-ref{display:inline-block;pointer-events:auto}.md-typeset .footnote-ref:before{display:inline;margin:0 .2em;border-left:.1rem solid rgba(0,0,0,.26);font-size:1.25em;content:"";vertical-align:-.5rem}.md-typeset .footnote-backref{display:inline-block;-webkit-transform:translateX(.5rem);transform:translateX(.5rem);-webkit-transition:color .25s,opacity .125s .125s,-webkit-transform .25s .125s;transition:color .25s,opacity .125s .125s,-webkit-transform .25s .125s;transition:transform .25s .125s,color .25s,opacity .125s .125s;transition:transform .25s .125s,color .25s,opacity .125s .125s,-webkit-transform .25s .125s;color:rgba(0,0,0,.26);font-size:0;opacity:0;vertical-align:text-bottom}.md-typeset .footnote-backref:before{font-size:1.6rem;content:"\E31B"}.md-typeset .headerlink{display:inline-block;margin-left:1rem;-webkit-transform:translateY(.5rem);transform:translateY(.5rem);-webkit-transition:color .25s,opacity .125s .25s,-webkit-transform .25s .25s;transition:color .25s,opacity .125s .25s,-webkit-transform .25s .25s;transition:transform .25s .25s,color .25s,opacity .125s .25s;transition:transform .25s .25s,color .25s,opacity .125s .25s,-webkit-transform .25s .25s;opacity:0}html body .md-typeset .headerlink{color:rgba(0,0,0,.26)}.md-typeset h1[id] .headerlink{display:none}.md-typeset h2[id]:before{display:block;margin-top:-.8rem;padding-top:.8rem;content:""}.md-typeset h2[id]:target:before{margin-top:-6.8rem;padding-top:6.8rem}.md-typeset h2[id] .headerlink:focus,.md-typeset h2[id]:hover .headerlink,.md-typeset h2[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h2[id] .headerlink:focus,.md-typeset h2[id]:hover .headerlink:hover,.md-typeset h2[id]:target .headerlink{color:#536dfe}.md-typeset h3[id]:before{display:block;margin-top:-.9rem;padding-top:.9rem;content:""}.md-typeset h3[id]:target:before{margin-top:-6.9rem;padding-top:6.9rem}.md-typeset h3[id] .headerlink:focus,.md-typeset h3[id]:hover .headerlink,.md-typeset h3[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h3[id] .headerlink:focus,.md-typeset h3[id]:hover .headerlink:hover,.md-typeset h3[id]:target .headerlink{color:#536dfe}.md-typeset h4[id]:before{display:block;margin-top:-.9rem;padding-top:.9rem;content:""}.md-typeset h4[id]:target:before{margin-top:-6.9rem;padding-top:6.9rem}.md-typeset h4[id] .headerlink:focus,.md-typeset h4[id]:hover .headerlink,.md-typeset h4[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h4[id] .headerlink:focus,.md-typeset h4[id]:hover .headerlink:hover,.md-typeset h4[id]:target .headerlink{color:#536dfe}.md-typeset h5[id]:before{display:block;margin-top:-1.1rem;padding-top:1.1rem;content:""}.md-typeset h5[id]:target:before{margin-top:-7.1rem;padding-top:7.1rem}.md-typeset h5[id] .headerlink:focus,.md-typeset h5[id]:hover .headerlink,.md-typeset h5[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h5[id] .headerlink:focus,.md-typeset h5[id]:hover .headerlink:hover,.md-typeset h5[id]:target .headerlink{color:#536dfe}.md-typeset h6[id]:before{display:block;margin-top:-1.1rem;padding-top:1.1rem;content:""}.md-typeset h6[id]:target:before{margin-top:-7.1rem;padding-top:7.1rem}.md-typeset h6[id] .headerlink:focus,.md-typeset h6[id]:hover .headerlink,.md-typeset h6[id]:target .headerlink{-webkit-transform:translate(0);transform:translate(0);opacity:1}.md-typeset h6[id] .headerlink:focus,.md-typeset h6[id]:hover .headerlink:hover,.md-typeset h6[id]:target .headerlink{color:#536dfe}.md-typeset .MJXc-display{margin:.75em 0;padding:.75em 0;overflow:auto;-webkit-overflow-scrolling:touch}.md-typeset .MathJax_CHTML{outline:0}.md-typeset .critic.comment,.md-typeset del.critic,.md-typeset ins.critic{margin:0 .25em;padding:.0625em 0;border-radius:.2rem;-webkit-box-decoration-break:clone;box-decoration-break:clone}.md-typeset del.critic{background-color:#fdd;-webkit-box-shadow:.25em 0 0 #fdd,-.25em 0 0 #fdd;box-shadow:.25em 0 0 #fdd,-.25em 0 0 #fdd}.md-typeset ins.critic{background-color:#dfd;-webkit-box-shadow:.25em 0 0 #dfd,-.25em 0 0 #dfd;box-shadow:.25em 0 0 #dfd,-.25em 0 0 #dfd}.md-typeset .critic.comment{background-color:hsla(0,0%,93%,.5);color:#37474f;-webkit-box-shadow:.25em 0 0 hsla(0,0%,93%,.5),-.25em 0 0 hsla(0,0%,93%,.5);box-shadow:.25em 0 0 hsla(0,0%,93%,.5),-.25em 0 0 hsla(0,0%,93%,.5)}.md-typeset .critic.comment:before{padding-right:.125em;color:rgba(0,0,0,.26);content:"\E0B7";vertical-align:-.125em}.md-typeset .critic.block{display:block;margin:1em 0;padding-right:1.6rem;padding-left:1.6rem;-webkit-box-shadow:none;box-shadow:none}.md-typeset .critic.block :first-child{margin-top:.5em}.md-typeset .critic.block :last-child{margin-bottom:.5em}.md-typeset details{padding-top:0}.md-typeset details[open]>summary:after{-webkit-transform:rotate(180deg);transform:rotate(180deg)}.md-typeset details:not([open]){padding-bottom:0}.md-typeset details:not([open])>summary{border-bottom:none}.md-typeset details summary{padding-right:4rem}.no-details .md-typeset details:not([open])>*{display:none}.no-details .md-typeset details:not([open]) summary{display:block}.md-typeset summary{display:block;outline:none;cursor:pointer}.md-typeset summary::-webkit-details-marker{display:none}.md-typeset summary:after{position:absolute;top:.8rem;right:1.2rem;color:rgba(0,0,0,.26);font-size:2rem;content:"\E313"}.md-typeset .emojione{width:2rem;vertical-align:text-top}.md-typeset code.codehilite,.md-typeset code.highlight{margin:0 .29412em;padding:.07353em 0}.md-typeset .task-list-item{position:relative;list-style-type:none}.md-typeset .task-list-item [type=checkbox]{position:absolute;top:.45em;left:-2em}.md-typeset .task-list-control .task-list-indicator:before{position:absolute;top:.15em;left:-1.25em;color:rgba(0,0,0,.26);font-size:1.25em;content:"\E835";vertical-align:-.25em}.md-typeset .task-list-control [type=checkbox]:checked+.task-list-indicator:before{content:"\E834"}.md-typeset .task-list-control [type=checkbox]{opacity:0;z-index:-1}@media print{.md-typeset a:after{color:rgba(0,0,0,.54);content:" [" attr(href) "]"}.md-typeset code,.md-typeset pre{white-space:pre-wrap}.md-typeset code{-webkit-box-shadow:none;box-shadow:none;-webkit-box-decoration-break:initial;box-decoration-break:slice}.md-clipboard,.md-content__icon,.md-footer,.md-header,.md-sidebar,.md-tabs,.md-typeset .headerlink{display:none}}@media only screen and (max-width:44.9375em){.md-typeset pre{margin:1em -1.6rem;border-radius:0}.md-typeset pre>code{padding:1.05rem 1.6rem}.md-footer-nav__link--prev .md-footer-nav__title{display:none}.md-search-result__teaser{max-height:5rem;-webkit-line-clamp:3}.codehilite .hll,.md-typeset .highlight .hll{margin:0 -1.6rem;padding:0 1.6rem}.md-typeset>.codehilite,.md-typeset>.highlight{margin:1em -1.6rem;border-radius:0}.md-typeset>.codehilite code,.md-typeset>.codehilite pre,.md-typeset>.highlight code,.md-typeset>.highlight pre{padding:1.05rem 1.6rem}.md-typeset>.codehilitetable{margin:1em -1.6rem;border-radius:0}.md-typeset>.codehilitetable .codehilite>code,.md-typeset>.codehilitetable .codehilite>pre,.md-typeset>.codehilitetable .highlight>code,.md-typeset>.codehilitetable .highlight>pre,.md-typeset>.codehilitetable .linenodiv{padding:1rem 1.6rem}.md-typeset>p>.MJXc-display{margin:.75em -1.6rem;padding:.25em 1.6rem}}@media only screen and (min-width:100em){html{font-size:68.75%}}@media only screen and (min-width:125em){html{font-size:75%}}@media only screen and (max-width:59.9375em){body[data-md-state=lock]{overflow:hidden}.ios body[data-md-state=lock] .md-container{display:none}html .md-nav__link[for=toc]{display:block;padding-right:4.8rem}html .md-nav__link[for=toc]:after{color:inherit;content:"\E8DE"}html .md-nav__link[for=toc]+.md-nav__link{display:none}html .md-nav__link[for=toc]~.md-nav{display:-webkit-box;display:-ms-flexbox;display:flex}.md-nav__source{display:block;padding:0 .4rem;background-color:rgba(50,64,144,.9675);color:#fff}.md-search__overlay{position:absolute;top:.4rem;left:.4rem;width:3.6rem;height:3.6rem;-webkit-transform-origin:center;transform-origin:center;-webkit-transition:opacity .2s .2s,-webkit-transform .3s .1s;transition:opacity .2s .2s,-webkit-transform .3s .1s;transition:transform .3s .1s,opacity .2s .2s;transition:transform .3s .1s,opacity .2s .2s,-webkit-transform .3s .1s;border-radius:2rem;background-color:#fff;overflow:hidden;pointer-events:none}[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transition:opacity .1s,-webkit-transform .4s;transition:opacity .1s,-webkit-transform .4s;transition:transform .4s,opacity .1s;transition:transform .4s,opacity .1s,-webkit-transform .4s;opacity:1}.md-search__inner{position:fixed;top:0;left:100%;width:100%;height:100%;-webkit-transform:translateX(5%);transform:translateX(5%);-webkit-transition:left 0s .3s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.4,0,.2,1) .15s;transition:left 0s .3s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.4,0,.2,1) .15s;transition:left 0s .3s,transform .15s cubic-bezier(.4,0,.2,1) .15s,opacity .15s .15s;transition:left 0s .3s,transform .15s cubic-bezier(.4,0,.2,1) .15s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.4,0,.2,1) .15s;opacity:0;z-index:2}[data-md-toggle=search]:checked~.md-header .md-search__inner{left:0;-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:left 0s 0s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1) .15s;transition:left 0s 0s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1) .15s;transition:left 0s 0s,transform .15s cubic-bezier(.1,.7,.1,1) .15s,opacity .15s .15s;transition:left 0s 0s,transform .15s cubic-bezier(.1,.7,.1,1) .15s,opacity .15s .15s,-webkit-transform .15s cubic-bezier(.1,.7,.1,1) .15s;opacity:1}.md-search__input{width:100%;height:4.8rem;font-size:1.8rem}.md-search__icon[for=search]{top:1.2rem;left:1.6rem}.md-search__icon[for=search][for=search]:before{content:"\E5C4"}.md-search__icon[type=reset]{top:1.2rem;right:1.6rem}.md-search__output{top:4.8rem;bottom:0}.md-search-result__article--document:before{display:none}}@media only screen and (max-width:76.1875em){[data-md-toggle=drawer]:checked~.md-overlay{width:100%;height:100%;-webkit-transition:width 0s,height 0s,opacity .25s;transition:width 0s,height 0s,opacity .25s;opacity:1}.md-header-nav__button.md-icon--home,.md-header-nav__button.md-logo{display:none}.md-hero__inner{margin-top:4.8rem;margin-bottom:2.4rem}.md-nav{background-color:#fff}.md-nav--primary,.md-nav--primary .md-nav{display:-webkit-box;display:-ms-flexbox;display:flex;position:absolute;top:0;right:0;left:0;-webkit-box-orient:vertical;-webkit-box-direction:normal;-ms-flex-direction:column;flex-direction:column;height:100%;z-index:1}.md-nav--primary .md-nav__item,.md-nav--primary .md-nav__title{font-size:1.6rem;line-height:1.5}html .md-nav--primary .md-nav__title{position:relative;height:11.2rem;padding:6rem 1.6rem .4rem;background-color:rgba(0,0,0,.07);color:rgba(0,0,0,.54);font-weight:400;line-height:4.8rem;white-space:nowrap;cursor:pointer}html .md-nav--primary .md-nav__title:before{display:block;position:absolute;top:.4rem;left:.4rem;width:4rem;height:4rem;color:rgba(0,0,0,.54)}html .md-nav--primary .md-nav__title~.md-nav__list{background-color:#fff;-webkit-box-shadow:0 .1rem 0 rgba(0,0,0,.07) inset;box-shadow:inset 0 .1rem 0 rgba(0,0,0,.07)}html .md-nav--primary .md-nav__title~.md-nav__list>.md-nav__item:first-child{border-top:0}html .md-nav--primary .md-nav__title--site{position:relative;background-color:#3f51b5;color:#fff}html .md-nav--primary .md-nav__title--site .md-nav__button{display:block;position:absolute;top:.4rem;left:.4rem;width:6.4rem;height:6.4rem;font-size:4.8rem}html .md-nav--primary .md-nav__title--site:before{display:none}.md-nav--primary .md-nav__list{-webkit-box-flex:1;-ms-flex:1;flex:1;overflow-y:auto}.md-nav--primary .md-nav__item{padding:0;border-top:.1rem solid rgba(0,0,0,.07)}.md-nav--primary .md-nav__item--nested>.md-nav__link{padding-right:4.8rem}.md-nav--primary .md-nav__item--nested>.md-nav__link:after{content:"\E315"}.md-nav--primary .md-nav__link{position:relative;margin-top:0;padding:1.2rem 1.6rem}.md-nav--primary .md-nav__link:after{position:absolute;top:50%;right:1.2rem;margin-top:-1.2rem;color:inherit;font-size:2.4rem}.md-nav--primary .md-nav--secondary .md-nav__link{position:static}.md-nav--primary .md-nav--secondary .md-nav{position:static;background-color:transparent}.md-nav--primary .md-nav--secondary .md-nav .md-nav__link{padding-left:2.8rem}.md-nav--primary .md-nav--secondary .md-nav .md-nav .md-nav__link{padding-left:4rem}.md-nav--primary .md-nav--secondary .md-nav .md-nav .md-nav .md-nav__link{padding-left:5.2rem}.md-nav--primary .md-nav--secondary .md-nav .md-nav .md-nav .md-nav .md-nav__link{padding-left:6.4rem}.md-nav__toggle~.md-nav{display:-webkit-box;display:-ms-flexbox;display:flex;-webkit-transform:translateX(100%);transform:translateX(100%);-webkit-transition:opacity .125s .05s,-webkit-transform .25s cubic-bezier(.8,0,.6,1);transition:opacity .125s .05s,-webkit-transform .25s cubic-bezier(.8,0,.6,1);transition:transform .25s cubic-bezier(.8,0,.6,1),opacity .125s .05s;transition:transform .25s cubic-bezier(.8,0,.6,1),opacity .125s .05s,-webkit-transform .25s cubic-bezier(.8,0,.6,1);opacity:0}.no-csstransforms3d .md-nav__toggle~.md-nav{display:none}.md-nav__toggle:checked~.md-nav{-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:opacity .125s .125s,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:opacity .125s .125s,-webkit-transform .25s cubic-bezier(.4,0,.2,1);transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .125s .125s;transition:transform .25s cubic-bezier(.4,0,.2,1),opacity .125s .125s,-webkit-transform .25s cubic-bezier(.4,0,.2,1);opacity:1}.no-csstransforms3d .md-nav__toggle:checked~.md-nav{display:-webkit-box;display:-ms-flexbox;display:flex}.md-sidebar--primary{position:fixed;top:0;left:-24.2rem;width:24.2rem;height:100%;-webkit-transform:translateX(0);transform:translateX(0);-webkit-transition:-webkit-transform .25s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .25s;transition:-webkit-transform .25s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .25s;transition:transform .25s cubic-bezier(.4,0,.2,1),box-shadow .25s;transition:transform .25s cubic-bezier(.4,0,.2,1),box-shadow .25s,-webkit-transform .25s cubic-bezier(.4,0,.2,1),-webkit-box-shadow .25s;background-color:#fff;z-index:3}.no-csstransforms3d .md-sidebar--primary{display:none}[data-md-toggle=drawer]:checked~.md-container .md-sidebar--primary{-webkit-box-shadow:0 8px 10px 1px rgba(0,0,0,.14),0 3px 14px 2px rgba(0,0,0,.12),0 5px 5px -3px rgba(0,0,0,.4);box-shadow:0 8px 10px 1px rgba(0,0,0,.14),0 3px 14px 2px rgba(0,0,0,.12),0 5px 5px -3px rgba(0,0,0,.4);-webkit-transform:translateX(24.2rem);transform:translateX(24.2rem)}.no-csstransforms3d [data-md-toggle=drawer]:checked~.md-container .md-sidebar--primary{display:block}.md-sidebar--primary .md-sidebar__scrollwrap{overflow:hidden;position:absolute;top:0;right:0;bottom:0;left:0;margin:0}.md-tabs{display:none}}@media only screen and (min-width:60em){.md-content{margin-right:24.2rem}.md-header-nav__button.md-icon--search{display:none}.md-header-nav__source{display:block;width:23rem;max-width:23rem;margin-left:2.8rem;padding-right:1.2rem}.md-search{padding:.4rem}.md-search__overlay{position:fixed;top:0;left:0;width:0;height:0;-webkit-transition:width 0s .25s,height 0s .25s,opacity .25s;transition:width 0s .25s,height 0s .25s,opacity .25s;background-color:rgba(0,0,0,.54);cursor:pointer}[data-md-toggle=search]:checked~.md-header .md-search__overlay{width:100%;height:100%;-webkit-transition:width 0s,height 0s,opacity .25s;transition:width 0s,height 0s,opacity .25s;opacity:1}.md-search__inner{position:relative;width:23rem;padding:.2rem 0;float:right;-webkit-transition:width .25s cubic-bezier(.1,.7,.1,1);transition:width .25s cubic-bezier(.1,.7,.1,1)}.md-search__form,.md-search__input{border-radius:.2rem}.md-search__input{width:100%;height:3.6rem;padding-left:4.4rem;-webkit-transition:background-color .25s cubic-bezier(.1,.7,.1,1),color .25s cubic-bezier(.1,.7,.1,1);transition:background-color .25s cubic-bezier(.1,.7,.1,1),color .25s cubic-bezier(.1,.7,.1,1);background-color:rgba(0,0,0,.26);color:inherit;font-size:1.6rem}.md-search__input+.md-search__icon{color:inherit}.md-search__input::-webkit-input-placeholder{color:hsla(0,0%,100%,.7)}.md-search__input:-ms-input-placeholder,.md-search__input::-ms-input-placeholder{color:hsla(0,0%,100%,.7)}.md-search__input::placeholder{color:hsla(0,0%,100%,.7)}.md-search__input:hover{background-color:hsla(0,0%,100%,.12)}[data-md-toggle=search]:checked~.md-header .md-search__input{border-radius:.2rem .2rem 0 0;background-color:#fff;color:rgba(0,0,0,.87);text-overflow:none}[data-md-toggle=search]:checked~.md-header .md-search__input+.md-search__icon,[data-md-toggle=search]:checked~.md-header .md-search__input::-webkit-input-placeholder{color:rgba(0,0,0,.54)}[data-md-toggle=search]:checked~.md-header .md-search__input+.md-search__icon,[data-md-toggle=search]:checked~.md-header .md-search__input:-ms-input-placeholder,[data-md-toggle=search]:checked~.md-header .md-search__input::-ms-input-placeholder{color:rgba(0,0,0,.54)}[data-md-toggle=search]:checked~.md-header .md-search__input+.md-search__icon,[data-md-toggle=search]:checked~.md-header .md-search__input::placeholder{color:rgba(0,0,0,.54)}.md-search__output{top:3.8rem;-webkit-transition:opacity .4s;transition:opacity .4s;opacity:0}[data-md-toggle=search]:checked~.md-header .md-search__output{-webkit-box-shadow:0 6px 10px 0 rgba(0,0,0,.14),0 1px 18px 0 rgba(0,0,0,.12),0 3px 5px -1px rgba(0,0,0,.4);box-shadow:0 6px 10px 0 rgba(0,0,0,.14),0 1px 18px 0 rgba(0,0,0,.12),0 3px 5px -1px rgba(0,0,0,.4);opacity:1}.md-search__scrollwrap{max-height:0}[data-md-toggle=search]:checked~.md-header .md-search__scrollwrap{max-height:75vh}.md-search__scrollwrap::-webkit-scrollbar{width:.4rem;height:.4rem}.md-search__scrollwrap::-webkit-scrollbar-thumb{background-color:rgba(0,0,0,.26)}.md-search__scrollwrap::-webkit-scrollbar-thumb:hover{background-color:#536dfe}.md-search-result__article,.md-search-result__meta{padding-left:4.4rem}.md-sidebar--secondary{display:block;margin-left:100%;-webkit-transform:translate(-100%);transform:translate(-100%)}}@media only screen and (min-width:76.25em){.md-content{margin-left:24.2rem}.md-content__inner{margin-right:2.4rem;margin-left:2.4rem}.md-header-nav__button.md-icon--menu{display:none}.md-nav[data-md-state=animate]{-webkit-transition:max-height .25s cubic-bezier(.86,0,.07,1);transition:max-height .25s cubic-bezier(.86,0,.07,1)}.md-nav__toggle~.md-nav{max-height:0;overflow:hidden}.md-nav[data-md-state=expand],.md-nav__toggle:checked~.md-nav{max-height:100%}.md-nav__item--nested>.md-nav>.md-nav__title{display:none}.md-nav__item--nested>.md-nav__link:after{display:inline-block;-webkit-transform-origin:.45em .45em;transform-origin:.45em .45em;-webkit-transform-style:preserve-3d;transform-style:preserve-3d;vertical-align:-.125em}.js .md-nav__item--nested>.md-nav__link:after{-webkit-transition:-webkit-transform .4s;transition:-webkit-transform .4s;transition:transform .4s;transition:transform .4s,-webkit-transform .4s}.md-nav__item--nested .md-nav__toggle:checked~.md-nav__link:after{-webkit-transform:rotateX(180deg);transform:rotateX(180deg)}.md-search__scrollwrap,[data-md-toggle=search]:checked~.md-header .md-search__inner{width:68.8rem}.md-sidebar--secondary{margin-left:122rem}.md-tabs~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested{font-size:0}.md-tabs--active~.md-main .md-nav--primary .md-nav__title--site{display:none}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item{font-size:0}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested{display:none;font-size:1.4rem;overflow:auto}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested>.md-nav__link{margin-top:0;font-weight:700;pointer-events:none}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--nested>.md-nav__link:after{display:none}.md-tabs--active~.md-main .md-nav--primary>.md-nav__list>.md-nav__item--active{display:block}.md-tabs--active~.md-main .md-nav[data-md-level="1"]{max-height:none}.md-tabs--active~.md-main .md-nav[data-md-level="1"]>.md-nav__list>.md-nav__item{padding-left:0}}@media only screen and (min-width:45em){.md-footer-nav__link{width:50%}.md-footer-copyright{max-width:75%;float:left}.md-footer-social{padding:1.2rem 0;float:right}}@media only screen and (max-width:29.9375em){[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transform:scale(45);transform:scale(45)}}@media only screen and (min-width:30em) and (max-width:44.9375em){[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transform:scale(60);transform:scale(60)}}@media only screen and (min-width:45em) and (max-width:59.9375em){[data-md-toggle=search]:checked~.md-header .md-search__overlay{-webkit-transform:scale(75);transform:scale(75)}}@media only screen and (min-width:60em) and (max-width:76.1875em){.md-search__scrollwrap,[data-md-toggle=search]:checked~.md-header .md-search__inner{width:46.8rem}.md-search-result__teaser{max-height:5rem;-webkit-line-clamp:3}} +/*# sourceMappingURL=data:application/json;charset=utf-8;base64,eyJ2ZXJzaW9uIjozLCJzb3VyY2VzIjpbXSwibmFtZXMiOltdLCJtYXBwaW5ncyI6IiIsImZpbGUiOiJhc3NldHMvc3R5bGVzaGVldHMvYXBwbGljYXRpb24uYWM2NDI1MWUuY3NzIiwic291cmNlUm9vdCI6IiJ9*/ diff --git a/docs/mkdocs-material-theme/assets/stylesheets/yandex_fonts.css b/docs/mkdocs-material-theme/assets/stylesheets/custom.css similarity index 77% rename from docs/mkdocs-material-theme/assets/stylesheets/yandex_fonts.css rename to docs/mkdocs-material-theme/assets/stylesheets/custom.css index b6c81f611d4..a908ec2f21a 100644 --- a/docs/mkdocs-material-theme/assets/stylesheets/yandex_fonts.css +++ b/docs/mkdocs-material-theme/assets/stylesheets/custom.css @@ -64,5 +64,58 @@ } body { - font-family: 'Yandex Sans Text Web'; + font: 300 14pt/200% 'Yandex Sans Text Web', Arial, sans-serif; +} + +a:link, a:visited { + color: #08f; + text-decoration: none; +} + +.md-nav__link { + color: #000 !important; +} + +.md-nav__link:hover, .md-nav__link:active { + color: #08f !important; + text-decoration: none; +} + +a:hover, a:active { + color: #f00; + text-decoration: underline; +} + +.md-typeset pre { + font: 13px/18px monospace, "Courier New"; + display: block; + padding: 1rem 3rem 1rem 1rem; + overflow: scroll; +} + +h1, h2, h3, .md-logo { + font-family: 'Yandex Sans Display Web', Arial, sans-serif; + color: #000 !important; +} + +.md-logo { + padding: 0; +} + +.md-header { + border-bottom: 1px solid #efefef; +} + +.md-header-nav__title { + font-size: 3rem; + font-family: 'Yandex Sans Display Web', Arial, sans-serif; +} + +.md-content__icon:hover { + text-decoration: none !important; + color: #08f !important; +} + +.md-search-result__link { + text-decoration: none !important; } diff --git a/docs/mkdocs-material-theme/base.html b/docs/mkdocs-material-theme/base.html index 5dd9d9bca96..a35e61dd354 100644 --- a/docs/mkdocs-material-theme/base.html +++ b/docs/mkdocs-material-theme/base.html @@ -134,8 +134,12 @@
{% block content %} - {% if page.edit_url %} - + {% if config.extra.single_page %} + + {% else %} + {% if page.edit_url %} + + {% endif %} {% endif %} {% if not "\x3ch1" in page.content %}

{{ page.title | default(config.site_name, true)}}

diff --git a/docs/mkdocs-material-theme/partials/footer.html b/docs/mkdocs-material-theme/partials/footer.html index 449d9dfcd5b..5ab451ef554 100644 --- a/docs/mkdocs-material-theme/partials/footer.html +++ b/docs/mkdocs-material-theme/partials/footer.html @@ -44,11 +44,6 @@ {{ config.copyright }}
{% endif %} - powered by - MkDocs - and - - Material for MkDocs {% block social %} {% include "partials/social.html" %} diff --git a/docs/mkdocs-material-theme/partials/header.html b/docs/mkdocs-material-theme/partials/header.html index 764a5b82231..3c954c8c5ac 100644 --- a/docs/mkdocs-material-theme/partials/header.html +++ b/docs/mkdocs-material-theme/partials/header.html @@ -33,11 +33,11 @@ diff --git a/docs/mkdocs_en.yml b/docs/mkdocs_en.yml index 59b500c6c98..ec933308103 100644 --- a/docs/mkdocs_en.yml +++ b/docs/mkdocs_en.yml @@ -1,15 +1,16 @@ -site_name: ClickHouse docs +site_name: ClickHouse Documentation +copyright: ©2016–2018 Yandex LLC docs_dir: en site_dir: build/docs/en -use_directory_urls: false +use_directory_urls: true repo_name: 'yandex/ClickHouse' repo_url: 'https://github.com/yandex/ClickHouse/' edit_uri: 'edit/master/docs/en' extra_css: - - assets/stylesheets/yandex_fonts.css + - assets/stylesheets/custom.css theme: name: null @@ -31,10 +32,12 @@ theme: - 404.html extra: + single_page: false search: language: 'en' - +markdown_extensions: + - codehilite pages: - 'ClickHouse': 'index.md' diff --git a/docs/mkdocs_en_single_page.yml b/docs/mkdocs_en_single_page.yml index 15088124fc3..5025d0cbc3f 100644 --- a/docs/mkdocs_en_single_page.yml +++ b/docs/mkdocs_en_single_page.yml @@ -1,4 +1,5 @@ -site_name: ClickHouse docs +site_name: ClickHouse Documentation +copyright: ©2016–2018 Yandex LLC docs_dir: en_single_page site_dir: build/docs/en/single @@ -8,7 +9,10 @@ repo_name: 'yandex/ClickHouse' repo_url: 'https://github.com/yandex/ClickHouse/' extra_css: - - assets/stylesheets/yandex_fonts.css + - assets/stylesheets/custom.css + +markdown_extensions: + - codehilite theme: name: null @@ -30,10 +34,9 @@ theme: - 404.html extra: + single_page: true search: language: 'en' - - pages: - 'Documentation': 'index.md' diff --git a/docs/mkdocs_ru.yml b/docs/mkdocs_ru.yml index 3d16b0526ed..ad26a510ad9 100644 --- a/docs/mkdocs_ru.yml +++ b/docs/mkdocs_ru.yml @@ -1,15 +1,19 @@ site_name: Документация ClickHouse +copyright: ©2016–2018 Yandex LLC docs_dir: ru site_dir: build/docs/ru -use_directory_urls: false +use_directory_urls: true repo_name: 'yandex/ClickHouse' repo_url: 'https://github.com/yandex/ClickHouse/' edit_uri: 'edit/master/docs/ru' extra_css: - - assets/stylesheets/yandex_fonts.css + - assets/stylesheets/custom.css + +markdown_extensions: + - codehilite theme: name: null @@ -31,6 +35,7 @@ theme: - 404.html extra: + single_page: false search: language: 'en, ru' diff --git a/docs/mkdocs_ru_single_page.yml b/docs/mkdocs_ru_single_page.yml index 9fefab45e0b..c66c4716eda 100644 --- a/docs/mkdocs_ru_single_page.yml +++ b/docs/mkdocs_ru_single_page.yml @@ -1,4 +1,5 @@ site_name: Документация ClickHouse +copyright: ©2016–2018 Yandex LLC docs_dir: ru_single_page site_dir: build/docs/ru/single @@ -8,7 +9,10 @@ repo_name: 'yandex/ClickHouse' repo_url: 'https://github.com/yandex/ClickHouse/' extra_css: - - assets/stylesheets/yandex_fonts.css + - assets/stylesheets/custom.css + +markdown_extensions: + - codehilite theme: name: null @@ -30,6 +34,7 @@ theme: - 404.html extra: + single_page: true search: language: 'en, ru' diff --git a/docs/requirements.txt b/docs/requirements.txt index ceaeff0fcf7..fbe704e75b2 100644 --- a/docs/requirements.txt +++ b/docs/requirements.txt @@ -1,20 +1,28 @@ alabaster==0.7.10 Babel==2.5.1 +backports-abc==0.5 certifi==2017.11.5 chardet==3.0.4 +click==6.7 CommonMark==0.5.4 docutils==0.14 idna==2.6 imagesize==0.7.1 Jinja2==2.10 +livereload==2.5.1 +Markdown==2.6.11 MarkupSafe==1.0 +mkdocs==0.17.2 Pygments==2.2.0 pytz==2017.3 +PyYAML==3.12 recommonmark==0.4.0 requests==2.18.4 +singledispatch==3.4.0.3 six==1.11.0 snowballstemmer==1.2.1 Sphinx==1.6.5 sphinxcontrib-websupport==1.0.1 +tornado==4.5.3 typing==3.6.2 urllib3==1.22 diff --git a/docs/ru/table_engines/distributed.md b/docs/ru/table_engines/distributed.md index 20d9bb40daa..3a0fbca5533 100644 --- a/docs/ru/table_engines/distributed.md +++ b/docs/ru/table_engines/distributed.md @@ -1,6 +1,6 @@ -# Distributed (Схема шардирования) +# Distributed **Движок Distributed не хранит данные самостоятельно**, а позволяет обрабатывать запросы распределённо, на нескольких серверах. Чтение автоматически распараллеливается. При чтении будут использованы индексы таблиц на удалённых серверах, если есть. diff --git a/docs/ru/table_engines/replication.md b/docs/ru/table_engines/replication.md index 99c37a8cfb3..f76f7644a77 100644 --- a/docs/ru/table_engines/replication.md +++ b/docs/ru/table_engines/replication.md @@ -2,16 +2,18 @@ # Репликация данных -## ReplicatedMergeTree +## ReplicatedAggregatingMergeTree ## ReplicatedCollapsingMergeTree -## ReplicatedAggregatingMergeTree +## ReplicatedGraphiteMergeTree + +## ReplicatedMergeTree + +## ReplicatedReplacingMergeTree ## ReplicatedSummingMergeTree -## ReplicatedGraphiteMergeTree - Репликация поддерживается только для таблиц семейства MergeTree. Репликация работает на уровне отдельных таблиц, а не всего сервера. То есть, на сервере могут быть расположены одновременно реплицируемые и не реплицируемые таблицы. Реплицируются INSERT, ALTER (см. подробности в описании запроса ALTER). Реплицируются сжатые данные, а не тексты запросов. diff --git a/docs/validate_headers_structures_in_md.py b/docs/validate_headers_structures_in_md.py index d4daabd3112..26df66450d5 100644 --- a/docs/validate_headers_structures_in_md.py +++ b/docs/validate_headers_structures_in_md.py @@ -1,58 +1,64 @@ #!/usr/bin/env python # -*- coding: utf-8 -*- +# Gets all the files in SOURCES_TREE directory, shows all level headers +# for each file and skip or process files by user's selection. - -# Gets all the files in SOURCES_TREE directory, shows all level headers for each file and skip or process files by user's selection. - -from os import walk +import os SOURCES_TREE = 'ru' STOP_AT_THE_FIRST_FILE = False -for (dirpath, dirnames, filenames) in walk(SOURCES_TREE): - for filename in filenames : - if filename == 'conf.py': continue +for (dirpath, dirnames, filenames) in os.walk(SOURCES_TREE): + for filename in filenames: + if filename == 'conf.py': + continue - print "=== "+ dirpath+'/'+filename + print '=== ' + dirpath + '/' + filename - f = open(dirpath+'/'+filename) + f = open(dirpath + '/' + filename) content = f.readlines() f.close() - # Showing headers structure in md-file count_lines = 0 for l in content: - if l.startswith('#'): print l - if l.startswith("==="): print content[count_lines - 1] + l - if l.startswith("---"): print content[count_lines - 1] + l + if l.startswith('#'): + print l + if l.startswith('==='): + print content[count_lines - 1] + l + if l.startswith('---'): + print content[count_lines - 1] + l count_lines += 1 # At this stage user should check the headers structucture and choose what to to # Replace headers markup or not - choise = raw_input("What to do with a file (pass(s) or process(p)): ") + choise = raw_input('What to do with a file (pass(s) or process(p)): ') - if choise == 's': continue + if choise == 's': + continue else: - print "processing..." + print 'processing...' count_lines = 0 for l in content: - if l.startswith("==="): - print count_lines, content[count_lines -1], content[count_lines] - content[count_lines - 1] = "# "+content[count_lines - 1] + if l.startswith('==='): + print count_lines, content[count_lines - 1], content[count_lines] + content[count_lines - 1] = '# ' + content[count_lines - 1] content.pop(count_lines) - if l.startswith("---"): - print count_lines, content[count_lines -1], content[count_lines] - content[count_lines - 1] = "## "+content[count_lines - 1] + if l.startswith('---'): + print count_lines, content[count_lines - 1], content[count_lines] + content[count_lines - 1] = '## ' + content[count_lines - 1] content.pop(count_lines) count_lines += 1 - f=open(dirpath+'/'+filename,"w") - for l in content: f.write(l) + f = open(dirpath + '/' + filename, 'w') + for l in content: + f.write(l) f.close() - if STOP_AT_THE_FIRST_FILE : break + if STOP_AT_THE_FIRST_FILE: + break - if STOP_AT_THE_FIRST_FILE : break + if STOP_AT_THE_FIRST_FILE: + break diff --git a/libs/libcommon/include/common/iostream_debug_helpers.h b/libs/libcommon/include/common/iostream_debug_helpers.h index 09ab4de83e6..739a35d398f 100644 --- a/libs/libcommon/include/common/iostream_debug_helpers.h +++ b/libs/libcommon/include/common/iostream_debug_helpers.h @@ -74,6 +74,14 @@ ostream & operator<<(ostream & stream, const map & what) return stream; } +template +ostream & operator<<(ostream & stream, const multimap & what) +{ + stream << "multimap(size = " << what.size() << ")"; + dumpContainer(stream, what); + return stream; +} + template ostream & operator<<(ostream & stream, const unordered_map & what) { @@ -82,6 +90,14 @@ ostream & operator<<(ostream & stream, const unordered_map & what) return stream; } +template +ostream & operator<<(ostream & stream, const unordered_multimap & what) +{ + stream << "unordered_multimap(size = " << what.size() << ")"; + dumpContainer(stream, what); + return stream; +} + template ostream & operator<<(ostream & stream, const set & what) { @@ -90,6 +106,14 @@ ostream & operator<<(ostream & stream, const set & what) return stream; } +template +ostream & operator<<(ostream & stream, const multiset & what) +{ + stream << "multiset(size = " << what.size() << ")"; + dumpContainer(stream, what); + return stream; +} + template ostream & operator<<(ostream & stream, const unordered_set & what) { @@ -98,6 +122,14 @@ ostream & operator<<(ostream & stream, const unordered_set & what) return stream; } +template +ostream & operator<<(ostream & stream, const unordered_multiset & what) +{ + stream << "unordered_multiset(size = " << what.size() << ")"; + dumpContainer(stream, what); + return stream; +} + template ostream & operator<<(ostream & stream, const list & what) { diff --git a/website/index.html b/website/index.html index e87c7774ba2..6e079bf6d4f 100644 --- a/website/index.html +++ b/website/index.html @@ -90,6 +90,7 @@
+

ClickHouse. Just makes you think faster.

@@ -449,7 +450,7 @@ clickhouse-client ClickHouse source code is published under Apache 2.0 License. Software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

- +
diff --git a/website/tutorial.html b/website/tutorial.html index 65e9f1ae518..da881ddacc5 100644 --- a/website/tutorial.html +++ b/website/tutorial.html @@ -596,7 +596,7 @@ ENGINE = ReplicatedMergeTree( ClickHouse source code is published under Apache 2.0 License. Software is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.

- +