From 5d34b00487376ab710448c6c66fd763f9231dc79 Mon Sep 17 00:00:00 2001 From: dmitrii Date: Thu, 18 Apr 2019 11:24:06 +0300 Subject: [PATCH 01/75] parser + ast --- dbms/src/Parsers/ASTSelectQuery.cpp | 2 ++ dbms/src/Parsers/ASTSelectQuery.h | 1 + dbms/src/Parsers/ParserSelectQuery.cpp | 22 +++++++++++++++++++--- 3 files changed, 22 insertions(+), 3 deletions(-) diff --git a/dbms/src/Parsers/ASTSelectQuery.cpp b/dbms/src/Parsers/ASTSelectQuery.cpp index da7277abee1..2218ef0e797 100644 --- a/dbms/src/Parsers/ASTSelectQuery.cpp +++ b/dbms/src/Parsers/ASTSelectQuery.cpp @@ -143,6 +143,8 @@ void ASTSelectQuery::formatImpl(const FormatSettings & s, FormatState & state, F s.ostr << ", "; } limitLength()->formatImpl(s, state, frame); + if (limit_with_ties) + s.ostr << (s.hilite ? hilite_keyword : "") << s.nl_or_ws << indent_str << " WITH TIES" << (s.hilite ? hilite_none : ""); } if (settings()) diff --git a/dbms/src/Parsers/ASTSelectQuery.h b/dbms/src/Parsers/ASTSelectQuery.h index 374933a1d57..f2577fa3264 100644 --- a/dbms/src/Parsers/ASTSelectQuery.h +++ b/dbms/src/Parsers/ASTSelectQuery.h @@ -41,6 +41,7 @@ public: bool group_by_with_totals = false; bool group_by_with_rollup = false; bool group_by_with_cube = false; + bool limit_with_ties = false; ASTPtr & refSelect() { return getExpression(Expression::SELECT); } ASTPtr & refTables() { return getExpression(Expression::TABLES); } diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index c521ba4ec40..f991cb17eac 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -41,6 +41,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_rollup("ROLLUP"); ParserKeyword s_cube("CUBE"); ParserKeyword s_top("TOP"); + ParserKeyword s_with_ties("WITH TIES"); ParserKeyword s_offset("OFFSET"); ParserNotEmptyExpressionList exp_list(false); @@ -75,7 +76,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } - /// SELECT [DISTINCT] [TOP N] expr list + /// SELECT [DISTINCT] [TOP N [WITH TIES]] expr list { if (!s_select.ignore(pos, expected)) return false; @@ -99,6 +100,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!num.parse(pos, limit_length, expected)) return false; } + + if (s_with_ties.ignore(pos, expected)) + select_query->limit_with_ties = true; } if (!exp_list_for_select_clause.parse(pos, select_expression_list, expected)) @@ -180,7 +184,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } - /// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list + /// LIMIT length [WITH TIES] | LIMIT offset, length [WITH TIES] | LIMIT count BY expr-list if (s_limit.ignore(pos, expected)) { if (limit_length) @@ -196,7 +200,12 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) limit_offset = limit_length; if (!exp_elem.parse(pos, limit_length, expected)) return false; + + if (s_with_ties.ignore(pos, expected)) + select_query->limit_with_ties = true; } + else if (s_with_ties.ignore(pos, expected)) + select_query->limit_with_ties = true; else if (s_by.ignore(pos, expected)) { limit_by_value = limit_length; @@ -212,7 +221,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) } } - /// LIMIT length | LIMIT offset, length + /// LIMIT length [WITH TIES] | LIMIT offset, length [WITH TIES] if (s_limit.ignore(pos, expected)) { if (!limit_by_value || limit_length) @@ -229,8 +238,15 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) if (!exp_elem.parse(pos, limit_length, expected)) return false; } + + if (s_with_ties.ignore(pos, expected)) + select_query->limit_with_ties = true; } + // WITH TIES was used without ORDER BY + if (!order_expression_list && select_query->limit_with_ties) + return false; + /// SETTINGS key1 = value1, key2 = value2, ... if (s_settings.ignore(pos, expected)) { From 8d9105eb25d40ebab20e3900c3b374bf2618fa18 Mon Sep 17 00:00:00 2001 From: dmitrii Date: Thu, 18 Apr 2019 15:34:10 +0300 Subject: [PATCH 02/75] return false -> throw exception --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Parsers/ParserSelectQuery.cpp | 5 +++-- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index bfe9049fcb3..f1bb78562a9 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -426,6 +426,7 @@ namespace ErrorCodes extern const int BROTLI_WRITE_FAILED = 449; extern const int BAD_TTL_EXPRESSION = 450; extern const int BAD_TTL_FILE = 451; + extern const int WITH_TIES_WITHOUT_ORDER_BY = 452; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index f991cb17eac..373066d0d4e 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -17,6 +17,7 @@ namespace ErrorCodes { extern const int SYNTAX_ERROR; extern const int TOP_AND_LIMIT_TOGETHER; + extern const int WITH_TIES_WITHOUT_ORDER_BY; } @@ -243,9 +244,9 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) select_query->limit_with_ties = true; } - // WITH TIES was used without ORDER BY + /// WITH TIES was used without ORDER BY if (!order_expression_list && select_query->limit_with_ties) - return false; + throw Exception("Can not use WITH TIES without ORDER BY", ErrorCodes::WITH_TIES_WITHOUT_ORDER_BY); /// SETTINGS key1 = value1, key2 = value2, ... if (s_settings.ignore(pos, expected)) From a130850e1534c1f99c0276f1c1ebc2511254999a Mon Sep 17 00:00:00 2001 From: dmitrii Date: Fri, 19 Apr 2019 16:38:25 +0300 Subject: [PATCH 03/75] limit with ties --- dbms/CMakeLists.txt | 2 +- dbms/src/Common/SharedBlockRowRef.h | 90 +++++++++++++++++++ .../AggregatingSortedBlockInputStream.h | 4 +- .../CollapsingSortedBlockInputStream.h | 10 +-- .../GraphiteRollupSortedBlockInputStream.cpp | 2 +- .../GraphiteRollupSortedBlockInputStream.h | 4 +- .../src/DataStreams/LimitBlockInputStream.cpp | 87 ++++++++++++++++-- dbms/src/DataStreams/LimitBlockInputStream.h | 9 +- .../MergingSortedBlockInputStream.h | 77 +--------------- .../ReplacingSortedBlockInputStream.h | 6 +- .../SummingSortedBlockInputStream.h | 4 +- ...sionedCollapsingSortedBlockInputStream.cpp | 4 +- ...ersionedCollapsingSortedBlockInputStream.h | 4 +- .../Interpreters/InterpreterSelectQuery.cpp | 19 ++-- 14 files changed, 214 insertions(+), 108 deletions(-) create mode 100644 dbms/src/Common/SharedBlockRowRef.h diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index 5c1d73b7a74..407ebc33370 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -134,7 +134,7 @@ list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctio list (APPEND dbms_sources src/Dictionaries/DictionaryFactory.cpp src/Dictionaries/DictionarySourceFactory.cpp src/Dictionaries/DictionaryStructure.cpp) list (APPEND dbms_headers src/Dictionaries/DictionaryFactory.h src/Dictionaries/DictionarySourceFactory.h src/Dictionaries/DictionaryStructure.h) -add_library(clickhouse_common_io ${LINK_MODE} ${clickhouse_common_io_headers} ${clickhouse_common_io_sources}) +add_library(clickhouse_common_io ${LINK_MODE} ${clickhouse_common_io_headers} ${clickhouse_common_io_sources} src/Common/SharedBlockRowRef.h) if (OS_FREEBSD) target_compile_definitions (clickhouse_common_io PUBLIC CLOCK_MONOTONIC_COARSE=CLOCK_MONOTONIC_FAST) diff --git a/dbms/src/Common/SharedBlockRowRef.h b/dbms/src/Common/SharedBlockRowRef.h new file mode 100644 index 00000000000..5aa994d22f4 --- /dev/null +++ b/dbms/src/Common/SharedBlockRowRef.h @@ -0,0 +1,90 @@ +#pragma once + +#include +#include "../Core/Block.h" +#include "../../../contrib/boost/boost/smart_ptr/intrusive_ptr.hpp" +#include "../Columns/IColumn.h" + + +namespace DB +{ + +/// Allows you refer to the row in the block and hold the block ownership, +/// and thus avoid creating a temporary row object. +/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; +/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; +/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; +/// The reference counter is not atomic, since it is used from one thread. +namespace detail +{ + struct SharedBlock : Block + { + int refcount = 0; + + ColumnRawPtrs all_columns; + ColumnRawPtrs sort_columns; + + SharedBlock(Block && block) : Block(std::move(block)) {} + }; +} + +inline void intrusive_ptr_add_ref(detail::SharedBlock * ptr) +{ + ++ptr->refcount; +} + +inline void intrusive_ptr_release(detail::SharedBlock * ptr) +{ + if (0 == --ptr->refcount) + delete ptr; +} + +using SharedBlockPtr = boost::intrusive_ptr; + +struct SharedBlockRowRef +{ + ColumnRawPtrs * columns = nullptr; + size_t row_num; + SharedBlockPtr shared_block; + + void swap(SharedBlockRowRef & other) + { + std::swap(columns, other.columns); + std::swap(row_num, other.row_num); + std::swap(shared_block, other.shared_block); + } + + /// The number and types of columns must match. + bool operator==(const SharedBlockRowRef & other) const + { + size_t size = columns->size(); + for (size_t i = 0; i < size; ++i) + if (0 != (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1)) + return false; + return true; + } + + bool operator!=(const SharedBlockRowRef & other) const + { + return !(*this == other); + } + + void reset() + { + SharedBlockRowRef empty; + swap(empty); + } + + bool empty() const { return columns == nullptr; } + size_t size() const { return empty() ? 0 : columns->size(); } +}; + +void setRowRef(SharedBlockRowRef & row_ref, SharedBlockPtr shared_block, ColumnRawPtrs * columns, size_t row_num) +{ + row_ref.row_num = row_num; + row_ref.columns = columns; + row_ref.shared_block = shared_block; +} + + +} \ No newline at end of file diff --git a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h index 522b54aeaec..18de871e021 100644 --- a/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/AggregatingSortedBlockInputStream.h @@ -43,8 +43,8 @@ private: ColumnNumbers column_numbers_not_to_aggregate; std::vector columns_to_aggregate; - RowRef current_key; /// The current primary key. - RowRef next_key; /// The primary key of the next row. + SharedBlockRowRef current_key; /// The current primary key. + SharedBlockRowRef next_key; /// The primary key of the next row. /** We support two different cursors - with Collation and without. * Templates are used instead of polymorphic SortCursor and calls to virtual functions. diff --git a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h index cf72df30dbd..9abdcc92be6 100644 --- a/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/CollapsingSortedBlockInputStream.h @@ -46,12 +46,12 @@ private: /// Read is finished. bool finished = false; - RowRef current_key; /// The current primary key. - RowRef next_key; /// The primary key of the next row. + SharedBlockRowRef current_key; /// The current primary key. + SharedBlockRowRef next_key; /// The primary key of the next row. - RowRef first_negative; /// The first negative row for the current primary key. - RowRef last_positive; /// The last positive row for the current primary key. - RowRef last_negative; /// Last negative row. It is only stored if there is not one row is written to output. + SharedBlockRowRef first_negative; /// The first negative row for the current primary key. + SharedBlockRowRef last_positive; /// The last positive row for the current primary key. + SharedBlockRowRef last_negative; /// Last negative row. It is only stored if there is not one row is written to output. size_t count_positive = 0; /// The number of positive rows for the current primary key. size_t count_negative = 0; /// The number of negative rows for the current primary key. diff --git a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp index fb24d8c37a4..c4494b760e7 100644 --- a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.cpp @@ -321,7 +321,7 @@ void GraphiteRollupSortedBlockInputStream::finishCurrentGroup(MutableColumns & m } -void GraphiteRollupSortedBlockInputStream::accumulateRow(RowRef & row) +void GraphiteRollupSortedBlockInputStream::accumulateRow(SharedBlockRowRef & row) { const Graphite::AggregationPattern * aggregation_pattern = std::get<1>(current_rule); if (aggregate_state_created) diff --git a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h index 00bd2f4b67e..040ae4f2c71 100644 --- a/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h +++ b/dbms/src/DataStreams/GraphiteRollupSortedBlockInputStream.h @@ -204,7 +204,7 @@ private: StringRef current_group_path; /// Last row with maximum version for current primary key (time bucket). - RowRef current_subgroup_newest_row; + SharedBlockRowRef current_subgroup_newest_row; /// Time of last read row time_t current_time = 0; @@ -236,7 +236,7 @@ private: void finishCurrentGroup(MutableColumns & merged_columns); /// Update the state of the aggregate function with the new `value`. - void accumulateRow(RowRef & row); + void accumulateRow(SharedBlockRowRef & row); }; } diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index b1b8f5c95d1..5f2f504e7b5 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -6,8 +6,33 @@ namespace DB { -LimitBlockInputStream::LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_, bool use_limit_as_total_rows_approx) - : limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_) +namespace detail +{ +ColumnRawPtrs getBlockColumns(const Block & block, const SortDescription description) +{ + size_t size = description.size(); + ColumnRawPtrs res; + res.reserve(size); + + for (size_t i = 0; i < size; ++i) + { + const IColumn * column = !description[i].column_name.empty() + ? block.getByName(description[i].column_name).column.get() + : block.safeGetByPosition(description[i].column_number).column.get(); + res.emplace_back(column); + } + + return res; +} +} + + + +LimitBlockInputStream::LimitBlockInputStream( + const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_, + bool use_limit_as_total_rows_approx, bool with_ties_, const SortDescription & description_) + : limit(limit_), offset(offset_), always_read_till_end(always_read_till_end_), with_ties(with_ties_) + , description(description_) { if (use_limit_as_total_rows_approx) { @@ -23,6 +48,36 @@ Block LimitBlockInputStream::readImpl() Block res; UInt64 rows = 0; + if (with_ties && tiesRowRef.shared_block) + { + res = children.back()->read(); + rows = res.rows(); + pos += rows; + + SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); + ColumnRawPtrs columns = getBlockColumns(*ptr, description); + UInt64 len; + + for (len = 0; len < rows; ++len) + { + SharedBlockRowRef currentRow; + setRowRef(currentRow, ptr, &columns, len); + if (currentRow != tiesRowRef) + { + tiesRowRef.reset(); + break; + } + } + + if (len < rows - 1) + { + for (size_t i = 0; i < ptr->columns(); ++i) + ptr->safeGetByPosition(i).column = ptr->safeGetByPosition(i).column->cut(0, len); + } + + return *ptr; + } + /// pos - how many lines were read, including the last read block if (pos >= offset + limit) @@ -46,6 +101,7 @@ Block LimitBlockInputStream::readImpl() pos += rows; } while (pos <= offset); + /// give away the whole block if (pos >= offset + rows && pos <= offset + limit) return res; @@ -60,13 +116,34 @@ Block LimitBlockInputStream::readImpl() static_cast(pos) - static_cast(offset), static_cast(limit) + static_cast(offset) - static_cast(pos) + static_cast(rows))); - for (size_t i = 0; i < res.columns(); ++i) - res.safeGetByPosition(i).column = res.safeGetByPosition(i).column->cut(start, length); + SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); + if (with_ties) + { + ColumnRawPtrs columns = getBlockColumns(*ptr, description); + setRowRef(tiesRowRef, ptr, &columns, start + length - 1); + + for (size_t i = tiesRowRef.row_num + 1; i < rows; ++i) + { + SharedBlockRowRef currentRow; + setRowRef(currentRow, ptr, &columns, i); + if (currentRow == tiesRowRef) + ++length; + else + { + tiesRowRef.reset(); + break; + } + } + } + + for (size_t i = 0; i < ptr->columns(); ++i) + ptr->safeGetByPosition(i).column = ptr->safeGetByPosition(i).column->cut(start, length); // TODO: we should provide feedback to child-block, so it will know how many rows are actually consumed. // It's crucial for streaming engines like Kafka. - return res; + return *ptr; } + } diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index ed6dac8c5ac..1760f791a7a 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -1,6 +1,7 @@ #pragma once #include +#include namespace DB @@ -18,7 +19,10 @@ public: * when otherwise, due to the cancellation of the request, we would not have received the data for GROUP BY WITH TOTALS from the remote server. * If use_limit_as_total_rows_approx = true, then addTotalRowsApprox is called to use the limit in progress & stats */ - LimitBlockInputStream(const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_ = false, bool use_limit_as_total_rows_approx = false); + LimitBlockInputStream( + const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, + bool always_read_till_end_ = false, bool use_limit_as_total_rows_approx = false, + bool with_ties_ = false, const SortDescription & description_ = {}); String getName() const override { return "Limit"; } @@ -32,6 +36,9 @@ private: UInt64 offset; UInt64 pos = 0; bool always_read_till_end; + bool with_ties; + const SortDescription description; + SharedBlockRowRef tiesRowRef; }; } diff --git a/dbms/src/DataStreams/MergingSortedBlockInputStream.h b/dbms/src/DataStreams/MergingSortedBlockInputStream.h index 68ea179d68d..903b9ef9fc5 100644 --- a/dbms/src/DataStreams/MergingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/MergingSortedBlockInputStream.h @@ -5,6 +5,7 @@ #include #include +#include #include #include @@ -24,39 +25,6 @@ namespace ErrorCodes } -/// Allows you refer to the row in the block and hold the block ownership, -/// and thus avoid creating a temporary row object. -/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; -/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; -/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; -/// The reference counter is not atomic, since it is used from one thread. -namespace detail -{ -struct SharedBlock : Block -{ - int refcount = 0; - - ColumnRawPtrs all_columns; - ColumnRawPtrs sort_columns; - - SharedBlock(Block && block) : Block(std::move(block)) {} -}; -} - -using SharedBlockPtr = boost::intrusive_ptr; - -inline void intrusive_ptr_add_ref(detail::SharedBlock * ptr) -{ - ++ptr->refcount; -} - -inline void intrusive_ptr_release(detail::SharedBlock * ptr) -{ - if (0 == --ptr->refcount) - delete ptr; -} - - /** Merges several sorted streams into one sorted stream. */ class MergingSortedBlockInputStream : public IBlockInputStream @@ -78,45 +46,6 @@ public: Block getHeader() const override { return header; } protected: - struct RowRef - { - ColumnRawPtrs * columns = nullptr; - size_t row_num; - SharedBlockPtr shared_block; - - void swap(RowRef & other) - { - std::swap(columns, other.columns); - std::swap(row_num, other.row_num); - std::swap(shared_block, other.shared_block); - } - - /// The number and types of columns must match. - bool operator==(const RowRef & other) const - { - size_t size = columns->size(); - for (size_t i = 0; i < size; ++i) - if (0 != (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1)) - return false; - return true; - } - - bool operator!=(const RowRef & other) const - { - return !(*this == other); - } - - void reset() - { - RowRef empty; - swap(empty); - } - - bool empty() const { return columns == nullptr; } - size_t size() const { return empty() ? 0 : columns->size(); } - }; - - Block readImpl() override; void readSuffixImpl() override; @@ -197,7 +126,7 @@ protected: } template - void setRowRef(RowRef & row_ref, TSortCursor & cursor) + void setRowRef(SharedBlockRowRef & row_ref, TSortCursor & cursor) { row_ref.row_num = cursor.impl->pos; row_ref.shared_block = source_blocks[cursor.impl->order]; @@ -205,7 +134,7 @@ protected: } template - void setPrimaryKeyRef(RowRef & row_ref, TSortCursor & cursor) + void setPrimaryKeyRef(SharedBlockRowRef & row_ref, TSortCursor & cursor) { row_ref.row_num = cursor.impl->pos; row_ref.shared_block = source_blocks[cursor.impl->order]; diff --git a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h index cd52c3a8d08..72d80d939ca 100644 --- a/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/ReplacingSortedBlockInputStream.h @@ -40,11 +40,11 @@ private: bool finished = false; /// Primary key of current row. - RowRef current_key; + SharedBlockRowRef current_key; /// Primary key of next row. - RowRef next_key; + SharedBlockRowRef next_key; /// Last row with maximum version for current primary key. - RowRef selected_row; + SharedBlockRowRef selected_row; /// The position (into current_row_sources) of the row with the highest version. size_t max_pos = 0; diff --git a/dbms/src/DataStreams/SummingSortedBlockInputStream.h b/dbms/src/DataStreams/SummingSortedBlockInputStream.h index 96f417eb113..4412e5529f8 100644 --- a/dbms/src/DataStreams/SummingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/SummingSortedBlockInputStream.h @@ -129,8 +129,8 @@ private: std::vector columns_to_aggregate; std::vector maps_to_sum; - RowRef current_key; /// The current primary key. - RowRef next_key; /// The primary key of the next row. + SharedBlockRowRef current_key; /// The current primary key. + SharedBlockRowRef next_key; /// The primary key of the next row. Row current_row; bool current_row_is_zero = true; /// Are all summed columns zero (or empty)? It is updated incrementally. diff --git a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp index fc24bef60bc..e5a501f019b 100644 --- a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp +++ b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.cpp @@ -46,7 +46,7 @@ void VersionedCollapsingSortedBlockInputStream::insertGap(size_t gap_size) } } -void VersionedCollapsingSortedBlockInputStream::insertRow(size_t skip_rows, const RowRef & row, MutableColumns & merged_columns) +void VersionedCollapsingSortedBlockInputStream::insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns) { const auto & columns = row.shared_block->all_columns; for (size_t i = 0; i < num_columns; ++i) @@ -109,7 +109,7 @@ void VersionedCollapsingSortedBlockInputStream::merge(MutableColumns & merged_co { SortCursor current = queue.top(); - RowRef next_key; + SharedBlockRowRef next_key; Int8 sign = static_cast(*current->all_columns[sign_column_number]).getData()[current->pos]; diff --git a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h index 573fd66920d..78db168bb8b 100644 --- a/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h +++ b/dbms/src/DataStreams/VersionedCollapsingSortedBlockInputStream.h @@ -197,7 +197,7 @@ private: Int8 sign_in_queue = 0; const size_t max_rows_in_queue; /// Rows with the same primary key and sign. - FixedSizeDequeWithGaps current_keys; + FixedSizeDequeWithGaps current_keys; size_t blocks_written = 0; @@ -207,7 +207,7 @@ private: void merge(MutableColumns & merged_columns, std::priority_queue & queue); /// Output to result row for the current primary key. - void insertRow(size_t skip_rows, const RowRef & row, MutableColumns & merged_columns); + void insertRow(size_t skip_rows, const SharedBlockRowRef & row, MutableColumns & merged_columns); void insertGap(size_t gap_size); }; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index fe5af40a6d3..1fb62ce7f10 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -803,7 +803,7 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt /** Optimization - if there are several sources and there is LIMIT, then first apply the preliminary LIMIT, * limiting the number of rows in each up to `offset + limit`. */ - if (query.limitLength() && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.has_limit_by && !settings.extremes) + if (query.limitLength() && !query.limit_with_ties && pipeline.hasMoreThanOneStream() && !query.distinct && !expressions.has_limit_by && !settings.extremes) { executePreLimit(pipeline); } @@ -879,8 +879,8 @@ static std::pair getLimitLengthAndOffset(const ASTSelectQuery & static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context) { - /// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY. - if (!query.distinct && !query.limitBy()) + /// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY or WITH TIES. + if (!query.distinct && !query.limitBy() && !query.limit_with_ties) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); return limit_length + limit_offset; @@ -1083,11 +1083,12 @@ void InterpreterSelectQuery::executeFetchColumns( auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY but LIMIT is specified, and limit + offset < max_block_size, + /** Optimization - if not specified DISTINCT, WHERE, GROUP, HAVING, ORDER, LIMIT BY, WITH TIES but LIMIT is specified, and limit + offset < max_block_size, * then as the block size we will use limit + offset (not to read more from the table than requested), * and also set the number of threads to 1. */ if (!query.distinct + && !query.limit_with_ties && !query.prewhere() && !query.where() && !query.groupBy() @@ -1511,7 +1512,7 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or UInt64 limit_for_distinct = 0; /// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows. - if (!query.orderBy() || !before_order) + if ((!query.orderBy() || !before_order) && !query.limit_with_ties) limit_for_distinct = limit_length + limit_offset; pipeline.transform([&](auto & stream) @@ -1546,8 +1547,8 @@ void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) { auto & query = getSelectQuery(); - /// If there is LIMIT - if (query.limitLength()) + /// If there is LIMIT and no WITH TIES + if (query.limitLength() && !query.limit_with_ties) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) @@ -1624,13 +1625,15 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query)) always_read_till_end = true; + SortDescription order_descr = getSortDescription(query); + UInt64 limit_length; UInt64 limit_offset; std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context); pipeline.transform([&](auto & stream) { - stream = std::make_shared(stream, limit_length, limit_offset, always_read_till_end); + stream = std::make_shared(stream, limit_length, limit_offset, always_read_till_end, false, query.limit_with_ties, order_descr); }); } } From 100cee18c38cd8b3ffbc5a75f6d89f5387c7659b Mon Sep 17 00:00:00 2001 From: dmitrii Date: Fri, 19 Apr 2019 18:45:42 +0300 Subject: [PATCH 04/75] comments + fix naming --- dbms/src/Common/SharedBlockRowRef.h | 6 ---- .../src/DataStreams/LimitBlockInputStream.cpp | 36 +++++++++++++------ dbms/src/DataStreams/LimitBlockInputStream.h | 4 ++- 3 files changed, 29 insertions(+), 17 deletions(-) diff --git a/dbms/src/Common/SharedBlockRowRef.h b/dbms/src/Common/SharedBlockRowRef.h index 5aa994d22f4..c73bf5e6400 100644 --- a/dbms/src/Common/SharedBlockRowRef.h +++ b/dbms/src/Common/SharedBlockRowRef.h @@ -79,12 +79,6 @@ struct SharedBlockRowRef size_t size() const { return empty() ? 0 : columns->size(); } }; -void setRowRef(SharedBlockRowRef & row_ref, SharedBlockPtr shared_block, ColumnRawPtrs * columns, size_t row_num) -{ - row_ref.row_num = row_num; - row_ref.columns = columns; - row_ref.shared_block = shared_block; -} } \ No newline at end of file diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 5f2f504e7b5..f3fb45f27cd 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -8,6 +8,8 @@ namespace DB namespace detail { + +/// gets pointers to all columns of block, which were used for ORDER BY ColumnRawPtrs getBlockColumns(const Block & block, const SortDescription description) { size_t size = description.size(); @@ -24,6 +26,16 @@ ColumnRawPtrs getBlockColumns(const Block & block, const SortDescription descrip return res; } + + +void setSharedBlockRowRef(SharedBlockRowRef &row_ref, SharedBlockPtr shared_block, ColumnRawPtrs *columns, + size_t row_num) +{ + row_ref.row_num = row_num; + row_ref.columns = columns; + row_ref.shared_block = shared_block; +} + } @@ -48,7 +60,9 @@ Block LimitBlockInputStream::readImpl() Block res; UInt64 rows = 0; - if (with_ties && tiesRowRef.shared_block) + /// pos >= offset + limit and all rows in previous block were equal to ties_row_ref + /// so we check current block + if (with_ties && ties_row_ref.shared_block) { res = children.back()->read(); rows = res.rows(); @@ -61,10 +75,10 @@ Block LimitBlockInputStream::readImpl() for (len = 0; len < rows; ++len) { SharedBlockRowRef currentRow; - setRowRef(currentRow, ptr, &columns, len); - if (currentRow != tiesRowRef) + setSharedBlockRowRef(currentRow, ptr, &columns, len); + if (currentRow != ties_row_ref) { - tiesRowRef.reset(); + ties_row_ref.reset(); break; } } @@ -117,20 +131,22 @@ Block LimitBlockInputStream::readImpl() static_cast(limit) + static_cast(offset) - static_cast(pos) + static_cast(rows))); SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); + + /// check if other rows in current block equals to last one in limit if (with_ties) { ColumnRawPtrs columns = getBlockColumns(*ptr, description); - setRowRef(tiesRowRef, ptr, &columns, start + length - 1); + setSharedBlockRowRef(ties_row_ref, ptr, &columns, start + length - 1); - for (size_t i = tiesRowRef.row_num + 1; i < rows; ++i) + for (size_t i = ties_row_ref.row_num + 1; i < rows; ++i) { - SharedBlockRowRef currentRow; - setRowRef(currentRow, ptr, &columns, i); - if (currentRow == tiesRowRef) + SharedBlockRowRef current_row; + setSharedBlockRowRef(current_row, ptr, &columns, i); + if (current_row == ties_row_ref) ++length; else { - tiesRowRef.reset(); + ties_row_ref.reset(); break; } } diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index 1760f791a7a..763cefc63d8 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -18,6 +18,8 @@ public: * If always_read_till_end = true - reads all the data to the end, but ignores them. This is necessary in rare cases: * when otherwise, due to the cancellation of the request, we would not have received the data for GROUP BY WITH TOTALS from the remote server. * If use_limit_as_total_rows_approx = true, then addTotalRowsApprox is called to use the limit in progress & stats + * with_ties = true, when query has WITH TIES modifier. If so,description should be provided + * description lets us know which row we should check for equality */ LimitBlockInputStream( const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, @@ -38,7 +40,7 @@ private: bool always_read_till_end; bool with_ties; const SortDescription description; - SharedBlockRowRef tiesRowRef; + SharedBlockRowRef ties_row_ref; }; } From 2e0374fbd4f6baf04ca5787b6337861301fa6a47 Mon Sep 17 00:00:00 2001 From: dmitriiut Date: Sun, 21 Apr 2019 05:33:03 +0300 Subject: [PATCH 05/75] fix WITH TIES --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 1fb62ce7f10..0cff5abd45f 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1547,8 +1547,8 @@ void InterpreterSelectQuery::executeUnion(Pipeline & pipeline) void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) { auto & query = getSelectQuery(); - /// If there is LIMIT and no WITH TIES - if (query.limitLength() && !query.limit_with_ties) + /// If there is LIMIT + if (query.limitLength()) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) From 8a7853b0a80773c1a9f5668c6d80a6006a4e5f49 Mon Sep 17 00:00:00 2001 From: dmitriiut Date: Sun, 21 Apr 2019 06:36:59 +0300 Subject: [PATCH 06/75] parser WITH FILL some interpreter changes --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Core/SortDescription.h | 21 ++++++++-- .../Interpreters/InterpreterSelectQuery.cpp | 40 ++++++++++++++++++- .../src/Interpreters/InterpreterSelectQuery.h | 3 ++ dbms/src/Parsers/ASTOrderByElement.cpp | 20 ++++++++++ dbms/src/Parsers/ASTOrderByElement.h | 12 +++++- dbms/src/Parsers/ExpressionElementParsers.cpp | 27 ++++++++++++- dbms/src/Parsers/ExpressionElementParsers.h | 1 + 8 files changed, 117 insertions(+), 8 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index f1bb78562a9..2468df697bb 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -427,6 +427,7 @@ namespace ErrorCodes extern const int BAD_TTL_EXPRESSION = 450; extern const int BAD_TTL_FILE = 451; extern const int WITH_TIES_WITHOUT_ORDER_BY = 452; + extern const int INVALID_WITH_FILL_EXPRESSION = 453; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index ebf3a401d9b..1cccbc2d766 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -20,12 +20,25 @@ struct SortColumnDescription int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less. /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite. std::shared_ptr collator; /// Collator for locale-specific comparison of strings + bool with_fill; /// If true, all missed values in range [FROM, TO] will be filled + /// Range [FROM, TO] respects sorting direction + double fill_from; /// Value >= FROM + double fill_to; /// Value + STEP <= TO + double fill_step; /// Default = 1 - SortColumnDescription(size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) - : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} + SortColumnDescription( + size_t column_number_, int direction_, int nulls_direction_, + const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, + double fill_from_ = 0, double fill_to_ = 0, double fill_step_ = 0) + : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) + , with_fill(with_fill_), fill_from(fill_from_), fill_to(fill_to_), fill_step(fill_step_) {} - SortColumnDescription(const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr) - : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) {} + SortColumnDescription( + const std::string & column_name_, int direction_, int nulls_direction_, + const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, + double fill_from_ = 0, double fill_to_ = 0, double fill_step_ = 0) + : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_) + , collator(collator_), with_fill(with_fill_), fill_from(fill_from_), fill_to(fill_to_), fill_step(fill_step_) {} bool operator == (const SortColumnDescription & other) const { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 0cff5abd45f..1b7cb4777e1 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -74,6 +74,7 @@ namespace ErrorCodes extern const int PARAMETER_OUT_OF_BOUND; extern const int ARGUMENT_OUT_OF_BOUND; extern const int INVALID_LIMIT_EXPRESSION; + extern const int INVALID_WITH_FILL_EXPRESSION; } namespace @@ -888,6 +889,34 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c return 0; } +static Float64 getWithFillFloatValue(const ASTPtr & node, const Context & context) +{ + const auto & [field, type] = evaluateConstantExpression(node, context); + + if (!isNumber(type)) + throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + Field converted = convertFieldToType(field, DataTypeFloat64()); + if (converted.isNull()) + throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of WITH FILL expression is not representable as Float64", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + return converted.safeGet(); +} + +static std::tuple getWithFillParameters(const ASTOrderByElement & node, const Context & context) +{ + Float64 fill_from = 0; + Float64 fill_to = 0; + Float64 fill_step = 1; + + if (node.fill_from) + fill_from = getWithFillFloatValue(node.fill_from, context); + if (node.fill_to) + fill_to = getWithFillFloatValue(node.fill_to, context); + if (node.fill_step) + fill_step = getWithFillFloatValue(node.fill_step, context); + return {fill_from, fill_to, fill_step}; +} void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, Pipeline & pipeline, @@ -1412,7 +1441,7 @@ void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const Expres } -static SortDescription getSortDescription(const ASTSelectQuery & query) +SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -1425,7 +1454,14 @@ static SortDescription getSortDescription(const ASTSelectQuery & query) if (order_by_elem.collation) collator = std::make_shared(order_by_elem.collation->as().value.get()); - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); + if (order_by_elem.with_fill) + { + auto[fill_from, fill_to, fill_step] = getWithFillParameters(order_by_elem, context); + order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, + true, fill_from, fill_to, fill_step); + } + else + order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); } return order_descr; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 1a72a5fb199..fa53a1f5d21 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -208,6 +208,9 @@ private: */ void initSettings(); + /// gets ORDER BY description + SortDescription getSortDescription(const ASTSelectQuery & query); + const SelectQueryOptions options; ASTPtr query_ptr; Context context; diff --git a/dbms/src/Parsers/ASTOrderByElement.cpp b/dbms/src/Parsers/ASTOrderByElement.cpp index 3ec5674ab27..ac57cfa437b 100644 --- a/dbms/src/Parsers/ASTOrderByElement.cpp +++ b/dbms/src/Parsers/ASTOrderByElement.cpp @@ -25,6 +25,26 @@ void ASTOrderByElement::formatImpl(const FormatSettings & settings, FormatState settings.ostr << (settings.hilite ? hilite_keyword : "") << " COLLATE " << (settings.hilite ? hilite_none : ""); collation->formatImpl(settings, state, frame); } + + if (with_fill) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " WITH FILL " << (settings.hilite ? hilite_none : ""); + if (fill_from) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " FROM " << (settings.hilite ? hilite_none : ""); + fill_from->formatImpl(settings, state, frame); + } + if (fill_to) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " TO " << (settings.hilite ? hilite_none : ""); + fill_to->formatImpl(settings, state, frame); + } + if (fill_step) + { + settings.ostr << (settings.hilite ? hilite_keyword : "") << " STEP " << (settings.hilite ? hilite_none : ""); + fill_step->formatImpl(settings, state, frame); + } + } } } diff --git a/dbms/src/Parsers/ASTOrderByElement.h b/dbms/src/Parsers/ASTOrderByElement.h index 729915400ce..982dd527abe 100644 --- a/dbms/src/Parsers/ASTOrderByElement.h +++ b/dbms/src/Parsers/ASTOrderByElement.h @@ -15,15 +15,25 @@ public: int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. bool nulls_direction_was_explicitly_specified; + bool with_fill; + ASTPtr fill_from; + ASTPtr fill_to; + ASTPtr fill_step; + /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ ASTPtr collation; ASTOrderByElement( - const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_, ASTPtr & collation_) + const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_, + ASTPtr & collation_, const bool with_fill_, ASTPtr & fill_from_, ASTPtr & fill_to_, ASTPtr & fill_step_) : direction(direction_) , nulls_direction(nulls_direction_) , nulls_direction_was_explicitly_specified(nulls_direction_was_explicitly_specified_) , collation(collation_) + , with_fill(with_fill_) + , fill_from(fill_from_) + , fill_to(fill_to_) + , fill_step(fill_step_) { } diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 2741aa0d491..7c3a3a05ee1 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -1283,7 +1283,12 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ParserKeyword first("FIRST"); ParserKeyword last("LAST"); ParserKeyword collate("COLLATE"); + ParserKeyword with_fill("WITH FILL"); + ParserKeyword from("FROM"); + ParserKeyword to("TO"); + ParserKeyword step("STEP"); ParserStringLiteral collate_locale_parser; + ParserExpressionWithOptionalAlias exp_parser(false); ASTPtr expr_elem; if (!elem_p.parse(pos, expr_elem, expected)) @@ -1318,7 +1323,27 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect return false; } - node = std::make_shared(direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node); + /// WITH FILL [FROM x] [TO y] [STEP z] + bool with_fill_val = false; + ASTPtr fill_from; + ASTPtr fill_to; + ASTPtr fill_step; + if (with_fill.ignore(pos)) + { + with_fill_val = true; + if (from.ignore(pos) && !exp_parser.parse(pos, fill_from, expected)) + return false; + + if (to.ignore(pos) && !exp_parser.parse(pos, fill_to, expected)) + return false; + + if (step.ignore(pos) && !exp_parser.parse(pos, fill_step, expected)) + return false; + } + + node = std::make_shared( + direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node, + with_fill, fill_from, fill_to, fill_step); node->children.push_back(expr_elem); if (locale_node) node->children.push_back(locale_node); diff --git a/dbms/src/Parsers/ExpressionElementParsers.h b/dbms/src/Parsers/ExpressionElementParsers.h index e3dc5ae44d0..41f69d91947 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.h +++ b/dbms/src/Parsers/ExpressionElementParsers.h @@ -273,6 +273,7 @@ protected: /** Element of ORDER BY expression - same as expression element, but in addition, ASC[ENDING] | DESC[ENDING] could be specified * and optionally, NULLS LAST|FIRST * and optionally, COLLATE 'locale'. + * and optionally, WITH FILL [FROM x] [TO y] [STEP z] */ class ParserOrderByElement : public IParserBase { From e94575e1da25cf598acb2c141e41b947ffaca596 Mon Sep 17 00:00:00 2001 From: dmitriiut Date: Sun, 21 Apr 2019 19:16:25 +0300 Subject: [PATCH 07/75] fix progress --- dbms/src/Common/ErrorCodes.cpp | 1 + dbms/src/Common/SharedBlockRowRef.h | 27 ++++++++++ dbms/src/Core/SortDescription.h | 4 +- .../DataStreams/FillingBlockInputStream.cpp | 18 +++++++ .../src/DataStreams/FillingBlockInputStream.h | 29 ++++++++++ .../src/DataStreams/LimitBlockInputStream.cpp | 43 ++------------- .../Interpreters/InterpreterSelectQuery.cpp | 54 ++++++++++++++++++- .../src/Interpreters/InterpreterSelectQuery.h | 1 + 8 files changed, 136 insertions(+), 41 deletions(-) create mode 100644 dbms/src/DataStreams/FillingBlockInputStream.cpp create mode 100644 dbms/src/DataStreams/FillingBlockInputStream.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 2468df697bb..ea21db02f6f 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -428,6 +428,7 @@ namespace ErrorCodes extern const int BAD_TTL_FILE = 451; extern const int WITH_TIES_WITHOUT_ORDER_BY = 452; extern const int INVALID_WITH_FILL_EXPRESSION = 453; + extern const int FILL_STEP_ZERO_VALUE = 454; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/SharedBlockRowRef.h b/dbms/src/Common/SharedBlockRowRef.h index c73bf5e6400..4fb24c49439 100644 --- a/dbms/src/Common/SharedBlockRowRef.h +++ b/dbms/src/Common/SharedBlockRowRef.h @@ -77,6 +77,33 @@ struct SharedBlockRowRef bool empty() const { return columns == nullptr; } size_t size() const { return empty() ? 0 : columns->size(); } + + /// gets pointers to all columns of block, which were used for ORDER BY + static ColumnRawPtrs getBlockColumns(const Block & block, const SortDescription description) + { + size_t size = description.size(); + ColumnRawPtrs res; + res.reserve(size); + + for (size_t i = 0; i < size; ++i) + { + const IColumn * column = !description[i].column_name.empty() + ? block.getByName(description[i].column_name).column.get() + : block.safeGetByPosition(description[i].column_number).column.get(); + res.emplace_back(column); + } + + return res; + } + + + static void setSharedBlockRowRef(SharedBlockRowRef &row_ref, SharedBlockPtr shared_block, ColumnRawPtrs *columns, + size_t row_num) + { + row_ref.row_num = row_num; + row_ref.columns = columns; + row_ref.shared_block = shared_block; + } }; diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index 1cccbc2d766..e0c658e61a4 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -22,8 +22,8 @@ struct SortColumnDescription std::shared_ptr collator; /// Collator for locale-specific comparison of strings bool with_fill; /// If true, all missed values in range [FROM, TO] will be filled /// Range [FROM, TO] respects sorting direction - double fill_from; /// Value >= FROM - double fill_to; /// Value + STEP <= TO + double fill_from; /// Fill value >= FROM + double fill_to; /// Fill value + STEP <= TO double fill_step; /// Default = 1 SortColumnDescription( diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp new file mode 100644 index 00000000000..e8355204dff --- /dev/null +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -0,0 +1,18 @@ +#include "FillingBlockInputStream.h" + +FillingBlockInputStream::FillingBlockInputStream( + const BlockInputStreamPtr & input, const SortDescription & description_) + : description(description_) +{ + children.push_back(input); +} + + +Block FillingBlockInputStream::readImpl() +{ + Block res; + UInt64 rows = 0; + + + +} \ No newline at end of file diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h new file mode 100644 index 00000000000..6afbeb1eafb --- /dev/null +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -0,0 +1,29 @@ +#pragma once + +#include +#include + +namespace DB +{ + +/** Implements the WITH FILL part of ORDER BY operation. +*/ +class FillingBlockInputStream : public IBlockInputStream +{ +public: + FillingBlockInputStream(const BlockInputStreamPtr & input, const SortDescription & description_); + + String getName() const override { return "With fill"; } + + Block getHeader() const override { return children.at(0)->getHeader(); } + +protected: + Block readImpl() override; + +private: + const SortDescription description; + SharedBlockRowRef last_row_ref; +}; + + +} \ No newline at end of file diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index f3fb45f27cd..16c5c94f16c 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -6,39 +6,6 @@ namespace DB { -namespace detail -{ - -/// gets pointers to all columns of block, which were used for ORDER BY -ColumnRawPtrs getBlockColumns(const Block & block, const SortDescription description) -{ - size_t size = description.size(); - ColumnRawPtrs res; - res.reserve(size); - - for (size_t i = 0; i < size; ++i) - { - const IColumn * column = !description[i].column_name.empty() - ? block.getByName(description[i].column_name).column.get() - : block.safeGetByPosition(description[i].column_number).column.get(); - res.emplace_back(column); - } - - return res; -} - - -void setSharedBlockRowRef(SharedBlockRowRef &row_ref, SharedBlockPtr shared_block, ColumnRawPtrs *columns, - size_t row_num) -{ - row_ref.row_num = row_num; - row_ref.columns = columns; - row_ref.shared_block = shared_block; -} - -} - - LimitBlockInputStream::LimitBlockInputStream( const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_, @@ -69,13 +36,13 @@ Block LimitBlockInputStream::readImpl() pos += rows; SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); - ColumnRawPtrs columns = getBlockColumns(*ptr, description); + ColumnRawPtrs columns = SharedBlockRowRef::getBlockColumns(*ptr, description); UInt64 len; for (len = 0; len < rows; ++len) { SharedBlockRowRef currentRow; - setSharedBlockRowRef(currentRow, ptr, &columns, len); + SharedBlockRowRef::setSharedBlockRowRef(currentRow, ptr, &columns, len); if (currentRow != ties_row_ref) { ties_row_ref.reset(); @@ -135,13 +102,13 @@ Block LimitBlockInputStream::readImpl() /// check if other rows in current block equals to last one in limit if (with_ties) { - ColumnRawPtrs columns = getBlockColumns(*ptr, description); - setSharedBlockRowRef(ties_row_ref, ptr, &columns, start + length - 1); + ColumnRawPtrs columns = SharedBlockRowRef::getBlockColumns(*ptr, description); + SharedBlockRowRef::setSharedBlockRowRef(ties_row_ref, ptr, &columns, start + length - 1); for (size_t i = ties_row_ref.row_num + 1; i < rows; ++i) { SharedBlockRowRef current_row; - setSharedBlockRowRef(current_row, ptr, &columns, i); + SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &columns, i); if (current_row == ties_row_ref) ++length; else diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 1b7cb4777e1..7e523015ccc 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -22,6 +22,7 @@ #include #include #include +#include #include #include @@ -75,6 +76,7 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; + extern const int FILL_STEP_ZERO_VALUE; } namespace @@ -715,6 +717,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt executeLimitBy(pipeline); } + executeWithFill(pipeline); + if (query.limitLength()) executePreLimit(pipeline); } @@ -840,6 +844,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt */ executeExtremes(pipeline); + executeWithFill(pipeline); + executeLimit(pipeline); } } @@ -915,6 +921,10 @@ static std::tuple getWithFillParameters(const ASTOrde fill_to = getWithFillFloatValue(node.fill_to, context); if (node.fill_step) fill_step = getWithFillFloatValue(node.fill_step, context); + + if (!fill_step) + throw Exception("STEP value can not be zero", ErrorCodes::FILL_STEP_ZERO_VALUE); + return {fill_from, fill_to, fill_step}; } @@ -1457,6 +1467,24 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery if (order_by_elem.with_fill) { auto[fill_from, fill_to, fill_step] = getWithFillParameters(order_by_elem, context); + + if (order_by_elem.direction == -1) + { + /// if DESC, then STEP < 0, FROM > TO + fill_step = std::min(fill_step, fill_step * -1); + auto from = fill_from; + fill_from = std::max(fill_from, fill_to); + fill_to = std::min(from, fill_to); + } + else + { + /// if ASC, then STEP > 0, FROM < TO + fill_step = std::max(fill_step, fill_step * -1); + auto from = fill_from; + fill_from = std::min(fill_from, fill_to); + fill_to = std::max(from, fill_to); + } + order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, true, fill_from, fill_to, fill_step); } @@ -1589,7 +1617,7 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) { - stream = std::make_shared(stream, limit, 0, false); + stream = std::make_shared(stream, limit, 0, false, query.limit_with_ties, getSortDescription(query)); }); } } @@ -1675,6 +1703,30 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) } +void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) +{ + auto & query = getSelectQuery(); + if (query.orderBy()) + { + SortDescription order_descr = getSortDescription(query); + SortDescription fill_descr; + for (auto & desc : order_descr) + { + if (desc.with_fill) + fill_descr.push_back(desc); + } + + if (!fill_descr.size()) + return; + + pipeline.transform([&](auto & stream) + { + stream = std::make_shared(stream, fill_descr); + }); + } +} + + void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline) { if (!context.getSettingsRef().extremes) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index fa53a1f5d21..2e838dbfec5 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -179,6 +179,7 @@ private: void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); void executeOrder(Pipeline & pipeline); + void executeWithFill(Pipeline & pipeline); void executeMergeSorted(Pipeline & pipeline); void executePreLimit(Pipeline & pipeline); void executeUnion(Pipeline & pipeline); From 50c322bc6dfe52fbf789a6d939353b9aa747d23f Mon Sep 17 00:00:00 2001 From: dmitriiut Date: Mon, 22 Apr 2019 02:04:23 +0300 Subject: [PATCH 08/75] WITH FILL version --- dbms/CMakeLists.txt | 2 +- dbms/src/Common/SharedBlockRowRef.h | 11 +- dbms/src/Core/SortDescription.h | 27 ++- .../DataStreams/FillingBlockInputStream.cpp | 157 +++++++++++++++++- .../src/DataStreams/FillingBlockInputStream.h | 9 +- .../src/DataStreams/LimitBlockInputStream.cpp | 9 +- .../Interpreters/InterpreterSelectQuery.cpp | 51 +++--- 7 files changed, 213 insertions(+), 53 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index a11e68ae0e5..2c9bfa48605 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -134,7 +134,7 @@ list (APPEND dbms_headers src/TableFunctions/ITableFunction.h src/TableFunctio list (APPEND dbms_sources src/Dictionaries/DictionaryFactory.cpp src/Dictionaries/DictionarySourceFactory.cpp src/Dictionaries/DictionaryStructure.cpp) list (APPEND dbms_headers src/Dictionaries/DictionaryFactory.h src/Dictionaries/DictionarySourceFactory.h src/Dictionaries/DictionaryStructure.h) -add_library(clickhouse_common_io ${LINK_MODE} ${clickhouse_common_io_headers} ${clickhouse_common_io_sources} src/Common/SharedBlockRowRef.h) +add_library(clickhouse_common_io ${LINK_MODE} ${clickhouse_common_io_headers} ${clickhouse_common_io_sources}) if (OS_FREEBSD) target_compile_definitions (clickhouse_common_io PUBLIC CLOCK_MONOTONIC_COARSE=CLOCK_MONOTONIC_FAST) diff --git a/dbms/src/Common/SharedBlockRowRef.h b/dbms/src/Common/SharedBlockRowRef.h index 4fb24c49439..8977fedc50e 100644 --- a/dbms/src/Common/SharedBlockRowRef.h +++ b/dbms/src/Common/SharedBlockRowRef.h @@ -1,9 +1,10 @@ #pragma once #include -#include "../Core/Block.h" -#include "../../../contrib/boost/boost/smart_ptr/intrusive_ptr.hpp" -#include "../Columns/IColumn.h" +#include +#include +#include +#include namespace DB @@ -97,7 +98,7 @@ struct SharedBlockRowRef } - static void setSharedBlockRowRef(SharedBlockRowRef &row_ref, SharedBlockPtr shared_block, ColumnRawPtrs *columns, + static void setSharedBlockRowRef(SharedBlockRowRef & row_ref, SharedBlockPtr & shared_block, ColumnRawPtrs * columns, size_t row_num) { row_ref.row_num = row_num; @@ -108,4 +109,4 @@ struct SharedBlockRowRef -} \ No newline at end of file +} diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index e0c658e61a4..bbf1f5ec859 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -11,6 +11,17 @@ class Collator; namespace DB { +struct FillColumnDescription +{ + /// All missed values in range [FROM, TO] will be filled + /// Range [FROM, TO] respects sorting direction + bool has_from = false; + bool has_to = false; + Field fill_from; /// Fill value >= FILL_FROM + Field fill_to; /// Fill value + STEP <= FILL_TO + Field fill_step; /// Default = 1 +}; + /// Description of the sorting rule by one column. struct SortColumnDescription { @@ -20,25 +31,23 @@ struct SortColumnDescription int nulls_direction; /// 1 - NULLs and NaNs are greater, -1 - less. /// To achieve NULLS LAST, set it equal to direction, to achieve NULLS FIRST, set it opposite. std::shared_ptr collator; /// Collator for locale-specific comparison of strings - bool with_fill; /// If true, all missed values in range [FROM, TO] will be filled - /// Range [FROM, TO] respects sorting direction - double fill_from; /// Fill value >= FROM - double fill_to; /// Fill value + STEP <= TO - double fill_step; /// Default = 1 + bool with_fill; + FillColumnDescription fill_description; + SortColumnDescription( size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, - double fill_from_ = 0, double fill_to_ = 0, double fill_step_ = 0) + FillColumnDescription fill_description_ = {}) : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) - , with_fill(with_fill_), fill_from(fill_from_), fill_to(fill_to_), fill_step(fill_step_) {} + , with_fill(with_fill_), fill_description(fill_description_) {} SortColumnDescription( const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, - double fill_from_ = 0, double fill_to_ = 0, double fill_step_ = 0) + FillColumnDescription fill_description_ = {}) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_) - , collator(collator_), with_fill(with_fill_), fill_from(fill_from_), fill_to(fill_to_), fill_step(fill_step_) {} + , collator(collator_), with_fill(with_fill_), fill_description(fill_description_) {} bool operator == (const SortColumnDescription & other) const { diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index e8355204dff..4ed2f921bbb 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -1,8 +1,76 @@ #include "FillingBlockInputStream.h" +namespace DB +{ + +namespace ErrorCodes +{ +extern const int INVALID_WITH_FILL_EXPRESSION; +} + +namespace detail +{ + +ColumnRawPtrs getColumnsExcept(SharedBlockPtr & block_ptr, ColumnRawPtrs & except_columns) +{ + ColumnRawPtrs res; + res.reserve(block_ptr->columns() - except_columns.size()); + + for (size_t i = 0; i < block_ptr->columns(); ++i) + { + const IColumn * raw_col = block_ptr->safeGetByPosition(i).column.get(); + if (std::find(except_columns.begin(), except_columns.end(), raw_col) != except_columns.end()) + res.emplace_back(raw_col); + } + + return res; +} + +void copyRowFromColumns(ColumnRawPtrs & from, ColumnRawPtrs & to, size_t row_num) +{ + for (size_t i = 0; i < from.size(); ++i) + const_cast(to[i])->insertFrom(*from[i], row_num); +} + +void fillRestOfRow( + size_t cols_copied, ColumnRawPtrs & res_fill_columns, ColumnRawPtrs & res_rest_columns, + ColumnRawPtrs & old_rest_columns, UInt64 & next_row_num) +{ + /// step_val was inserted, fill all other columns with default values + if (cols_copied < res_fill_columns.size()) + { + for (; cols_copied < res_fill_columns.size(); ++cols_copied) + const_cast(res_fill_columns[cols_copied])->insertDefault(); + for (size_t it = 0; it < res_rest_columns.size(); ++it) + const_cast(res_rest_columns[it])->insertDefault(); + + return; + } + + /// fill row wasn't created, copy rest values from row + detail::copyRowFromColumns(old_rest_columns, res_rest_columns, next_row_num); + ++next_row_num; +} + +Field sumTwoFields(Field & a, Field & b) +{ + switch (a.getType()) + { + case Field::Types::Null: return a; + case Field::Types::UInt64: return a.get() + b.get(); + case Field::Types::Int64: return a.get() + b.get(); + case Field::Types::Int128: return a.get() + b.get(); + case Field::Types::Float64: return a.get() + b.get(); + default: + throw Exception("WITH FILL can be used only with numeric types", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } +} + +} + FillingBlockInputStream::FillingBlockInputStream( - const BlockInputStreamPtr & input, const SortDescription & description_) - : description(description_) + const BlockInputStreamPtr & input, const SortDescription & fill_description_) + : fill_description(fill_description_) { children.push_back(input); } @@ -10,9 +78,92 @@ FillingBlockInputStream::FillingBlockInputStream( Block FillingBlockInputStream::readImpl() { - Block res; + Block cur_block; UInt64 rows = 0; + cur_block = children.back()->read(); + if (!cur_block) + return cur_block; + + Block res_block = cur_block.cloneEmpty(); + + rows = cur_block.rows(); + + SharedBlockPtr old_block_ptr = new detail::SharedBlock(std::move(cur_block)); + ColumnRawPtrs old_fill_columns = SharedBlockRowRef::getBlockColumns(*old_block_ptr, fill_description); + ColumnRawPtrs old_rest_columns = detail::getColumnsExcept(old_block_ptr, old_fill_columns); + + SharedBlockPtr res_block_ptr = new detail::SharedBlock(std::move(res_block)); + ColumnRawPtrs res_fill_columns = SharedBlockRowRef::getBlockColumns(*res_block_ptr, fill_description); + ColumnRawPtrs res_rest_columns = detail::getColumnsExcept(res_block_ptr, res_fill_columns); + + /// number of next row in current block + UInt64 next_row_num = 0; + + /// read first block + if (!pos) + { + ++next_row_num; + /// create row number 0 in result block here + detail::copyRowFromColumns(old_fill_columns, res_fill_columns, 0); + detail::copyRowFromColumns(old_rest_columns, res_rest_columns, 0); + } + pos += rows; + + /// current block is not first, need to compare with row in other block + if (!next_row_num) + { + size_t cnt_cols = 0; + size_t fill_columns_size = old_fill_columns.size(); + for (; cnt_cols < fill_columns_size; ++cnt_cols) + { + Field step = fill_description[cnt_cols].fill_description.fill_step; + Field prev_val = (*last_row_ref.columns)[cnt_cols][last_row_ref.row_num]; + Field step_val = detail::sumTwoFields(prev_val, step); + Field next_val = old_fill_columns[cnt_cols][next_row_num]; + if (step_val >= next_val) + const_cast(res_fill_columns[cnt_cols])->insertFrom(*old_fill_columns[cnt_cols], next_row_num); + else + { + const_cast(res_fill_columns[cnt_cols])->insert(step_val); + break; + } + } + /// create row number 0 in result block here + detail::fillRestOfRow(cnt_cols, res_fill_columns, res_rest_columns, old_rest_columns, next_row_num); + } + + /// number of last added row in result block + UInt64 last_row_num = 0; + + while (next_row_num < rows) + { + size_t cnt_cols = 0; + size_t fill_columns_size = old_fill_columns.size(); + for (; cnt_cols < fill_columns_size; ++cnt_cols) + { + Field step = fill_description[cnt_cols].fill_description.fill_step; + Field prev_val = res_fill_columns[cnt_cols][last_row_num]; + Field step_val = detail::sumTwoFields(prev_val, step); + Field next_val = old_fill_columns[cnt_cols][next_row_num]; + if (step_val >= next_val) + const_cast(res_fill_columns[cnt_cols])->insertFrom(*old_fill_columns[cnt_cols], next_row_num); + else + { + const_cast(res_fill_columns[cnt_cols])->insert(step_val); + break; + } + } + + /// create new row in result block, increment last_row_num + detail::fillRestOfRow(cnt_cols, res_fill_columns, res_rest_columns, old_rest_columns, next_row_num); + ++last_row_num; + } + + /// finished current block, need to remember last row + SharedBlockRowRef::setSharedBlockRowRef(last_row_ref, res_block_ptr, & res_fill_columns, last_row_num); + return *res_block_ptr; +} } \ No newline at end of file diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index 6afbeb1eafb..1e4cd5bdb6a 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -11,9 +11,9 @@ namespace DB class FillingBlockInputStream : public IBlockInputStream { public: - FillingBlockInputStream(const BlockInputStreamPtr & input, const SortDescription & description_); + FillingBlockInputStream(const BlockInputStreamPtr & input, const SortDescription & fill_description_); - String getName() const override { return "With fill"; } + String getName() const override { return "WithFill"; } Block getHeader() const override { return children.at(0)->getHeader(); } @@ -21,8 +21,9 @@ protected: Block readImpl() override; private: - const SortDescription description; - SharedBlockRowRef last_row_ref; + UInt64 pos = 0; /// total number of read rows + const SortDescription fill_description; /// contains only rows with WITH_FILL + SharedBlockRowRef last_row_ref; /// ref to last written row }; diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 16c5c94f16c..4dbecc4dcc7 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -31,9 +31,10 @@ Block LimitBlockInputStream::readImpl() /// so we check current block if (with_ties && ties_row_ref.shared_block) { - res = children.back()->read(); rows = res.rows(); pos += rows; + res = children.back()->read(); + SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); ColumnRawPtrs columns = SharedBlockRowRef::getBlockColumns(*ptr, description); @@ -41,9 +42,9 @@ Block LimitBlockInputStream::readImpl() for (len = 0; len < rows; ++len) { - SharedBlockRowRef currentRow; - SharedBlockRowRef::setSharedBlockRowRef(currentRow, ptr, &columns, len); - if (currentRow != ties_row_ref) + SharedBlockRowRef current_row; + SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &columns, len); + if (current_row != ties_row_ref) { ties_row_ref.reset(); break; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 575f0f6f690..cda5de3a471 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -895,37 +895,38 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c return 0; } -static Float64 getWithFillFloatValue(const ASTPtr & node, const Context & context) +static Field getWithFillFieldValue(const ASTPtr & node, const Context & context) { const auto & [field, type] = evaluateConstantExpression(node, context); if (!isNumber(type)) throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - Field converted = convertFieldToType(field, DataTypeFloat64()); - if (converted.isNull()) - throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of WITH FILL expression is not representable as Float64", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - - return converted.safeGet(); + return field; } -static std::tuple getWithFillParameters(const ASTOrderByElement & node, const Context & context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement &node, const Context &context) { - Float64 fill_from = 0; - Float64 fill_to = 0; - Float64 fill_step = 1; - + FillColumnDescription descr; if (node.fill_from) - fill_from = getWithFillFloatValue(node.fill_from, context); + { + descr.has_from = true; + descr.fill_from = getWithFillFieldValue(node.fill_from, context); + } if (node.fill_to) - fill_to = getWithFillFloatValue(node.fill_to, context); + { + descr.has_to = true; + descr.fill_to = getWithFillFieldValue(node.fill_to, context); + } if (node.fill_step) - fill_step = getWithFillFloatValue(node.fill_step, context); + descr.fill_step = getWithFillFieldValue(node.fill_step, context); + else + descr.fill_step = 1; - if (!fill_step) + if (descr.fill_step == 0) throw Exception("STEP value can not be zero", ErrorCodes::FILL_STEP_ZERO_VALUE); - return {fill_from, fill_to, fill_step}; + return descr; } void InterpreterSelectQuery::executeFetchColumns( @@ -1466,27 +1467,23 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery if (order_by_elem.with_fill) { - auto[fill_from, fill_to, fill_step] = getWithFillParameters(order_by_elem, context); + FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); if (order_by_elem.direction == -1) { /// if DESC, then STEP < 0, FROM > TO - fill_step = std::min(fill_step, fill_step * -1); - auto from = fill_from; - fill_from = std::max(fill_from, fill_to); - fill_to = std::min(from, fill_to); + if (fill_desc.has_from && fill_desc.has_to && fill_desc.fill_from < fill_desc.fill_to) + std::swap(fill_desc.fill_from, fill_desc.fill_to); } else { /// if ASC, then STEP > 0, FROM < TO - fill_step = std::max(fill_step, fill_step * -1); - auto from = fill_from; - fill_from = std::min(fill_from, fill_to); - fill_to = std::max(from, fill_to); + if (fill_desc.has_from && fill_desc.has_to && fill_desc.fill_from > fill_desc.fill_to) + std::swap(fill_desc.fill_from, fill_desc.fill_to); } order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, - true, fill_from, fill_to, fill_step); + true, fill_desc); } else order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); @@ -1717,7 +1714,7 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) fill_descr.push_back(desc); } - if (!fill_descr.size()) + if (fill_descr.empty()) return; pipeline.transform([&](auto & stream) From a2c3fd2057483396a936ceb0db8082e92ced0a01 Mon Sep 17 00:00:00 2001 From: dmitriiut Date: Tue, 23 Apr 2019 04:48:51 +0300 Subject: [PATCH 09/75] bug fixes --- dbms/src/Core/SortDescription.h | 8 ++-- .../DataStreams/FillingBlockInputStream.cpp | 37 +++++++++++-------- .../Interpreters/InterpreterSelectQuery.cpp | 13 ++----- dbms/src/Parsers/ASTOrderByElement.h | 6 +-- dbms/src/Parsers/ExpressionElementParsers.cpp | 2 +- 5 files changed, 33 insertions(+), 33 deletions(-) diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index bbf1f5ec859..8a30e98203c 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -4,7 +4,7 @@ #include #include #include - +#include class Collator; @@ -15,8 +15,6 @@ struct FillColumnDescription { /// All missed values in range [FROM, TO] will be filled /// Range [FROM, TO] respects sorting direction - bool has_from = false; - bool has_to = false; Field fill_from; /// Fill value >= FILL_FROM Field fill_to; /// Fill value + STEP <= FILL_TO Field fill_step; /// Default = 1 @@ -38,14 +36,14 @@ struct SortColumnDescription SortColumnDescription( size_t column_number_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, - FillColumnDescription fill_description_ = {}) + const FillColumnDescription & fill_description_ = {}) : column_number(column_number_), direction(direction_), nulls_direction(nulls_direction_), collator(collator_) , with_fill(with_fill_), fill_description(fill_description_) {} SortColumnDescription( const std::string & column_name_, int direction_, int nulls_direction_, const std::shared_ptr & collator_ = nullptr, bool with_fill_ = false, - FillColumnDescription fill_description_ = {}) + const FillColumnDescription & fill_description_ = {}) : column_name(column_name_), column_number(0), direction(direction_), nulls_direction(nulls_direction_) , collator(collator_), with_fill(with_fill_), fill_description(fill_description_) {} diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index 4ed2f921bbb..ffb7cffba0e 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -19,7 +19,7 @@ ColumnRawPtrs getColumnsExcept(SharedBlockPtr & block_ptr, ColumnRawPtrs & excep for (size_t i = 0; i < block_ptr->columns(); ++i) { const IColumn * raw_col = block_ptr->safeGetByPosition(i).column.get(); - if (std::find(except_columns.begin(), except_columns.end(), raw_col) != except_columns.end()) + if (std::find(except_columns.begin(), except_columns.end(), raw_col) == except_columns.end()) res.emplace_back(raw_col); } @@ -78,18 +78,18 @@ FillingBlockInputStream::FillingBlockInputStream( Block FillingBlockInputStream::readImpl() { - Block cur_block; + Block old_block; UInt64 rows = 0; - cur_block = children.back()->read(); - if (!cur_block) - return cur_block; + old_block = children.back()->read(); + if (!old_block) + return old_block; - Block res_block = cur_block.cloneEmpty(); + Block res_block = old_block.cloneEmpty(); - rows = cur_block.rows(); + rows = old_block.rows(); - SharedBlockPtr old_block_ptr = new detail::SharedBlock(std::move(cur_block)); + SharedBlockPtr old_block_ptr = new detail::SharedBlock(std::move(old_block)); ColumnRawPtrs old_fill_columns = SharedBlockRowRef::getBlockColumns(*old_block_ptr, fill_description); ColumnRawPtrs old_rest_columns = detail::getColumnsExcept(old_block_ptr, old_fill_columns); @@ -108,7 +108,6 @@ Block FillingBlockInputStream::readImpl() detail::copyRowFromColumns(old_fill_columns, res_fill_columns, 0); detail::copyRowFromColumns(old_rest_columns, res_rest_columns, 0); } - pos += rows; /// current block is not first, need to compare with row in other block if (!next_row_num) @@ -118,11 +117,14 @@ Block FillingBlockInputStream::readImpl() for (; cnt_cols < fill_columns_size; ++cnt_cols) { Field step = fill_description[cnt_cols].fill_description.fill_step; - Field prev_val = (*last_row_ref.columns)[cnt_cols][last_row_ref.row_num]; + Field next_val; + Field prev_val; + old_fill_columns[cnt_cols]->get(next_row_num, next_val); + (*last_row_ref.columns)[cnt_cols]->get(last_row_ref.row_num, prev_val); Field step_val = detail::sumTwoFields(prev_val, step); - Field next_val = old_fill_columns[cnt_cols][next_row_num]; if (step_val >= next_val) - const_cast(res_fill_columns[cnt_cols])->insertFrom(*old_fill_columns[cnt_cols], next_row_num); + const_cast(res_fill_columns[cnt_cols])->insertFrom( + *old_fill_columns[cnt_cols], next_row_num); else { const_cast(res_fill_columns[cnt_cols])->insert(step_val); @@ -131,6 +133,7 @@ Block FillingBlockInputStream::readImpl() } /// create row number 0 in result block here detail::fillRestOfRow(cnt_cols, res_fill_columns, res_rest_columns, old_rest_columns, next_row_num); + ++pos; } /// number of last added row in result block @@ -143,11 +146,14 @@ Block FillingBlockInputStream::readImpl() for (; cnt_cols < fill_columns_size; ++cnt_cols) { Field step = fill_description[cnt_cols].fill_description.fill_step; - Field prev_val = res_fill_columns[cnt_cols][last_row_num]; + Field prev_val; + res_fill_columns[cnt_cols]->get(last_row_num, prev_val); Field step_val = detail::sumTwoFields(prev_val, step); - Field next_val = old_fill_columns[cnt_cols][next_row_num]; + Field next_val; + old_fill_columns[cnt_cols]->get(next_row_num, next_val); if (step_val >= next_val) - const_cast(res_fill_columns[cnt_cols])->insertFrom(*old_fill_columns[cnt_cols], next_row_num); + const_cast(res_fill_columns[cnt_cols])->insertFrom( + *old_fill_columns[cnt_cols], next_row_num); else { const_cast(res_fill_columns[cnt_cols])->insert(step_val); @@ -158,6 +164,7 @@ Block FillingBlockInputStream::readImpl() /// create new row in result block, increment last_row_num detail::fillRestOfRow(cnt_cols, res_fill_columns, res_rest_columns, old_rest_columns, next_row_num); ++last_row_num; + ++pos; } /// finished current block, need to remember last row diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index cda5de3a471..077181171ea 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -909,15 +909,9 @@ static FillColumnDescription getWithFillDescription(const ASTOrderByElement &nod { FillColumnDescription descr; if (node.fill_from) - { - descr.has_from = true; descr.fill_from = getWithFillFieldValue(node.fill_from, context); - } if (node.fill_to) - { - descr.has_to = true; descr.fill_to = getWithFillFieldValue(node.fill_to, context); - } if (node.fill_step) descr.fill_step = getWithFillFieldValue(node.fill_step, context); else @@ -1472,13 +1466,13 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery if (order_by_elem.direction == -1) { /// if DESC, then STEP < 0, FROM > TO - if (fill_desc.has_from && fill_desc.has_to && fill_desc.fill_from < fill_desc.fill_to) + if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from < fill_desc.fill_to) std::swap(fill_desc.fill_from, fill_desc.fill_to); } else { /// if ASC, then STEP > 0, FROM < TO - if (fill_desc.has_from && fill_desc.has_to && fill_desc.fill_from > fill_desc.fill_to) + if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from > fill_desc.fill_to) std::swap(fill_desc.fill_from, fill_desc.fill_to); } @@ -1613,9 +1607,10 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) if (query.limitLength()) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); + SortDescription sort_descr = getSortDescription(query); pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) { - stream = std::make_shared(stream, limit, 0, false, query.limit_with_ties, getSortDescription(query)); + stream = std::make_shared(stream, limit, 0, false, false, query.limit_with_ties, sort_descr); }); } } diff --git a/dbms/src/Parsers/ASTOrderByElement.h b/dbms/src/Parsers/ASTOrderByElement.h index 982dd527abe..30da8172f52 100644 --- a/dbms/src/Parsers/ASTOrderByElement.h +++ b/dbms/src/Parsers/ASTOrderByElement.h @@ -15,14 +15,14 @@ public: int nulls_direction; /// Same as direction for NULLS LAST, opposite for NULLS FIRST. bool nulls_direction_was_explicitly_specified; + /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ + ASTPtr collation; + bool with_fill; ASTPtr fill_from; ASTPtr fill_to; ASTPtr fill_step; - /** Collation for locale-specific string comparison. If empty, then sorting done by bytes. */ - ASTPtr collation; - ASTOrderByElement( const int direction_, const int nulls_direction_, const bool nulls_direction_was_explicitly_specified_, ASTPtr & collation_, const bool with_fill_, ASTPtr & fill_from_, ASTPtr & fill_to_, ASTPtr & fill_step_) diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 7c3a3a05ee1..1fc70745b0b 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -1343,7 +1343,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect node = std::make_shared( direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node, - with_fill, fill_from, fill_to, fill_step); + with_fill_val, fill_from, fill_to, fill_step); node->children.push_back(expr_elem); if (locale_node) node->children.push_back(locale_node); From 64d65c01231597345576da25b184ebc3d1f20182 Mon Sep 17 00:00:00 2001 From: qianlixiang Date: Fri, 26 Jul 2019 18:48:42 +0800 Subject: [PATCH 10/75] Fixed mismatched header of prewhere with sample column --- .../Interpreters/InterpreterSelectQuery.cpp | 26 +++++++++++++++++++ .../00975_sample_prewhere.reference | 2 ++ .../0_stateless/00975_sample_prewhere.sql | 17 ++++++++++++ 3 files changed, 45 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00975_sample_prewhere.reference create mode 100644 dbms/tests/queries/0_stateless/00975_sample_prewhere.sql diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index d6e46fbedb3..a31554dd630 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -666,6 +666,21 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS pipeline.streams.back() = std::make_shared( pipeline.streams.back(), expressions.prewhere_info->prewhere_actions, expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column); + + // To remove additional columns in dry run + // For example, sample column which can be removed in this stage + if (expressions.prewhere_info->remove_columns_actions) + { + if constexpr (pipeline_with_processors) + { + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, expressions.prewhere_info->remove_columns_actions); + }); + } + else + pipeline.streams.back() = std::make_shared(pipeline.streams.back(), expressions.prewhere_info->remove_columns_actions); + } } } else @@ -1269,12 +1284,23 @@ void InterpreterSelectQuery::executeFetchColumns( streams = {std::make_shared(storage->getSampleBlockForColumns(required_columns))}; if (query_info.prewhere_info) + { streams.back() = std::make_shared( streams.back(), prewhere_info->prewhere_actions, prewhere_info->prewhere_column_name, prewhere_info->remove_prewhere_column); + // To remove additional columns + // In some cases, we did not read any marks so that the pipeline.streams is empty + // Thus, some columns in prewhere are not removed as expected + // This leads to mismatched header in distributed table + if (query_info.prewhere_info->remove_columns_actions) + { + streams.back() = std::make_shared(streams.back(), query_info.prewhere_info->remove_columns_actions); + } + } + } for (auto & stream : streams) diff --git a/dbms/tests/queries/0_stateless/00975_sample_prewhere.reference b/dbms/tests/queries/0_stateless/00975_sample_prewhere.reference new file mode 100644 index 00000000000..2559e5c49e7 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_sample_prewhere.reference @@ -0,0 +1,2 @@ +3 +6 diff --git a/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql b/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql new file mode 100644 index 00000000000..5d202a9d12c --- /dev/null +++ b/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql @@ -0,0 +1,17 @@ +drop table if exists test.sample_prewhere; +drop table if exists test.sample_prewhere_all; + +create table if not exists test.sample_prewhere (date Date, id Int32, time Int64) engine = MergeTree partition by date order by (id, time, intHash64(time)) sample by intHash64(time); +insert into test.sample_prewhere values ('2019-01-01', 2, 1564028096); +insert into test.sample_prewhere values ('2019-01-01', 1, 1564028096); +insert into test.sample_prewhere values ('2019-01-02', 3, 1564028096); + +create table if not exists test.sample_prewhere_all as test.sample_prewhere engine = Distributed('test_cluster_two_shards_localhost', 'test', 'sample_prewhere'); + +select id from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00' limit 0, 1; +select id from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) > '2019-07-20 00:00:00' limit 0, 1; +select id from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00'; +select count() from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) > '2019-07-20 00:00:00' limit 0, 1; + +drop table if exists test.sample_prewhere; +drop table if exists test.sample_prewhere_all; From 72b80fe1f7f084ace71266c50ee400c1ad765e27 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 14 Aug 2019 20:01:47 +0300 Subject: [PATCH 11/75] merging with master --- .../Interpreters/InterpreterSelectQuery.cpp | 173 +++++++----------- .../src/Interpreters/InterpreterSelectQuery.h | 8 +- dbms/src/Parsers/ParserSelectQuery.cpp | 4 - 3 files changed, 65 insertions(+), 120 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index d1cafd3674d..6cad9b682c6 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -604,8 +604,35 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage return res; } +static Field getWithFillFieldValue(const ASTPtr & node, const Context & context) +{ + const auto & [field, type] = evaluateConstantExpression(node, context); -static SortDescription getSortDescription(const ASTSelectQuery & query) + if (!isNumber(type)) + throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + return field; +} + +static FillColumnDescription getWithFillDescription(const ASTOrderByElement &node, const Context &context) +{ + FillColumnDescription descr; + if (node.fill_from) + descr.fill_from = getWithFillFieldValue(node.fill_from, context); + if (node.fill_to) + descr.fill_to = getWithFillFieldValue(node.fill_to, context); + if (node.fill_step) + descr.fill_step = getWithFillFieldValue(node.fill_step, context); + else + descr.fill_step = 1; + + if (descr.fill_step == 0) + throw Exception("STEP value can not be zero", ErrorCodes::FILL_STEP_ZERO_VALUE); + + return descr; +} + +static SortDescription getSortDescription(const ASTSelectQuery & query, const Context & context) { SortDescription order_descr; order_descr.reserve(query.orderBy()->children.size()); @@ -618,13 +645,33 @@ static SortDescription getSortDescription(const ASTSelectQuery & query) if (order_by_elem.collation) collator = std::make_shared(order_by_elem.collation->as().value.get()); - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); + if (order_by_elem.with_fill) + { + FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); + + if (order_by_elem.direction == -1) + { + /// if DESC, then STEP < 0, FROM > TO + if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from < fill_desc.fill_to) + std::swap(fill_desc.fill_from, fill_desc.fill_to); + } + else + { + /// if ASC, then STEP > 0, FROM < TO + if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from > fill_desc.fill_to) + std::swap(fill_desc.fill_from, fill_desc.fill_to); + } + + order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, + true, fill_desc); + } + else + order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); } return order_descr; } - static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context) { const auto & [field, type] = evaluateConstantExpression(node, context); @@ -673,7 +720,7 @@ static SortingInfoPtr optimizeReadInOrder(const MergeTreeData & merge_tree, cons if (!merge_tree.hasSortingKey()) return {}; - auto order_descr = getSortDescription(query); + auto order_descr = getSortDescription(query, context); SortDescription prefix_order_descr; int read_direction = order_descr.at(0).direction; @@ -1150,75 +1197,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeSubqueriesInSetsAndJoins(pipeline, expressions.subqueries_for_sets); } - -static UInt64 getLimitUIntValue(const ASTPtr & node, const Context & context) -{ - const auto & [field, type] = evaluateConstantExpression(node, context); - - if (!isNumber(type)) - throw Exception("Illegal type " + type->getName() + " of LIMIT expression, must be numeric type", ErrorCodes::INVALID_LIMIT_EXPRESSION); - - Field converted = convertFieldToType(field, DataTypeUInt64()); - if (converted.isNull()) - throw Exception("The value " + applyVisitor(FieldVisitorToString(), field) + " of LIMIT expression is not representable as UInt64", ErrorCodes::INVALID_LIMIT_EXPRESSION); - - return converted.safeGet(); -} - -static std::pair getLimitLengthAndOffset(const ASTSelectQuery & query, const Context & context) -{ - UInt64 length = 0; - UInt64 offset = 0; - - if (query.limitLength()) - { - length = getLimitUIntValue(query.limitLength(), context); - if (query.limitOffset()) - offset = getLimitUIntValue(query.limitOffset(), context); - } - - return {length, offset}; -} - -static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context) -{ - /// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY or WITH TIES. - if (!query.distinct && !query.limitBy() && !query.limit_with_ties) - { - auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - return limit_length + limit_offset; - } - return 0; -} - -static Field getWithFillFieldValue(const ASTPtr & node, const Context & context) -{ - const auto & [field, type] = evaluateConstantExpression(node, context); - - if (!isNumber(type)) - throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - - return field; -} - -static FillColumnDescription getWithFillDescription(const ASTOrderByElement &node, const Context &context) -{ - FillColumnDescription descr; - if (node.fill_from) - descr.fill_from = getWithFillFieldValue(node.fill_from, context); - if (node.fill_to) - descr.fill_to = getWithFillFieldValue(node.fill_to, context); - if (node.fill_step) - descr.fill_step = getWithFillFieldValue(node.fill_step, context); - else - descr.fill_step = 1; - - if (descr.fill_step == 0) - throw Exception("STEP value can not be zero", ErrorCodes::FILL_STEP_ZERO_VALUE); - - return descr; -} - template void InterpreterSelectQuery::executeFetchColumns( QueryProcessingStage::Enum processing_stage, TPipeline & pipeline, @@ -2001,14 +1979,10 @@ void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const E }); } -<<<<<<< HEAD void InterpreterSelectQuery::executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info) -======= -SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery & query) ->>>>>>> a2c3fd2057483396a936ceb0db8082e92ced0a01 { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query); + SortDescription order_descr = getSortDescription(query, context); const Settings & settings = context.getSettingsRef(); UInt64 limit = getLimitForSorting(query, context); @@ -2038,36 +2012,11 @@ SortDescription InterpreterSelectQuery::getSortDescription(const ASTSelectQuery stream = std::make_shared(stream); }); -<<<<<<< HEAD pipeline.firstStream() = std::make_shared( pipeline.streams, order_descr, settings.max_block_size, limit); pipeline.streams.resize(1); } -======= - if (order_by_elem.with_fill) - { - FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); - - if (order_by_elem.direction == -1) - { - /// if DESC, then STEP < 0, FROM > TO - if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from < fill_desc.fill_to) - std::swap(fill_desc.fill_from, fill_desc.fill_to); - } - else - { - /// if ASC, then STEP > 0, FROM < TO - if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from > fill_desc.fill_to) - std::swap(fill_desc.fill_from, fill_desc.fill_to); - } - - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, - true, fill_desc); - } - else - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); ->>>>>>> a2c3fd2057483396a936ceb0db8082e92ced0a01 } else { @@ -2100,7 +2049,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP /// TODO: Implement optimization using sorting_info auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query); + SortDescription order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); const Settings & settings = context.getSettingsRef(); @@ -2137,7 +2086,7 @@ void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline, SortingInfoP void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query); + SortDescription order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); const Settings & settings = context.getSettingsRef(); @@ -2164,7 +2113,7 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline) void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline) { auto & query = getSelectQuery(); - SortDescription order_descr = getSortDescription(query); + SortDescription order_descr = getSortDescription(query, context); UInt64 limit = getLimitForSorting(query, context); const Settings & settings = context.getSettingsRef(); @@ -2277,7 +2226,7 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) if (query.limitLength()) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - SortDescription sort_descr = getSortDescription(query); + SortDescription sort_descr = getSortDescription(query, context); pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) { stream = std::make_shared(stream, limit, 0, false, false, query.limit_with_ties, sort_descr); @@ -2392,7 +2341,7 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query)) always_read_till_end = true; - SortDescription order_descr = getSortDescription(query); + SortDescription order_descr = getSortDescription(query, context); UInt64 limit_length; UInt64 limit_offset; @@ -2411,7 +2360,7 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) auto & query = getSelectQuery(); if (query.orderBy()) { - SortDescription order_descr = getSortDescription(query); + SortDescription order_descr = getSortDescription(query, context); SortDescription fill_descr; for (auto & desc : order_descr) { @@ -2429,6 +2378,12 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) } } +void InterpreterSelectQuery::executeWithFill(QueryPipeline &) +{ + throw Exception("Unsupported WITH FILL with processors", ErrorCodes::NOT_IMPLEMENTED); +} + + void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline) { auto & query = getSelectQuery(); diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index 19c98677455..1262d79cada 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -195,12 +195,8 @@ private: void executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final); void executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression); -<<<<<<< HEAD void executeOrder(Pipeline & pipeline, SortingInfoPtr sorting_info); -======= - void executeOrder(Pipeline & pipeline); void executeWithFill(Pipeline & pipeline); ->>>>>>> a2c3fd2057483396a936ceb0db8082e92ced0a01 void executeMergeSorted(Pipeline & pipeline); void executePreLimit(Pipeline & pipeline); void executeUnion(Pipeline & pipeline); @@ -218,6 +214,7 @@ private: void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info); + void executeWithFill(QueryPipeline & pipeline); void executeMergeSorted(QueryPipeline & pipeline); void executePreLimit(QueryPipeline & pipeline); void executeLimitBy(QueryPipeline & pipeline); @@ -248,9 +245,6 @@ private: */ void initSettings(); - /// gets ORDER BY description - SortDescription getSortDescription(const ASTSelectQuery & query); - const SelectQueryOptions options; ASTPtr query_ptr; Context context; diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 2fd0a22fca0..e65ef9d3c92 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -186,11 +186,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) return false; } -<<<<<<< HEAD /// LIMIT length | LIMIT offset, length | LIMIT count BY expr-list | LIMIT offset, length BY expr-list -======= - /// LIMIT length [WITH TIES] | LIMIT offset, length [WITH TIES] | LIMIT count BY expr-list ->>>>>>> a2c3fd2057483396a936ceb0db8082e92ced0a01 if (s_limit.ignore(pos, expected)) { if (limit_length) From 00d98c51c37d1f42b91d2bacde657eb5cf7759c1 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 15 Aug 2019 14:09:41 +0300 Subject: [PATCH 12/75] fix limit without order --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 9b82984ab0f..fd1197fe347 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2342,7 +2342,13 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline) if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query)) always_read_till_end = true; - SortDescription order_descr = getSortDescription(query, context); + SortDescription order_descr; + if (query.limit_with_ties) + { + if (!query.orderBy()) + throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); + order_descr = getSortDescription(query, context); + } UInt64 limit_length; UInt64 limit_offset; From dbf1417bd6229ba8512ff17d67e3dd5e0efafe45 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Sun, 21 Jul 2019 16:15:04 +0300 Subject: [PATCH 13/75] Globs in storage file --- dbms/src/Common/parseGlobs.cpp | 74 +++++++++++++++++++++++++++++++ dbms/src/Common/parseGlobs.h | 9 ++++ dbms/src/Storages/StorageFile.cpp | 58 +++++++++++++++++++----- dbms/src/Storages/StorageFile.h | 3 ++ 4 files changed, 134 insertions(+), 10 deletions(-) create mode 100644 dbms/src/Common/parseGlobs.cpp create mode 100644 dbms/src/Common/parseGlobs.h diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp new file mode 100644 index 00000000000..216ce2a9cf6 --- /dev/null +++ b/dbms/src/Common/parseGlobs.cpp @@ -0,0 +1,74 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +std::string makeRegexpPatternFromGlobs(const std::string & initial_str) +{ + std::string first_prepare; + first_prepare.reserve(initial_str.size()); + for (const auto & letter : initial_str) + { + if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+')) + first_prepare.push_back('\\'); + first_prepare.push_back(letter); + } + re2::RE2 char_range(R"(({[^*?/\\]\.\.[^*?/\\]}))"); + re2::StringPiece input_for_range(first_prepare); + re2::StringPiece matched_range(first_prepare); + + std::string second_prepare; + second_prepare.reserve(first_prepare.size()); + size_t current_index = 0; + size_t pos; + while (RE2::FindAndConsume(&input_for_range, char_range, &matched_range)) + { + pos = matched_range.data() - first_prepare.data(); + second_prepare += first_prepare.substr(current_index, pos - current_index); + second_prepare.append({'[', matched_range.ToString()[1], '-', matched_range.ToString()[4], ']'}); + current_index = input_for_range.data() - first_prepare.data(); + } + second_prepare += first_prepare.substr(current_index); + std::cout << second_prepare < + +namespace DB +{ +/* Parse globs in string and make a regexp for it. + */ +std::string makeRegexpPatternFromGlobs(const std::string & ipath); +} diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 5162e667133..69bde384de0 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -18,12 +18,19 @@ #include #include +#include #include #include #include +#include +#include +#include + +namespace fs = boost::filesystem; + namespace DB { @@ -87,7 +94,24 @@ StorageFile::StorageFile( poco_path = Poco::Path(db_dir_path, poco_path); path = poco_path.absolute().toString(); - checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); + if ((path.find('*') != std::string::npos) || (path.find('?') != std::string::npos) || (path.find('{') != std::string::npos)) + { + path_with_globs = true; + std::string path_pattern = makeRegexpPatternFromGlobs(path); + re2::RE2 matcher(path_pattern); + fs::path cur_dir(db_dir_path); + fs::directory_iterator end; + for (fs::directory_iterator it(cur_dir); it != end; ++it) + { + const fs::path file = (*it); + if (re2::RE2::FullMatch(file.string(), matcher)) + { + matched_paths.push_back(file.string()); + checkCreationIsAllowed(context_global, db_dir_path, matched_paths.back(), table_fd); + } + } + } else + checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); is_db_table = false; } else /// Is DB's file @@ -117,7 +141,7 @@ StorageFile::StorageFile( class StorageFileBlockInputStream : public IBlockInputStream { public: - StorageFileBlockInputStream(StorageFile & storage_, const Context & context, UInt64 max_block_size) + StorageFileBlockInputStream(StorageFile & storage_, const Context & context, UInt64 max_block_size, size_t num_of_path) : storage(storage_) { if (storage.use_table_fd) @@ -143,8 +167,8 @@ public: else { shared_lock = std::shared_lock(storage.rwlock); - - read_buf = std::make_unique(storage.path); + std::string path_to_read = storage.path_with_globs ? storage.matched_paths[num_of_path] : storage.path; + read_buf = std::make_unique(path_to_read); } reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size); @@ -191,12 +215,26 @@ BlockInputStreams StorageFile::read( size_t max_block_size, unsigned /*num_streams*/) { - BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size); - const ColumnsDescription & columns_ = getColumns(); - auto column_defaults = columns_.getDefaults(); - if (column_defaults.empty()) - return {block_input}; - return {std::make_shared(block_input, column_defaults, context)}; + const ColumnsDescription & columns = getColumns(); + auto column_defaults = columns.getDefaults(); + if (!path_with_globs) + { + BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size, 0); + if (column_defaults.empty()) + return {block_input}; + return {std::make_shared(block_input, column_defaults, context)}; + } + BlockInputStreams blocks_input; + blocks_input.reserve(matched_paths.size()); + for (size_t i = 0; i < matched_paths.size(); ++i) + { + BlockInputStreamPtr cur_block = std::make_shared(*this, context, max_block_size, i); + if (column_defaults.empty()) + blocks_input.push_back(cur_block); + else + blocks_input.push_back(std::make_shared(cur_block, column_defaults, context)); + } + return blocks_input; } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index cc5878520ce..177f13d3d27 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -71,6 +71,9 @@ private: std::string path; int table_fd = -1; + bool path_with_globs = false; + std::vector matched_paths; + bool is_db_table = true; /// Table is stored in real database, not user's file bool use_table_fd = false; /// Use table_fd insted of path std::atomic table_fd_was_used{false}; /// To detect repeating reads from stdin From 6055c61e7fb82db5b97dc0f282a417604b8bb651 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Wed, 24 Jul 2019 12:51:02 +0300 Subject: [PATCH 14/75] wip hdfs --- dbms/CMakeLists.txt | 4 +- dbms/src/IO/HDFSCommon.h | 20 ++++++++++ dbms/src/Storages/StorageHDFS.cpp | 65 +++++++++++++++++++++++++++---- 3 files changed, 80 insertions(+), 9 deletions(-) diff --git a/dbms/CMakeLists.txt b/dbms/CMakeLists.txt index b589c398238..4a0652e86f2 100644 --- a/dbms/CMakeLists.txt +++ b/dbms/CMakeLists.txt @@ -390,8 +390,8 @@ if (USE_PROTOBUF) endif () if (USE_HDFS) - target_link_libraries (clickhouse_common_io PRIVATE ${HDFS3_LIBRARY}) - target_include_directories (clickhouse_common_io SYSTEM BEFORE PRIVATE ${HDFS3_INCLUDE_DIR}) + target_link_libraries (clickhouse_common_io PUBLIC ${HDFS3_LIBRARY}) + target_include_directories (clickhouse_common_io SYSTEM BEFORE PUBLIC ${HDFS3_INCLUDE_DIR}) endif() if (USE_BROTLI) diff --git a/dbms/src/IO/HDFSCommon.h b/dbms/src/IO/HDFSCommon.h index 8c526d908bb..0be00de08dd 100644 --- a/dbms/src/IO/HDFSCommon.h +++ b/dbms/src/IO/HDFSCommon.h @@ -27,6 +27,26 @@ struct HDFSFsDeleter } +struct HDFSFileInfo +{ + hdfsFileInfo * file_info; + int length; + + HDFSFileInfo() + : file_info(nullptr) + , length(0) + { + } + HDFSFileInfo(const HDFSFileInfo & other) = delete; + HDFSFileInfo(HDFSFileInfo && other) = default; + HDFSFileInfo & operator=(const HDFSFileInfo & other) = delete; + HDFSFileInfo & operator=(HDFSFileInfo && other) = default; + + ~HDFSFileInfo() + { + hdfsFreeFileInfo(file_info, length); + } +}; using HDFSBuilderPtr = std::unique_ptr; using HDFSFSPtr = std::unique_ptr, detail::HDFSFsDeleter>; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 8c87f4ccd6a..7d9d123e5a3 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -9,12 +9,17 @@ #include #include #include +#include #include #include #include #include #include - +#include +#include +#include +#include +#include namespace DB { @@ -127,6 +132,13 @@ private: BlockOutputStreamPtr writer; }; +//static Strings recursiveLSWithRegexpMatching(const String & cur_path, hdfsFS fs, const re2::RE2 & matcher) +//{ +// HDFSFileInfo ls; +// ls.file_info = hdfsListDirectory(fs.get(), path_without_globs, ls.length); +// +//} + } @@ -138,12 +150,51 @@ BlockInputStreams StorageHDFS::read( size_t max_block_size, unsigned /*num_streams*/) { - return {std::make_shared( - uri, - format_name, - getSampleBlock(), - context_, - max_block_size)}; + Strings path_parts; + Poco::URI poco_uri(uri); + poco_uri.getPathSegments(path_parts); + String path_without_globs; + for (const auto & part : path_parts) + { + if ((part.find('*') != std::string::npos) || (part.find('?') != std::string::npos) || (part.find('{') != std::string::npos)) + break; + path_without_globs.push_back('/'); + path_without_globs.append(part); + } + if (path_without_globs == poco_uri.getPath()) + return {std::make_shared( + uri, + format_name, + getSampleBlock(), + context_, + max_block_size)}; + + String path_pattern = makeRegexpPatternFromGlobs(poco_uri.getPath()); + re2::RE2 matcher(path_pattern); + path_without_globs.push_back('/'); + poco_uri.setPath(path_without_globs); + HDFSBuilderPtr builder = createHDFSBuilder(poco_uri); + HDFSFSPtr fs = createHDFSFS(builder.get()); +// Strings res_paths = recursiveLSWithRegexpMatching(path_without_globs, fs.get(), matcher); + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), path_without_globs.data(), &ls.length); + BlockInputStreams result; + for (int i = 0; i < ls.length; ++i) + { + if (ls.file_info[i].mKind == 'F') + { + String cur_path = path_without_globs + String(ls.file_info[i].mName); + if (re2::RE2::FullMatch(cur_path, matcher)) + { + poco_uri.setPath(cur_path); + result.push_back( + std::make_shared(poco_uri.toString(), format_name, getSampleBlock(), context_, + max_block_size)); + } + } + } + + return result; } void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name) From e6d85de79aa4922eca8e3224eac326a55ea684d5 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Sun, 28 Jul 2019 01:04:52 +0300 Subject: [PATCH 15/75] wip small change --- dbms/src/Storages/StorageFile.cpp | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 69bde384de0..4f71fa14f32 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -97,16 +97,15 @@ StorageFile::StorageFile( if ((path.find('*') != std::string::npos) || (path.find('?') != std::string::npos) || (path.find('{') != std::string::npos)) { path_with_globs = true; - std::string path_pattern = makeRegexpPatternFromGlobs(path); - re2::RE2 matcher(path_pattern); + re2::RE2 matcher(makeRegexpPatternFromGlobs(path)); fs::path cur_dir(db_dir_path); fs::directory_iterator end; for (fs::directory_iterator it(cur_dir); it != end; ++it) { - const fs::path file = (*it); - if (re2::RE2::FullMatch(file.string(), matcher)) + std::string file = it->path().string(); + if (re2::RE2::FullMatch(file, matcher)) { - matched_paths.push_back(file.string()); + matched_paths.push_back(file); checkCreationIsAllowed(context_global, db_dir_path, matched_paths.back(), table_fd); } } From a3b35d5569e4e4bb7d5588e42172b141d90c379e Mon Sep 17 00:00:00 2001 From: stavrolia Date: Tue, 30 Jul 2019 21:46:49 +0300 Subject: [PATCH 16/75] WIP --- dbms/src/Storages/StorageFile.cpp | 2 +- dbms/src/Storages/StorageHDFS.cpp | 41 +++++++++++-------- .../integration/test_storage_hdfs/test.py | 6 ++- 3 files changed, 28 insertions(+), 21 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 4f71fa14f32..f2ecc1ea881 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -94,7 +94,7 @@ StorageFile::StorageFile( poco_path = Poco::Path(db_dir_path, poco_path); path = poco_path.absolute().toString(); - if ((path.find('*') != std::string::npos) || (path.find('?') != std::string::npos) || (path.find('{') != std::string::npos)) + if (path.find_first_of("*?{") != std::string::npos) { path_with_globs = true; re2::RE2 matcher(makeRegexpPatternFromGlobs(path)); diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 7d9d123e5a3..64a4496ae43 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -151,17 +151,9 @@ BlockInputStreams StorageHDFS::read( unsigned /*num_streams*/) { Strings path_parts; - Poco::URI poco_uri(uri); - poco_uri.getPathSegments(path_parts); - String path_without_globs; - for (const auto & part : path_parts) - { - if ((part.find('*') != std::string::npos) || (part.find('?') != std::string::npos) || (part.find('{') != std::string::npos)) - break; - path_without_globs.push_back('/'); - path_without_globs.append(part); - } - if (path_without_globs == poco_uri.getPath()) + size_t first_glob = uri.find_first_of("*?{"); + + if (first_glob == std::string::npos) return {std::make_shared( uri, format_name, @@ -169,26 +161,39 @@ BlockInputStreams StorageHDFS::read( context_, max_block_size)}; - String path_pattern = makeRegexpPatternFromGlobs(poco_uri.getPath()); + String uri_without_globs = uri.substr(0, first_glob); + size_t end_of_path_without_globs = uri_without_globs.rfind('/'); + uri_without_globs = uri_without_globs.substr(0, end_of_path_without_globs + 1); + + size_t begin_of_path = uri.find('/', uri.find("//") + 2); + String path_from_uri = uri.substr(begin_of_path); + String uri_without_path = uri.substr(0, begin_of_path); + + String path_pattern = makeRegexpPatternFromGlobs(path_from_uri); re2::RE2 matcher(path_pattern); - path_without_globs.push_back('/'); - poco_uri.setPath(path_without_globs); - HDFSBuilderPtr builder = createHDFSBuilder(poco_uri); + + HDFSBuilderPtr builder = createHDFSBuilder(Poco::URI(uri_without_globs)); HDFSFSPtr fs = createHDFSFS(builder.get()); // Strings res_paths = recursiveLSWithRegexpMatching(path_without_globs, fs.get(), matcher); + HDFSFileInfo ls; + String path_without_globs = uri_without_globs.substr(begin_of_path); ls.file_info = hdfsListDirectory(fs.get(), path_without_globs.data(), &ls.length); BlockInputStreams result; for (int i = 0; i < ls.length; ++i) { if (ls.file_info[i].mKind == 'F') { - String cur_path = path_without_globs + String(ls.file_info[i].mName); + String cur_path = String(ls.file_info[i].mName); + if (cur_path[1] == '/') + { + cur_path = cur_path.substr(1); + } + if (re2::RE2::FullMatch(cur_path, matcher)) { - poco_uri.setPath(cur_path); result.push_back( - std::make_shared(poco_uri.toString(), format_name, getSampleBlock(), context_, + std::make_shared(uri_without_path + String(ls.file_info[i].mName), format_name, getSampleBlock(), context_, max_block_size)); } } diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index 173c2d77b3f..4a1c33da627 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -41,10 +41,12 @@ def test_read_write_table(started_cluster): hdfs_api = HDFSApi("root") data = "1\tSerialize\t555.222\n2\tData\t777.333\n" hdfs_api.write_data("/simple_table_function", data) + hdfs_api.write_data("/dir/file", data) - assert hdfs_api.read_data("/simple_table_function") == data + assert hdfs_api.read_data("/dir/file") == data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_functio?', 'TSV', 'id UInt64, text String, number Float64')") == data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/fil?', 'TSV', 'id UInt64, text String, number Float64')") == data def test_write_table(started_cluster): From 7addd50c16bb2aff2ef4b175ea4baae307229d68 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Thu, 1 Aug 2019 18:46:54 +0300 Subject: [PATCH 17/75] recursive version for StorageHDFS and some tests for it --- dbms/src/Storages/StorageHDFS.cpp | 69 +++++++++++-------- .../integration/test_storage_hdfs/test.py | 32 +++++++-- 2 files changed, 68 insertions(+), 33 deletions(-) diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 64a4496ae43..22c80bf4386 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -132,12 +132,41 @@ private: BlockOutputStreamPtr writer; }; -//static Strings recursiveLSWithRegexpMatching(const String & cur_path, hdfsFS fs, const re2::RE2 & matcher) -//{ -// HDFSFileInfo ls; -// ls.file_info = hdfsListDirectory(fs.get(), path_without_globs, ls.length); -// -//} +static Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) +{ + HDFSFileInfo ls; + ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); + Strings result; + size_t next_slash = for_match.find('/', 1); + String cur_item_for_match = for_match.substr(0, next_slash); + String part_pattern = makeRegexpPatternFromGlobs(cur_item_for_match); + re2::RE2 matcher(part_pattern); + + for (int i = 0; i < ls.length; ++i) + { + String cur_path = String(ls.file_info[i].mName); + size_t last_slash = cur_path.rfind('/'); + String cur_path_item = cur_path.substr(last_slash); + + if ((ls.file_info[i].mKind == 'F') && (next_slash == std::string::npos)) + { + if (re2::RE2::FullMatch(cur_path_item, matcher)) + { + result.push_back(String(ls.file_info[i].mName)); + } + } + else if ((ls.file_info[i].mKind == 'D') && (next_slash != std::string::npos)) + { + if (re2::RE2::FullMatch(cur_path_item, matcher)) + { + Strings result_part = LSWithRegexpMatching(cur_path, fs, for_match.substr(next_slash)); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + } + + return result; +} } @@ -163,40 +192,22 @@ BlockInputStreams StorageHDFS::read( String uri_without_globs = uri.substr(0, first_glob); size_t end_of_path_without_globs = uri_without_globs.rfind('/'); + String part_with_glob = uri.substr(end_of_path_without_globs); uri_without_globs = uri_without_globs.substr(0, end_of_path_without_globs + 1); size_t begin_of_path = uri.find('/', uri.find("//") + 2); String path_from_uri = uri.substr(begin_of_path); String uri_without_path = uri.substr(0, begin_of_path); - String path_pattern = makeRegexpPatternFromGlobs(path_from_uri); - re2::RE2 matcher(path_pattern); - + String path_without_globs = uri_without_globs.substr(begin_of_path); HDFSBuilderPtr builder = createHDFSBuilder(Poco::URI(uri_without_globs)); HDFSFSPtr fs = createHDFSFS(builder.get()); -// Strings res_paths = recursiveLSWithRegexpMatching(path_without_globs, fs.get(), matcher); - - HDFSFileInfo ls; - String path_without_globs = uri_without_globs.substr(begin_of_path); - ls.file_info = hdfsListDirectory(fs.get(), path_without_globs.data(), &ls.length); + Strings res_paths = LSWithRegexpMatching(path_without_globs, fs, part_with_glob); BlockInputStreams result; - for (int i = 0; i < ls.length; ++i) + for (const auto & res_path : res_paths) { - if (ls.file_info[i].mKind == 'F') - { - String cur_path = String(ls.file_info[i].mName); - if (cur_path[1] == '/') - { - cur_path = cur_path.substr(1); - } - - if (re2::RE2::FullMatch(cur_path, matcher)) - { - result.push_back( - std::make_shared(uri_without_path + String(ls.file_info[i].mName), format_name, getSampleBlock(), context_, + result.push_back(std::make_shared(uri_without_path + res_path, format_name, getSampleBlock(), context_, max_block_size)); - } - } } return result; diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index 4a1c33da627..f219e1262ae 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -41,12 +41,10 @@ def test_read_write_table(started_cluster): hdfs_api = HDFSApi("root") data = "1\tSerialize\t555.222\n2\tData\t777.333\n" hdfs_api.write_data("/simple_table_function", data) - hdfs_api.write_data("/dir/file", data) - assert hdfs_api.read_data("/dir/file") == data + assert hdfs_api.read_data("/simple_table_function") == data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_functio?', 'TSV', 'id UInt64, text String, number Float64')") == data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/fil?', 'TSV', 'id UInt64, text String, number Float64')") == data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/simple_table_function', 'TSV', 'id UInt64, text String, number Float64')") == data def test_write_table(started_cluster): @@ -76,3 +74,29 @@ def test_bad_hdfs_uri(started_cluster): except Exception as ex: print ex assert 'Unable to open HDFS file' in str(ex) + +def test_globs_in_read_table(started_cluster): + hdfs_api = HDFSApi("root") + some_data = "1\tSerialize\t555.222\n2\tData\t777.333\n" + hdfs_api.write_data("/simple_table_function", some_data) + hdfs_api.write_data("/dir/file", some_data) + hdfs_api.write_data("/some_dir/dir1/file", some_data) + hdfs_api.write_data("/some_dir/dir2/file", some_data) + hdfs_api.write_data("/some_dir/file", some_data) + hdfs_api.write_data("/table1_function", some_data) + hdfs_api.write_data("/table2_function", some_data) + hdfs_api.write_data("/table3_function", some_data) + + assert hdfs_api.read_data("/dir/file") == some_data + + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/*_table_functio?', 'TSV', 'id UInt64, text String, number Float64')") == some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/fil?', 'TSV', 'id UInt64, text String, number Float64')") == some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/table{3..8}_function', 'TSV', 'id UInt64, text String, number Float64')") == some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/table{2..8}_function', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*', 'TSV', 'id UInt64, text String, number Float64')") == some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*?*?*?*?*', 'TSV', 'id UInt64, text String, number Float64')") == some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*?*?*?*?*?*', 'TSV', 'id UInt64, text String, number Float64')") == "" + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*{a..z}*{a..z}*{a..z}*{a..z}*', 'TSV', 'id UInt64, text String, number Float64')") == some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/some_dir/*/file', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/some_dir/dir?/*', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data + assert node1.query("select * from hdfs('hdfs://hdfs1:9000/*/*/*', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data From 95cb95105dd20c517e5b4a4dcb65ca41aad5a4db Mon Sep 17 00:00:00 2001 From: stavrolia Date: Fri, 2 Aug 2019 18:00:12 +0300 Subject: [PATCH 18/75] Add recursive version for storage file --- dbms/src/Storages/StorageFile.cpp | 53 ++++++++++++++++++++++++------- dbms/src/Storages/StorageHDFS.cpp | 3 +- 2 files changed, 43 insertions(+), 13 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index f2ecc1ea881..dd02d4665c2 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -28,6 +28,7 @@ #include #include #include +#include namespace fs = boost::filesystem; @@ -46,6 +47,43 @@ namespace ErrorCodes extern const int EMPTY_LIST_OF_COLUMNS_PASSED; } +namespace +{ + +Strings LSWithRegexpMatching(const String & path_for_ls, const String & for_match) +{ + Strings result; + size_t next_slash = for_match.find('/', 1); + String cur_item_for_match = for_match.substr(0, next_slash); + String part_pattern = makeRegexpPatternFromGlobs(cur_item_for_match); + re2::RE2 matcher(part_pattern); + + fs::directory_iterator end; + for (fs::directory_iterator it(path_for_ls); it != end; ++it) + { + std::string cur_path = it->path().string(); + size_t last_slash = cur_path.rfind('/'); + String cur_path_item = cur_path.substr(last_slash); + + if ((!is_directory(it->path())) && (next_slash == std::string::npos)) + { + if (re2::RE2::FullMatch(cur_path_item, matcher)) + { + result.push_back(it->path().string()); + } + } + else if ((is_directory(it->path())) && (next_slash != std::string::npos)) + { + if (re2::RE2::FullMatch(cur_path_item, matcher)) + { + Strings result_part = LSWithRegexpMatching(cur_path, for_match); + std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); + } + } + } + return result; +} +} static std::string getTablePath(const std::string & db_dir_path, const std::string & table_name, const std::string & format_name) { @@ -97,18 +135,9 @@ StorageFile::StorageFile( if (path.find_first_of("*?{") != std::string::npos) { path_with_globs = true; - re2::RE2 matcher(makeRegexpPatternFromGlobs(path)); - fs::path cur_dir(db_dir_path); - fs::directory_iterator end; - for (fs::directory_iterator it(cur_dir); it != end; ++it) - { - std::string file = it->path().string(); - if (re2::RE2::FullMatch(file, matcher)) - { - matched_paths.push_back(file); - checkCreationIsAllowed(context_global, db_dir_path, matched_paths.back(), table_fd); - } - } + matched_paths = LSWithRegexpMatching(db_dir_path, path); + for (const auto & cur_path : matched_paths) + checkCreationIsAllowed(context_global, db_dir_path, cur_path, table_fd); } else checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); is_db_table = false; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 22c80bf4386..6322d676503 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -132,10 +132,11 @@ private: BlockOutputStreamPtr writer; }; -static Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) +Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) { HDFSFileInfo ls; ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); + Strings result; size_t next_slash = for_match.find('/', 1); String cur_item_for_match = for_match.substr(0, next_slash); From b386b3090cf92a92245a8ae8275f9c966713ec68 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Mon, 5 Aug 2019 14:43:39 +0300 Subject: [PATCH 19/75] Impove HDFS version --- dbms/src/Storages/StorageHDFS.cpp | 25 +++++++++++++------------ 1 file changed, 13 insertions(+), 12 deletions(-) diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 6322d676503..ef9ae9c6bfe 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -134,12 +134,18 @@ private: Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) { + size_t first_glob = for_match.find_first_of("*?{"); + + size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + String path_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' + String path_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' + HDFSFileInfo ls; - ls.file_info = hdfsListDirectory(fs.get(), path_for_ls.data(), &ls.length); + ls.file_info = hdfsListDirectory(fs.get(), path_without_globs.data(), &ls.length); Strings result; - size_t next_slash = for_match.find('/', 1); - String cur_item_for_match = for_match.substr(0, next_slash); + size_t next_slash = path_with_globs.find('/', 1); + String cur_item_for_match = path_with_globs.substr(0, next_slash); /// without '/' at the end String part_pattern = makeRegexpPatternFromGlobs(cur_item_for_match); re2::RE2 matcher(part_pattern); @@ -160,7 +166,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c { if (re2::RE2::FullMatch(cur_path_item, matcher)) { - Strings result_part = LSWithRegexpMatching(cur_path, fs, for_match.substr(next_slash)); + Strings result_part = LSWithRegexpMatching(cur_path + "/", fs, path_with_globs.substr(next_slash)); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } @@ -191,19 +197,14 @@ BlockInputStreams StorageHDFS::read( context_, max_block_size)}; - String uri_without_globs = uri.substr(0, first_glob); - size_t end_of_path_without_globs = uri_without_globs.rfind('/'); - String part_with_glob = uri.substr(end_of_path_without_globs); - uri_without_globs = uri_without_globs.substr(0, end_of_path_without_globs + 1); - size_t begin_of_path = uri.find('/', uri.find("//") + 2); String path_from_uri = uri.substr(begin_of_path); String uri_without_path = uri.substr(0, begin_of_path); - String path_without_globs = uri_without_globs.substr(begin_of_path); - HDFSBuilderPtr builder = createHDFSBuilder(Poco::URI(uri_without_globs)); + HDFSBuilderPtr builder = createHDFSBuilder(Poco::URI(uri_without_path + "/")); HDFSFSPtr fs = createHDFSFS(builder.get()); - Strings res_paths = LSWithRegexpMatching(path_without_globs, fs, part_with_glob); + + Strings res_paths = LSWithRegexpMatching("/", fs, path_from_uri); BlockInputStreams result; for (const auto & res_path : res_paths) { From f9037fee2c0a6098d809f85d6764f244ecd76423 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Tue, 6 Aug 2019 02:10:19 +0300 Subject: [PATCH 20/75] Fix File version --- dbms/src/Storages/StorageFile.cpp | 25 ++++++++++++++----------- 1 file changed, 14 insertions(+), 11 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index dd02d4665c2..0ab56a0e4e5 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -53,18 +53,22 @@ namespace Strings LSWithRegexpMatching(const String & path_for_ls, const String & for_match) { Strings result; - size_t next_slash = for_match.find('/', 1); - String cur_item_for_match = for_match.substr(0, next_slash); + size_t first_glob = for_match.find_first_of("*?{"); + + size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + String path_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' + String path_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' + + size_t next_slash = path_with_globs.find('/', 1); + String cur_item_for_match = path_with_globs.substr(0, next_slash); /// without '/' at the end String part_pattern = makeRegexpPatternFromGlobs(cur_item_for_match); re2::RE2 matcher(part_pattern); - fs::directory_iterator end; for (fs::directory_iterator it(path_for_ls); it != end; ++it) { std::string cur_path = it->path().string(); size_t last_slash = cur_path.rfind('/'); String cur_path_item = cur_path.substr(last_slash); - if ((!is_directory(it->path())) && (next_slash == std::string::npos)) { if (re2::RE2::FullMatch(cur_path_item, matcher)) @@ -76,7 +80,7 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const String & for_matc { if (re2::RE2::FullMatch(cur_path_item, matcher)) { - Strings result_part = LSWithRegexpMatching(cur_path, for_match); + Strings result_part = LSWithRegexpMatching(cur_path + "/", path_with_globs.substr(next_slash)); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } @@ -132,13 +136,15 @@ StorageFile::StorageFile( poco_path = Poco::Path(db_dir_path, poco_path); path = poco_path.absolute().toString(); - if (path.find_first_of("*?{") != std::string::npos) + size_t first_glob = path.find_first_of("*?{"); + if (first_glob != std::string::npos) { path_with_globs = true; matched_paths = LSWithRegexpMatching(db_dir_path, path); for (const auto & cur_path : matched_paths) checkCreationIsAllowed(context_global, db_dir_path, cur_path, table_fd); - } else + } + else checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); is_db_table = false; } @@ -257,10 +263,7 @@ BlockInputStreams StorageFile::read( for (size_t i = 0; i < matched_paths.size(); ++i) { BlockInputStreamPtr cur_block = std::make_shared(*this, context, max_block_size, i); - if (column_defaults.empty()) - blocks_input.push_back(cur_block); - else - blocks_input.push_back(std::make_shared(cur_block, column_defaults, context)); + blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared(cur_block, column_defaults, context)); } return blocks_input; } From 229dd7c2b685be9a804d6c849916860bb4cb2bf2 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Thu, 8 Aug 2019 17:26:02 +0300 Subject: [PATCH 21/75] wip --- dbms/src/Common/parseGlobs.cpp | 6 ------ dbms/src/Common/parseGlobs.h | 3 ++- dbms/src/Storages/StorageFile.cpp | 29 ++++++++++++++--------------- dbms/src/Storages/StorageHDFS.cpp | 27 ++++++++++++--------------- 4 files changed, 28 insertions(+), 37 deletions(-) diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp index 216ce2a9cf6..ad353429acb 100644 --- a/dbms/src/Common/parseGlobs.cpp +++ b/dbms/src/Common/parseGlobs.cpp @@ -2,7 +2,6 @@ #include #include #include -#include namespace DB { @@ -33,7 +32,6 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str) current_index = input_for_range.data() - first_prepare.data(); } second_prepare += first_prepare.substr(current_index); - std::cout << second_prepare < +#include namespace DB { /* Parse globs in string and make a regexp for it. */ -std::string makeRegexpPatternFromGlobs(const std::string & ipath); +std::string makeRegexpPatternFromGlobs(const std::string & path); } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 0ab56a0e4e5..4eba9e730f3 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -50,37 +50,36 @@ namespace ErrorCodes namespace { -Strings LSWithRegexpMatching(const String & path_for_ls, const String & for_match) +std::vector LSWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match) { - Strings result; + size_t first_glob = for_match.find_first_of("*?{"); size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); - String path_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - String path_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' + std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - size_t next_slash = path_with_globs.find('/', 1); - String cur_item_for_match = path_with_globs.substr(0, next_slash); /// without '/' at the end - String part_pattern = makeRegexpPatternFromGlobs(cur_item_for_match); - re2::RE2 matcher(part_pattern); + size_t next_slash = suffix_with_globs.find('/', 1); + re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); + + std::vector result; fs::directory_iterator end; - for (fs::directory_iterator it(path_for_ls); it != end; ++it) + for (fs::directory_iterator it(path_for_ls + for_match.substr(1, end_of_path_without_globs)); it != end; ++it) { - std::string cur_path = it->path().string(); - size_t last_slash = cur_path.rfind('/'); - String cur_path_item = cur_path.substr(last_slash); + std::string full_path = it->path().string(); + size_t last_slash = full_path.rfind('/'); + String file_name = full_path.substr(last_slash); if ((!is_directory(it->path())) && (next_slash == std::string::npos)) { - if (re2::RE2::FullMatch(cur_path_item, matcher)) + if (re2::RE2::FullMatch(file_name, matcher)) { result.push_back(it->path().string()); } } else if ((is_directory(it->path())) && (next_slash != std::string::npos)) { - if (re2::RE2::FullMatch(cur_path_item, matcher)) + if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(cur_path + "/", path_with_globs.substr(next_slash)); + Strings result_part = LSWithRegexpMatching(full_path + "/", suffix_with_globs.substr(next_slash)); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index ef9ae9c6bfe..e492da080c1 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -137,36 +137,33 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c size_t first_glob = for_match.find_first_of("*?{"); size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); - String path_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - String path_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' + String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' + String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' + + size_t next_slash = suffix_with_globs.find('/', 1); + re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); HDFSFileInfo ls; - ls.file_info = hdfsListDirectory(fs.get(), path_without_globs.data(), &ls.length); - + ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length); Strings result; - size_t next_slash = path_with_globs.find('/', 1); - String cur_item_for_match = path_with_globs.substr(0, next_slash); /// without '/' at the end - String part_pattern = makeRegexpPatternFromGlobs(cur_item_for_match); - re2::RE2 matcher(part_pattern); - for (int i = 0; i < ls.length; ++i) { - String cur_path = String(ls.file_info[i].mName); - size_t last_slash = cur_path.rfind('/'); - String cur_path_item = cur_path.substr(last_slash); + String full_path = String(ls.file_info[i].mName); + size_t last_slash = full_path.rfind('/'); + String file_name = full_path.substr(last_slash); if ((ls.file_info[i].mKind == 'F') && (next_slash == std::string::npos)) { - if (re2::RE2::FullMatch(cur_path_item, matcher)) + if (re2::RE2::FullMatch(file_name, matcher)) { result.push_back(String(ls.file_info[i].mName)); } } else if ((ls.file_info[i].mKind == 'D') && (next_slash != std::string::npos)) { - if (re2::RE2::FullMatch(cur_path_item, matcher)) + if (re2::RE2::FullMatch(file_name, matcher)) { - Strings result_part = LSWithRegexpMatching(cur_path + "/", fs, path_with_globs.substr(next_slash)); + Strings result_part = LSWithRegexpMatching(full_path + "/", fs, suffix_with_globs.substr(next_slash)); std::move(result_part.begin(), result_part.end(), std::back_inserter(result)); } } From 32fad1e961b06b6035d47fb44d9a8bee23abfef0 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Thu, 8 Aug 2019 22:16:17 +0300 Subject: [PATCH 22/75] fix --- dbms/src/Storages/StorageFile.cpp | 13 ++++++------- 1 file changed, 6 insertions(+), 7 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 4eba9e730f3..d06e071b915 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -63,7 +63,7 @@ std::vector LSWithRegexpMatching(const std::string & path_for_ls, c std::vector result; fs::directory_iterator end; - for (fs::directory_iterator it(path_for_ls + for_match.substr(1, end_of_path_without_globs)); it != end; ++it) + for (fs::directory_iterator it(path_for_ls); it != end; ++it) { std::string full_path = it->path().string(); size_t last_slash = full_path.rfind('/'); @@ -174,7 +174,7 @@ StorageFile::StorageFile( class StorageFileBlockInputStream : public IBlockInputStream { public: - StorageFileBlockInputStream(StorageFile & storage_, const Context & context, UInt64 max_block_size, size_t num_of_path) + StorageFileBlockInputStream(StorageFile & storage_, const Context & context, UInt64 max_block_size, std::string file_path) : storage(storage_) { if (storage.use_table_fd) @@ -200,8 +200,7 @@ public: else { shared_lock = std::shared_lock(storage.rwlock); - std::string path_to_read = storage.path_with_globs ? storage.matched_paths[num_of_path] : storage.path; - read_buf = std::make_unique(path_to_read); + read_buf = std::make_unique(file_path); } reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size); @@ -252,16 +251,16 @@ BlockInputStreams StorageFile::read( auto column_defaults = columns.getDefaults(); if (!path_with_globs) { - BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size, 0); + BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size, path); if (column_defaults.empty()) return {block_input}; return {std::make_shared(block_input, column_defaults, context)}; } BlockInputStreams blocks_input; blocks_input.reserve(matched_paths.size()); - for (size_t i = 0; i < matched_paths.size(); ++i) + for (const auto & file_path : matched_paths) { - BlockInputStreamPtr cur_block = std::make_shared(*this, context, max_block_size, i); + BlockInputStreamPtr cur_block = std::make_shared(*this, context, max_block_size, file_path); blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared(cur_block, column_defaults, context)); } return blocks_input; From 2f2bf953ebfe786e16ad811479db2032b2e2ba8d Mon Sep 17 00:00:00 2001 From: stavrolia Date: Fri, 9 Aug 2019 20:06:29 +0300 Subject: [PATCH 23/75] fix and add test for storage file version --- dbms/src/Common/parseGlobs.cpp | 5 ++- dbms/src/Storages/StorageFile.cpp | 4 +- .../test_globs_in_filepath/__init__.py | 0 .../test_globs_in_filepath/test.py | 42 +++++++++++++++++++ 4 files changed, 48 insertions(+), 3 deletions(-) create mode 100644 dbms/tests/integration/test_globs_in_filepath/__init__.py create mode 100644 dbms/tests/integration/test_globs_in_filepath/test.py diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp index ad353429acb..02775d25268 100644 --- a/dbms/src/Common/parseGlobs.cpp +++ b/dbms/src/Common/parseGlobs.cpp @@ -5,7 +5,10 @@ namespace DB { - +/* Because of difference between grep-wildcard-syntax and perl-regexp one we need some transformation of string to use RE2 library for matching. + * It couldn't be one pass because of various configurations of braces in filenames (Linux allow almost any symbols in paths). + * So there are some iterations of escaping and replacements to make correct perl-regexp. + */ std::string makeRegexpPatternFromGlobs(const std::string & initial_str) { std::string first_prepare; diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index d06e071b915..5516182385c 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -63,7 +63,7 @@ std::vector LSWithRegexpMatching(const std::string & path_for_ls, c std::vector result; fs::directory_iterator end; - for (fs::directory_iterator it(path_for_ls); it != end; ++it) + for (fs::directory_iterator it(path_for_ls + for_match.substr(1, end_of_path_without_globs)); it != end; ++it) { std::string full_path = it->path().string(); size_t last_slash = full_path.rfind('/'); @@ -139,7 +139,7 @@ StorageFile::StorageFile( if (first_glob != std::string::npos) { path_with_globs = true; - matched_paths = LSWithRegexpMatching(db_dir_path, path); + matched_paths = LSWithRegexpMatching("/", path); for (const auto & cur_path : matched_paths) checkCreationIsAllowed(context_global, db_dir_path, cur_path, table_fd); } diff --git a/dbms/tests/integration/test_globs_in_filepath/__init__.py b/dbms/tests/integration/test_globs_in_filepath/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_globs_in_filepath/test.py b/dbms/tests/integration/test_globs_in_filepath/test.py new file mode 100644 index 00000000000..c7d6e26c58e --- /dev/null +++ b/dbms/tests/integration/test_globs_in_filepath/test.py @@ -0,0 +1,42 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node') + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_globs(start_cluster): + some_data = "\t555.222\nData\t777.333" + path_to_userfiles = "/var/lib/clickhouse/user_files/" + dirs = ["dir1/", "dir2/"] + + for dir in dirs: + node.exec_in_container(['bash', '-c', 'mkdir {}{}'.format(path_to_userfiles, dir)], privileged=True, user='root') + + # all directories appeared in files must be listed in dirs + files = ["dir1/file1", "dir1/file2", + "dir2/file1", "dir2/file2", "dir2/file11", + "file1"] + + for filename in files: + node.exec_in_container(['bash', '-c', 'echo "{}{}" > {}{}'.format(filename, some_data, path_to_userfiles, filename)], privileged=True, user='root') + + test_requests = [("dir1/file*", "4"), + ("dir1/file?", "4"), + ("dir1/file{0..9}", "4"), + ("dir2/file*", "6"), + ("dir2/file?", "4"), + ("*", "2")] + + for pattern, value in test_requests: + assert node.query(''' + select count(*) from file('{}', 'TSV', 'text String, number Float64') + '''.format(pattern)) == '{}\n'.format(value) \ No newline at end of file From 2bfd3981079bf4279ae67e0eae320b65d423c340 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Fri, 9 Aug 2019 20:25:29 +0300 Subject: [PATCH 24/75] fix test --- .../integration/test_storage_hdfs/test.py | 37 +++++++++---------- 1 file changed, 17 insertions(+), 20 deletions(-) diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index f219e1262ae..75e16558688 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -78,25 +78,22 @@ def test_bad_hdfs_uri(started_cluster): def test_globs_in_read_table(started_cluster): hdfs_api = HDFSApi("root") some_data = "1\tSerialize\t555.222\n2\tData\t777.333\n" - hdfs_api.write_data("/simple_table_function", some_data) - hdfs_api.write_data("/dir/file", some_data) - hdfs_api.write_data("/some_dir/dir1/file", some_data) - hdfs_api.write_data("/some_dir/dir2/file", some_data) - hdfs_api.write_data("/some_dir/file", some_data) - hdfs_api.write_data("/table1_function", some_data) - hdfs_api.write_data("/table2_function", some_data) - hdfs_api.write_data("/table3_function", some_data) + globs_dir = "/dir_for_test_with_globs/" + files = ["simple_table_function", "dir/file", "some_dir/dir1/file", "some_dir/dir2/file", "some_dir/file", "table1_function", "table2_function", "table3_function"] + for filename in files: + hdfs_api.write_data(globs_dir + filename, some_data) - assert hdfs_api.read_data("/dir/file") == some_data + test_requests = [("*_table_functio?", 1), + ("dir/fil?", 1), + ("table{3..8}_function", 1), + ("table{2..8}_function", 2), + ("dir/*", 1), + ("dir/*?*?*?*?*", 1), + ("dir/*?*?*?*?*?*", 0), + ("dir/*{a..z}*{a..z}*{a..z}*{a..z}*", 1), + ("some_dir/*/file", 2), + ("some_dir/dir?/*", 2), + ("*/*/*", 2)] - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/*_table_functio?', 'TSV', 'id UInt64, text String, number Float64')") == some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/fil?', 'TSV', 'id UInt64, text String, number Float64')") == some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/table{3..8}_function', 'TSV', 'id UInt64, text String, number Float64')") == some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/table{2..8}_function', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*', 'TSV', 'id UInt64, text String, number Float64')") == some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*?*?*?*?*', 'TSV', 'id UInt64, text String, number Float64')") == some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*?*?*?*?*?*', 'TSV', 'id UInt64, text String, number Float64')") == "" - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/dir/*{a..z}*{a..z}*{a..z}*{a..z}*', 'TSV', 'id UInt64, text String, number Float64')") == some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/some_dir/*/file', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/some_dir/dir?/*', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data - assert node1.query("select * from hdfs('hdfs://hdfs1:9000/*/*/*', 'TSV', 'id UInt64, text String, number Float64')") == 2 * some_data + for pattern, value in test_requests: + assert node1.query("select * from hdfs('hdfs://hdfs1:9000" + globs_dir + pattern + "', 'TSV', 'id UInt64, text String, number Float64')") == value * some_data \ No newline at end of file From b3fd5bc14173c5240cf21fdfbbbede5f8bf68e93 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Sat, 10 Aug 2019 19:00:01 +0300 Subject: [PATCH 25/75] Add tests, comments and fix --- dbms/src/Storages/StorageFile.cpp | 18 ++- dbms/src/Storages/StorageHDFS.cpp | 6 +- .../test_globs_in_filepath/test.py | 111 +++++++++++++++--- .../integration/test_storage_hdfs/test.py | 8 +- 4 files changed, 117 insertions(+), 26 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 5516182385c..89e0230fb8e 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -49,7 +49,9 @@ namespace ErrorCodes namespace { - +/* Recursive directory listing with matched paths as a result. + * Have the same method in StorageHDFS. + */ std::vector LSWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match) { @@ -62,20 +64,27 @@ std::vector LSWithRegexpMatching(const std::string & path_for_ls, c re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); std::vector result; + std::string preffix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); + if (!fs::exists(fs::path(preffix_without_globs.data()))) + { + return result; + } fs::directory_iterator end; - for (fs::directory_iterator it(path_for_ls + for_match.substr(1, end_of_path_without_globs)); it != end; ++it) + for (fs::directory_iterator it(preffix_without_globs); it != end; ++it) { std::string full_path = it->path().string(); size_t last_slash = full_path.rfind('/'); String file_name = full_path.substr(last_slash); - if ((!is_directory(it->path())) && (next_slash == std::string::npos)) + /// Condition with next_slash means what we are looking for (it is from current position in psttern of path) + /// Condition is_directory means what kind of path is it in current iteration of ls + if ((!fs::is_directory(it->path())) && (next_slash == std::string::npos)) { if (re2::RE2::FullMatch(file_name, matcher)) { result.push_back(it->path().string()); } } - else if ((is_directory(it->path())) && (next_slash != std::string::npos)) + else if ((fs::is_directory(it->path())) && (next_slash != std::string::npos)) { if (re2::RE2::FullMatch(file_name, matcher)) { @@ -130,6 +139,7 @@ StorageFile::StorageFile( if (!table_path_.empty()) /// Is user's file { + /// rwlock is common because path with globs and many files in this case is readonly now Poco::Path poco_path = Poco::Path(table_path_); if (poco_path.isRelative()) poco_path = Poco::Path(db_dir_path, poco_path); diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index e492da080c1..d5668dbf1ae 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -132,6 +132,9 @@ private: BlockOutputStreamPtr writer; }; +/* Recursive directory listing with matched paths as a result. + * Have the same method in StorageFile. + */ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) { size_t first_glob = for_match.find_first_of("*?{"); @@ -151,7 +154,8 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c String full_path = String(ls.file_info[i].mName); size_t last_slash = full_path.rfind('/'); String file_name = full_path.substr(last_slash); - + /// Condition with next_slash means what we are looking for (it is from current position in psttern of path) + /// Condition with type of current file_info means what kind of path is it in current iteration of ls if ((ls.file_info[i].mKind == 'F') && (next_slash == std::string::npos)) { if (re2::RE2::FullMatch(file_name, matcher)) diff --git a/dbms/tests/integration/test_globs_in_filepath/test.py b/dbms/tests/integration/test_globs_in_filepath/test.py index c7d6e26c58e..e6f28fedfe9 100644 --- a/dbms/tests/integration/test_globs_in_filepath/test.py +++ b/dbms/tests/integration/test_globs_in_filepath/test.py @@ -4,6 +4,7 @@ from helpers.cluster import ClickHouseCluster cluster = ClickHouseCluster(__file__) node = cluster.add_instance('node') +path_to_userfiles_from_defaut_config = "/var/lib/clickhouse/user_files/" # should be the same as in config file @pytest.fixture(scope="module") def start_cluster(): @@ -13,30 +14,104 @@ def start_cluster(): finally: cluster.shutdown() -def test_globs(start_cluster): - some_data = "\t555.222\nData\t777.333" - path_to_userfiles = "/var/lib/clickhouse/user_files/" - dirs = ["dir1/", "dir2/"] +def test_strange_filenames(start_cluster): + # 2 rows data + some_data = "\t111.222\nData\t333.444" - for dir in dirs: - node.exec_in_container(['bash', '-c', 'mkdir {}{}'.format(path_to_userfiles, dir)], privileged=True, user='root') + node.exec_in_container(['bash', '-c', 'mkdir {}strange_names/'.format(path_to_userfiles_from_defaut_config)], privileged=True, user='root') - # all directories appeared in files must be listed in dirs - files = ["dir1/file1", "dir1/file2", - "dir2/file1", "dir2/file2", "dir2/file11", - "file1"] + files = ["p.o.i.n.t.s", + "b}{ra{ces", + "b}.o{t.h"] + # filename inside testing data for debug simplicity for filename in files: - node.exec_in_container(['bash', '-c', 'echo "{}{}" > {}{}'.format(filename, some_data, path_to_userfiles, filename)], privileged=True, user='root') + node.exec_in_container(['bash', '-c', 'echo "{}{}" > {}strange_names/{}'.format(filename, some_data, path_to_userfiles_from_defaut_config, filename)], privileged=True, user='root') - test_requests = [("dir1/file*", "4"), - ("dir1/file?", "4"), - ("dir1/file{0..9}", "4"), - ("dir2/file*", "6"), - ("dir2/file?", "4"), - ("*", "2")] + test_requests = [("p.o.??n.t.s", "2"), + ("p.o.*t.s", "2"), + ("b}{r?{ces", "2"), + ("b}*ces", "2"), + ("b}{r{a..z}{ces", "2"), + ("b}.?{t.h", "2"), + ("b}.?{t.{a..z}", "2")] + + for pattern, value in test_requests: + assert node.query(''' + select count(*) from file('strange_names/{}', 'TSV', 'text String, number Float64') + '''.format(pattern)) == '{}\n'.format(value) + assert node.query(''' + select count(*) from file('{}strange_names/{}', 'TSV', 'text String, number Float64') + '''.format(path_to_userfiles_from_defaut_config, pattern)) == '{}\n'.format(value) + +def test_linear_structure(start_cluster): + # 2 rows data + some_data = "\t123.456\nData\t789.012" + + files = ["file1", "file2", "file3", "file4", "file5", + "file000", "file111", "file222", "file333", "file444", + "a_file", "b_file", "c_file", "d_file", "e_file", + "a_data", "b_data", "c_data", "d_data", "e_data"] + + # filename inside testing data for debug simplicity + for filename in files: + node.exec_in_container(['bash', '-c', 'echo "{}{}" > {}{}'.format(filename, some_data, path_to_userfiles_from_defaut_config, filename)], privileged=True, user='root') + + test_requests = [("file{0..9}", "10"), + ("file?", "10"), + ("file{0..9}{0..9}{0..9}", "10"), + ("file???", "10"), + ("file*", "20"), + ("a_{file,data}", "4"), + ("?_{file,data}", "20"), + ("{a..z}_{file,data}", "20"), + ("{a..z}?{file,data}", "20"), + ("*", "40")] for pattern, value in test_requests: assert node.query(''' select count(*) from file('{}', 'TSV', 'text String, number Float64') - '''.format(pattern)) == '{}\n'.format(value) \ No newline at end of file + '''.format(pattern)) == '{}\n'.format(value) + assert node.query(''' + select count(*) from file('{}{}', 'TSV', 'text String, number Float64') + '''.format(path_to_userfiles_from_defaut_config, pattern)) == '{}\n'.format(value) + +def test_deep_structure(start_cluster): + # 2 rows data + some_data = "\t135.791\nData\t246.802" + dirs = ["directory1/", "directory2/", "some_more_dir/", "we/", + "directory1/big_dir/", + "directory1/dir1/", "directory1/dir2/", "directory1/dir3/", + "directory2/dir1/", "directory2/dir2/", "directory2/one_more_dir/", + "some_more_dir/yet_another_dir/", + "we/need/", "we/need/to/", "we/need/to/go/", "we/need/to/go/deeper/"] + + for dir in dirs: + node.exec_in_container(['bash', '-c', 'mkdir {}{}'.format(path_to_userfiles_from_defaut_config, dir)], privileged=True, user='root') + + # all directories appeared in files must be listed in dirs + files = [] + for i in range(10): + for j in range(10): + for k in range(10): + files.append("directory1/big_dir/file"+str(i)+str(j)+str(k)) + + for dir in dirs: + files.append(dir+"file") + + # filename inside testing data for debug simplicity + for filename in files: + node.exec_in_container(['bash', '-c', 'echo "{}{}" > {}{}'.format(filename, some_data, path_to_userfiles_from_defaut_config, filename)], privileged=True, user='root') + + test_requests = [ ("directory{1..5}/big_dir/*", "2002"), ("directory{0..6}/big_dir/*{0..9}{0..9}{0..9}", "2000"), + ("?", "0"), + ("directory{0..5}/dir{1..3}/file", "10"), ("directory{0..5}/dir?/file", "10"), + ("we/need/to/go/deeper/file", "2"), ("*/*/*/*/*/*", "2"), ("we/n{a..z}ed/to/*/deeper/{a..z}{a..z}{a..z}{a..z}", "2"), ("we/need/??/go/deeper/*?*?*?*?*", "2")] + + for pattern, value in test_requests: + assert node.query(''' + select count(*) from file('{}', 'TSV', 'text String, number Float64') + '''.format(pattern)) == '{}\n'.format(value) + assert node.query(''' + select count(*) from file('{}{}', 'TSV', 'text String, number Float64') + '''.format(path_to_userfiles_from_defaut_config, pattern)) == '{}\n'.format(value) \ No newline at end of file diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index 75e16558688..5bbb86a7fcd 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -79,11 +79,12 @@ def test_globs_in_read_table(started_cluster): hdfs_api = HDFSApi("root") some_data = "1\tSerialize\t555.222\n2\tData\t777.333\n" globs_dir = "/dir_for_test_with_globs/" - files = ["simple_table_function", "dir/file", "some_dir/dir1/file", "some_dir/dir2/file", "some_dir/file", "table1_function", "table2_function", "table3_function"] + files = ["dir1/dir_dir/file1", "dir2/file2", "simple_table_function", "dir/file", "some_dir/dir1/file", "some_dir/dir2/file", "some_dir/file", "table1_function", "table2_function", "table3_function"] for filename in files: hdfs_api.write_data(globs_dir + filename, some_data) - test_requests = [("*_table_functio?", 1), + test_requests = [("dir{1..5}/dir_dir/file1", 1), + ("*_table_functio?", 1), ("dir/fil?", 1), ("table{3..8}_function", 1), ("table{2..8}_function", 2), @@ -93,7 +94,8 @@ def test_globs_in_read_table(started_cluster): ("dir/*{a..z}*{a..z}*{a..z}*{a..z}*", 1), ("some_dir/*/file", 2), ("some_dir/dir?/*", 2), - ("*/*/*", 2)] + ("*/*/*", 3), + ("?", 0)] for pattern, value in test_requests: assert node1.query("select * from hdfs('hdfs://hdfs1:9000" + globs_dir + pattern + "', 'TSV', 'id UInt64, text String, number Float64')") == value * some_data \ No newline at end of file From 1190e866aa13dd9ed23b7c988e4adf7071e59cef Mon Sep 17 00:00:00 2001 From: stavrolia Date: Sun, 11 Aug 2019 01:19:47 +0300 Subject: [PATCH 26/75] Fix --- dbms/src/Storages/StorageFile.cpp | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index 89e0230fb8e..e6a53d4fcf2 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -257,8 +257,8 @@ BlockInputStreams StorageFile::read( size_t max_block_size, unsigned /*num_streams*/) { - const ColumnsDescription & columns = getColumns(); - auto column_defaults = columns.getDefaults(); + const ColumnsDescription & columns_ = getColumns(); + auto column_defaults = columns_.getDefaults(); if (!path_with_globs) { BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size, path); From 9fdb9e580557a4bfda71dc582b11ad9f3b501383 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 19 Aug 2019 23:22:45 +0300 Subject: [PATCH 27/75] implement modifier 'with fill' for 'order by' clause --- dbms/src/Core/SortDescription.h | 8 +- .../DataStreams/FillingBlockInputStream.cpp | 298 +++++++++++------- .../src/DataStreams/FillingBlockInputStream.h | 41 ++- .../Interpreters/InterpreterSelectQuery.cpp | 75 +++-- dbms/src/Parsers/ExpressionElementParsers.cpp | 6 +- 5 files changed, 264 insertions(+), 164 deletions(-) diff --git a/dbms/src/Core/SortDescription.h b/dbms/src/Core/SortDescription.h index 8a30e98203c..e1ec142f645 100644 --- a/dbms/src/Core/SortDescription.h +++ b/dbms/src/Core/SortDescription.h @@ -13,11 +13,11 @@ namespace DB struct FillColumnDescription { - /// All missed values in range [FROM, TO] will be filled - /// Range [FROM, TO] respects sorting direction + /// All missed values in range [FROM, TO) will be filled + /// Range [FROM, TO) respects sorting direction Field fill_from; /// Fill value >= FILL_FROM - Field fill_to; /// Fill value + STEP <= FILL_TO - Field fill_step; /// Default = 1 + Field fill_to; /// Fill value + STEP < FILL_TO + Field fill_step; /// Default = 1 or -1 according to direction }; /// Description of the sorting rule by one column. diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index ffb7cffba0e..3f4e109e644 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -1,176 +1,238 @@ #include "FillingBlockInputStream.h" +#include namespace DB { namespace ErrorCodes { -extern const int INVALID_WITH_FILL_EXPRESSION; + extern const int INVALID_WITH_FILL_EXPRESSION; } -namespace detail +static bool less(const Field & lhs, const Field & rhs, int direction) { + if (direction == -1) + return applyVisitor(FieldVisitorAccurateLess(), rhs, lhs); -ColumnRawPtrs getColumnsExcept(SharedBlockPtr & block_ptr, ColumnRawPtrs & except_columns) + return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); +} + +static bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); } + +FillingRow::FillingRow(const SortDescription & description_) : description(description_) { - ColumnRawPtrs res; - res.reserve(block_ptr->columns() - except_columns.size()); + row.resize(description.size()); +} - for (size_t i = 0; i < block_ptr->columns(); ++i) +bool FillingRow::next(const FillingRow & to_row) +{ + size_t pos = 0; + for (; pos < row.size(); ++pos) + if (!row[pos].isNull() && !to_row[pos].isNull() && !equals(row[pos], to_row[pos])) + break; + + if (pos == row.size() || less(to_row[pos], row[pos], getDirection(pos))) + return false; + + for (size_t i = row.size() - 1; i > pos; --i) { - const IColumn * raw_col = block_ptr->safeGetByPosition(i).column.get(); - if (std::find(except_columns.begin(), except_columns.end(), raw_col) == except_columns.end()) - res.emplace_back(raw_col); + if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) + continue; + + auto next_value = row[i]; + applyVisitor(FieldVisitorSum(getFillDescription(i).fill_step), next_value); + if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) + { + initFromDefaults(i + 1); + row[i] = next_value; + return true; + } } - return res; -} + auto next_value = row[pos]; + applyVisitor(FieldVisitorSum(getFillDescription(pos).fill_step), next_value); -void copyRowFromColumns(ColumnRawPtrs & from, ColumnRawPtrs & to, size_t row_num) -{ - for (size_t i = 0; i < from.size(); ++i) - const_cast(to[i])->insertFrom(*from[i], row_num); -} - -void fillRestOfRow( - size_t cols_copied, ColumnRawPtrs & res_fill_columns, ColumnRawPtrs & res_rest_columns, - ColumnRawPtrs & old_rest_columns, UInt64 & next_row_num) -{ - /// step_val was inserted, fill all other columns with default values - if (cols_copied < res_fill_columns.size()) + if (equals(next_value, to_row[pos])) { - for (; cols_copied < res_fill_columns.size(); ++cols_copied) - const_cast(res_fill_columns[cols_copied])->insertDefault(); - for (size_t it = 0; it < res_rest_columns.size(); ++it) - const_cast(res_rest_columns[it])->insertDefault(); + bool is_less = false; + for (size_t i = pos + 1; i < row.size(); ++i) + { + const auto & fill_from = getFillDescription(i).fill_from; + if (!fill_from.isNull() && less(fill_from, to_row[i], getDirection(i))) + { + is_less = true; + row[i] = fill_from; + } + else + row[i] = to_row[i]; + } - return; + row[pos] = next_value; + return is_less; } - /// fill row wasn't created, copy rest values from row - detail::copyRowFromColumns(old_rest_columns, res_rest_columns, next_row_num); - ++next_row_num; + if (less(next_value, to_row[pos], getDirection(pos))) + { + initFromDefaults(pos + 1); + row[pos] = next_value; + return true; + } + + return false; } -Field sumTwoFields(Field & a, Field & b) +void FillingRow::initFromColumns(const Columns & columns, size_t row_num, size_t from_pos, bool ignore_default_from) { - switch (a.getType()) + for (size_t i = from_pos; i < columns.size(); ++i) { - case Field::Types::Null: return a; - case Field::Types::UInt64: return a.get() + b.get(); - case Field::Types::Int64: return a.get() + b.get(); - case Field::Types::Int128: return a.get() + b.get(); - case Field::Types::Float64: return a.get() + b.get(); - default: - throw Exception("WITH FILL can be used only with numeric types", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + columns[i]->get(row_num, row[i]); + const auto & fill_from = getFillDescription(i).fill_from; + if (!ignore_default_from && !fill_from.isNull() && less(fill_from, row[i], getDirection(i))) + row[i] = fill_from; } } +void FillingRow::initFromDefaults(size_t from_pos) +{ + for (size_t i = from_pos; i < row.size(); ++i) + row[i] = getFillDescription(i).fill_from; } + +static void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row) +{ + for (size_t i = 0; i < filling_columns.size(); ++i) + { + if (filling_row[i].isNull()) + filling_columns[i]->insertDefault(); + else + filling_columns[i]->insert(filling_row[i]); + } + + for (size_t i = 0; i < other_columns.size(); ++i) + other_columns[i]->insertDefault(); + +} + +static void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num) +{ + for (size_t i = 0; i < source.size(); ++i) + dest[i]->insertFrom(*source[i], row_num); +} + + FillingBlockInputStream::FillingBlockInputStream( const BlockInputStreamPtr & input, const SortDescription & fill_description_) - : fill_description(fill_description_) + : fill_description(fill_description_), filling_row(fill_description_), next_row(fill_description_) { children.push_back(input); + header = children.at(0)->getHeader(); + + std::vector is_fill_column(header.columns()); + for (const auto & elem : fill_description) + { + size_t pos = header.getPositionByName(elem.column_name); + fill_column_positions.push_back(pos); + is_fill_column[pos] = true; + } + + for (size_t i = 0; i < header.columns(); ++i) + { + if (is_fill_column[i]) + { + if (!isNumber(header.getByPosition(i).type)) + throw Exception("WITH FILL can be used only with numeric types, but is set for column with type " + + header.getByPosition(i).type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } + else + other_column_positions.push_back(i); + } } Block FillingBlockInputStream::readImpl() { - Block old_block; - UInt64 rows = 0; + Columns old_fill_columns; + Columns old_other_columns; + MutableColumns res_fill_columns; + MutableColumns res_other_columns; - old_block = children.back()->read(); - if (!old_block) - return old_block; - - Block res_block = old_block.cloneEmpty(); - - rows = old_block.rows(); - - SharedBlockPtr old_block_ptr = new detail::SharedBlock(std::move(old_block)); - ColumnRawPtrs old_fill_columns = SharedBlockRowRef::getBlockColumns(*old_block_ptr, fill_description); - ColumnRawPtrs old_rest_columns = detail::getColumnsExcept(old_block_ptr, old_fill_columns); - - SharedBlockPtr res_block_ptr = new detail::SharedBlock(std::move(res_block)); - ColumnRawPtrs res_fill_columns = SharedBlockRowRef::getBlockColumns(*res_block_ptr, fill_description); - ColumnRawPtrs res_rest_columns = detail::getColumnsExcept(res_block_ptr, res_fill_columns); - - /// number of next row in current block - UInt64 next_row_num = 0; - - /// read first block - if (!pos) + auto init_columns_by_positions = [](const Block & block, Columns & columns, + MutableColumns & mutable_columns, const Positions & positions) { - ++next_row_num; - /// create row number 0 in result block here - detail::copyRowFromColumns(old_fill_columns, res_fill_columns, 0); - detail::copyRowFromColumns(old_rest_columns, res_rest_columns, 0); - } - - /// current block is not first, need to compare with row in other block - if (!next_row_num) - { - size_t cnt_cols = 0; - size_t fill_columns_size = old_fill_columns.size(); - for (; cnt_cols < fill_columns_size; ++cnt_cols) + for (size_t pos : positions) { - Field step = fill_description[cnt_cols].fill_description.fill_step; - Field next_val; - Field prev_val; - old_fill_columns[cnt_cols]->get(next_row_num, next_val); - (*last_row_ref.columns)[cnt_cols]->get(last_row_ref.row_num, prev_val); - Field step_val = detail::sumTwoFields(prev_val, step); - if (step_val >= next_val) - const_cast(res_fill_columns[cnt_cols])->insertFrom( - *old_fill_columns[cnt_cols], next_row_num); - else - { - const_cast(res_fill_columns[cnt_cols])->insert(step_val); - break; - } + auto column = block.getByPosition(pos).column; + columns.push_back(column); + mutable_columns.push_back(column->cloneEmpty()->assumeMutable()); } - /// create row number 0 in result block here - detail::fillRestOfRow(cnt_cols, res_fill_columns, res_rest_columns, old_rest_columns, next_row_num); - ++pos; + }; + + auto block = children.back()->read(); + if (!block) + { + init_columns_by_positions(header, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(header, old_other_columns, res_other_columns, other_column_positions); + + bool generated = false; + for (size_t i = 0; i < filling_row.size(); ++i) + next_row[i] = filling_row.getFillDescription(i).fill_to; + + while (filling_row.next(next_row)) + { + generated = true; + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + } + + if (generated) + return createResultBlock(res_fill_columns, res_other_columns); + + return block; } - /// number of last added row in result block - UInt64 last_row_num = 0; + size_t rows = block.rows(); + init_columns_by_positions(block, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(block, old_other_columns, res_other_columns, other_column_positions); - while (next_row_num < rows) + if (first) { - size_t cnt_cols = 0; - size_t fill_columns_size = old_fill_columns.size(); - for (; cnt_cols < fill_columns_size; ++cnt_cols) + filling_row.initFromColumns(old_fill_columns, 0); + for (size_t i = 0; i < filling_row.size(); ++i) { - Field step = fill_description[cnt_cols].fill_description.fill_step; - Field prev_val; - res_fill_columns[cnt_cols]->get(last_row_num, prev_val); - Field step_val = detail::sumTwoFields(prev_val, step); - Field next_val; - old_fill_columns[cnt_cols]->get(next_row_num, next_val); - if (step_val >= next_val) - const_cast(res_fill_columns[cnt_cols])->insertFrom( - *old_fill_columns[cnt_cols], next_row_num); - else + if (less(filling_row[i], (*old_fill_columns[i])[0], filling_row.getDirection(i))) { - const_cast(res_fill_columns[cnt_cols])->insert(step_val); + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); break; } } - /// create new row in result block, increment last_row_num - detail::fillRestOfRow(cnt_cols, res_fill_columns, res_rest_columns, old_rest_columns, next_row_num); - ++last_row_num; - ++pos; + first = false; } - /// finished current block, need to remember last row - SharedBlockRowRef::setSharedBlockRowRef(last_row_ref, res_block_ptr, & res_fill_columns, last_row_num); - return *res_block_ptr; + for (size_t row_ind = 0; row_ind < rows; ++row_ind) + { + next_row.initFromColumns(old_fill_columns, row_ind, 0, true); + + /// Comment + while (filling_row.next(next_row)) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + + copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); + copyRowFromColumns(res_other_columns, old_other_columns, row_ind); + } + + return createResultBlock(res_fill_columns, res_other_columns); } +Block FillingBlockInputStream::createResultBlock(MutableColumns & fill_columns, MutableColumns & other_columns) const +{ + MutableColumns result_columns(header.columns()); + for (size_t i = 0; i < fill_columns.size(); ++i) + result_columns[fill_column_positions[i]] = std::move(fill_columns[i]); + for (size_t i = 0; i < other_columns.size(); ++i) + result_columns[other_column_positions[i]] = std::move(other_columns[i]); -} \ No newline at end of file + return header.cloneWithColumns(std::move(result_columns)); +} + +} diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index 1e4cd5bdb6a..b847878d2f2 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -1,11 +1,31 @@ #pragma once #include -#include namespace DB { +class FillingRow +{ +public: + FillingRow(const SortDescription & sort_description); + + bool next(const FillingRow & to_row); + void initFromColumns(const Columns & columns, size_t row_num, size_t from_pos = 0, bool ignore_default_from = false); + void initFromDefaults(size_t from_pos = 0); + + Field & operator[](size_t ind) { return row[ind]; } + const Field & operator[](size_t ind) const { return row[ind]; } + size_t size() const { return row.size(); } + + int getDirection(size_t ind) { return description[ind].direction; } + const FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } + +private: + std::vector row; + SortDescription description; +}; + /** Implements the WITH FILL part of ORDER BY operation. */ class FillingBlockInputStream : public IBlockInputStream @@ -13,18 +33,25 @@ class FillingBlockInputStream : public IBlockInputStream public: FillingBlockInputStream(const BlockInputStreamPtr & input, const SortDescription & fill_description_); - String getName() const override { return "WithFill"; } + String getName() const override { return "Filling"; } - Block getHeader() const override { return children.at(0)->getHeader(); } + Block getHeader() const override { return header; } protected: Block readImpl() override; private: - UInt64 pos = 0; /// total number of read rows + Block createResultBlock(MutableColumns & fill_columns, MutableColumns & other_columns) const; + const SortDescription fill_description; /// contains only rows with WITH_FILL - SharedBlockRowRef last_row_ref; /// ref to last written row + FillingRow filling_row; + FillingRow next_row; + Block header; + + using Positions = std::vector; + Positions fill_column_positions; + Positions other_column_positions; + bool first = true; }; - -} \ No newline at end of file +} diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index fd1197fe347..cece47d05e2 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -58,6 +58,7 @@ #include #include #include +#include #include #include #include @@ -102,7 +103,6 @@ namespace ErrorCodes extern const int ARGUMENT_OUT_OF_BOUND; extern const int INVALID_LIMIT_EXPRESSION; extern const int INVALID_WITH_FILL_EXPRESSION; - extern const int FILL_STEP_ZERO_VALUE; } namespace @@ -614,20 +614,47 @@ static Field getWithFillFieldValue(const ASTPtr & node, const Context & context) return field; } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement &node, const Context &context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, const Context &context) { FillColumnDescription descr; - if (node.fill_from) - descr.fill_from = getWithFillFieldValue(node.fill_from, context); - if (node.fill_to) - descr.fill_to = getWithFillFieldValue(node.fill_to, context); - if (node.fill_step) - descr.fill_step = getWithFillFieldValue(node.fill_step, context); + if (order_by_elem.fill_from) + descr.fill_from = getWithFillFieldValue(order_by_elem.fill_from, context); + if (order_by_elem.fill_to) + descr.fill_to = getWithFillFieldValue(order_by_elem.fill_to, context); + if (order_by_elem.fill_step) + descr.fill_step = getWithFillFieldValue(order_by_elem.fill_step, context); else - descr.fill_step = 1; + descr.fill_step = order_by_elem.direction; - if (descr.fill_step == 0) - throw Exception("STEP value can not be zero", ErrorCodes::FILL_STEP_ZERO_VALUE); + if (applyVisitor(FieldVisitorAccurateEquals(), descr.fill_step, Field{0})) + throw Exception("WITH FILL STEP value cannot be zero", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + if (order_by_elem.direction == 1) + { + if (applyVisitor(FieldVisitorAccurateLess(), descr.fill_step, Field{0})) + throw Exception("WITH FILL STEP value cannot be negative for sorting in ascending direction", + ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + if (!descr.fill_from.isNull() && !descr.fill_to.isNull() && + applyVisitor(FieldVisitorAccurateLess(), descr.fill_to, descr.fill_from)) + { + throw Exception("WITH FILL TO value cannot be less than FROM value for sorting in ascending direction", + ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } + } + else + { + if (applyVisitor(FieldVisitorAccurateLess(), Field{0}, descr.fill_step)) + throw Exception("WITH FILL STEP value cannot be positive for sorting in descending direction", + ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + if (!descr.fill_from.isNull() && !descr.fill_to.isNull() && + applyVisitor(FieldVisitorAccurateLess(), descr.fill_from, descr.fill_to)) + { + throw Exception("WITH FILL FROM value cannot be less than TO value for sorting in descending direction", + ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } + } return descr; } @@ -648,22 +675,8 @@ static SortDescription getSortDescription(const ASTSelectQuery & query, const Co if (order_by_elem.with_fill) { FillColumnDescription fill_desc = getWithFillDescription(order_by_elem, context); - - if (order_by_elem.direction == -1) - { - /// if DESC, then STEP < 0, FROM > TO - if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from < fill_desc.fill_to) - std::swap(fill_desc.fill_from, fill_desc.fill_to); - } - else - { - /// if ASC, then STEP > 0, FROM < TO - if (!fill_desc.fill_from.isNull() && !fill_desc.fill_to.isNull() && fill_desc.fill_from > fill_desc.fill_to) - std::swap(fill_desc.fill_from, fill_desc.fill_to); - } - - order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator, - true, fill_desc); + order_descr.emplace_back(name, order_by_elem.direction, + order_by_elem.nulls_direction, collator, true, fill_desc); } else order_descr.emplace_back(name, order_by_elem.direction, order_by_elem.nulls_direction, collator); @@ -1055,8 +1068,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeLimitBy(pipeline); } - executeWithFill(pipeline); - if (query.limitLength()) executePreLimit(pipeline); } @@ -1180,6 +1191,8 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS executeLimitBy(pipeline); } + executeWithFill(pipeline); + /** We must do projection after DISTINCT because projection may remove some columns. */ executeProjection(pipeline, expressions.final_projection); @@ -1188,8 +1201,6 @@ void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputS */ executeExtremes(pipeline); - executeWithFill(pipeline); - executeLimit(pipeline); } } @@ -2389,7 +2400,7 @@ void InterpreterSelectQuery::executeWithFill(QueryPipeline &) { throw Exception("Unsupported WITH FILL with processors", ErrorCodes::NOT_IMPLEMENTED); } - + void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline) { diff --git a/dbms/src/Parsers/ExpressionElementParsers.cpp b/dbms/src/Parsers/ExpressionElementParsers.cpp index 435140f25ba..8f0f7fffce2 100644 --- a/dbms/src/Parsers/ExpressionElementParsers.cpp +++ b/dbms/src/Parsers/ExpressionElementParsers.cpp @@ -1401,13 +1401,13 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect } /// WITH FILL [FROM x] [TO y] [STEP z] - bool with_fill_val = false; + bool has_with_fill = false; ASTPtr fill_from; ASTPtr fill_to; ASTPtr fill_step; if (with_fill.ignore(pos)) { - with_fill_val = true; + has_with_fill = true; if (from.ignore(pos) && !exp_parser.parse(pos, fill_from, expected)) return false; @@ -1420,7 +1420,7 @@ bool ParserOrderByElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expect node = std::make_shared( direction, nulls_direction, nulls_direction_was_explicitly_specified, locale_node, - with_fill_val, fill_from, fill_to, fill_step); + has_with_fill, fill_from, fill_to, fill_step); node->children.push_back(expr_elem); if (locale_node) node->children.push_back(locale_node); From 9a29c218fff83d47b96752113b73f36ca25c936d Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 21 Aug 2019 19:18:43 +0300 Subject: [PATCH 28/75] improvents and fixes of modifier 'with fill' for 'order by' clause --- dbms/src/Common/ErrorCodes.cpp | 1 - .../DataStreams/FillingBlockInputStream.cpp | 57 ++++++++++++++----- .../src/DataStreams/FillingBlockInputStream.h | 11 ++-- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- 4 files changed, 50 insertions(+), 21 deletions(-) diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 1856edee08e..e23f555663d 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -445,7 +445,6 @@ namespace ErrorCodes extern const int CANNOT_PTHREAD_ATTR = 468; extern const int WITH_TIES_WITHOUT_ORDER_BY = 469; extern const int INVALID_WITH_FILL_EXPRESSION = 470; - extern const int FILL_STEP_ZERO_VALUE = 471; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index 3f4e109e644..aefcb4bf982 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -9,6 +9,7 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; } +/// Compares fields in terms of sorting order, considering direction. static bool less(const Field & lhs, const Field & rhs, int direction) { if (direction == -1) @@ -21,12 +22,27 @@ static bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(F FillingRow::FillingRow(const SortDescription & description_) : description(description_) { + for (size_t i = 0; i < description.size(); ++i) + { + auto & fill_from = description[i].fill_description.fill_from; + auto & fill_to = description[i].fill_description.fill_to; + + /// Cast fields to same types. Otherwise, there will be troubles, when we reach zero, while generating rows. + if (fill_to.getType() == Field::Types::Int64 && fill_from.getType() == Field::Types::UInt64) + fill_from = fill_from.get(); + + if (fill_from.getType() == Field::Types::Int64 && fill_to.getType() == Field::Types::UInt64) + fill_to = fill_to.get(); + + } row.resize(description.size()); } bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; + + /// Find position we need to increment for generating next row. for (; pos < row.size(); ++pos) if (!row[pos].isNull() && !to_row[pos].isNull() && !equals(row[pos], to_row[pos])) break; @@ -34,6 +50,8 @@ bool FillingRow::next(const FillingRow & to_row) if (pos == row.size() || less(to_row[pos], row[pos], getDirection(pos))) return false; + /// If we have any 'fill_to' value at position greater than 'pos', + /// we need to generate rows up to 'fill_to' value. for (size_t i = row.size() - 1; i > pos; --i) { if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) @@ -58,10 +76,11 @@ bool FillingRow::next(const FillingRow & to_row) for (size_t i = pos + 1; i < row.size(); ++i) { const auto & fill_from = getFillDescription(i).fill_from; - if (!fill_from.isNull() && less(fill_from, to_row[i], getDirection(i))) + if (!fill_from.isNull() && !to_row[i].isNull() && less(fill_from, to_row[i], getDirection(i))) { is_less = true; - row[i] = fill_from; + initFromDefaults(i); + break; } else row[i] = to_row[i]; @@ -81,15 +100,10 @@ bool FillingRow::next(const FillingRow & to_row) return false; } -void FillingRow::initFromColumns(const Columns & columns, size_t row_num, size_t from_pos, bool ignore_default_from) +void FillingRow::initFromColumns(const Columns & columns, size_t row_num, size_t from_pos) { for (size_t i = from_pos; i < columns.size(); ++i) - { columns[i]->get(row_num, row[i]); - const auto & fill_from = getFillDescription(i).fill_from; - if (!ignore_default_from && !fill_from.isNull() && less(fill_from, row[i], getDirection(i))) - row[i] = fill_from; - } } void FillingRow::initFromDefaults(size_t from_pos) @@ -122,14 +136,14 @@ static void copyRowFromColumns(MutableColumns & dest, const Columns & source, si FillingBlockInputStream::FillingBlockInputStream( - const BlockInputStreamPtr & input, const SortDescription & fill_description_) - : fill_description(fill_description_), filling_row(fill_description_), next_row(fill_description_) + const BlockInputStreamPtr & input, const SortDescription & sort_description_) + : sort_description(sort_description_), filling_row(sort_description_), next_row(sort_description_) { children.push_back(input); header = children.at(0)->getHeader(); std::vector is_fill_column(header.columns()); - for (const auto & elem : fill_description) + for (const auto & elem : sort_description) { size_t pos = header.getPositionByName(elem.column_name); fill_column_positions.push_back(pos); @@ -140,9 +154,20 @@ FillingBlockInputStream::FillingBlockInputStream( { if (is_fill_column[i]) { - if (!isNumber(header.getByPosition(i).type)) + auto type = header.getByPosition(i).type; + if (!isColumnedAsNumber(header.getByPosition(i).type)) throw Exception("WITH FILL can be used only with numeric types, but is set for column with type " + header.getByPosition(i).type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + const auto & fill_from = sort_description[i].fill_description.fill_from; + const auto & fill_to = sort_description[i].fill_description.fill_to; + if (type->isValueRepresentedByUnsignedInteger() && + ((!fill_from.isNull() && less(fill_from, Field{0}, 1)) || + (!fill_to.isNull() && less(fill_to, Field{0}, 1)))) + { + throw Exception("WITH FILL bound values cannot be negative for unsigned type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } } else other_column_positions.push_back(i); @@ -199,8 +224,10 @@ Block FillingBlockInputStream::readImpl() filling_row.initFromColumns(old_fill_columns, 0); for (size_t i = 0; i < filling_row.size(); ++i) { - if (less(filling_row[i], (*old_fill_columns[i])[0], filling_row.getDirection(i))) + if (!filling_row.getFillDescription(i).fill_from.isNull() && + less(filling_row.getFillDescription(i).fill_from, (*old_fill_columns[i])[0], filling_row.getDirection(i))) { + filling_row.initFromDefaults(i); insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); break; } @@ -211,9 +238,9 @@ Block FillingBlockInputStream::readImpl() for (size_t row_ind = 0; row_ind < rows; ++row_ind) { - next_row.initFromColumns(old_fill_columns, row_ind, 0, true); + next_row.initFromColumns(old_fill_columns, row_ind); - /// Comment + /// Insert generated filling row to block, while it is less than current row in block. while (filling_row.next(next_row)) insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index b847878d2f2..5097fe5c6b8 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -10,8 +10,11 @@ class FillingRow public: FillingRow(const SortDescription & sort_description); + /// Generates next row according to fill from, to and step values. + /// Returns true, if generated row less than to_row in terms of sorting order. bool next(const FillingRow & to_row); - void initFromColumns(const Columns & columns, size_t row_num, size_t from_pos = 0, bool ignore_default_from = false); + + void initFromColumns(const Columns & columns, size_t row_num, size_t from_pos = 0); void initFromDefaults(size_t from_pos = 0); Field & operator[](size_t ind) { return row[ind]; } @@ -43,9 +46,9 @@ protected: private: Block createResultBlock(MutableColumns & fill_columns, MutableColumns & other_columns) const; - const SortDescription fill_description; /// contains only rows with WITH_FILL - FillingRow filling_row; - FillingRow next_row; + const SortDescription sort_description; /// Contains only rows with WITH FILL. + FillingRow filling_row; /// Current row, which is used to fill gaps. + FillingRow next_row; /// Row to which we need to generate filling rows. Block header; using Positions = std::vector; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index cece47d05e2..9cc4a9b4bb4 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -608,7 +608,7 @@ static Field getWithFillFieldValue(const ASTPtr & node, const Context & context) { const auto & [field, type] = evaluateConstantExpression(node, context); - if (!isNumber(type)) + if (!isColumnedAsNumber(type)) throw Exception("Illegal type " + type->getName() + " of WITH FILL expression, must be numeric type", ErrorCodes::INVALID_WITH_FILL_EXPRESSION); return field; From 2aa4243c25598830ade1822ebc14b1b312c54dd8 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 22 Aug 2019 17:09:56 +0300 Subject: [PATCH 29/75] add test for modifier 'with fill' --- .../00995_order_by_with_fill.reference | 710 ++++++++++++++++++ .../0_stateless/00995_order_by_with_fill.sql | 46 ++ 2 files changed, 756 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference create mode 100644 dbms/tests/queries/0_stateless/00995_order_by_with_fill.sql diff --git a/dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference b/dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference new file mode 100644 index 00000000000..df73b0d1bc3 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference @@ -0,0 +1,710 @@ +*** table without fill to compare *** +2019-05-07 18 prh +2019-05-07 26 2ke +2019-05-08 28 otf +2019-05-09 25 798 +2019-05-10 1 myj +2019-05-10 16 vp7 +2019-05-11 18 3s2 +2019-05-15 27 enb +2019-05-19 20 yfh +2019-05-23 15 01v +2019-05-23 29 72y +2019-05-24 13 sd0 +2019-05-25 17 0ei +2019-05-30 18 3kd +2019-06-04 5 6az +*** date WITH FILL, val *** +2019-05-07 18 prh +2019-05-07 26 2ke +2019-05-08 28 otf +2019-05-09 25 798 +2019-05-10 1 myj +2019-05-10 16 vp7 +2019-05-11 18 3s2 +2019-05-12 0 +2019-05-13 0 +2019-05-14 0 +2019-05-15 27 enb +2019-05-16 0 +2019-05-17 0 +2019-05-18 0 +2019-05-19 20 yfh +2019-05-20 0 +2019-05-21 0 +2019-05-22 0 +2019-05-23 15 01v +2019-05-23 29 72y +2019-05-24 13 sd0 +2019-05-25 17 0ei +2019-05-26 0 +2019-05-27 0 +2019-05-28 0 +2019-05-29 0 +2019-05-30 18 3kd +2019-05-31 0 +2019-06-01 0 +2019-06-02 0 +2019-06-03 0 +2019-06-04 5 6az +*** date WITH FILL FROM 2019-05-01 TO 2019-05-31, val WITH FILL *** +2019-05-01 0 +2019-05-02 0 +2019-05-03 0 +2019-05-04 0 +2019-05-05 0 +2019-05-06 0 +2019-05-07 18 prh +2019-05-07 19 +2019-05-07 20 +2019-05-07 21 +2019-05-07 22 +2019-05-07 23 +2019-05-07 24 +2019-05-07 25 +2019-05-07 26 2ke +2019-05-08 28 otf +2019-05-09 25 798 +2019-05-10 1 myj +2019-05-10 2 +2019-05-10 3 +2019-05-10 4 +2019-05-10 5 +2019-05-10 6 +2019-05-10 7 +2019-05-10 8 +2019-05-10 9 +2019-05-10 10 +2019-05-10 11 +2019-05-10 12 +2019-05-10 13 +2019-05-10 14 +2019-05-10 15 +2019-05-10 16 vp7 +2019-05-11 18 3s2 +2019-05-12 0 +2019-05-13 0 +2019-05-14 0 +2019-05-15 27 enb +2019-05-16 0 +2019-05-17 0 +2019-05-18 0 +2019-05-19 20 yfh +2019-05-20 0 +2019-05-21 0 +2019-05-22 0 +2019-05-23 15 01v +2019-05-23 16 +2019-05-23 17 +2019-05-23 18 +2019-05-23 19 +2019-05-23 20 +2019-05-23 21 +2019-05-23 22 +2019-05-23 23 +2019-05-23 24 +2019-05-23 25 +2019-05-23 26 +2019-05-23 27 +2019-05-23 28 +2019-05-23 29 72y +2019-05-24 13 sd0 +2019-05-25 17 0ei +2019-05-26 0 +2019-05-27 0 +2019-05-28 0 +2019-05-29 0 +2019-05-30 18 3kd +2019-05-31 0 +2019-06-01 0 +2019-06-02 0 +2019-06-03 0 +2019-06-04 5 6az +*** date DESC WITH FILL, val WITH FILL FROM 1 TO 6 *** +2019-06-04 1 +2019-06-04 2 +2019-06-04 3 +2019-06-04 4 +2019-06-04 5 6az +2019-06-03 1 +2019-06-03 2 +2019-06-03 3 +2019-06-03 4 +2019-06-03 5 +2019-06-02 1 +2019-06-02 2 +2019-06-02 3 +2019-06-02 4 +2019-06-02 5 +2019-06-01 1 +2019-06-01 2 +2019-06-01 3 +2019-06-01 4 +2019-06-01 5 +2019-05-31 1 +2019-05-31 2 +2019-05-31 3 +2019-05-31 4 +2019-05-31 5 +2019-05-30 1 +2019-05-30 2 +2019-05-30 3 +2019-05-30 4 +2019-05-30 5 +2019-05-30 6 +2019-05-30 7 +2019-05-30 8 +2019-05-30 9 +2019-05-30 10 +2019-05-30 11 +2019-05-30 12 +2019-05-30 13 +2019-05-30 14 +2019-05-30 15 +2019-05-30 16 +2019-05-30 17 +2019-05-30 18 3kd +2019-05-29 1 +2019-05-29 2 +2019-05-29 3 +2019-05-29 4 +2019-05-29 5 +2019-05-28 1 +2019-05-28 2 +2019-05-28 3 +2019-05-28 4 +2019-05-28 5 +2019-05-27 1 +2019-05-27 2 +2019-05-27 3 +2019-05-27 4 +2019-05-27 5 +2019-05-26 1 +2019-05-26 2 +2019-05-26 3 +2019-05-26 4 +2019-05-26 5 +2019-05-25 1 +2019-05-25 2 +2019-05-25 3 +2019-05-25 4 +2019-05-25 5 +2019-05-25 6 +2019-05-25 7 +2019-05-25 8 +2019-05-25 9 +2019-05-25 10 +2019-05-25 11 +2019-05-25 12 +2019-05-25 13 +2019-05-25 14 +2019-05-25 15 +2019-05-25 16 +2019-05-25 17 0ei +2019-05-24 1 +2019-05-24 2 +2019-05-24 3 +2019-05-24 4 +2019-05-24 5 +2019-05-24 6 +2019-05-24 7 +2019-05-24 8 +2019-05-24 9 +2019-05-24 10 +2019-05-24 11 +2019-05-24 12 +2019-05-24 13 sd0 +2019-05-23 1 +2019-05-23 2 +2019-05-23 3 +2019-05-23 4 +2019-05-23 5 +2019-05-23 6 +2019-05-23 7 +2019-05-23 8 +2019-05-23 9 +2019-05-23 10 +2019-05-23 11 +2019-05-23 12 +2019-05-23 13 +2019-05-23 14 +2019-05-23 15 01v +2019-05-23 16 +2019-05-23 17 +2019-05-23 18 +2019-05-23 19 +2019-05-23 20 +2019-05-23 21 +2019-05-23 22 +2019-05-23 23 +2019-05-23 24 +2019-05-23 25 +2019-05-23 26 +2019-05-23 27 +2019-05-23 28 +2019-05-23 29 72y +2019-05-22 1 +2019-05-22 2 +2019-05-22 3 +2019-05-22 4 +2019-05-22 5 +2019-05-21 1 +2019-05-21 2 +2019-05-21 3 +2019-05-21 4 +2019-05-21 5 +2019-05-20 1 +2019-05-20 2 +2019-05-20 3 +2019-05-20 4 +2019-05-20 5 +2019-05-19 1 +2019-05-19 2 +2019-05-19 3 +2019-05-19 4 +2019-05-19 5 +2019-05-19 6 +2019-05-19 7 +2019-05-19 8 +2019-05-19 9 +2019-05-19 10 +2019-05-19 11 +2019-05-19 12 +2019-05-19 13 +2019-05-19 14 +2019-05-19 15 +2019-05-19 16 +2019-05-19 17 +2019-05-19 18 +2019-05-19 19 +2019-05-19 20 yfh +2019-05-18 1 +2019-05-18 2 +2019-05-18 3 +2019-05-18 4 +2019-05-18 5 +2019-05-17 1 +2019-05-17 2 +2019-05-17 3 +2019-05-17 4 +2019-05-17 5 +2019-05-16 1 +2019-05-16 2 +2019-05-16 3 +2019-05-16 4 +2019-05-16 5 +2019-05-15 1 +2019-05-15 2 +2019-05-15 3 +2019-05-15 4 +2019-05-15 5 +2019-05-15 6 +2019-05-15 7 +2019-05-15 8 +2019-05-15 9 +2019-05-15 10 +2019-05-15 11 +2019-05-15 12 +2019-05-15 13 +2019-05-15 14 +2019-05-15 15 +2019-05-15 16 +2019-05-15 17 +2019-05-15 18 +2019-05-15 19 +2019-05-15 20 +2019-05-15 21 +2019-05-15 22 +2019-05-15 23 +2019-05-15 24 +2019-05-15 25 +2019-05-15 26 +2019-05-15 27 enb +2019-05-14 1 +2019-05-14 2 +2019-05-14 3 +2019-05-14 4 +2019-05-14 5 +2019-05-13 1 +2019-05-13 2 +2019-05-13 3 +2019-05-13 4 +2019-05-13 5 +2019-05-12 1 +2019-05-12 2 +2019-05-12 3 +2019-05-12 4 +2019-05-12 5 +2019-05-11 1 +2019-05-11 2 +2019-05-11 3 +2019-05-11 4 +2019-05-11 5 +2019-05-11 6 +2019-05-11 7 +2019-05-11 8 +2019-05-11 9 +2019-05-11 10 +2019-05-11 11 +2019-05-11 12 +2019-05-11 13 +2019-05-11 14 +2019-05-11 15 +2019-05-11 16 +2019-05-11 17 +2019-05-11 18 3s2 +2019-05-10 1 myj +2019-05-10 2 +2019-05-10 3 +2019-05-10 4 +2019-05-10 5 +2019-05-10 6 +2019-05-10 7 +2019-05-10 8 +2019-05-10 9 +2019-05-10 10 +2019-05-10 11 +2019-05-10 12 +2019-05-10 13 +2019-05-10 14 +2019-05-10 15 +2019-05-10 16 vp7 +2019-05-09 1 +2019-05-09 2 +2019-05-09 3 +2019-05-09 4 +2019-05-09 5 +2019-05-09 6 +2019-05-09 7 +2019-05-09 8 +2019-05-09 9 +2019-05-09 10 +2019-05-09 11 +2019-05-09 12 +2019-05-09 13 +2019-05-09 14 +2019-05-09 15 +2019-05-09 16 +2019-05-09 17 +2019-05-09 18 +2019-05-09 19 +2019-05-09 20 +2019-05-09 21 +2019-05-09 22 +2019-05-09 23 +2019-05-09 24 +2019-05-09 25 798 +2019-05-08 1 +2019-05-08 2 +2019-05-08 3 +2019-05-08 4 +2019-05-08 5 +2019-05-08 6 +2019-05-08 7 +2019-05-08 8 +2019-05-08 9 +2019-05-08 10 +2019-05-08 11 +2019-05-08 12 +2019-05-08 13 +2019-05-08 14 +2019-05-08 15 +2019-05-08 16 +2019-05-08 17 +2019-05-08 18 +2019-05-08 19 +2019-05-08 20 +2019-05-08 21 +2019-05-08 22 +2019-05-08 23 +2019-05-08 24 +2019-05-08 25 +2019-05-08 26 +2019-05-08 27 +2019-05-08 28 otf +2019-05-07 1 +2019-05-07 2 +2019-05-07 3 +2019-05-07 4 +2019-05-07 5 +2019-05-07 6 +2019-05-07 7 +2019-05-07 8 +2019-05-07 9 +2019-05-07 10 +2019-05-07 11 +2019-05-07 12 +2019-05-07 13 +2019-05-07 14 +2019-05-07 15 +2019-05-07 16 +2019-05-07 17 +2019-05-07 18 prh +2019-05-07 19 +2019-05-07 20 +2019-05-07 21 +2019-05-07 22 +2019-05-07 23 +2019-05-07 24 +2019-05-07 25 +2019-05-07 26 2ke +*** date DESC WITH FILL TO 2019-05-01 STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3 *** +2019-06-04 10 +2019-06-04 7 +2019-06-04 5 6az +2019-06-04 4 +2019-06-04 1 +2019-06-04 -2 +2019-06-02 10 +2019-06-02 7 +2019-06-02 4 +2019-06-02 1 +2019-06-02 -2 +2019-05-31 10 +2019-05-31 7 +2019-05-31 4 +2019-05-31 1 +2019-05-31 -2 +2019-05-30 18 3kd +2019-05-29 10 +2019-05-29 7 +2019-05-29 4 +2019-05-29 1 +2019-05-29 -2 +2019-05-27 10 +2019-05-27 7 +2019-05-27 4 +2019-05-27 1 +2019-05-27 -2 +2019-05-25 17 0ei +2019-05-25 14 +2019-05-25 11 +2019-05-25 8 +2019-05-25 5 +2019-05-25 2 +2019-05-25 -1 +2019-05-25 -4 +2019-05-24 13 sd0 +2019-05-23 29 72y +2019-05-23 26 +2019-05-23 23 +2019-05-23 20 +2019-05-23 17 +2019-05-23 15 01v +2019-05-23 14 +2019-05-23 11 +2019-05-23 8 +2019-05-23 5 +2019-05-23 2 +2019-05-23 -1 +2019-05-23 -4 +2019-05-21 10 +2019-05-21 7 +2019-05-21 4 +2019-05-21 1 +2019-05-21 -2 +2019-05-19 20 yfh +2019-05-19 17 +2019-05-19 14 +2019-05-19 11 +2019-05-19 8 +2019-05-19 5 +2019-05-19 2 +2019-05-19 -1 +2019-05-19 -4 +2019-05-17 10 +2019-05-17 7 +2019-05-17 4 +2019-05-17 1 +2019-05-17 -2 +2019-05-15 27 enb +2019-05-15 24 +2019-05-15 21 +2019-05-15 18 +2019-05-15 15 +2019-05-15 12 +2019-05-15 9 +2019-05-15 6 +2019-05-15 3 +2019-05-15 0 +2019-05-15 -3 +2019-05-13 10 +2019-05-13 7 +2019-05-13 4 +2019-05-13 1 +2019-05-13 -2 +2019-05-11 18 3s2 +2019-05-11 15 +2019-05-11 12 +2019-05-11 9 +2019-05-11 6 +2019-05-11 3 +2019-05-11 0 +2019-05-11 -3 +2019-05-10 16 vp7 +2019-05-10 1 myj +2019-05-09 25 798 +2019-05-09 22 +2019-05-09 19 +2019-05-09 16 +2019-05-09 13 +2019-05-09 10 +2019-05-09 7 +2019-05-09 4 +2019-05-09 1 +2019-05-09 -2 +2019-05-08 28 otf +2019-05-07 26 2ke +2019-05-07 23 +2019-05-07 20 +2019-05-07 18 prh +2019-05-07 17 +2019-05-07 14 +2019-05-07 11 +2019-05-07 8 +2019-05-07 5 +2019-05-07 2 +2019-05-07 -1 +2019-05-07 -4 +2019-05-05 10 +2019-05-05 7 +2019-05-05 4 +2019-05-05 1 +2019-05-05 -2 +2019-05-03 10 +2019-05-03 7 +2019-05-03 4 +2019-05-03 1 +2019-05-03 -2 +2019-05-01 10 +2019-05-01 7 +2019-05-01 4 +2019-05-01 1 +2019-05-01 -2 +*** date WITH FILL TO 2019-06-23 STEP 3, val WITH FILL FROM -10 STEP 2 +2019-05-07 -10 +2019-05-07 -8 +2019-05-07 -6 +2019-05-07 -4 +2019-05-07 -2 +2019-05-07 0 +2019-05-07 2 +2019-05-07 4 +2019-05-07 6 +2019-05-07 8 +2019-05-07 10 +2019-05-07 12 +2019-05-07 14 +2019-05-07 16 +2019-05-07 18 prh +2019-05-07 20 +2019-05-07 22 +2019-05-07 24 +2019-05-07 26 2ke +2019-05-08 28 otf +2019-05-09 25 798 +2019-05-10 -10 +2019-05-10 -8 +2019-05-10 -6 +2019-05-10 -4 +2019-05-10 -2 +2019-05-10 0 +2019-05-10 1 myj +2019-05-10 2 +2019-05-10 4 +2019-05-10 6 +2019-05-10 8 +2019-05-10 10 +2019-05-10 12 +2019-05-10 14 +2019-05-10 16 vp7 +2019-05-11 18 3s2 +2019-05-13 -10 +2019-05-15 27 enb +2019-05-16 -10 +2019-05-19 -10 +2019-05-19 -8 +2019-05-19 -6 +2019-05-19 -4 +2019-05-19 -2 +2019-05-19 0 +2019-05-19 2 +2019-05-19 4 +2019-05-19 6 +2019-05-19 8 +2019-05-19 10 +2019-05-19 12 +2019-05-19 14 +2019-05-19 16 +2019-05-19 18 +2019-05-19 20 yfh +2019-05-22 -10 +2019-05-23 15 01v +2019-05-23 29 72y +2019-05-24 13 sd0 +2019-05-25 -10 +2019-05-25 -8 +2019-05-25 -6 +2019-05-25 -4 +2019-05-25 -2 +2019-05-25 0 +2019-05-25 2 +2019-05-25 4 +2019-05-25 6 +2019-05-25 8 +2019-05-25 10 +2019-05-25 12 +2019-05-25 14 +2019-05-25 16 +2019-05-25 17 0ei +2019-05-28 -10 +2019-05-30 18 3kd +2019-05-31 -10 +2019-06-03 -10 +2019-06-04 5 6az +2019-06-06 -10 +2019-06-09 -10 +2019-06-12 -10 +2019-06-15 -10 +2019-06-18 -10 +2019-06-21 -10 +*** table without fill to compare *** +1 -2 +1 3 +3 2 +5 -1 +6 5 +8 0 +*** a WITH FILL, b WITH fill *** +1 -2 +1 -1 +1 0 +1 1 +1 2 +1 3 +2 0 +3 2 +4 0 +5 -1 +6 5 +7 0 +8 0 +*** a WITH FILL, b WITH fill TO 6 STEP 2 *** +1 -2 +1 0 +1 2 +1 3 +1 4 +2 0 +3 2 +3 4 +4 0 +5 -1 +5 1 +5 3 +5 5 +6 5 +7 0 +8 0 +8 2 +8 4 diff --git a/dbms/tests/queries/0_stateless/00995_order_by_with_fill.sql b/dbms/tests/queries/0_stateless/00995_order_by_with_fill.sql new file mode 100644 index 00000000000..e46ed4390fc --- /dev/null +++ b/dbms/tests/queries/0_stateless/00995_order_by_with_fill.sql @@ -0,0 +1,46 @@ +DROP TABLE IF EXISTS fill; +CREATE TABLE fill (date Date, val Int, str String) ENGINE = Memory; +INSERT INTO fill VALUES (toDate('2019-05-24'), 13, 'sd0')(toDate('2019-05-10'), 16, 'vp7')(toDate('2019-05-25'), 17, '0ei')(toDate('2019-05-30'), 18, '3kd')(toDate('2019-05-15'), 27, 'enb')(toDate('2019-06-04'), 5, '6az')(toDate('2019-05-23'), 15, '01v')(toDate('2019-05-08'), 28, 'otf')(toDate('2019-05-19'), 20, 'yfh')(toDate('2019-05-07'), 26, '2ke')(toDate('2019-05-07'), 18, 'prh')(toDate('2019-05-09'), 25, '798')(toDate('2019-05-10'), 1, 'myj')(toDate('2019-05-11'), 18, '3s2')(toDate('2019-05-23'), 29, '72y'); + +SELECT '*** table without fill to compare ***'; +SELECT * FROM fill ORDER BY date, val; + +-- Some useful cases + +SELECT '*** date WITH FILL, val ***'; +SELECT * FROM fill ORDER BY date WITH FILL, val; + +SELECT '*** date WITH FILL FROM 2019-05-01 TO 2019-05-31, val WITH FILL ***'; +SELECT * FROM fill ORDER BY date WITH FILL FROM toDate('2019-05-01') TO toDate('2019-05-31'), val WITH FILL; + +SELECT '*** date DESC WITH FILL, val WITH FILL FROM 1 TO 6 ***'; +SELECT * FROM fill ORDER BY date DESC WITH FILL, val WITH FILL FROM 1 TO 6; + +-- Some weird cases + +SELECT '*** date DESC WITH FILL TO 2019-05-01 STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3 ***'; +SELECT * FROM fill ORDER BY date DESC WITH FILL TO toDate('2019-05-01') STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3; + +SELECT '*** date WITH FILL TO 2019-06-23 STEP 3, val WITH FILL FROM -10 STEP 2'; +SELECT * FROM fill ORDER BY date WITH FILL TO toDate('2019-06-23') STEP 3, val WITH FILL FROM -10 STEP 2; + + +DROP TABLE fill; +CREATE TABLE fill (a UInt32, b Int32) ENGINE = Memory; +INSERT INTO fill VALUES (1, -2), (1, 3), (3, 2), (5, -1), (6, 5), (8, 0); + +SELECT '*** table without fill to compare ***'; +SELECT * FROM fill ORDER BY a, b; + +SELECT '*** a WITH FILL, b WITH fill ***'; +SELECT * FROM fill ORDER BY a WITH FILL, b WITH fill; + +SELECT '*** a WITH FILL, b WITH fill TO 6 STEP 2 ***'; +SELECT * FROM fill ORDER BY a WITH FILL, b WITH fill TO 6 STEP 2; + +SELECT * FROM fill ORDER BY a WITH FILL STEP -1; -- { serverError 470 } +SELECT * FROM fill ORDER BY a WITH FILL FROM 10 TO 1; -- { serverError 470 } +SELECT * FROM fill ORDER BY a DESC WITH FILL FROM 1 TO 10; -- { serverError 470 } +SELECT * FROM fill ORDER BY a WITH FILL FROM -10 to 10; -- { serverError 470 } + +DROP TABLE fill; From e069fc495dbaaf0225ae19aa24180e874de760f6 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 23 Aug 2019 02:16:00 +0300 Subject: [PATCH 30/75] fix limit with ties --- dbms/src/Common/SharedBlockRowRef.h | 27 +-------- .../src/DataStreams/LimitBlockInputStream.cpp | 60 +++++++++++++------ dbms/src/DataStreams/LimitBlockInputStream.h | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 10 +++- 4 files changed, 55 insertions(+), 44 deletions(-) diff --git a/dbms/src/Common/SharedBlockRowRef.h b/dbms/src/Common/SharedBlockRowRef.h index 8977fedc50e..d37bcbe193a 100644 --- a/dbms/src/Common/SharedBlockRowRef.h +++ b/dbms/src/Common/SharedBlockRowRef.h @@ -79,27 +79,8 @@ struct SharedBlockRowRef bool empty() const { return columns == nullptr; } size_t size() const { return empty() ? 0 : columns->size(); } - /// gets pointers to all columns of block, which were used for ORDER BY - static ColumnRawPtrs getBlockColumns(const Block & block, const SortDescription description) - { - size_t size = description.size(); - ColumnRawPtrs res; - res.reserve(size); - - for (size_t i = 0; i < size; ++i) - { - const IColumn * column = !description[i].column_name.empty() - ? block.getByName(description[i].column_name).column.get() - : block.safeGetByPosition(description[i].column_number).column.get(); - res.emplace_back(column); - } - - return res; - } - - - static void setSharedBlockRowRef(SharedBlockRowRef & row_ref, SharedBlockPtr & shared_block, ColumnRawPtrs * columns, - size_t row_num) + static void setSharedBlockRowRef(SharedBlockRowRef & row_ref, + SharedBlockPtr & shared_block,ColumnRawPtrs * columns, size_t row_num) { row_ref.row_num = row_num; row_ref.columns = columns; @@ -107,6 +88,4 @@ struct SharedBlockRowRef } }; - - -} +} \ No newline at end of file diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 4dbecc4dcc7..ea794425bae 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -6,6 +6,24 @@ namespace DB { +/// gets pointers to all columns of block, which were used for ORDER BY +static ColumnRawPtrs getSortColumns(const Block & block, const SortDescription & description) +{ + size_t size = description.size(); + ColumnRawPtrs res; + res.reserve(size); + + for (size_t i = 0; i < size; ++i) + { + const IColumn * column = !description[i].column_name.empty() + ? block.getByName(description[i].column_name).column.get() + : block.safeGetByPosition(description[i].column_number).column.get(); + res.emplace_back(column); + } + + return res; +} + LimitBlockInputStream::LimitBlockInputStream( const BlockInputStreamPtr & input, UInt64 limit_, UInt64 offset_, bool always_read_till_end_, @@ -21,29 +39,30 @@ LimitBlockInputStream::LimitBlockInputStream( children.push_back(input); } - Block LimitBlockInputStream::readImpl() { Block res; UInt64 rows = 0; - /// pos >= offset + limit and all rows in previous block were equal to ties_row_ref - /// so we check current block - if (with_ties && ties_row_ref.shared_block) + /// pos >= offset + limit and all rows in the end of previous block were equal + /// to last row at 'limit' position. So we check current block. + if (!ties_row_ref.empty() && pos >= offset + limit) { - rows = res.rows(); - pos += rows; res = children.back()->read(); + rows = res.rows(); + if (!res) + return res; SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); - ColumnRawPtrs columns = SharedBlockRowRef::getBlockColumns(*ptr, description); - UInt64 len; + ptr->sort_columns = getSortColumns(*ptr, description); + UInt64 len; for (len = 0; len < rows; ++len) { SharedBlockRowRef current_row; - SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &columns, len); + SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &ptr->sort_columns, len); + if (current_row != ties_row_ref) { ties_row_ref.reset(); @@ -51,7 +70,7 @@ Block LimitBlockInputStream::readImpl() } } - if (len < rows - 1) + if (len < rows) { for (size_t i = 0; i < ptr->columns(); ++i) ptr->safeGetByPosition(i).column = ptr->safeGetByPosition(i).column->cut(0, len); @@ -60,8 +79,6 @@ Block LimitBlockInputStream::readImpl() return *ptr; } - /// pos - how many lines were read, including the last read block - if (pos >= offset + limit) { if (!always_read_till_end) @@ -83,10 +100,18 @@ Block LimitBlockInputStream::readImpl() pos += rows; } while (pos <= offset); + SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); + if (with_ties) + ptr->sort_columns = getSortColumns(*ptr, description); /// give away the whole block if (pos >= offset + rows && pos <= offset + limit) - return res; + { + /// Save rowref for last row, because probalbly next block begins with the same row. + if (with_ties && pos == offset + limit) + SharedBlockRowRef::setSharedBlockRowRef(ties_row_ref, ptr, &ptr->sort_columns, rows - 1); + return *ptr; + } /// give away a piece of the block UInt64 start = std::max( @@ -98,18 +123,16 @@ Block LimitBlockInputStream::readImpl() static_cast(pos) - static_cast(offset), static_cast(limit) + static_cast(offset) - static_cast(pos) + static_cast(rows))); - SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); /// check if other rows in current block equals to last one in limit if (with_ties) { - ColumnRawPtrs columns = SharedBlockRowRef::getBlockColumns(*ptr, description); - SharedBlockRowRef::setSharedBlockRowRef(ties_row_ref, ptr, &columns, start + length - 1); + SharedBlockRowRef::setSharedBlockRowRef(ties_row_ref, ptr, &ptr->sort_columns, start + length - 1); for (size_t i = ties_row_ref.row_num + 1; i < rows; ++i) { SharedBlockRowRef current_row; - SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &columns, i); + SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &ptr->sort_columns, i); if (current_row == ties_row_ref) ++length; else @@ -120,6 +143,9 @@ Block LimitBlockInputStream::readImpl() } } + if (start + length == rows) + return *ptr; + for (size_t i = 0; i < ptr->columns(); ++i) ptr->safeGetByPosition(i).column = ptr->safeGetByPosition(i).column->cut(start, length); diff --git a/dbms/src/DataStreams/LimitBlockInputStream.h b/dbms/src/DataStreams/LimitBlockInputStream.h index 763cefc63d8..6c5f76cdaaf 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.h +++ b/dbms/src/DataStreams/LimitBlockInputStream.h @@ -18,7 +18,7 @@ public: * If always_read_till_end = true - reads all the data to the end, but ignores them. This is necessary in rare cases: * when otherwise, due to the cancellation of the request, we would not have received the data for GROUP BY WITH TOTALS from the remote server. * If use_limit_as_total_rows_approx = true, then addTotalRowsApprox is called to use the limit in progress & stats - * with_ties = true, when query has WITH TIES modifier. If so,description should be provided + * with_ties = true, when query has WITH TIES modifier. If so, description should be provided * description lets us know which row we should check for equality */ LimitBlockInputStream( diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 9cc4a9b4bb4..d94aa47fec3 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -719,7 +719,7 @@ static std::pair getLimitLengthAndOffset(const ASTSelectQuery & static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & context) { /// Partial sort can be done if there is LIMIT but no DISTINCT or LIMIT BY. - if (!query.distinct && !query.limitBy()) + if (!query.distinct && !query.limitBy() && !query.limit_with_ties) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); return limit_length + limit_offset; @@ -2238,7 +2238,13 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline) if (query.limitLength()) { auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context); - SortDescription sort_descr = getSortDescription(query, context); + SortDescription sort_descr; + if (query.limit_with_ties) + { + if (!query.orderBy()) + throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); + sort_descr = getSortDescription(query, context); + } pipeline.transform([&, limit = limit_length + limit_offset](auto & stream) { stream = std::make_shared(stream, limit, 0, false, false, query.limit_with_ties, sort_descr); From 659d3857abb682732ae6a1d6c30226c886af1da4 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 23 Aug 2019 02:48:20 +0300 Subject: [PATCH 31/75] fix build for clang --- dbms/src/Common/SharedBlockRowRef.h | 2 +- dbms/src/Interpreters/InterpreterSelectQuery.h | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dbms/src/Common/SharedBlockRowRef.h b/dbms/src/Common/SharedBlockRowRef.h index d37bcbe193a..c2cf975d2b4 100644 --- a/dbms/src/Common/SharedBlockRowRef.h +++ b/dbms/src/Common/SharedBlockRowRef.h @@ -88,4 +88,4 @@ struct SharedBlockRowRef } }; -} \ No newline at end of file +} diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index b1707db3480..d3d1bc99bfd 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -222,7 +222,7 @@ private: void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info); - void executeWithFill(QueryPipeline & pipeline); + [[noreturn]] void executeWithFill(QueryPipeline & pipeline); void executeMergeSorted(QueryPipeline & pipeline); void executePreLimit(QueryPipeline & pipeline); void executeLimitBy(QueryPipeline & pipeline); From 811dab72a8d5716b90e8fb7b8fdbdef7649416e9 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 23 Aug 2019 14:11:50 +0300 Subject: [PATCH 32/75] fix limit with offset --- dbms/src/DataStreams/LimitBlockInputStream.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index ea794425bae..276f2c6ee41 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -143,7 +143,7 @@ Block LimitBlockInputStream::readImpl() } } - if (start + length == rows) + if (length == rows) return *ptr; for (size_t i = 0; i < ptr->columns(); ++i) From 5e6ea650eb2dee8dfea1435f57bd0ed3dae1112a Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 23 Aug 2019 14:13:57 +0300 Subject: [PATCH 33/75] update comments --- dbms/src/DataStreams/FillingBlockInputStream.cpp | 1 + dbms/src/DataStreams/FillingBlockInputStream.h | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index aefcb4bf982..6c0ee82c690 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -227,6 +227,7 @@ Block FillingBlockInputStream::readImpl() if (!filling_row.getFillDescription(i).fill_from.isNull() && less(filling_row.getFillDescription(i).fill_from, (*old_fill_columns[i])[0], filling_row.getDirection(i))) { + /// Insert filling row, if it's less than first row in block, because of set 'fill_from' value. filling_row.initFromDefaults(i); insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); break; diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index 5097fe5c6b8..b7079fba2cc 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -10,7 +10,7 @@ class FillingRow public: FillingRow(const SortDescription & sort_description); - /// Generates next row according to fill from, to and step values. + /// Generates next row according to fill 'from', 'to' and 'step' values. /// Returns true, if generated row less than to_row in terms of sorting order. bool next(const FillingRow & to_row); From 1e1d4eb7c17bb9e923ca84b4fbecc8def1ccf77b Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 23 Aug 2019 14:20:53 +0300 Subject: [PATCH 34/75] temporarily make a dummy for 'with fill' with processors --- .../src/Interpreters/InterpreterSelectQuery.cpp | 17 ++++++++++++++++- dbms/src/Interpreters/InterpreterSelectQuery.h | 2 +- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 7c0cb1e3fc3..78dd2d8caae 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2493,7 +2493,22 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) void InterpreterSelectQuery::executeWithFill(QueryPipeline &) { - throw Exception("Unsupported WITH FILL with processors", ErrorCodes::NOT_IMPLEMENTED); + auto & query = getSelectQuery(); + if (query.orderBy()) + { + SortDescription order_descr = getSortDescription(query, context); + SortDescription fill_descr; + for (auto & desc : order_descr) + { + if (desc.with_fill) + fill_descr.push_back(desc); + } + + if (fill_descr.empty()) + return; + + throw Exception("Unsupported WITH FILL with processors", ErrorCodes::NOT_IMPLEMENTED); + } } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.h b/dbms/src/Interpreters/InterpreterSelectQuery.h index d3d1bc99bfd..b1707db3480 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.h +++ b/dbms/src/Interpreters/InterpreterSelectQuery.h @@ -222,7 +222,7 @@ private: void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression); void executeOrder(QueryPipeline & pipeline, SortingInfoPtr sorting_info); - [[noreturn]] void executeWithFill(QueryPipeline & pipeline); + void executeWithFill(QueryPipeline & pipeline); void executeMergeSorted(QueryPipeline & pipeline); void executePreLimit(QueryPipeline & pipeline); void executeLimitBy(QueryPipeline & pipeline); From 5470db34e736fc4a685142b950b967865ad424bd Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 27 Aug 2019 00:00:16 +0300 Subject: [PATCH 35/75] add test for limit with fill --- .../00996_limit_with_ties.reference | 87 +++++++++++++++++++ .../0_stateless/00996_limit_with_ties.sql | 46 ++++++++++ 2 files changed, 133 insertions(+) create mode 100644 dbms/tests/queries/0_stateless/00996_limit_with_ties.reference create mode 100644 dbms/tests/queries/0_stateless/00996_limit_with_ties.sql diff --git a/dbms/tests/queries/0_stateless/00996_limit_with_ties.reference b/dbms/tests/queries/0_stateless/00996_limit_with_ties.reference new file mode 100644 index 00000000000..940a15c7159 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00996_limit_with_ties.reference @@ -0,0 +1,87 @@ +1 +1 +* +1 +1 +2 +2 +2 +* +1 +1 +2 +2 +2 +* +1 +1 +* +1 +1 +* +1 +1 +2 +2 +2 +* +1 +1 +2 +2 +2 +* +1 +1 +2 +2 +2 +3 +3 +* +1 +1 +* +1 +1 +* +1 +1 +2 +2 +2 +* +1 +1 +2 +2 +2 +* +1 +1 +2 +2 +2 +3 +3 +* +1 +1 +2 +2 +2 +* +1 +1 +2 +2 +2 +* +1 +1 +2 +2 +2 +3 +3 +* diff --git a/dbms/tests/queries/0_stateless/00996_limit_with_ties.sql b/dbms/tests/queries/0_stateless/00996_limit_with_ties.sql new file mode 100644 index 00000000000..5d36bb6daea --- /dev/null +++ b/dbms/tests/queries/0_stateless/00996_limit_with_ties.sql @@ -0,0 +1,46 @@ +DROP TABLE IF EXISTS ties; +CREATE TABLE ties (a Int) ENGINE = Memory; + +INSERT INTO ties VALUES (1), (1), (2), (2), (2), (3), (3); + +SELECT a FROM ties order by a limit 1 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 3 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 5 with ties; +SELECT '*'; + +SET max_block_size = 2; +SELECT a FROM ties order by a limit 1 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 2 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 3 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 4 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 6 with ties; +SELECT '*'; + +SET max_block_size = 3; +SELECT a FROM ties order by a limit 1 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 2 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 3 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 4 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 6 with ties; +SELECT '*'; + + +SET max_block_size = 1; +SELECT a FROM ties order by a limit 3 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 5 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 6 with ties; +SELECT '*'; + +DROP TABLE ties; From f4e0dceddb7762c79ac9b8aef3d8bb974662a0af Mon Sep 17 00:00:00 2001 From: stavrolia Date: Tue, 27 Aug 2019 18:20:31 +0300 Subject: [PATCH 36/75] some changes --- dbms/src/Common/parseGlobs.cpp | 88 ++++++++++--------- dbms/src/Storages/StorageFile.cpp | 5 +- .../test_globs_in_filepath/test.py | 10 +-- .../integration/test_storage_hdfs/test.py | 1 - 4 files changed, 52 insertions(+), 52 deletions(-) diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp index 02775d25268..db9aff95975 100644 --- a/dbms/src/Common/parseGlobs.cpp +++ b/dbms/src/Common/parseGlobs.cpp @@ -5,56 +5,60 @@ namespace DB { -/* Because of difference between grep-wildcard-syntax and perl-regexp one we need some transformation of string to use RE2 library for matching. - * It couldn't be one pass because of various configurations of braces in filenames (Linux allow almost any symbols in paths). - * So there are some iterations of escaping and replacements to make correct perl-regexp. +/* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library fo matching + * with such steps: + * 1) search intervals and enums in {}, replace them by regexp with pipe (expr1|expr2|expr3), + * 2) search and replace "*" and "?". + * Before each search need to escape symbols that we would not search. */ -std::string makeRegexpPatternFromGlobs(const std::string & initial_str) +std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs) { - std::string first_prepare; - first_prepare.reserve(initial_str.size()); - for (const auto & letter : initial_str) + std::string escaped_with_globs; + escaped_with_globs.reserve(initial_str_with_globs.size()); + /// Escaping only characters that not used in glob syntax + for (const auto & letter : initial_str_with_globs) { - if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+')) - first_prepare.push_back('\\'); - first_prepare.push_back(letter); + if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')')) + escaped_with_globs.push_back('\\'); + escaped_with_globs.push_back(letter); } - re2::RE2 char_range(R"(({[^*?/\\]\.\.[^*?/\\]}))"); - re2::StringPiece input_for_range(first_prepare); - re2::StringPiece matched_range(first_prepare); - std::string second_prepare; - second_prepare.reserve(first_prepare.size()); + re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*, + re2::StringPiece input(escaped_with_globs); + re2::StringPiece matched(escaped_with_globs); size_t current_index = 0; - size_t pos; - while (RE2::FindAndConsume(&input_for_range, char_range, &matched_range)) + std::string almost_regexp; + almost_regexp.reserve(escaped_with_globs.size()); + while (RE2::FindAndConsume(&input, enum_or_range, &matched)) { - pos = matched_range.data() - first_prepare.data(); - second_prepare += first_prepare.substr(current_index, pos - current_index); - second_prepare.append({'[', matched_range.ToString()[1], '-', matched_range.ToString()[4], ']'}); - current_index = input_for_range.data() - first_prepare.data(); + std::string buffer = matched.ToString(); + almost_regexp.append(escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1)); + + if (buffer.find(',') == std::string::npos) + { + size_t first_point = buffer.find('.'); + std::string first_number = buffer.substr(0, first_point); + std::string second_number = buffer.substr(first_point + 2, buffer.size() - first_point - 2); + size_t range_begin = std::stoull(first_number); + size_t range_end = std::stoull(second_number); + buffer = std::to_string(range_begin); + for (size_t i = range_begin + 1; i <= range_end; ++i) + { + buffer += "|"; + buffer += std::to_string(i); + } + } + else + { + std::replace(buffer.begin(), buffer.end(), ',', '|'); + } + almost_regexp.append("(" + buffer + ")"); + current_index = input.data() - escaped_with_globs.data(); } - second_prepare += first_prepare.substr(current_index); - re2::RE2 enumeration(R"(({[^{}*,]+,[^{}*]*[^{}*,]}))"); - re2::StringPiece input_enum(second_prepare); - re2::StringPiece matched_enum(second_prepare); - current_index = 0; - std::string third_prepare; - while (RE2::FindAndConsume(&input_enum, enumeration, &matched_enum)) - { - pos = matched_enum.data() - second_prepare.data(); - third_prepare.append(second_prepare.substr(current_index, pos - current_index)); - std::string buffer = matched_enum.ToString(); - buffer[0] = '('; - buffer.back() = ')'; - std::replace(buffer.begin(), buffer.end(), ',', '|'); - third_prepare.append(buffer); - current_index = input_enum.data() - second_prepare.data(); - } - third_prepare += second_prepare.substr(current_index); + almost_regexp += escaped_with_globs.substr(current_index); ///// std::string result; - result.reserve(third_prepare.size()); - for (const auto & letter : third_prepare) + result.reserve(almost_regexp.size()); + for (const auto & letter : almost_regexp) { if ((letter == '?') || (letter == '*')) { @@ -62,7 +66,7 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str) if (letter == '?') continue; } - if ((letter == '.') || (letter == '{') || (letter == '}')) + if ((letter == '.') || (letter == '{') || (letter == '}') || (letter == '\\')) result.push_back('\\'); result.push_back(letter); } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index e6a53d4fcf2..f7fe833f64d 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -27,10 +27,9 @@ #include #include -#include -#include +#include -namespace fs = boost::filesystem; +namespace fs = std::filesystem; namespace DB { diff --git a/dbms/tests/integration/test_globs_in_filepath/test.py b/dbms/tests/integration/test_globs_in_filepath/test.py index e6f28fedfe9..db794c35d47 100644 --- a/dbms/tests/integration/test_globs_in_filepath/test.py +++ b/dbms/tests/integration/test_globs_in_filepath/test.py @@ -32,9 +32,7 @@ def test_strange_filenames(start_cluster): ("p.o.*t.s", "2"), ("b}{r?{ces", "2"), ("b}*ces", "2"), - ("b}{r{a..z}{ces", "2"), - ("b}.?{t.h", "2"), - ("b}.?{t.{a..z}", "2")] + ("b}.?{t.h", "2")] for pattern, value in test_requests: assert node.query(''' @@ -64,8 +62,8 @@ def test_linear_structure(start_cluster): ("file*", "20"), ("a_{file,data}", "4"), ("?_{file,data}", "20"), - ("{a..z}_{file,data}", "20"), - ("{a..z}?{file,data}", "20"), + ("{a,b,c,d,e}_{file,data}", "20"), + ("{a,b,c,d,e}?{file,data}", "20"), ("*", "40")] for pattern, value in test_requests: @@ -106,7 +104,7 @@ def test_deep_structure(start_cluster): test_requests = [ ("directory{1..5}/big_dir/*", "2002"), ("directory{0..6}/big_dir/*{0..9}{0..9}{0..9}", "2000"), ("?", "0"), ("directory{0..5}/dir{1..3}/file", "10"), ("directory{0..5}/dir?/file", "10"), - ("we/need/to/go/deeper/file", "2"), ("*/*/*/*/*/*", "2"), ("we/n{a..z}ed/to/*/deeper/{a..z}{a..z}{a..z}{a..z}", "2"), ("we/need/??/go/deeper/*?*?*?*?*", "2")] + ("we/need/to/go/deeper/file", "2"), ("*/*/*/*/*/*", "2"), ("we/need/??/go/deeper/*?*?*?*?*", "2")] for pattern, value in test_requests: assert node.query(''' diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index 5bbb86a7fcd..a9d34926346 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -91,7 +91,6 @@ def test_globs_in_read_table(started_cluster): ("dir/*", 1), ("dir/*?*?*?*?*", 1), ("dir/*?*?*?*?*?*", 0), - ("dir/*{a..z}*{a..z}*{a..z}*{a..z}*", 1), ("some_dir/*/file", 2), ("some_dir/dir?/*", 2), ("*/*/*", 3), From 50d870705ccab765a64d12d6fe80e1cae961419d Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 27 Aug 2019 20:48:42 +0300 Subject: [PATCH 37/75] limit with ties for processors --- dbms/src/Common/ErrorCodes.cpp | 1 - dbms/src/Common/SharedBlockRowRef.h | 10 +- .../src/DataStreams/LimitBlockInputStream.cpp | 17 ++-- .../Interpreters/InterpreterSelectQuery.cpp | 10 +- dbms/src/Processors/LimitTransform.cpp | 95 +++++++++++++++++-- dbms/src/Processors/LimitTransform.h | 13 ++- dbms/src/Processors/SharedChunk.h | 91 ++++++++++++++++++ .../Transforms/MergingSortedTransform.h | 35 +------ 8 files changed, 212 insertions(+), 60 deletions(-) create mode 100644 dbms/src/Processors/SharedChunk.h diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 6a6fb136019..8ce607b5bd4 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -443,7 +443,6 @@ namespace ErrorCodes extern const int INSECURE_PATH = 466; extern const int CANNOT_PARSE_BOOL = 467; extern const int CANNOT_PTHREAD_ATTR = 468; - extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 469; extern const int VIOLATED_CONSTRAINT = 469; extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470; extern const int SETTINGS_ARE_NOT_SUPPORTED = 471; diff --git a/dbms/src/Common/SharedBlockRowRef.h b/dbms/src/Common/SharedBlockRowRef.h index c2cf975d2b4..193f7e4dd05 100644 --- a/dbms/src/Common/SharedBlockRowRef.h +++ b/dbms/src/Common/SharedBlockRowRef.h @@ -2,7 +2,6 @@ #include #include -#include #include #include @@ -79,12 +78,11 @@ struct SharedBlockRowRef bool empty() const { return columns == nullptr; } size_t size() const { return empty() ? 0 : columns->size(); } - static void setSharedBlockRowRef(SharedBlockRowRef & row_ref, - SharedBlockPtr & shared_block,ColumnRawPtrs * columns, size_t row_num) + void set(SharedBlockPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_) { - row_ref.row_num = row_num; - row_ref.columns = columns; - row_ref.shared_block = shared_block; + shared_block = shared_block_; + columns = columns_; + row_num = row_num_; } }; diff --git a/dbms/src/DataStreams/LimitBlockInputStream.cpp b/dbms/src/DataStreams/LimitBlockInputStream.cpp index 276f2c6ee41..5e262e921e8 100644 --- a/dbms/src/DataStreams/LimitBlockInputStream.cpp +++ b/dbms/src/DataStreams/LimitBlockInputStream.cpp @@ -7,7 +7,7 @@ namespace DB { /// gets pointers to all columns of block, which were used for ORDER BY -static ColumnRawPtrs getSortColumns(const Block & block, const SortDescription & description) +static ColumnRawPtrs extractSortColumns(const Block & block, const SortDescription & description) { size_t size = description.size(); ColumnRawPtrs res; @@ -45,7 +45,7 @@ Block LimitBlockInputStream::readImpl() UInt64 rows = 0; /// pos >= offset + limit and all rows in the end of previous block were equal - /// to last row at 'limit' position. So we check current block. + /// to row at 'limit' position. So we check current block. if (!ties_row_ref.empty() && pos >= offset + limit) { res = children.back()->read(); @@ -55,13 +55,13 @@ Block LimitBlockInputStream::readImpl() return res; SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); - ptr->sort_columns = getSortColumns(*ptr, description); + ptr->sort_columns = extractSortColumns(*ptr, description); UInt64 len; for (len = 0; len < rows; ++len) { SharedBlockRowRef current_row; - SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &ptr->sort_columns, len); + current_row.set(ptr, &ptr->sort_columns, len); if (current_row != ties_row_ref) { @@ -102,14 +102,14 @@ Block LimitBlockInputStream::readImpl() SharedBlockPtr ptr = new detail::SharedBlock(std::move(res)); if (with_ties) - ptr->sort_columns = getSortColumns(*ptr, description); + ptr->sort_columns = extractSortColumns(*ptr, description); /// give away the whole block if (pos >= offset + rows && pos <= offset + limit) { /// Save rowref for last row, because probalbly next block begins with the same row. if (with_ties && pos == offset + limit) - SharedBlockRowRef::setSharedBlockRowRef(ties_row_ref, ptr, &ptr->sort_columns, rows - 1); + ties_row_ref.set(ptr, &ptr->sort_columns, rows - 1); return *ptr; } @@ -127,12 +127,12 @@ Block LimitBlockInputStream::readImpl() /// check if other rows in current block equals to last one in limit if (with_ties) { - SharedBlockRowRef::setSharedBlockRowRef(ties_row_ref, ptr, &ptr->sort_columns, start + length - 1); + ties_row_ref.set(ptr, &ptr->sort_columns, start + length - 1); for (size_t i = ties_row_ref.row_num + 1; i < rows; ++i) { SharedBlockRowRef current_row; - SharedBlockRowRef::setSharedBlockRowRef(current_row, ptr, &ptr->sort_columns, i); + current_row.set(ptr, &ptr->sort_columns, i); if (current_row == ties_row_ref) ++length; else @@ -155,5 +155,4 @@ Block LimitBlockInputStream::readImpl() return *ptr; } - } diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 14ea1989783..90854dfaab7 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -2539,13 +2539,21 @@ void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline) UInt64 limit_offset; std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context); + SortDescription order_descr; + if (query.limit_with_ties) + { + if (!query.orderBy()) + throw Exception("LIMIT WITH TIES without ORDER BY", ErrorCodes::LOGICAL_ERROR); + order_descr = getSortDescription(query, context); + } + pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr { if (stream_type != QueryPipeline::StreamType::Main) return nullptr; return std::make_shared( - header, limit_length, limit_offset, always_read_till_end); + header, limit_length, limit_offset, always_read_till_end, query.limit_with_ties, order_descr); }); } } diff --git a/dbms/src/Processors/LimitTransform.cpp b/dbms/src/Processors/LimitTransform.cpp index 1be10c405bb..266267d4e56 100644 --- a/dbms/src/Processors/LimitTransform.cpp +++ b/dbms/src/Processors/LimitTransform.cpp @@ -6,19 +6,26 @@ namespace DB LimitTransform::LimitTransform( const Block & header_, size_t limit_, size_t offset_, - bool always_read_till_end_) + bool always_read_till_end_, bool with_ties_, + const SortDescription & description_) : IProcessor({header_}, {header_}) , input(inputs.front()), output(outputs.front()) , limit(limit_), offset(offset_) , always_read_till_end(always_read_till_end_) + , with_ties(with_ties_), description(description_) { + for (const auto & desc : description) + { + if (!desc.column_name.empty()) + sort_column_positions.push_back(header_.getPositionByName(desc.column_name)); + else + sort_column_positions.push_back(desc.column_number); + } } LimitTransform::Status LimitTransform::prepare() { - - /// Check can output. bool output_finished = false; if (output.isFinished()) @@ -46,7 +53,7 @@ LimitTransform::Status LimitTransform::prepare() } /// Check if we are done with pushing. - bool pushing_is_finished = rows_read >= offset + limit; + bool pushing_is_finished = (rows_read >= offset + limit) && ties_row_ref.empty(); if (pushing_is_finished) { if (!always_read_till_end) @@ -116,6 +123,13 @@ LimitTransform::Status LimitTransform::prepare() if (output.hasData()) return Status::PortFull; + if (with_ties && rows_read == offset + limit) + { + SharedChunkPtr shared_chunk = new detail::SharedChunk(current_chunk.clone()); + shared_chunk->sort_columns = extractSortColumns(shared_chunk->getColumns()); + ties_row_ref.set(shared_chunk, &shared_chunk->sort_columns, shared_chunk->getNumRows() - 1); + } + output.push(std::move(current_chunk)); has_block = false; @@ -132,8 +146,39 @@ LimitTransform::Status LimitTransform::prepare() void LimitTransform::work() { - size_t num_rows = current_chunk.getNumRows(); - size_t num_columns = current_chunk.getNumColumns(); + SharedChunkPtr shared_chunk = new detail::SharedChunk(std::move(current_chunk)); + shared_chunk->sort_columns = extractSortColumns(shared_chunk->getColumns()); + + size_t num_rows = shared_chunk->getNumRows(); + size_t num_columns = shared_chunk->getNumColumns(); + + if (!ties_row_ref.empty() && rows_read >= offset + limit) + { + UInt64 len; + for (len = 0; len < num_rows; ++len) + { + SharedChunkRowRef current_row; + current_row.set(shared_chunk, &shared_chunk->sort_columns, len); + + if (current_row != ties_row_ref) + { + ties_row_ref.reset(); + break; + } + } + + auto columns = shared_chunk->detachColumns(); + + if (len < num_rows) + { + for (size_t i = 0; i < num_columns; ++i) + columns[i] = columns[i]->cut(0, len); + } + + current_chunk.setColumns(std::move(columns), len); + block_processed = true; + return; + } /// return a piece of the block size_t start = std::max( @@ -145,7 +190,33 @@ void LimitTransform::work() static_cast(rows_read) - static_cast(offset), static_cast(limit) + static_cast(offset) - static_cast(rows_read) + static_cast(num_rows))); - auto columns = current_chunk.detachColumns(); + /// check if other rows in current block equals to last one in limit + if (with_ties) + { + ties_row_ref.set(shared_chunk, &shared_chunk->sort_columns, start + length - 1); + SharedChunkRowRef current_row; + + for (size_t i = ties_row_ref.row_num + 1; i < num_rows; ++i) + { + current_row.set(shared_chunk, &shared_chunk->sort_columns, i); + if (current_row == ties_row_ref) + ++length; + else + { + ties_row_ref.reset(); + break; + } + } + } + + if (length == num_rows) + { + current_chunk = std::move(*shared_chunk); + block_processed = true; + return; + } + + auto columns = shared_chunk->detachColumns(); for (size_t i = 0; i < num_columns; ++i) columns[i] = columns[i]->cut(start, length); @@ -155,5 +226,15 @@ void LimitTransform::work() block_processed = true; } +ColumnRawPtrs LimitTransform::extractSortColumns(const Columns & columns) +{ + ColumnRawPtrs res; + res.reserve(description.size()); + for (size_t pos : sort_column_positions) + res.push_back(columns[pos].get()); + + return res; +} + } diff --git a/dbms/src/Processors/LimitTransform.h b/dbms/src/Processors/LimitTransform.h index f80ca263c95..3df5e3cc049 100644 --- a/dbms/src/Processors/LimitTransform.h +++ b/dbms/src/Processors/LimitTransform.h @@ -1,7 +1,8 @@ #pragma once #include - +#include +#include namespace DB { @@ -23,10 +24,18 @@ private: UInt64 rows_before_limit_at_least = 0; + bool with_ties; + const SortDescription description; + SharedChunkRowRef ties_row_ref; + + std::vector sort_column_positions; + ColumnRawPtrs extractSortColumns(const Columns & columns); + public: LimitTransform( const Block & header_, size_t limit_, size_t offset_, - bool always_read_till_end_ = false); + bool always_read_till_end_ = false, bool with_ties_ = false, + const SortDescription & description_ = {}); String getName() const override { return "Limit"; } diff --git a/dbms/src/Processors/SharedChunk.h b/dbms/src/Processors/SharedChunk.h new file mode 100644 index 00000000000..c6fe3c12f89 --- /dev/null +++ b/dbms/src/Processors/SharedChunk.h @@ -0,0 +1,91 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +/// Allows you refer to the row in the block and hold the block ownership, +/// and thus avoid creating a temporary row object. +/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; +/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; +/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; +/// The reference counter is not atomic, since it is used from one thread. +namespace detail +{ +struct SharedChunk : Chunk +{ + int refcount = 0; + + ColumnRawPtrs all_columns; + ColumnRawPtrs sort_columns; + + SharedChunk(Chunk && chunk) : Chunk(std::move(chunk)) {} +}; + +} + +inline void intrusive_ptr_add_ref(detail::SharedChunk * ptr) +{ + ++ptr->refcount; +} + +inline void intrusive_ptr_release(detail::SharedChunk * ptr) +{ + if (0 == --ptr->refcount) + delete ptr; +} + +using SharedChunkPtr = boost::intrusive_ptr; + + +struct SharedChunkRowRef +{ + ColumnRawPtrs * columns = nullptr; + size_t row_num; + SharedChunkPtr shared_block; + + void swap(SharedChunkRowRef & other) + { + std::swap(columns, other.columns); + std::swap(row_num, other.row_num); + std::swap(shared_block, other.shared_block); + } + + /// The number and types of columns must match. + bool operator==(const SharedChunkRowRef & other) const + { + size_t size = columns->size(); + for (size_t i = 0; i < size; ++i) + if (0 != (*columns)[i]->compareAt(row_num, other.row_num, *(*other.columns)[i], 1)) + return false; + return true; + } + + bool operator!=(const SharedChunkRowRef & other) const + { + return !(*this == other); + } + + void reset() + { + SharedChunkRowRef empty; + swap(empty); + } + + bool empty() const { return columns == nullptr; } + size_t size() const { return empty() ? 0 : columns->size(); } + + void set(SharedChunkPtr & shared_block_, ColumnRawPtrs * columns_, size_t row_num_) + { + shared_block = shared_block_; + columns = columns_; + row_num = row_num_; + } +}; + +} diff --git a/dbms/src/Processors/Transforms/MergingSortedTransform.h b/dbms/src/Processors/Transforms/MergingSortedTransform.h index f1175c8d347..0991835bfaf 100644 --- a/dbms/src/Processors/Transforms/MergingSortedTransform.h +++ b/dbms/src/Processors/Transforms/MergingSortedTransform.h @@ -2,46 +2,13 @@ #include #include #include +#include #include namespace DB { -/// Allows you refer to the row in the block and hold the block ownership, -/// and thus avoid creating a temporary row object. -/// Do not use std::shared_ptr, since there is no need for a place for `weak_count` and `deleter`; -/// does not use Poco::SharedPtr, since you need to allocate a block and `refcount` in one piece; -/// does not use Poco::AutoPtr, since it does not have a `move` constructor and there are extra checks for nullptr; -/// The reference counter is not atomic, since it is used from one thread. -namespace detail -{ -struct SharedChunk : Chunk -{ - int refcount = 0; - - ColumnRawPtrs all_columns; - ColumnRawPtrs sort_columns; - - SharedChunk(Chunk && chunk) : Chunk(std::move(chunk)) {} -}; - -} - -using SharedChunkPtr = boost::intrusive_ptr; - - -inline void intrusive_ptr_add_ref(detail::SharedChunk * ptr) -{ - ++ptr->refcount; -} - -inline void intrusive_ptr_release(detail::SharedChunk * ptr) -{ - if (0 == --ptr->refcount) - delete ptr; -} - class MergingSortedTransform : public IProcessor { public: From a169213691521e4f3debc288f2fa1e6ef6394303 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 27 Aug 2019 21:11:02 +0300 Subject: [PATCH 38/75] test for limit with ties --- .../00996_limit_with_ties.reference | 61 ++++--------------- .../0_stateless/00996_limit_with_ties.sql | 29 +++------ 2 files changed, 22 insertions(+), 68 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00996_limit_with_ties.reference b/dbms/tests/queries/0_stateless/00996_limit_with_ties.reference index 940a15c7159..aa5d102bc9b 100644 --- a/dbms/tests/queries/0_stateless/00996_limit_with_ties.reference +++ b/dbms/tests/queries/0_stateless/00996_limit_with_ties.reference @@ -6,21 +6,27 @@ 2 2 2 +2 * 1 1 2 2 2 +2 +* +1 +* +1 +2 +2 +2 +2 * 1 1 * -1 -1 -* -1 -1 +2 2 2 2 @@ -30,58 +36,17 @@ 2 2 2 -* -1 -1 -2 -2 -2 -3 -3 -* -1 -1 -* -1 -1 -* -1 -1 -2 -2 2 * 1 1 +* +2 2 2 2 * -1 -1 -2 -2 -2 -3 -3 -* -1 -1 2 2 2 * -1 -1 -2 -2 -2 -* -1 -1 -2 -2 -2 -3 -3 -* diff --git a/dbms/tests/queries/0_stateless/00996_limit_with_ties.sql b/dbms/tests/queries/0_stateless/00996_limit_with_ties.sql index 5d36bb6daea..3e4813bc6b5 100644 --- a/dbms/tests/queries/0_stateless/00996_limit_with_ties.sql +++ b/dbms/tests/queries/0_stateless/00996_limit_with_ties.sql @@ -1,7 +1,9 @@ DROP TABLE IF EXISTS ties; CREATE TABLE ties (a Int) ENGINE = Memory; -INSERT INTO ties VALUES (1), (1), (2), (2), (2), (3), (3); +-- SET experimental_use_processors=1; + +INSERT INTO ties VALUES (1), (1), (2), (2), (2), (2) (3), (3); SELECT a FROM ties order by a limit 1 with ties; SELECT '*'; @@ -11,36 +13,23 @@ SELECT a FROM ties order by a limit 5 with ties; SELECT '*'; SET max_block_size = 2; -SELECT a FROM ties order by a limit 1 with ties; +SELECT a FROM ties order by a limit 1, 1 with ties; +SELECT '*'; +SELECT a FROM ties order by a limit 1, 2 with ties; SELECT '*'; SELECT a FROM ties order by a limit 2 with ties; SELECT '*'; -SELECT a FROM ties order by a limit 3 with ties; +SELECT a FROM ties order by a limit 2, 3 with ties; SELECT '*'; SELECT a FROM ties order by a limit 4 with ties; SELECT '*'; -SELECT a FROM ties order by a limit 6 with ties; -SELECT '*'; SET max_block_size = 3; SELECT a FROM ties order by a limit 1 with ties; SELECT '*'; -SELECT a FROM ties order by a limit 2 with ties; +SELECT a FROM ties order by a limit 2, 3 with ties; SELECT '*'; -SELECT a FROM ties order by a limit 3 with ties; -SELECT '*'; -SELECT a FROM ties order by a limit 4 with ties; -SELECT '*'; -SELECT a FROM ties order by a limit 6 with ties; -SELECT '*'; - - -SET max_block_size = 1; -SELECT a FROM ties order by a limit 3 with ties; -SELECT '*'; -SELECT a FROM ties order by a limit 5 with ties; -SELECT '*'; -SELECT a FROM ties order by a limit 6 with ties; +SELECT a FROM ties order by a limit 3, 2 with ties; SELECT '*'; DROP TABLE ties; From d7a712e7ae37c88afbfe120678509bc8ffbe60e9 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Thu, 29 Aug 2019 18:19:12 +0300 Subject: [PATCH 39/75] update logic of 'with fill' modifier --- .../DataStreams/FillingBlockInputStream.cpp | 109 +++++--- .../src/DataStreams/FillingBlockInputStream.h | 9 +- .../00995_order_by_with_fill.reference | 260 ++---------------- 3 files changed, 105 insertions(+), 273 deletions(-) diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index 6c0ee82c690..7113b0aa438 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -24,20 +24,42 @@ FillingRow::FillingRow(const SortDescription & description_) : description(descr { for (size_t i = 0; i < description.size(); ++i) { - auto & fill_from = description[i].fill_description.fill_from; - auto & fill_to = description[i].fill_description.fill_to; + auto & descr = description[i].fill_description; /// Cast fields to same types. Otherwise, there will be troubles, when we reach zero, while generating rows. - if (fill_to.getType() == Field::Types::Int64 && fill_from.getType() == Field::Types::UInt64) - fill_from = fill_from.get(); - - if (fill_from.getType() == Field::Types::Int64 && fill_to.getType() == Field::Types::UInt64) - fill_to = fill_to.get(); - + if (descr.fill_to.getType() == Field::Types::Int64 + || descr.fill_from.getType() == Field::Types::Int64 || descr.fill_step.getType() == Field::Types::Int64) + { + if (descr.fill_to.getType() == Field::Types::UInt64) + descr.fill_to = descr.fill_to.get(); + if (descr.fill_from.getType() == Field::Types::UInt64) + descr.fill_from = descr.fill_from.get(); + if (descr.fill_step.getType() == Field::Types::UInt64) + descr.fill_step = descr.fill_step.get(); + } } row.resize(description.size()); } +bool FillingRow::operator<(const FillingRow & other) const +{ + for (size_t i = 0; i < size(); ++i) + { + if (row[i].isNull() || other[i].isNull() || equals(row[i], other[i])) + continue; + return less(row[i], other[i], getDirection(i)); + } + return false; +} + +bool FillingRow::operator==(const FillingRow & other) const +{ + for (size_t i = 0; i < size(); ++i) + if (!equals(row[i], other[i])) + return false; + return true; +} + bool FillingRow::next(const FillingRow & to_row) { size_t pos = 0; @@ -61,8 +83,8 @@ bool FillingRow::next(const FillingRow & to_row) applyVisitor(FieldVisitorSum(getFillDescription(i).fill_step), next_value); if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) { - initFromDefaults(i + 1); row[i] = next_value; + initFromDefaults(i + 1); return true; } } @@ -70,41 +92,30 @@ bool FillingRow::next(const FillingRow & to_row) auto next_value = row[pos]; applyVisitor(FieldVisitorSum(getFillDescription(pos).fill_step), next_value); - if (equals(next_value, to_row[pos])) + if (less(to_row[pos], next_value, getDirection(pos))) + return false; + + row[pos] = next_value; + if (equals(row[pos], to_row[pos])) { bool is_less = false; - for (size_t i = pos + 1; i < row.size(); ++i) + for (size_t i = pos + 1; i < size(); ++i) { const auto & fill_from = getFillDescription(i).fill_from; - if (!fill_from.isNull() && !to_row[i].isNull() && less(fill_from, to_row[i], getDirection(i))) - { - is_less = true; - initFromDefaults(i); - break; - } + if (!fill_from.isNull()) + row[i] = fill_from; else row[i] = to_row[i]; + is_less |= less(row[i], to_row[i], getDirection(i)); } - row[pos] = next_value; return is_less; } - if (less(next_value, to_row[pos], getDirection(pos))) - { - initFromDefaults(pos + 1); - row[pos] = next_value; - return true; - } - - return false; + initFromDefaults(pos + 1); + return true; } -void FillingRow::initFromColumns(const Columns & columns, size_t row_num, size_t from_pos) -{ - for (size_t i = from_pos; i < columns.size(); ++i) - columns[i]->get(row_num, row[i]); -} void FillingRow::initFromDefaults(size_t from_pos) { @@ -125,7 +136,6 @@ static void insertFromFillingRow(MutableColumns & filling_columns, MutableColumn for (size_t i = 0; i < other_columns.size(); ++i) other_columns[i]->insertDefault(); - } static void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num) @@ -199,10 +209,15 @@ Block FillingBlockInputStream::readImpl() init_columns_by_positions(header, old_fill_columns, res_fill_columns, fill_column_positions); init_columns_by_positions(header, old_other_columns, res_other_columns, other_column_positions); + bool should_insert_first = next_row < filling_row; + bool generated = false; for (size_t i = 0; i < filling_row.size(); ++i) next_row[i] = filling_row.getFillDescription(i).fill_to; + if (should_insert_first && filling_row < next_row) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + while (filling_row.next(next_row)) { generated = true; @@ -221,25 +236,41 @@ Block FillingBlockInputStream::readImpl() if (first) { - filling_row.initFromColumns(old_fill_columns, 0); for (size_t i = 0; i < filling_row.size(); ++i) { - if (!filling_row.getFillDescription(i).fill_from.isNull() && - less(filling_row.getFillDescription(i).fill_from, (*old_fill_columns[i])[0], filling_row.getDirection(i))) + auto current_value = (*old_fill_columns[i])[0]; + const auto & fill_from = filling_row.getFillDescription(i).fill_from; + if (!fill_from.isNull() && !equals(current_value, fill_from)) { - /// Insert filling row, if it's less than first row in block, because of set 'fill_from' value. filling_row.initFromDefaults(i); - insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + if (less(fill_from, current_value, filling_row.getDirection(i))) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); break; } + filling_row[i] = current_value; } - first = false; } for (size_t row_ind = 0; row_ind < rows; ++row_ind) { - next_row.initFromColumns(old_fill_columns, row_ind); + bool should_insert_first = next_row < filling_row; + + for (size_t i = 0; i < filling_row.size(); ++i) + { + auto current_value = (*old_fill_columns[i])[row_ind]; + const auto & fill_to = filling_row.getFillDescription(i).fill_to; + + if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) + next_row[i] = current_value; + else + next_row[i] = fill_to; + } + + /// A case, when at previous step row was initialized from defaults 'fill_from' values + /// and probably we need to insert it to block. + if (should_insert_first && filling_row < next_row) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); /// Insert generated filling row to block, while it is less than current row in block. while (filling_row.next(next_row)) diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index b7079fba2cc..7ba89f3d13a 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -11,18 +11,19 @@ public: FillingRow(const SortDescription & sort_description); /// Generates next row according to fill 'from', 'to' and 'step' values. - /// Returns true, if generated row less than to_row in terms of sorting order. bool next(const FillingRow & to_row); - void initFromColumns(const Columns & columns, size_t row_num, size_t from_pos = 0); void initFromDefaults(size_t from_pos = 0); + void initFromColumns(const Columns & columns, size_t row_ind, size_t from_pos); Field & operator[](size_t ind) { return row[ind]; } const Field & operator[](size_t ind) const { return row[ind]; } size_t size() const { return row.size(); } + bool operator<(const FillingRow & other) const; + bool operator==(const FillingRow & other) const; - int getDirection(size_t ind) { return description[ind].direction; } - const FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } + int getDirection(size_t ind) const { return description[ind].direction; } + const FillColumnDescription & getFillDescription(size_t ind) const { return description[ind].fill_description; } private: std::vector row; diff --git a/dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference b/dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference index df73b0d1bc3..adb0e1aa2c3 100644 --- a/dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference +++ b/dbms/tests/queries/0_stateless/00995_order_by_with_fill.reference @@ -115,10 +115,6 @@ 2019-05-28 0 2019-05-29 0 2019-05-30 18 3kd -2019-05-31 0 -2019-06-01 0 -2019-06-02 0 -2019-06-03 0 2019-06-04 5 6az *** date DESC WITH FILL, val WITH FILL FROM 1 TO 6 *** 2019-06-04 1 @@ -151,18 +147,6 @@ 2019-05-30 3 2019-05-30 4 2019-05-30 5 -2019-05-30 6 -2019-05-30 7 -2019-05-30 8 -2019-05-30 9 -2019-05-30 10 -2019-05-30 11 -2019-05-30 12 -2019-05-30 13 -2019-05-30 14 -2019-05-30 15 -2019-05-30 16 -2019-05-30 17 2019-05-30 18 3kd 2019-05-29 1 2019-05-29 2 @@ -189,59 +173,19 @@ 2019-05-25 3 2019-05-25 4 2019-05-25 5 -2019-05-25 6 -2019-05-25 7 -2019-05-25 8 -2019-05-25 9 -2019-05-25 10 -2019-05-25 11 -2019-05-25 12 -2019-05-25 13 -2019-05-25 14 -2019-05-25 15 -2019-05-25 16 2019-05-25 17 0ei 2019-05-24 1 2019-05-24 2 2019-05-24 3 2019-05-24 4 2019-05-24 5 -2019-05-24 6 -2019-05-24 7 -2019-05-24 8 -2019-05-24 9 -2019-05-24 10 -2019-05-24 11 -2019-05-24 12 2019-05-24 13 sd0 2019-05-23 1 2019-05-23 2 2019-05-23 3 2019-05-23 4 2019-05-23 5 -2019-05-23 6 -2019-05-23 7 -2019-05-23 8 -2019-05-23 9 -2019-05-23 10 -2019-05-23 11 -2019-05-23 12 -2019-05-23 13 -2019-05-23 14 2019-05-23 15 01v -2019-05-23 16 -2019-05-23 17 -2019-05-23 18 -2019-05-23 19 -2019-05-23 20 -2019-05-23 21 -2019-05-23 22 -2019-05-23 23 -2019-05-23 24 -2019-05-23 25 -2019-05-23 26 -2019-05-23 27 -2019-05-23 28 2019-05-23 29 72y 2019-05-22 1 2019-05-22 2 @@ -263,20 +207,6 @@ 2019-05-19 3 2019-05-19 4 2019-05-19 5 -2019-05-19 6 -2019-05-19 7 -2019-05-19 8 -2019-05-19 9 -2019-05-19 10 -2019-05-19 11 -2019-05-19 12 -2019-05-19 13 -2019-05-19 14 -2019-05-19 15 -2019-05-19 16 -2019-05-19 17 -2019-05-19 18 -2019-05-19 19 2019-05-19 20 yfh 2019-05-18 1 2019-05-18 2 @@ -298,27 +228,6 @@ 2019-05-15 3 2019-05-15 4 2019-05-15 5 -2019-05-15 6 -2019-05-15 7 -2019-05-15 8 -2019-05-15 9 -2019-05-15 10 -2019-05-15 11 -2019-05-15 12 -2019-05-15 13 -2019-05-15 14 -2019-05-15 15 -2019-05-15 16 -2019-05-15 17 -2019-05-15 18 -2019-05-15 19 -2019-05-15 20 -2019-05-15 21 -2019-05-15 22 -2019-05-15 23 -2019-05-15 24 -2019-05-15 25 -2019-05-15 26 2019-05-15 27 enb 2019-05-14 1 2019-05-14 2 @@ -340,113 +249,31 @@ 2019-05-11 3 2019-05-11 4 2019-05-11 5 -2019-05-11 6 -2019-05-11 7 -2019-05-11 8 -2019-05-11 9 -2019-05-11 10 -2019-05-11 11 -2019-05-11 12 -2019-05-11 13 -2019-05-11 14 -2019-05-11 15 -2019-05-11 16 -2019-05-11 17 2019-05-11 18 3s2 2019-05-10 1 myj 2019-05-10 2 2019-05-10 3 2019-05-10 4 2019-05-10 5 -2019-05-10 6 -2019-05-10 7 -2019-05-10 8 -2019-05-10 9 -2019-05-10 10 -2019-05-10 11 -2019-05-10 12 -2019-05-10 13 -2019-05-10 14 -2019-05-10 15 2019-05-10 16 vp7 2019-05-09 1 2019-05-09 2 2019-05-09 3 2019-05-09 4 2019-05-09 5 -2019-05-09 6 -2019-05-09 7 -2019-05-09 8 -2019-05-09 9 -2019-05-09 10 -2019-05-09 11 -2019-05-09 12 -2019-05-09 13 -2019-05-09 14 -2019-05-09 15 -2019-05-09 16 -2019-05-09 17 -2019-05-09 18 -2019-05-09 19 -2019-05-09 20 -2019-05-09 21 -2019-05-09 22 -2019-05-09 23 -2019-05-09 24 2019-05-09 25 798 2019-05-08 1 2019-05-08 2 2019-05-08 3 2019-05-08 4 2019-05-08 5 -2019-05-08 6 -2019-05-08 7 -2019-05-08 8 -2019-05-08 9 -2019-05-08 10 -2019-05-08 11 -2019-05-08 12 -2019-05-08 13 -2019-05-08 14 -2019-05-08 15 -2019-05-08 16 -2019-05-08 17 -2019-05-08 18 -2019-05-08 19 -2019-05-08 20 -2019-05-08 21 -2019-05-08 22 -2019-05-08 23 -2019-05-08 24 -2019-05-08 25 -2019-05-08 26 -2019-05-08 27 2019-05-08 28 otf 2019-05-07 1 2019-05-07 2 2019-05-07 3 2019-05-07 4 2019-05-07 5 -2019-05-07 6 -2019-05-07 7 -2019-05-07 8 -2019-05-07 9 -2019-05-07 10 -2019-05-07 11 -2019-05-07 12 -2019-05-07 13 -2019-05-07 14 -2019-05-07 15 -2019-05-07 16 -2019-05-07 17 2019-05-07 18 prh -2019-05-07 19 -2019-05-07 20 -2019-05-07 21 -2019-05-07 22 -2019-05-07 23 -2019-05-07 24 -2019-05-07 25 2019-05-07 26 2ke *** date DESC WITH FILL TO 2019-05-01 STEP -2, val DESC WITH FILL FROM 10 TO -5 STEP -3 *** 2019-06-04 10 @@ -477,77 +304,55 @@ 2019-05-27 1 2019-05-27 -2 2019-05-25 17 0ei -2019-05-25 14 -2019-05-25 11 -2019-05-25 8 -2019-05-25 5 -2019-05-25 2 -2019-05-25 -1 -2019-05-25 -4 +2019-05-25 10 +2019-05-25 7 +2019-05-25 4 +2019-05-25 1 +2019-05-25 -2 2019-05-24 13 sd0 2019-05-23 29 72y -2019-05-23 26 -2019-05-23 23 -2019-05-23 20 -2019-05-23 17 2019-05-23 15 01v -2019-05-23 14 -2019-05-23 11 -2019-05-23 8 -2019-05-23 5 -2019-05-23 2 -2019-05-23 -1 -2019-05-23 -4 +2019-05-23 10 +2019-05-23 7 +2019-05-23 4 +2019-05-23 1 +2019-05-23 -2 2019-05-21 10 2019-05-21 7 2019-05-21 4 2019-05-21 1 2019-05-21 -2 2019-05-19 20 yfh -2019-05-19 17 -2019-05-19 14 -2019-05-19 11 -2019-05-19 8 -2019-05-19 5 -2019-05-19 2 -2019-05-19 -1 -2019-05-19 -4 +2019-05-19 10 +2019-05-19 7 +2019-05-19 4 +2019-05-19 1 +2019-05-19 -2 2019-05-17 10 2019-05-17 7 2019-05-17 4 2019-05-17 1 2019-05-17 -2 2019-05-15 27 enb -2019-05-15 24 -2019-05-15 21 -2019-05-15 18 -2019-05-15 15 -2019-05-15 12 -2019-05-15 9 -2019-05-15 6 -2019-05-15 3 -2019-05-15 0 -2019-05-15 -3 +2019-05-15 10 +2019-05-15 7 +2019-05-15 4 +2019-05-15 1 +2019-05-15 -2 2019-05-13 10 2019-05-13 7 2019-05-13 4 2019-05-13 1 2019-05-13 -2 2019-05-11 18 3s2 -2019-05-11 15 -2019-05-11 12 -2019-05-11 9 -2019-05-11 6 -2019-05-11 3 -2019-05-11 0 -2019-05-11 -3 +2019-05-11 10 +2019-05-11 7 +2019-05-11 4 +2019-05-11 1 +2019-05-11 -2 2019-05-10 16 vp7 2019-05-10 1 myj 2019-05-09 25 798 -2019-05-09 22 -2019-05-09 19 -2019-05-09 16 -2019-05-09 13 2019-05-09 10 2019-05-09 7 2019-05-09 4 @@ -555,17 +360,12 @@ 2019-05-09 -2 2019-05-08 28 otf 2019-05-07 26 2ke -2019-05-07 23 -2019-05-07 20 2019-05-07 18 prh -2019-05-07 17 -2019-05-07 14 -2019-05-07 11 -2019-05-07 8 -2019-05-07 5 -2019-05-07 2 -2019-05-07 -1 -2019-05-07 -4 +2019-05-07 10 +2019-05-07 7 +2019-05-07 4 +2019-05-07 1 +2019-05-07 -2 2019-05-05 10 2019-05-05 7 2019-05-05 4 From 011c752ed74b4ce544f45a18273bad783591a027 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Thu, 29 Aug 2019 18:38:33 +0300 Subject: [PATCH 40/75] Add unit test --- dbms/src/Common/parseGlobs.cpp | 10 ++++++---- .../gtest_makeRegexpPatternFromGlobs.cpp | 20 +++++++++++++++++++ 2 files changed, 26 insertions(+), 4 deletions(-) create mode 100644 dbms/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp index db9aff95975..2722c5fc655 100644 --- a/dbms/src/Common/parseGlobs.cpp +++ b/dbms/src/Common/parseGlobs.cpp @@ -8,8 +8,10 @@ namespace DB /* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library fo matching * with such steps: * 1) search intervals and enums in {}, replace them by regexp with pipe (expr1|expr2|expr3), - * 2) search and replace "*" and "?". + * 2) search and replace "*" and "?" with "". * Before each search need to escape symbols that we would not search. + * + * There are few examples in unit tests. */ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs) { @@ -55,18 +57,18 @@ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_glob almost_regexp.append("(" + buffer + ")"); current_index = input.data() - escaped_with_globs.data(); } - almost_regexp += escaped_with_globs.substr(current_index); ///// + almost_regexp += escaped_with_globs.substr(current_index); std::string result; result.reserve(almost_regexp.size()); for (const auto & letter : almost_regexp) { if ((letter == '?') || (letter == '*')) { - result += "[^/]"; + result += "[^/]"; /// '?' is any symbol except '/' if (letter == '?') continue; } - if ((letter == '.') || (letter == '{') || (letter == '}') || (letter == '\\')) + if ((letter == '.') || (letter == '{') || (letter == '}')) result.push_back('\\'); result.push_back(letter); } diff --git a/dbms/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp b/dbms/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp new file mode 100644 index 00000000000..db695b965a1 --- /dev/null +++ b/dbms/src/Common/tests/gtest_makeRegexpPatternFromGlobs.cpp @@ -0,0 +1,20 @@ +#include +#include +#include + + +using namespace DB; + + +TEST(Common, makeRegexpPatternFromGlobs) +{ + EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..09}"), "f(1|2|3|4|5|6|7|8|9)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{01..9}"), "f(1|2|3|4|5|6|7|8|9)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{0001..0000009}"), "f(1|2|3|4|5|6|7|8|9)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..2}{1..2}"), "f(1|2)(1|2)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{1..1}{1..1}"), "f(1)(1)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("f{0..0}{0..0}"), "f(0)(0)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("file{1..5}"),"file(1|2|3|4|5)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("file{1,2,3}"),"file(1|2|3)"); + EXPECT_EQ(makeRegexpPatternFromGlobs("{1,2,3}blabla{a.x,b.x,c.x}smth[]_else{aa,bb}?*"), "(1|2|3)blabla(a\\.x|b\\.x|c\\.x)smth\\[\\]_else(aa|bb)[^/][^/]*"); +} From 048d3da9ab193b4436480acdace41f4d9d42ac7f Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Fri, 30 Aug 2019 01:24:33 +0300 Subject: [PATCH 41/75] better types comparison and conversion in FillingBlockInputStream --- .../DataStreams/FillingBlockInputStream.cpp | 64 +++++++++++-------- .../src/DataStreams/FillingBlockInputStream.h | 2 +- .../Interpreters/InterpreterSelectQuery.cpp | 2 +- 3 files changed, 39 insertions(+), 29 deletions(-) diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index 7113b0aa438..35166e3bd15 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -1,5 +1,7 @@ #include "FillingBlockInputStream.h" #include +#include +#include namespace DB { @@ -22,22 +24,6 @@ static bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(F FillingRow::FillingRow(const SortDescription & description_) : description(description_) { - for (size_t i = 0; i < description.size(); ++i) - { - auto & descr = description[i].fill_description; - - /// Cast fields to same types. Otherwise, there will be troubles, when we reach zero, while generating rows. - if (descr.fill_to.getType() == Field::Types::Int64 - || descr.fill_from.getType() == Field::Types::Int64 || descr.fill_step.getType() == Field::Types::Int64) - { - if (descr.fill_to.getType() == Field::Types::UInt64) - descr.fill_to = descr.fill_to.get(); - if (descr.fill_from.getType() == Field::Types::UInt64) - descr.fill_from = descr.fill_from.get(); - if (descr.fill_step.getType() == Field::Types::UInt64) - descr.fill_step = descr.fill_step.get(); - } - } row.resize(description.size()); } @@ -154,30 +140,54 @@ FillingBlockInputStream::FillingBlockInputStream( std::vector is_fill_column(header.columns()); for (const auto & elem : sort_description) + is_fill_column[header.getPositionByName(elem.column_name)] = true; + + auto try_convert_fields = [](FillColumnDescription & descr, const DataTypePtr & type) { - size_t pos = header.getPositionByName(elem.column_name); - fill_column_positions.push_back(pos); - is_fill_column[pos] = true; - } + auto max_type = Field::Types::Null; + WhichDataType which(type); + DataTypePtr to_type; + if (isInteger(type) || which.isDateOrDateTime()) + { + max_type = Field::Types::Int64; + to_type = std::make_shared(); + } + else if (which.isFloat()) + { + max_type = Field::Types::Float64; + to_type = std::make_shared(); + } + + if (descr.fill_from.getType() > max_type || descr.fill_to.getType() > max_type + || descr.fill_step.getType() > max_type) + return false; + + descr.fill_from = convertFieldToType(descr.fill_from, *to_type); + descr.fill_to = convertFieldToType(descr.fill_to, *to_type); + descr.fill_step = convertFieldToType(descr.fill_step, *to_type); + + return true; + }; for (size_t i = 0; i < header.columns(); ++i) { if (is_fill_column[i]) { auto type = header.getByPosition(i).type; - if (!isColumnedAsNumber(header.getByPosition(i).type)) - throw Exception("WITH FILL can be used only with numeric types, but is set for column with type " - + header.getByPosition(i).type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + auto & descr = filling_row.getFillDescription(i); + if (!try_convert_fields(descr, type)) + throw Exception("Incompatible types of WITH FILL expression values with column type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); - const auto & fill_from = sort_description[i].fill_description.fill_from; - const auto & fill_to = sort_description[i].fill_description.fill_to; if (type->isValueRepresentedByUnsignedInteger() && - ((!fill_from.isNull() && less(fill_from, Field{0}, 1)) || - (!fill_to.isNull() && less(fill_to, Field{0}, 1)))) + ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || + (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) { throw Exception("WITH FILL bound values cannot be negative for unsigned type " + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); } + + fill_column_positions.push_back(i); } else other_column_positions.push_back(i); diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index 7ba89f3d13a..279b0aea901 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -23,7 +23,7 @@ public: bool operator==(const FillingRow & other) const; int getDirection(size_t ind) const { return description[ind].direction; } - const FillColumnDescription & getFillDescription(size_t ind) const { return description[ind].fill_description; } + FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } private: std::vector row; diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 90854dfaab7..587c30fcf7b 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -693,7 +693,7 @@ static Field getWithFillFieldValue(const ASTPtr & node, const Context & context) return field; } -static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, const Context &context) +static FillColumnDescription getWithFillDescription(const ASTOrderByElement & order_by_elem, const Context & context) { FillColumnDescription descr; if (order_by_elem.fill_from) From f51901bb3f7bd6ac59fb26caee149c628b9d2a77 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Fri, 30 Aug 2019 16:27:05 +0300 Subject: [PATCH 42/75] fix comment --- dbms/src/Common/parseGlobs.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp index 2722c5fc655..1c1033dc2b4 100644 --- a/dbms/src/Common/parseGlobs.cpp +++ b/dbms/src/Common/parseGlobs.cpp @@ -8,7 +8,7 @@ namespace DB /* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library fo matching * with such steps: * 1) search intervals and enums in {}, replace them by regexp with pipe (expr1|expr2|expr3), - * 2) search and replace "*" and "?" with "". + * 2) search and replace "*" and "?". * Before each search need to escape symbols that we would not search. * * There are few examples in unit tests. From 974789d3794e9c5ad95e6ab97d6498b00f9c4f36 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Fri, 30 Aug 2019 18:19:05 +0300 Subject: [PATCH 43/75] add const --- dbms/src/Storages/StorageFile.cpp | 28 ++++++++++++++-------------- dbms/src/Storages/StorageHDFS.cpp | 22 +++++++++++----------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index f7fe833f64d..c1da0f1253d 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -54,36 +54,36 @@ namespace std::vector LSWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match) { - size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob = for_match.find_first_of("*?{"); - size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); - std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' + const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const std::string suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - size_t next_slash = suffix_with_globs.find('/', 1); + const size_t next_slash = suffix_with_globs.find('/', 1); re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); std::vector result; - std::string preffix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); - if (!fs::exists(fs::path(preffix_without_globs.data()))) + const std::string prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); + if (!fs::exists(fs::path(prefix_without_globs.data()))) { return result; } - fs::directory_iterator end; - for (fs::directory_iterator it(preffix_without_globs); it != end; ++it) + const fs::directory_iterator end; + for (fs::directory_iterator it(prefix_without_globs); it != end; ++it) { - std::string full_path = it->path().string(); - size_t last_slash = full_path.rfind('/'); - String file_name = full_path.substr(last_slash); - /// Condition with next_slash means what we are looking for (it is from current position in psttern of path) + const std::string full_path = it->path().string(); + const size_t last_slash = full_path.rfind('/'); + const String file_name = full_path.substr(last_slash); + const bool looking_for_directory = next_slash != std::string::npos; /// Condition is_directory means what kind of path is it in current iteration of ls - if ((!fs::is_directory(it->path())) && (next_slash == std::string::npos)) + if (!fs::is_directory(it->path()) && !looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) { result.push_back(it->path().string()); } } - else if ((fs::is_directory(it->path())) && (next_slash != std::string::npos)) + else if (fs::is_directory(it->path()) && looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) { diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index d5668dbf1ae..eb38c579b7a 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -137,13 +137,13 @@ private: */ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match) { - size_t first_glob = for_match.find_first_of("*?{"); + const size_t first_glob = for_match.find_first_of("*?{"); - size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); - String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' - String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' + const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); + const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/' + const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/' - size_t next_slash = suffix_with_globs.find('/', 1); + const size_t next_slash = suffix_with_globs.find('/', 1); re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash))); HDFSFileInfo ls; @@ -151,19 +151,19 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c Strings result; for (int i = 0; i < ls.length; ++i) { - String full_path = String(ls.file_info[i].mName); - size_t last_slash = full_path.rfind('/'); - String file_name = full_path.substr(last_slash); - /// Condition with next_slash means what we are looking for (it is from current position in psttern of path) + const String full_path = String(ls.file_info[i].mName); + const size_t last_slash = full_path.rfind('/'); + const String file_name = full_path.substr(last_slash); + const bool looking_for_directory = next_slash != std::string::npos; /// Condition with type of current file_info means what kind of path is it in current iteration of ls - if ((ls.file_info[i].mKind == 'F') && (next_slash == std::string::npos)) + if ((ls.file_info[i].mKind == 'F') && !looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) { result.push_back(String(ls.file_info[i].mName)); } } - else if ((ls.file_info[i].mKind == 'D') && (next_slash != std::string::npos)) + else if ((ls.file_info[i].mKind == 'D') && looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) { From da11fb86b73f2ad7f3576dfac404bc2181d2584c Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 2 Sep 2019 17:29:51 +0300 Subject: [PATCH 44/75] implement 'with fill' modifier with processors --- dbms/src/Common/FillingRow.cpp | 127 +++++++++++ dbms/src/Common/FillingRow.h | 40 ++++ .../DataStreams/FillingBlockInputStream.cpp | 121 ----------- .../src/DataStreams/FillingBlockInputStream.h | 26 +-- .../Interpreters/InterpreterSelectQuery.cpp | 8 +- .../Transforms/FillingTransform.cpp | 200 ++++++++++++++++++ .../Processors/Transforms/FillingTransform.h | 40 ++++ 7 files changed, 414 insertions(+), 148 deletions(-) create mode 100644 dbms/src/Common/FillingRow.cpp create mode 100644 dbms/src/Common/FillingRow.h create mode 100644 dbms/src/Processors/Transforms/FillingTransform.cpp create mode 100644 dbms/src/Processors/Transforms/FillingTransform.h diff --git a/dbms/src/Common/FillingRow.cpp b/dbms/src/Common/FillingRow.cpp new file mode 100644 index 00000000000..cd150688246 --- /dev/null +++ b/dbms/src/Common/FillingRow.cpp @@ -0,0 +1,127 @@ +#include + +namespace DB +{ + +bool less(const Field & lhs, const Field & rhs, int direction) +{ + if (direction == -1) + return applyVisitor(FieldVisitorAccurateLess(), rhs, lhs); + + return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); +} + +bool equals(const Field & lhs, const Field & rhs) +{ + return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); +} + + +FillingRow::FillingRow(const SortDescription & description_) : description(description_) +{ + row.resize(description.size()); +} + +bool FillingRow::operator<(const FillingRow & other) const +{ + for (size_t i = 0; i < size(); ++i) + { + if (row[i].isNull() || other[i].isNull() || equals(row[i], other[i])) + continue; + return less(row[i], other[i], getDirection(i)); + } + return false; +} + +bool FillingRow::operator==(const FillingRow & other) const +{ + for (size_t i = 0; i < size(); ++i) + if (!equals(row[i], other[i])) + return false; + return true; +} + +bool FillingRow::next(const FillingRow & to_row) +{ + size_t pos = 0; + + /// Find position we need to increment for generating next row. + for (; pos < row.size(); ++pos) + if (!row[pos].isNull() && !to_row[pos].isNull() && !equals(row[pos], to_row[pos])) + break; + + if (pos == row.size() || less(to_row[pos], row[pos], getDirection(pos))) + return false; + + /// If we have any 'fill_to' value at position greater than 'pos', + /// we need to generate rows up to 'fill_to' value. + for (size_t i = row.size() - 1; i > pos; --i) + { + if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) + continue; + + auto next_value = row[i]; + applyVisitor(FieldVisitorSum(getFillDescription(i).fill_step), next_value); + if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) + { + row[i] = next_value; + initFromDefaults(i + 1); + return true; + } + } + + auto next_value = row[pos]; + applyVisitor(FieldVisitorSum(getFillDescription(pos).fill_step), next_value); + + if (less(to_row[pos], next_value, getDirection(pos))) + return false; + + row[pos] = next_value; + if (equals(row[pos], to_row[pos])) + { + bool is_less = false; + for (size_t i = pos + 1; i < size(); ++i) + { + const auto & fill_from = getFillDescription(i).fill_from; + if (!fill_from.isNull()) + row[i] = fill_from; + else + row[i] = to_row[i]; + is_less |= less(row[i], to_row[i], getDirection(i)); + } + + return is_less; + } + + initFromDefaults(pos + 1); + return true; +} + +void FillingRow::initFromDefaults(size_t from_pos) +{ + for (size_t i = from_pos; i < row.size(); ++i) + row[i] = getFillDescription(i).fill_from; +} + + +void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row) +{ + for (size_t i = 0; i < filling_columns.size(); ++i) + { + if (filling_row[i].isNull()) + filling_columns[i]->insertDefault(); + else + filling_columns[i]->insert(filling_row[i]); + } + + for (size_t i = 0; i < other_columns.size(); ++i) + other_columns[i]->insertDefault(); +} + +void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num) +{ + for (size_t i = 0; i < source.size(); ++i) + dest[i]->insertFrom(*source[i], row_num); +} + +} \ No newline at end of file diff --git a/dbms/src/Common/FillingRow.h b/dbms/src/Common/FillingRow.h new file mode 100644 index 00000000000..15a94431fd8 --- /dev/null +++ b/dbms/src/Common/FillingRow.h @@ -0,0 +1,40 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +/// Compares fields in terms of sorting order, considering direction. +bool less(const Field & lhs, const Field & rhs, int direction); +bool equals(const Field & lhs, const Field & rhs); + +class FillingRow +{ +public: + FillingRow(const SortDescription & sort_description); + + /// Generates next row according to fill 'from', 'to' and 'step' values. + bool next(const FillingRow & to_row); + + void initFromDefaults(size_t from_pos = 0); + + Field & operator[](size_t ind) { return row[ind]; } + const Field & operator[](size_t ind) const { return row[ind]; } + size_t size() const { return row.size(); } + bool operator<(const FillingRow & other) const; + bool operator==(const FillingRow & other) const; + + int getDirection(size_t ind) const { return description[ind].direction; } + FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } + +private: + std::vector row; + SortDescription description; +}; + +void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); +void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num); + +} \ No newline at end of file diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index 35166e3bd15..62db3eb0f0d 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -1,5 +1,4 @@ #include "FillingBlockInputStream.h" -#include #include #include @@ -11,126 +10,6 @@ namespace ErrorCodes extern const int INVALID_WITH_FILL_EXPRESSION; } -/// Compares fields in terms of sorting order, considering direction. -static bool less(const Field & lhs, const Field & rhs, int direction) -{ - if (direction == -1) - return applyVisitor(FieldVisitorAccurateLess(), rhs, lhs); - - return applyVisitor(FieldVisitorAccurateLess(), lhs, rhs); -} - -static bool equals(const Field & lhs, const Field & rhs) { return applyVisitor(FieldVisitorAccurateEquals(), lhs, rhs); } - -FillingRow::FillingRow(const SortDescription & description_) : description(description_) -{ - row.resize(description.size()); -} - -bool FillingRow::operator<(const FillingRow & other) const -{ - for (size_t i = 0; i < size(); ++i) - { - if (row[i].isNull() || other[i].isNull() || equals(row[i], other[i])) - continue; - return less(row[i], other[i], getDirection(i)); - } - return false; -} - -bool FillingRow::operator==(const FillingRow & other) const -{ - for (size_t i = 0; i < size(); ++i) - if (!equals(row[i], other[i])) - return false; - return true; -} - -bool FillingRow::next(const FillingRow & to_row) -{ - size_t pos = 0; - - /// Find position we need to increment for generating next row. - for (; pos < row.size(); ++pos) - if (!row[pos].isNull() && !to_row[pos].isNull() && !equals(row[pos], to_row[pos])) - break; - - if (pos == row.size() || less(to_row[pos], row[pos], getDirection(pos))) - return false; - - /// If we have any 'fill_to' value at position greater than 'pos', - /// we need to generate rows up to 'fill_to' value. - for (size_t i = row.size() - 1; i > pos; --i) - { - if (getFillDescription(i).fill_to.isNull() || row[i].isNull()) - continue; - - auto next_value = row[i]; - applyVisitor(FieldVisitorSum(getFillDescription(i).fill_step), next_value); - if (less(next_value, getFillDescription(i).fill_to, getDirection(i))) - { - row[i] = next_value; - initFromDefaults(i + 1); - return true; - } - } - - auto next_value = row[pos]; - applyVisitor(FieldVisitorSum(getFillDescription(pos).fill_step), next_value); - - if (less(to_row[pos], next_value, getDirection(pos))) - return false; - - row[pos] = next_value; - if (equals(row[pos], to_row[pos])) - { - bool is_less = false; - for (size_t i = pos + 1; i < size(); ++i) - { - const auto & fill_from = getFillDescription(i).fill_from; - if (!fill_from.isNull()) - row[i] = fill_from; - else - row[i] = to_row[i]; - is_less |= less(row[i], to_row[i], getDirection(i)); - } - - return is_less; - } - - initFromDefaults(pos + 1); - return true; -} - - -void FillingRow::initFromDefaults(size_t from_pos) -{ - for (size_t i = from_pos; i < row.size(); ++i) - row[i] = getFillDescription(i).fill_from; -} - - -static void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row) -{ - for (size_t i = 0; i < filling_columns.size(); ++i) - { - if (filling_row[i].isNull()) - filling_columns[i]->insertDefault(); - else - filling_columns[i]->insert(filling_row[i]); - } - - for (size_t i = 0; i < other_columns.size(); ++i) - other_columns[i]->insertDefault(); -} - -static void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num) -{ - for (size_t i = 0; i < source.size(); ++i) - dest[i]->insertFrom(*source[i], row_num); -} - - FillingBlockInputStream::FillingBlockInputStream( const BlockInputStreamPtr & input, const SortDescription & sort_description_) : sort_description(sort_description_), filling_row(sort_description_), next_row(sort_description_) diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index 279b0aea901..5623cf9087b 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -1,35 +1,11 @@ #pragma once #include +#include namespace DB { -class FillingRow -{ -public: - FillingRow(const SortDescription & sort_description); - - /// Generates next row according to fill 'from', 'to' and 'step' values. - bool next(const FillingRow & to_row); - - void initFromDefaults(size_t from_pos = 0); - void initFromColumns(const Columns & columns, size_t row_ind, size_t from_pos); - - Field & operator[](size_t ind) { return row[ind]; } - const Field & operator[](size_t ind) const { return row[ind]; } - size_t size() const { return row.size(); } - bool operator<(const FillingRow & other) const; - bool operator==(const FillingRow & other) const; - - int getDirection(size_t ind) const { return description[ind].direction; } - FillColumnDescription & getFillDescription(size_t ind) { return description[ind].fill_description; } - -private: - std::vector row; - SortDescription description; -}; - /** Implements the WITH FILL part of ORDER BY operation. */ class FillingBlockInputStream : public IBlockInputStream diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index 587c30fcf7b..e928246c630 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -83,6 +83,7 @@ #include #include #include +#include #include #include #include @@ -2491,7 +2492,7 @@ void InterpreterSelectQuery::executeWithFill(Pipeline & pipeline) } } -void InterpreterSelectQuery::executeWithFill(QueryPipeline &) +void InterpreterSelectQuery::executeWithFill(QueryPipeline & pipeline) { auto & query = getSelectQuery(); if (query.orderBy()) @@ -2507,7 +2508,10 @@ void InterpreterSelectQuery::executeWithFill(QueryPipeline &) if (fill_descr.empty()) return; - throw Exception("Unsupported WITH FILL with processors", ErrorCodes::NOT_IMPLEMENTED); + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, fill_descr); + }); } } diff --git a/dbms/src/Processors/Transforms/FillingTransform.cpp b/dbms/src/Processors/Transforms/FillingTransform.cpp new file mode 100644 index 00000000000..077d9e0a248 --- /dev/null +++ b/dbms/src/Processors/Transforms/FillingTransform.cpp @@ -0,0 +1,200 @@ +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_WITH_FILL_EXPRESSION; +} + + +FillingTransform::FillingTransform( + const Block & header_, const SortDescription & sort_description_) + : ISimpleTransform(header_, header_, true) + , sort_description(sort_description_) + , filling_row(sort_description_) + , next_row(sort_description_) +{ + std::vector is_fill_column(header_.columns()); + for (const auto & elem : sort_description) + is_fill_column[header_.getPositionByName(elem.column_name)] = true; + + auto try_convert_fields = [](FillColumnDescription & descr, const DataTypePtr & type) + { + auto max_type = Field::Types::Null; + WhichDataType which(type); + DataTypePtr to_type; + if (isInteger(type) || which.isDateOrDateTime()) + { + max_type = Field::Types::Int64; + to_type = std::make_shared(); + } + else if (which.isFloat()) + { + max_type = Field::Types::Float64; + to_type = std::make_shared(); + } + + if (descr.fill_from.getType() > max_type || descr.fill_to.getType() > max_type + || descr.fill_step.getType() > max_type) + return false; + + descr.fill_from = convertFieldToType(descr.fill_from, *to_type); + descr.fill_to = convertFieldToType(descr.fill_to, *to_type); + descr.fill_step = convertFieldToType(descr.fill_step, *to_type); + + return true; + }; + + for (size_t i = 0; i < header_.columns(); ++i) + { + if (is_fill_column[i]) + { + auto type = header_.getByPosition(i).type; + auto & descr = filling_row.getFillDescription(i); + if (!try_convert_fields(descr, type)) + throw Exception("Incompatible types of WITH FILL expression values with column type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + + if (type->isValueRepresentedByUnsignedInteger() && + ((!descr.fill_from.isNull() && less(descr.fill_from, Field{0}, 1)) || + (!descr.fill_to.isNull() && less(descr.fill_to, Field{0}, 1)))) + { + throw Exception("WITH FILL bound values cannot be negative for unsigned type " + + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); + } + + fill_column_positions.push_back(i); + } + else + other_column_positions.push_back(i); + } +} + +IProcessor::Status FillingTransform::prepare() +{ + if (input.isFinished() && !output.isFinished() && !has_input && !generate_suffix) + { + should_insert_first = next_row < filling_row; + + for (size_t i = 0; i < filling_row.size(); ++i) + next_row[i] = filling_row.getFillDescription(i).fill_to; + + if (filling_row < next_row) + { + generate_suffix = true; + return Status::Ready; + } + } + + return ISimpleTransform::prepare(); +} + + +void FillingTransform::transform(Chunk & chunk) +{ + Columns old_fill_columns; + Columns old_other_columns; + MutableColumns res_fill_columns; + MutableColumns res_other_columns; + + auto init_columns_by_positions = [](const Columns & old_columns, Columns & new_columns, + MutableColumns & new_mutable_columns, const Positions & positions) + { + for (size_t pos : positions) + { + new_columns.push_back(old_columns[pos]); + new_mutable_columns.push_back(old_columns[pos]->cloneEmpty()->assumeMutable()); + } + }; + + if (generate_suffix) + { + const auto & empty_columns = inputs.front().getHeader().getColumns(); + init_columns_by_positions(empty_columns, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(empty_columns, old_other_columns, res_other_columns, other_column_positions); + + if (should_insert_first && filling_row < next_row) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + + while (filling_row.next(next_row)) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + + setResultColumns(chunk, res_fill_columns, res_other_columns); + return; + } + + size_t num_rows = chunk.getNumRows(); + auto old_columns = chunk.detachColumns(); + + init_columns_by_positions(old_columns, old_fill_columns, res_fill_columns, fill_column_positions); + init_columns_by_positions(old_columns, old_other_columns, res_other_columns, other_column_positions); + + if (first) + { + for (size_t i = 0; i < filling_row.size(); ++i) + { + auto current_value = (*old_fill_columns[i])[0]; + const auto & fill_from = filling_row.getFillDescription(i).fill_from; + + if (!fill_from.isNull() && !equals(current_value, fill_from)) + { + filling_row.initFromDefaults(i); + if (less(fill_from, current_value, filling_row.getDirection(i))) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + break; + } + filling_row[i] = current_value; + } + first = false; + } + + for (size_t row_ind = 0; row_ind < num_rows; ++row_ind) + { + should_insert_first = next_row < filling_row; + + for (size_t i = 0; i < filling_row.size(); ++i) + { + auto current_value = (*old_fill_columns[i])[row_ind]; + const auto & fill_to = filling_row.getFillDescription(i).fill_to; + + if (fill_to.isNull() || less(current_value, fill_to, filling_row.getDirection(i))) + next_row[i] = current_value; + else + next_row[i] = fill_to; + } + + /// A case, when at previous step row was initialized from defaults 'fill_from' values + /// and probably we need to insert it to block. + if (should_insert_first && filling_row < next_row) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + + /// Insert generated filling row to block, while it is less than current row in block. + while (filling_row.next(next_row)) + insertFromFillingRow(res_fill_columns, res_other_columns, filling_row); + + copyRowFromColumns(res_fill_columns, old_fill_columns, row_ind); + copyRowFromColumns(res_other_columns, old_other_columns, row_ind); + } + + setResultColumns(chunk, res_fill_columns, res_other_columns); +} + +void FillingTransform::setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const +{ + MutableColumns result_columns(fill_columns.size() + other_columns.size()); + /// fill_columns always non-empty. + size_t num_rows = fill_columns[0]->size(); + + for (size_t i = 0; i < fill_columns.size(); ++i) + result_columns[fill_column_positions[i]] = std::move(fill_columns[i]); + for (size_t i = 0; i < other_columns.size(); ++i) + result_columns[other_column_positions[i]] = std::move(other_columns[i]); + + chunk.setColumns(std::move(result_columns), num_rows); +} + +} diff --git a/dbms/src/Processors/Transforms/FillingTransform.h b/dbms/src/Processors/Transforms/FillingTransform.h new file mode 100644 index 00000000000..2a615cd0bb0 --- /dev/null +++ b/dbms/src/Processors/Transforms/FillingTransform.h @@ -0,0 +1,40 @@ +#pragma once +#include +#include +#include + +namespace DB +{ + +/** Implements the WITH FILL part of ORDER BY operation. +*/ +class FillingTransform : public ISimpleTransform +{ +public: + FillingTransform(const Block & header_, const SortDescription & fill_description_); + + String getName() const override { return "FillingTransform"; } + + Status prepare() override; + +protected: + void transform(Chunk & Chunk) override; + +private: + void setResultColumns(Chunk & chunk, MutableColumns & fill_columns, MutableColumns & other_columns) const; + + const SortDescription sort_description; /// Contains only rows with WITH FILL. + FillingRow filling_row; /// Current row, which is used to fill gaps. + FillingRow next_row; /// Row to which we need to generate filling rows. + + using Positions = std::vector; + Positions fill_column_positions; + Positions other_column_positions; + bool first = true; + bool generate_suffix = false; + + /// Determines should we insert filling row before start generating next rows. + bool should_insert_first = false; +}; + +} From e122d54dcb0da383966a648051f0e7379c70ba9e Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 2 Sep 2019 18:22:27 +0300 Subject: [PATCH 45/75] update comments --- dbms/src/Common/FillingRow.h | 4 ++++ dbms/src/DataStreams/FillingBlockInputStream.h | 6 ++++-- dbms/src/Processors/Transforms/FillingTransform.h | 6 ++++-- 3 files changed, 12 insertions(+), 4 deletions(-) diff --git a/dbms/src/Common/FillingRow.h b/dbms/src/Common/FillingRow.h index 15a94431fd8..e7998139437 100644 --- a/dbms/src/Common/FillingRow.h +++ b/dbms/src/Common/FillingRow.h @@ -10,6 +10,10 @@ namespace DB bool less(const Field & lhs, const Field & rhs, int direction); bool equals(const Field & lhs, const Field & rhs); +/** Helps to implement modifier WITH FILL for ORDER BY clause. + * Stores row as array of fields and provides functions to generate next row for filling gaps and for comparing rows. + * Used in FillingBlockInputStream and in FillingTransform. + */ class FillingRow { public: diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index 5623cf9087b..00308d27471 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -6,8 +6,10 @@ namespace DB { -/** Implements the WITH FILL part of ORDER BY operation. -*/ +/** Implements modifier WITH FILL of ORDER BY clause. + * It fills gaps in data stream by rows with missing values in columns with set WITH FILL and deafults in other columns. + * Optionally FROM, TO and STEP values can be specified. + */ class FillingBlockInputStream : public IBlockInputStream { public: diff --git a/dbms/src/Processors/Transforms/FillingTransform.h b/dbms/src/Processors/Transforms/FillingTransform.h index 2a615cd0bb0..d49d0b844f6 100644 --- a/dbms/src/Processors/Transforms/FillingTransform.h +++ b/dbms/src/Processors/Transforms/FillingTransform.h @@ -6,8 +6,10 @@ namespace DB { -/** Implements the WITH FILL part of ORDER BY operation. -*/ +/** Implements modifier WITH FILL of ORDER BY clause. + * It fills gaps in data stream by rows with missing values in columns with set WITH FILL and deafult values in other columns. + * Optionally FROM, TO and STEP values can be specified. + */ class FillingTransform : public ISimpleTransform { public: From 4fea3e96881bb9f73fa0f0369219941d088af2ef Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Mon, 2 Sep 2019 20:18:44 +0300 Subject: [PATCH 46/75] style fixes --- dbms/src/Common/FillingRow.cpp | 2 +- dbms/src/Common/FillingRow.h | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/src/Common/FillingRow.cpp b/dbms/src/Common/FillingRow.cpp index cd150688246..f0edfc958a0 100644 --- a/dbms/src/Common/FillingRow.cpp +++ b/dbms/src/Common/FillingRow.cpp @@ -124,4 +124,4 @@ void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t ro dest[i]->insertFrom(*source[i], row_num); } -} \ No newline at end of file +} diff --git a/dbms/src/Common/FillingRow.h b/dbms/src/Common/FillingRow.h index e7998139437..1753508e139 100644 --- a/dbms/src/Common/FillingRow.h +++ b/dbms/src/Common/FillingRow.h @@ -10,7 +10,7 @@ namespace DB bool less(const Field & lhs, const Field & rhs, int direction); bool equals(const Field & lhs, const Field & rhs); -/** Helps to implement modifier WITH FILL for ORDER BY clause. +/** Helps to implement modifier WITH FILL for ORDER BY clause. * Stores row as array of fields and provides functions to generate next row for filling gaps and for comparing rows. * Used in FillingBlockInputStream and in FillingTransform. */ @@ -41,4 +41,4 @@ private: void insertFromFillingRow(MutableColumns & filling_columns, MutableColumns & other_columns, const FillingRow & filling_row); void copyRowFromColumns(MutableColumns & dest, const Columns & source, size_t row_num); -} \ No newline at end of file +} From 6d821c4877573afba4f80fdcebe6e8aa66fc88b3 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Tue, 3 Sep 2019 13:07:31 +0300 Subject: [PATCH 47/75] fix build --- dbms/src/DataStreams/FillingBlockInputStream.cpp | 2 +- dbms/src/DataStreams/FillingBlockInputStream.h | 2 +- dbms/src/{Common => Interpreters}/FillingRow.cpp | 2 +- dbms/src/{Common => Interpreters}/FillingRow.h | 0 dbms/src/Processors/Transforms/FillingTransform.h | 2 +- 5 files changed, 4 insertions(+), 4 deletions(-) rename dbms/src/{Common => Interpreters}/FillingRow.cpp (98%) rename dbms/src/{Common => Interpreters}/FillingRow.h (100%) diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index 62db3eb0f0d..c73fb899048 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -1,4 +1,4 @@ -#include "FillingBlockInputStream.h" +#include #include #include diff --git a/dbms/src/DataStreams/FillingBlockInputStream.h b/dbms/src/DataStreams/FillingBlockInputStream.h index 00308d27471..3cc4702e374 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.h +++ b/dbms/src/DataStreams/FillingBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include namespace DB { diff --git a/dbms/src/Common/FillingRow.cpp b/dbms/src/Interpreters/FillingRow.cpp similarity index 98% rename from dbms/src/Common/FillingRow.cpp rename to dbms/src/Interpreters/FillingRow.cpp index f0edfc958a0..9d4c81dc70b 100644 --- a/dbms/src/Common/FillingRow.cpp +++ b/dbms/src/Interpreters/FillingRow.cpp @@ -1,4 +1,4 @@ -#include +#include namespace DB { diff --git a/dbms/src/Common/FillingRow.h b/dbms/src/Interpreters/FillingRow.h similarity index 100% rename from dbms/src/Common/FillingRow.h rename to dbms/src/Interpreters/FillingRow.h diff --git a/dbms/src/Processors/Transforms/FillingTransform.h b/dbms/src/Processors/Transforms/FillingTransform.h index d49d0b844f6..5c4c78701f5 100644 --- a/dbms/src/Processors/Transforms/FillingTransform.h +++ b/dbms/src/Processors/Transforms/FillingTransform.h @@ -1,7 +1,7 @@ #pragma once #include #include -#include +#include namespace DB { From 97ac979645e9ca21ba8034cdd9447d83f47cd7bc Mon Sep 17 00:00:00 2001 From: stavrolia Date: Tue, 3 Sep 2019 17:23:51 +0300 Subject: [PATCH 48/75] Add docs for hdfs and fix some review comments --- dbms/src/Common/parseGlobs.cpp | 54 +++++++++---------- docs/en/operations/table_engines/file.md | 10 ++-- docs/en/operations/table_engines/hdfs.md | 45 ++++++++++++++++ .../en/query_language/table_functions/file.md | 2 +- .../en/query_language/table_functions/hdfs.md | 37 +++++++++++++ docs/ru/operations/table_engines/hdfs.md | 42 +++++++++++++++ .../ru/query_language/table_functions/hdfs.md | 36 +++++++++++++ 7 files changed, 192 insertions(+), 34 deletions(-) create mode 100644 docs/en/operations/table_engines/hdfs.md create mode 100644 docs/en/query_language/table_functions/hdfs.md create mode 100644 docs/ru/operations/table_engines/hdfs.md create mode 100644 docs/ru/query_language/table_functions/hdfs.md diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp index 1c1033dc2b4..cdb1dc6cedd 100644 --- a/dbms/src/Common/parseGlobs.cpp +++ b/dbms/src/Common/parseGlobs.cpp @@ -2,12 +2,13 @@ #include #include #include +#include namespace DB { /* Transforms string from grep-wildcard-syntax ("{N..M}", "{a,b,c}" as in remote table function and "*", "?") to perl-regexp for using re2 library fo matching * with such steps: - * 1) search intervals and enums in {}, replace them by regexp with pipe (expr1|expr2|expr3), + * 1) search intervals like {0..9} and enums like {abc,xyz,qwe} in {}, replace them by regexp with pipe (expr1|expr2|expr3), * 2) search and replace "*" and "?". * Before each search need to escape symbols that we would not search. * @@ -15,63 +16,60 @@ namespace DB */ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs) { - std::string escaped_with_globs; - escaped_with_globs.reserve(initial_str_with_globs.size()); + std::ostringstream oss; /// Escaping only characters that not used in glob syntax for (const auto & letter : initial_str_with_globs) { if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')')) - escaped_with_globs.push_back('\\'); - escaped_with_globs.push_back(letter); + oss << '\\'; + oss << letter; } - - re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*, + std::string escaped_with_globs = oss.str(); + oss.str(""); + static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*, re2::StringPiece input(escaped_with_globs); - re2::StringPiece matched(escaped_with_globs); + re2::StringPiece matched; size_t current_index = 0; - std::string almost_regexp; - almost_regexp.reserve(escaped_with_globs.size()); while (RE2::FindAndConsume(&input, enum_or_range, &matched)) { std::string buffer = matched.ToString(); - almost_regexp.append(escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1)); + oss << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; if (buffer.find(',') == std::string::npos) { - size_t first_point = buffer.find('.'); - std::string first_number = buffer.substr(0, first_point); - std::string second_number = buffer.substr(first_point + 2, buffer.size() - first_point - 2); - size_t range_begin = std::stoull(first_number); - size_t range_end = std::stoull(second_number); - buffer = std::to_string(range_begin); + size_t range_begin, range_end; + char point; + std::istringstream iss(buffer); + iss >> range_begin >> point >> point >> range_end; + oss << range_begin; for (size_t i = range_begin + 1; i <= range_end; ++i) { - buffer += "|"; - buffer += std::to_string(i); + oss << '|' << i; } } else { std::replace(buffer.begin(), buffer.end(), ',', '|'); + oss << buffer; } - almost_regexp.append("(" + buffer + ")"); + oss << ")"; current_index = input.data() - escaped_with_globs.data(); } - almost_regexp += escaped_with_globs.substr(current_index); - std::string result; - result.reserve(almost_regexp.size()); - for (const auto & letter : almost_regexp) + oss << escaped_with_globs.substr(current_index); + std::string almost_res = oss.str(); + oss.str(""); + for (const auto & letter : almost_res) { if ((letter == '?') || (letter == '*')) { - result += "[^/]"; /// '?' is any symbol except '/' + oss << "[^/]"; /// '?' is any symbol except '/' if (letter == '?') continue; } if ((letter == '.') || (letter == '{') || (letter == '}')) - result.push_back('\\'); - result.push_back(letter); + oss << '\\'; + oss << letter; } - return result; + return oss.str(); } } diff --git a/docs/en/operations/table_engines/file.md b/docs/en/operations/table_engines/file.md index 9e3c5b3400b..bd7ee3cb90e 100644 --- a/docs/en/operations/table_engines/file.md +++ b/docs/en/operations/table_engines/file.md @@ -27,7 +27,7 @@ When creating table using `File(Format)` it creates empty subdirectory in that f You may manually create this subfolder and file in server filesystem and then [ATTACH](../../query_language/misc.md) it to table information with matching name, so you can query data from that file. !!! warning - Be careful with this funcionality, because ClickHouse does not keep track of external changes to such files. The result of simultaneous writes via ClickHouse and outside of ClickHouse is undefined. + Be careful with this functionality, because ClickHouse does not keep track of external changes to such files. The result of simultaneous writes via ClickHouse and outside of ClickHouse is undefined. **Example:** @@ -73,9 +73,9 @@ $ echo -e "1,2\n3,4" | clickhouse-local -q "CREATE TABLE table (a Int64, b Int64 - Multiple `SELECT` queries can be performed concurrently, but `INSERT` queries will wait each other. - Not supported: - - `ALTER` - - `SELECT ... SAMPLE` - - Indices - - Replication + - `ALTER` + - `SELECT ... SAMPLE` + - Indices + - Replication [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/file/) diff --git a/docs/en/operations/table_engines/hdfs.md b/docs/en/operations/table_engines/hdfs.md new file mode 100644 index 00000000000..54d7cda27aa --- /dev/null +++ b/docs/en/operations/table_engines/hdfs.md @@ -0,0 +1,45 @@ +# HDFS {#table_engines-hdfs} + +Manages data on HDFS. This engine is similar +to the [File](file.md) and [URL](url.md) engine. + +## Usage + +``` +ENGINE = HDFS(URI, format) +``` + +The `format` parameter specifies one of the available file formats. To perform +`SELECT` queries, the format must be supported for input, and to perform +`INSERT` queries -- for output. The available formats are listed in the +[Formats](../../interfaces/formats.md#formats) section. + +**Example:** + +**1.** Set up the `HDFS_engine_table` table: + +``` sql +CREATE TABLE hdfs_engine_table (name String, value UInt32) ENGINE=HDFS('hdfs://hdfs1:9000/other_storage', 'TSV') +``` +**2.** Query the data: + +``` sql +SELECT * FROM hdfs_engine_table LIMIT 2 +``` + +``` +┌─name─┬─value─┐ +│ one │ 1 │ +│ two │ 2 │ +└──────┴───────┘ +``` + +## Details of Implementation + +- Reads and writes can be parallel +- Not supported: + - `ALTER` and `SELECT...SAMPLE` operations. + - Indexes. + - Replication. + +[Original article](https://clickhouse.yandex/docs/en/operations/table_engines/hdfs/) diff --git a/docs/en/query_language/table_functions/file.md b/docs/en/query_language/table_functions/file.md index ff9ba31cb28..4a1a7080f30 100644 --- a/docs/en/query_language/table_functions/file.md +++ b/docs/en/query_language/table_functions/file.md @@ -11,7 +11,7 @@ file(path, format, structure) - `path` — The relative path to the file from [user_files_path](../../operations/server_settings/settings.md#server_settings-user_files_path). - `format` — The [format](../../interfaces/formats.md#formats) of the file. -- `structure` — Structure of the table. Format `'colunmn1_name column1_ype, column2_name column2_type, ...'`. +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. **Returned value** diff --git a/docs/en/query_language/table_functions/hdfs.md b/docs/en/query_language/table_functions/hdfs.md new file mode 100644 index 00000000000..16c262aed7e --- /dev/null +++ b/docs/en/query_language/table_functions/hdfs.md @@ -0,0 +1,37 @@ + +# hdfs + +Creates a table from a file in hdfs. + +``` +hdfs(URI, format, structure) +``` + +**Input parameters** + +- `URI` — The relative URI to the file in HDFS. +- `format` — The [format](../../interfaces/formats.md#formats) of the file. +- `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. + +**Returned value** + +A table with the specified structure for reading or writing data in the specified file. + +**Example** + +Table from `hdfs://hdfs1:9000/test` and selection of the first two rows from it: + +```sql +SELECT * +FROM hdfs('hdfs://hdfs1:9000/test', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') +LIMIT 2 +``` + +``` +┌─column1─┬─column2─┬─column3─┐ +│ 1 │ 2 │ 3 │ +│ 3 │ 2 │ 1 │ +└─────────┴─────────┴─────────┘ +``` + +[Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/hdfs/) diff --git a/docs/ru/operations/table_engines/hdfs.md b/docs/ru/operations/table_engines/hdfs.md new file mode 100644 index 00000000000..a77d923985c --- /dev/null +++ b/docs/ru/operations/table_engines/hdfs.md @@ -0,0 +1,42 @@ +# HDFS {#table_engines-hdfs} + +Управляет данными в HDFS. Данный движок похож на движок [File](file.md) и на движок [URL](url.md). + +## Использование движка + +``` +ENGINE = HDFS(URI, format) +``` + +Параметр `format` должен быть таким, который ClickHouse может использовать и в запросах `INSERT`, и в запросах `SELECT`. Полный список поддерживаемых форматов смотрите в разделе [Форматы](../../interfaces/formats.md#formats). + +**Пример:** + +**1.** Создадим на сервере таблицу `hdfs_engine_table`: + +``` sql +CREATE TABLE hdfs_engine_table (name String, value UInt32) ENGINE=HDFS('hdfs://hdfs1:9000/other_storage', 'TSV') +``` + +**2.** Запросим данные: + +``` sql +SELECT * FROM hdfs_engine_table LIMIT 2 +``` + +``` +┌─name─┬─value─┐ +│ one │ 1 │ +│ two │ 2 │ +└──────┴───────┘ +``` + +## Детали реализации + +- Поддерживается многопоточное чтение и запись. +- Не поддерживается: + - использование операций `ALTER` и `SELECT...SAMPLE`; + - индексы; + - репликация. + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/hdfs/) diff --git a/docs/ru/query_language/table_functions/hdfs.md b/docs/ru/query_language/table_functions/hdfs.md new file mode 100644 index 00000000000..36a75ffbf42 --- /dev/null +++ b/docs/ru/query_language/table_functions/hdfs.md @@ -0,0 +1,36 @@ + +# hdfs + +Создаёт таблицу из файла. + +``` +hdfs(URI, format, structure) +``` + +**Входные параметры** + +- `URI` — URI до файла в HDFS. +- `format` — [формат](../../interfaces/formats.md#formats) файла. +- `structure` — структура таблицы. Формат `'column1_name column1_type, column2_name column2_type, ...'`. + +**Возвращаемое значение** + +Таблица с указанной структурой, предназначенная для чтения или записи данных в указанном файле. + +**Пример** + +Таблица из `hdfs://hdfs1:9000/test` и выборка первых двух строк из неё: + +``` sql +SELECT * +FROM hdfs('hdfs://hdfs1:9000/test', 'TSV', 'column1 UInt32, column2 UInt32, column3 UInt32') +LIMIT 2 +``` +``` +┌─column1─┬─column2─┬─column3─┐ +│ 1 │ 2 │ 3 │ +│ 3 │ 2 │ 1 │ +└─────────┴─────────┴─────────┘ +``` + +[Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/hdfs/) From 9cfcbd66499a017b0d6ce36c86b02e910ddc5805 Mon Sep 17 00:00:00 2001 From: CurtizJ Date: Wed, 4 Sep 2019 01:01:09 +0300 Subject: [PATCH 49/75] fix bug in FillingBlockInputStream --- dbms/src/DataStreams/FillingBlockInputStream.cpp | 4 ++-- dbms/src/Processors/Transforms/FillingTransform.cpp | 3 ++- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/DataStreams/FillingBlockInputStream.cpp b/dbms/src/DataStreams/FillingBlockInputStream.cpp index c73fb899048..ec026d56ad0 100644 --- a/dbms/src/DataStreams/FillingBlockInputStream.cpp +++ b/dbms/src/DataStreams/FillingBlockInputStream.cpp @@ -40,7 +40,6 @@ FillingBlockInputStream::FillingBlockInputStream( if (descr.fill_from.getType() > max_type || descr.fill_to.getType() > max_type || descr.fill_step.getType() > max_type) return false; - descr.fill_from = convertFieldToType(descr.fill_from, *to_type); descr.fill_to = convertFieldToType(descr.fill_to, *to_type); descr.fill_step = convertFieldToType(descr.fill_step, *to_type); @@ -52,8 +51,9 @@ FillingBlockInputStream::FillingBlockInputStream( { if (is_fill_column[i]) { + size_t pos = fill_column_positions.size(); + auto & descr = filling_row.getFillDescription(pos); auto type = header.getByPosition(i).type; - auto & descr = filling_row.getFillDescription(i); if (!try_convert_fields(descr, type)) throw Exception("Incompatible types of WITH FILL expression values with column type " + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); diff --git a/dbms/src/Processors/Transforms/FillingTransform.cpp b/dbms/src/Processors/Transforms/FillingTransform.cpp index 077d9e0a248..50fac121819 100644 --- a/dbms/src/Processors/Transforms/FillingTransform.cpp +++ b/dbms/src/Processors/Transforms/FillingTransform.cpp @@ -53,8 +53,9 @@ FillingTransform::FillingTransform( { if (is_fill_column[i]) { + size_t pos = fill_column_positions.size(); + auto & descr = filling_row.getFillDescription(pos); auto type = header_.getByPosition(i).type; - auto & descr = filling_row.getFillDescription(i); if (!try_convert_fields(descr, type)) throw Exception("Incompatible types of WITH FILL expression values with column type " + type->getName(), ErrorCodes::INVALID_WITH_FILL_EXPRESSION); From 904cfed046946dcda32a83374c84b1171fea1489 Mon Sep 17 00:00:00 2001 From: chertus Date: Tue, 3 Sep 2019 19:56:32 +0300 Subject: [PATCH 50/75] better includes --- dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp | 3 ++- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 1 + dbms/src/Interpreters/ExpressionAnalyzer.h | 3 --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 1 + dbms/src/Interpreters/QueryNormalizer.cpp | 1 - dbms/src/Interpreters/SubqueryForSet.cpp | 1 - dbms/src/Interpreters/SyntaxAnalyzer.cpp | 1 + dbms/src/Interpreters/SyntaxAnalyzer.h | 5 ++++- 8 files changed, 9 insertions(+), 7 deletions(-) diff --git a/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp b/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp index 4b0a76cfb37..ec29fbf40c1 100644 --- a/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp +++ b/dbms/src/Interpreters/DatabaseAndTableWithAlias.cpp @@ -1,6 +1,5 @@ #include #include -#include /// for getNamesAndTypeListFromTableExpression #include #include @@ -15,6 +14,8 @@ namespace DB NameSet removeDuplicateColumns(NamesAndTypesList & columns); +struct ASTTableExpression; +NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context); DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database) { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index d82169cf8e4..d2994ec22a1 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -29,6 +29,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index aebbaf038cc..7a5054ea040 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -26,9 +26,6 @@ class ASTExpressionList; class ASTSelectQuery; struct ASTTablesInSelectQueryElement; -struct SyntaxAnalyzerResult; -using SyntaxAnalyzerResultPtr = std::shared_ptr; - /// ExpressionAnalyzer sources, intermediates and results. It splits data and logic, allows to test them separately. struct ExpressionAnalyzerData { diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index e1470f33ca2..c93ba4a0680 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -44,6 +44,7 @@ #include #include #include +#include #include #include diff --git a/dbms/src/Interpreters/QueryNormalizer.cpp b/dbms/src/Interpreters/QueryNormalizer.cpp index ffa94f3d700..c2991885cf3 100644 --- a/dbms/src/Interpreters/QueryNormalizer.cpp +++ b/dbms/src/Interpreters/QueryNormalizer.cpp @@ -3,7 +3,6 @@ #include #include #include -#include #include #include #include diff --git a/dbms/src/Interpreters/SubqueryForSet.cpp b/dbms/src/Interpreters/SubqueryForSet.cpp index f6528bf110c..5572d60d3e9 100644 --- a/dbms/src/Interpreters/SubqueryForSet.cpp +++ b/dbms/src/Interpreters/SubqueryForSet.cpp @@ -1,5 +1,4 @@ #include -#include #include #include diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 3419a5baba3..dd0c37c50b5 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -17,6 +17,7 @@ #include #include #include +#include #include /// getSmallestColumn() #include diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index a2187ee2ef0..44fdc61ded3 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -1,7 +1,7 @@ #pragma once +#include #include -#include #include #include @@ -11,6 +11,9 @@ namespace DB NameSet removeDuplicateColumns(NamesAndTypesList & columns); class ASTFunction; +class AnalyzedJoin; +class Context; +struct SelectQueryOptions; struct SyntaxAnalyzerResult { From 32bf915610d48abef46d212f39a837e1d614a5b1 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Wed, 4 Sep 2019 14:11:30 +0300 Subject: [PATCH 51/75] More docs --- dbms/src/Storages/StorageFile.cpp | 44 +++++++------------ dbms/src/Storages/StorageFile.h | 6 +-- dbms/src/Storages/StorageHDFS.cpp | 18 ++------ .../en/query_language/table_functions/file.md | 14 +++++- .../en/query_language/table_functions/hdfs.md | 16 ++++++- .../ru/query_language/table_functions/file.md | 14 +++++- .../ru/query_language/table_functions/hdfs.md | 16 ++++++- 7 files changed, 76 insertions(+), 52 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index c1da0f1253d..bc745092e0b 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -53,7 +53,6 @@ namespace */ std::vector LSWithRegexpMatching(const std::string & path_for_ls, const std::string & for_match) { - const size_t first_glob = for_match.find_first_of("*?{"); const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/'); @@ -138,22 +137,14 @@ StorageFile::StorageFile( if (!table_path_.empty()) /// Is user's file { - /// rwlock is common because path with globs and many files in this case is readonly now Poco::Path poco_path = Poco::Path(table_path_); if (poco_path.isRelative()) poco_path = Poco::Path(db_dir_path, poco_path); - path = poco_path.absolute().toString(); - size_t first_glob = path.find_first_of("*?{"); - if (first_glob != std::string::npos) - { - path_with_globs = true; - matched_paths = LSWithRegexpMatching("/", path); - for (const auto & cur_path : matched_paths) - checkCreationIsAllowed(context_global, db_dir_path, cur_path, table_fd); - } - else - checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); + const std::string path = poco_path.absolute().toString(); + paths = LSWithRegexpMatching("/", path); + for (const auto & cur_path : paths) + checkCreationIsAllowed(context_global, db_dir_path, cur_path, table_fd); is_db_table = false; } else /// Is DB's file @@ -161,14 +152,14 @@ StorageFile::StorageFile( if (db_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - path = getTablePath(db_dir_path, table_name, format_name); + paths[0] = getTablePath(db_dir_path, table_name, format_name); is_db_table = true; - Poco::File(Poco::Path(path).parent()).createDirectories(); + Poco::File(Poco::Path(paths[0]).parent()).createDirectories(); } } else /// Will use FD { - checkCreationIsAllowed(context_global, db_dir_path, path, table_fd); + checkCreationIsAllowed(context_global, db_dir_path, paths[0], table_fd); is_db_table = false; use_table_fd = true; @@ -258,16 +249,9 @@ BlockInputStreams StorageFile::read( { const ColumnsDescription & columns_ = getColumns(); auto column_defaults = columns_.getDefaults(); - if (!path_with_globs) - { - BlockInputStreamPtr block_input = std::make_shared(*this, context, max_block_size, path); - if (column_defaults.empty()) - return {block_input}; - return {std::make_shared(block_input, column_defaults, context)}; - } BlockInputStreams blocks_input; - blocks_input.reserve(matched_paths.size()); - for (const auto & file_path : matched_paths) + blocks_input.reserve(paths.size()); + for (const auto & file_path : paths) { BlockInputStreamPtr cur_block = std::make_shared(*this, context, max_block_size, file_path); blocks_input.push_back(column_defaults.empty() ? cur_block : std::make_shared(cur_block, column_defaults, context)); @@ -293,7 +277,8 @@ public: } else { - write_buf = std::make_unique(storage.path, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); + if (storage.paths.size() != 1) throw Exception("Table '" + storage.table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + write_buf = std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); } writer = FormatFactory::instance().getOutput(storage.format_name, *write_buf, storage.getSampleBlock(), storage.context_global); @@ -347,13 +332,16 @@ void StorageFile::rename(const String & new_path_to_db, const String & new_datab if (!is_db_table) throw Exception("Can't rename table '" + table_name + "' binded to user-defined file (or FD)", ErrorCodes::DATABASE_ACCESS_DENIED); + if (paths.size() != 1) + throw Exception("Can't rename table '" + table_name + "' in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + std::unique_lock lock(rwlock); std::string path_new = getTablePath(new_path_to_db, new_table_name, format_name); Poco::File(Poco::Path(path_new).parent()).createDirectories(); - Poco::File(path).renameTo(path_new); + Poco::File(paths[0]).renameTo(path_new); - path = std::move(path_new); + paths[0] = std::move(path_new); table_name = new_table_name; database_name = new_database_name; } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 177f13d3d27..589de3314fd 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -41,7 +41,7 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; - String getDataPath() const override { return path; } + String getDataPath() const override { return paths[0]; } protected: friend class StorageFileBlockInputStream; @@ -68,11 +68,9 @@ private: std::string format_name; Context & context_global; - std::string path; int table_fd = -1; - bool path_with_globs = false; - std::vector matched_paths; + std::vector paths{""}; bool is_db_table = true; /// Table is stored in real database, not user's file bool use_table_fd = false; /// Use table_fd insted of path diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index eb38c579b7a..04767c28cd2 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -188,24 +188,14 @@ BlockInputStreams StorageHDFS::read( unsigned /*num_streams*/) { Strings path_parts; - size_t first_glob = uri.find_first_of("*?{"); - - if (first_glob == std::string::npos) - return {std::make_shared( - uri, - format_name, - getSampleBlock(), - context_, - max_block_size)}; - - size_t begin_of_path = uri.find('/', uri.find("//") + 2); - String path_from_uri = uri.substr(begin_of_path); - String uri_without_path = uri.substr(0, begin_of_path); + const size_t begin_of_path = uri.find('/', uri.find("//") + 2); + const String path_from_uri = uri.substr(begin_of_path); + const String uri_without_path = uri.substr(0, begin_of_path); HDFSBuilderPtr builder = createHDFSBuilder(Poco::URI(uri_without_path + "/")); HDFSFSPtr fs = createHDFSFS(builder.get()); - Strings res_paths = LSWithRegexpMatching("/", fs, path_from_uri); + const Strings res_paths = LSWithRegexpMatching("/", fs, path_from_uri); BlockInputStreams result; for (const auto & res_path : res_paths) { diff --git a/docs/en/query_language/table_functions/file.md b/docs/en/query_language/table_functions/file.md index 4a1a7080f30..0cb1f0d36bf 100644 --- a/docs/en/query_language/table_functions/file.md +++ b/docs/en/query_language/table_functions/file.md @@ -9,7 +9,7 @@ file(path, format, structure) **Input parameters** -- `path` — The relative path to the file from [user_files_path](../../operations/server_settings/settings.md#server_settings-user_files_path). +- `path` — The relative path to the file from [user_files_path](../../operations/server_settings/settings.md#server_settings-user_files_path). Path to file support following globs in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, ``'abc', 'def'` — strings. - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -51,4 +51,16 @@ LIMIT 2 SELECT * FROM file('test.csv', 'CSV', 'column1 UInt32, column2 UInt32, column3 UInt32') LIMIT 10 ``` +**Globs in path** + +- `*` — Matches any number of any characters including none. +- `?` — Matches any single character. +- `{some_string,another_string,yet_another_one}` — Matches any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{N..M}` — Matches any number in range from N to M including both borders. + +!!! warning + If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. + +Multiple path components can have globs. For being processed file should exists and matches to the whole path pattern. + [Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/file/) diff --git a/docs/en/query_language/table_functions/hdfs.md b/docs/en/query_language/table_functions/hdfs.md index 16c262aed7e..cce9b308101 100644 --- a/docs/en/query_language/table_functions/hdfs.md +++ b/docs/en/query_language/table_functions/hdfs.md @@ -1,7 +1,7 @@ # hdfs -Creates a table from a file in hdfs. +Creates a table from a file in HDFS. ``` hdfs(URI, format, structure) @@ -9,7 +9,7 @@ hdfs(URI, format, structure) **Input parameters** -- `URI` — The relative URI to the file in HDFS. +- `URI` — The relative URI to the file in HDFS. Path to file support following globs in readonly mode: `*`, `?`, `{abc,def}` and `{N..M}` where `N`, `M` — numbers, ``'abc', 'def'` — strings. - `format` — The [format](../../interfaces/formats.md#formats) of the file. - `structure` — Structure of the table. Format `'column1_name column1_type, column2_name column2_type, ...'`. @@ -34,4 +34,16 @@ LIMIT 2 └─────────┴─────────┴─────────┘ ``` +**Globs in path** + +- `*` — Matches any number of any characters including none. +- `?` — Matches any single character. +- `{some_string,another_string,yet_another_one}` — Matches any of strings `'some_string', 'another_string', 'yet_another_one'`. +- `{N..M}` — Matches any number in range from N to M including both borders. + +!!! warning + If your listing of files contains number ranges with leading zeros, use the construction with braces for each digit separately or use `?`. + +Multiple path components can have globs. For being processed file should exists and matches to the whole path pattern. + [Original article](https://clickhouse.yandex/docs/en/query_language/table_functions/hdfs/) diff --git a/docs/ru/query_language/table_functions/file.md b/docs/ru/query_language/table_functions/file.md index 0fb16bf5a48..9fc82b151b8 100644 --- a/docs/ru/query_language/table_functions/file.md +++ b/docs/ru/query_language/table_functions/file.md @@ -9,7 +9,7 @@ file(path, format, structure) **Входные параметры** -- `path` — относительный путь до файла от [user_files_path](../../operations/server_settings/settings.md#server_settings-user_files_path). +- `path` — относительный путь до файла от [user_files_path](../../operations/server_settings/settings.md#server_settings-user_files_path). Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, ``'abc', 'def'` — строки. - `format` — [формат](../../interfaces/formats.md#formats) файла. - `structure` — структура таблицы. Формат `'colunmn1_name column1_ype, column2_name column2_type, ...'`. @@ -45,4 +45,16 @@ LIMIT 2 └─────────┴─────────┴─────────┘ ``` +**Шаблоны в пути файла** + +- `*` — Матчит любое количество любых символов, включая отсутствие символов. +- `?` — Матчит ровно один любой символ. +- `{some_string,another_string,yet_another_one}` — Матчит любую из строк `'some_string', 'another_string', 'yet_another_one'`. +- `{N..M}` — Матчит любое число в интервале от `N` до `M` включительно. + +!!! warning + Если ваш список файлов содержит интервал с ведущими нулями, используйте конструкцию с фигурными скобками для каждой цифры по отдельности или используйте `?`. + +Шаблоны могут содержаться в разных частях пути. Обрабатываться будут ровно те файлы, которые и удовлетворяют всему шаблону пути, и существуют в файловой системе. + [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/file/) diff --git a/docs/ru/query_language/table_functions/hdfs.md b/docs/ru/query_language/table_functions/hdfs.md index 36a75ffbf42..ae881edea35 100644 --- a/docs/ru/query_language/table_functions/hdfs.md +++ b/docs/ru/query_language/table_functions/hdfs.md @@ -1,7 +1,7 @@ # hdfs -Создаёт таблицу из файла. +Создаёт таблицу из файла в HDFS. ``` hdfs(URI, format, structure) @@ -9,7 +9,7 @@ hdfs(URI, format, structure) **Входные параметры** -- `URI` — URI до файла в HDFS. +- `URI` — URI файла в HDFS. Путь к файлу поддерживает следующие шаблоны в режиме доступа только для чтения `*`, `?`, `{abc,def}` и `{N..M}`, где `N`, `M` — числа, ``'abc', 'def'` — строки. - `format` — [формат](../../interfaces/formats.md#formats) файла. - `structure` — структура таблицы. Формат `'column1_name column1_type, column2_name column2_type, ...'`. @@ -33,4 +33,16 @@ LIMIT 2 └─────────┴─────────┴─────────┘ ``` +**Шаблоны в пути файла** + +- `*` — Матчит любое количество любых символов, включая отсутствие символов. +- `?` — Матчит ровно один любой символ. +- `{some_string,another_string,yet_another_one}` — Матчит любую из строк `'some_string', 'another_string', 'yet_another_one'`. +- `{N..M}` — Матчит любое число в интервале от `N` до `M` включительно. + +!!! warning + Если ваш список файлов содержит интервал с ведущими нулями, используйте конструкцию с фигурными скобками для каждой цифры по отдельности или используйте `?`. + +Шаблоны могут содержаться в разных частях пути. Обрабатываться будут ровно те файлы, которые и удовлетворяют всему шаблону пути, и существуют в файловой системе. + [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/table_functions/hdfs/) From 282a1e1691fbc18935c4ec1513aef35d54da9860 Mon Sep 17 00:00:00 2001 From: Olga Khvostikova Date: Wed, 4 Sep 2019 16:24:41 +0300 Subject: [PATCH 52/75] Update docs/en/operations/table_engines/hdfs.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/table_engines/hdfs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/table_engines/hdfs.md b/docs/en/operations/table_engines/hdfs.md index 54d7cda27aa..4e366fe5d02 100644 --- a/docs/en/operations/table_engines/hdfs.md +++ b/docs/en/operations/table_engines/hdfs.md @@ -1,6 +1,6 @@ # HDFS {#table_engines-hdfs} -Manages data on HDFS. This engine is similar +This engine provides integration with [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) ecosystem by allowing to manage data on [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.htmll)via ClickHouse. This engine is similar to the [File](file.md) and [URL](url.md) engine. ## Usage From ec53b27473238fc9ab923209fa05ed5ceaef9b01 Mon Sep 17 00:00:00 2001 From: Olga Khvostikova Date: Wed, 4 Sep 2019 16:25:07 +0300 Subject: [PATCH 53/75] Update docs/en/operations/table_engines/hdfs.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/table_engines/hdfs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/table_engines/hdfs.md b/docs/en/operations/table_engines/hdfs.md index 4e366fe5d02..f322e7f91ed 100644 --- a/docs/en/operations/table_engines/hdfs.md +++ b/docs/en/operations/table_engines/hdfs.md @@ -1,7 +1,7 @@ # HDFS {#table_engines-hdfs} This engine provides integration with [Apache Hadoop](https://en.wikipedia.org/wiki/Apache_Hadoop) ecosystem by allowing to manage data on [HDFS](https://hadoop.apache.org/docs/current/hadoop-project-dist/hadoop-hdfs/HdfsDesign.htmll)via ClickHouse. This engine is similar -to the [File](file.md) and [URL](url.md) engine. +to the [File](file.md) and [URL](url.md) engines, but provides Hadoop-specific features. ## Usage From f2857ac6f2f6774619b1778592f74588905b74cd Mon Sep 17 00:00:00 2001 From: Olga Khvostikova Date: Wed, 4 Sep 2019 16:26:03 +0300 Subject: [PATCH 54/75] Update docs/en/operations/table_engines/hdfs.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/table_engines/hdfs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/table_engines/hdfs.md b/docs/en/operations/table_engines/hdfs.md index f322e7f91ed..7d6966db983 100644 --- a/docs/en/operations/table_engines/hdfs.md +++ b/docs/en/operations/table_engines/hdfs.md @@ -34,7 +34,7 @@ SELECT * FROM hdfs_engine_table LIMIT 2 └──────┴───────┘ ``` -## Details of Implementation +## Implementation Details - Reads and writes can be parallel - Not supported: From 600de9353216eef29e3b688631ad02fed76e8cdd Mon Sep 17 00:00:00 2001 From: Olga Khvostikova Date: Wed, 4 Sep 2019 16:26:52 +0300 Subject: [PATCH 55/75] Update docs/en/operations/table_engines/hdfs.md Co-Authored-By: Ivan Blinkov --- docs/en/operations/table_engines/hdfs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/operations/table_engines/hdfs.md b/docs/en/operations/table_engines/hdfs.md index 7d6966db983..4ffdf2f4fb3 100644 --- a/docs/en/operations/table_engines/hdfs.md +++ b/docs/en/operations/table_engines/hdfs.md @@ -16,7 +16,7 @@ The `format` parameter specifies one of the available file formats. To perform **Example:** -**1.** Set up the `HDFS_engine_table` table: +**1.** Set up the `hdfs_engine_table` table: ``` sql CREATE TABLE hdfs_engine_table (name String, value UInt32) ENGINE=HDFS('hdfs://hdfs1:9000/other_storage', 'TSV') From b854c945ad99478f95635eba75debbef356d5f46 Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 4 Sep 2019 19:20:02 +0300 Subject: [PATCH 56/75] some more ExpressionAnalyzer refactoring --- dbms/src/Core/Names.h | 3 + dbms/src/Interpreters/AnalyzedJoin.cpp | 16 ++--- dbms/src/Interpreters/AnalyzedJoin.h | 4 +- dbms/src/Interpreters/ExpressionActions.h | 3 - dbms/src/Interpreters/ExpressionAnalyzer.cpp | 67 ++++++++------------ dbms/src/Interpreters/ExpressionAnalyzer.h | 5 +- dbms/src/Interpreters/SubqueryForSet.cpp | 18 +----- dbms/src/Interpreters/SubqueryForSet.h | 2 +- dbms/src/Interpreters/interpretSubquery.cpp | 13 ++++ 9 files changed, 61 insertions(+), 70 deletions(-) diff --git a/dbms/src/Core/Names.h b/dbms/src/Core/Names.h index 61220779f7b..5489a233b6e 100644 --- a/dbms/src/Core/Names.h +++ b/dbms/src/Core/Names.h @@ -16,4 +16,7 @@ using NameOrderedSet = std::set; using NameToNameMap = std::unordered_map; using NameToNameSetMap = std::unordered_map; +using NameWithAlias = std::pair; +using NamesWithAliases = std::vector; + } diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index f60afe81276..7deb21d0dcc 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -93,14 +93,14 @@ NameSet AnalyzedJoin::getOriginalColumnsSet() const return out; } -std::unordered_map AnalyzedJoin::getOriginalColumnsMap(const NameSet & required_columns) const +NamesWithAliases AnalyzedJoin::getNamesWithAliases(const NameSet & required_columns) const { - std::unordered_map out; + NamesWithAliases out; for (const auto & column : required_columns) { auto it = original_names.find(column); if (it != original_names.end()) - out.insert(*it); + out.emplace_back(it->second, it->first); /// {original_name, name} } return out; } @@ -129,15 +129,15 @@ Names AnalyzedJoin::requiredJoinedNames() const return Names(required_columns_set.begin(), required_columns_set.end()); } -void AnalyzedJoin::appendRequiredColumns(const Block & sample, NameSet & required_columns) const +NamesWithAliases AnalyzedJoin::getRequiredColumns(const Block & sample, const Names & action_required_columns) const { - for (auto & column : key_names_right) + NameSet required_columns(action_required_columns.begin(), action_required_columns.end()); + + for (auto & column : requiredJoinedNames()) if (!sample.has(column)) required_columns.insert(column); - for (auto & column : columns_added_by_join) - if (!sample.has(column.name)) - required_columns.insert(column.name); + return getNamesWithAliases(required_columns); } void AnalyzedJoin::addJoinedColumn(const NameAndTypePair & joined_column) diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index 2622f35a941..bea430de479 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -64,12 +64,12 @@ public: NameSet getQualifiedColumnsSet() const; NameSet getOriginalColumnsSet() const; - std::unordered_map getOriginalColumnsMap(const NameSet & required_columns) const; + NamesWithAliases getNamesWithAliases(const NameSet & required_columns) const; + NamesWithAliases getRequiredColumns(const Block & sample, const Names & action_columns) const; void deduplicateAndQualifyColumnNames(const NameSet & left_table_columns, const String & right_table_prefix); size_t rightKeyInclusion(const String & name) const; - void appendRequiredColumns(const Block & sample, NameSet & required_columns) const; void addJoinedColumn(const NameAndTypePair & joined_column); void addJoinedColumnsAndCorrectNullability(Block & sample_block) const; diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 6997c3ef759..20acd1a95c8 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -20,9 +20,6 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -using NameWithAlias = std::pair; -using NamesWithAliases = std::vector; - class AnalyzedJoin; class IPreparedFunction; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index d2994ec22a1..3b70ea519b1 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -418,8 +418,7 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b if (!ast_join) return false; - SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join); - syntax->analyzed_join->setHashJoin(subquery_for_set.join); + makeTableJoin(*ast_join); initChain(chain, sourceColumns()); ExpressionActionsChain::Step & step = chain.steps.back(); @@ -457,7 +456,7 @@ static JoinPtr tryGetStorageJoin(const ASTTablesInSelectQueryElement & join_elem return {}; } -SubqueryForSet & SelectQueryExpressionAnalyzer::getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element) +void SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQueryElement & join_element) { /// Two JOINs are not supported with the same subquery, but different USINGs. auto join_hash = join_element.getTreeHash(); @@ -470,55 +469,45 @@ SubqueryForSet & SelectQueryExpressionAnalyzer::getSubqueryForJoin(const ASTTabl subquery_for_set.join = tryGetStorageJoin(join_element, context); if (!subquery_for_set.join) - makeHashJoin(join_element, subquery_for_set); + { + /// Actions which need to be calculated on joined block. + ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(); - return subquery_for_set; + if (!subquery_for_set.source) + makeSubqueryForJoin(join_element, joined_block_actions, subquery_for_set); + + /// Test actions on sample block (early error detection) + Block sample_block = subquery_for_set.renamedSampleBlock(); + joined_block_actions->execute(sample_block); + + /// TODO You do not need to set this up when JOIN is only needed on remote servers. + subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join); + subquery_for_set.joined_block_actions = joined_block_actions; + } + + syntax->analyzed_join->setHashJoin(subquery_for_set.join); } -void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryElement & join_element, - SubqueryForSet & subquery_for_set) const +void SelectQueryExpressionAnalyzer::makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, + const ExpressionActionsPtr & joined_block_actions, + SubqueryForSet & subquery_for_set) const { - /// Actions which need to be calculated on joined block. - ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(); - /** For GLOBAL JOINs (in the case, for example, of the push method for executing GLOBAL subqueries), the following occurs * - in the addExternalStorage function, the JOIN (SELECT ...) subquery is replaced with JOIN _data1, * in the subquery_for_set object this subquery is exposed as source and the temporary table _data1 as the `table`. * - this function shows the expression JOIN _data1. */ - if (!subquery_for_set.source) - { - ASTPtr table; - auto & table_to_join = join_element.table_expression->as(); - if (table_to_join.subquery) - table = table_to_join.subquery; - else if (table_to_join.table_function) - table = table_to_join.table_function; - else if (table_to_join.database_and_table_name) - table = table_to_join.database_and_table_name; + NamesWithAliases required_columns_with_aliases = + analyzedJoin().getRequiredColumns(joined_block_actions->getSampleBlock(), joined_block_actions->getRequiredColumns()); - Names action_columns = joined_block_actions->getRequiredColumns(); - NameSet required_columns(action_columns.begin(), action_columns.end()); + Names original_columns; + for (auto & pr : required_columns_with_aliases) + original_columns.push_back(pr.first); - analyzedJoin().appendRequiredColumns(joined_block_actions->getSampleBlock(), required_columns); + auto interpreter = interpretSubquery(join_element.table_expression, context, subquery_depth, original_columns); - auto original_map = analyzedJoin().getOriginalColumnsMap(required_columns); - Names original_columns; - for (auto & pr : original_map) - original_columns.push_back(pr.second); - - auto interpreter = interpretSubquery(table, context, subquery_depth, original_columns); - - subquery_for_set.makeSource(interpreter, original_map); - } - - Block sample_block = subquery_for_set.renamedSampleBlock(); - joined_block_actions->execute(sample_block); - - /// TODO You do not need to set this up when JOIN is only needed on remote servers. - subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join); - subquery_for_set.joined_block_actions = joined_block_actions; + subquery_for_set.makeSource(interpreter, std::move(required_columns_with_aliases)); } ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 7a5054ea040..d80e7f68943 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -219,9 +219,10 @@ private: */ void tryMakeSetForIndexFromSubquery(const ASTPtr & subquery_or_table_name); - SubqueryForSet & getSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element); + void makeTableJoin(const ASTTablesInSelectQueryElement & join_element); + void makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, const ExpressionActionsPtr & joined_block_actions, + SubqueryForSet & subquery_for_set) const; ExpressionActionsPtr createJoinedBlockActions() const; - void makeHashJoin(const ASTTablesInSelectQueryElement & join_element, SubqueryForSet & subquery_for_set) const; const ASTSelectQuery * getAggregatingQuery() const; }; diff --git a/dbms/src/Interpreters/SubqueryForSet.cpp b/dbms/src/Interpreters/SubqueryForSet.cpp index 5572d60d3e9..6e0cd540db4 100644 --- a/dbms/src/Interpreters/SubqueryForSet.cpp +++ b/dbms/src/Interpreters/SubqueryForSet.cpp @@ -6,26 +6,14 @@ namespace DB { void SubqueryForSet::makeSource(std::shared_ptr & interpreter, - const std::unordered_map & name_to_origin) + NamesWithAliases && joined_block_aliases_) { + joined_block_aliases = std::move(joined_block_aliases_); source = std::make_shared(interpreter->getSampleBlock(), [interpreter]() mutable { return interpreter->execute().in; }); - for (const auto & names : name_to_origin) - joined_block_aliases.emplace_back(names.second, names.first); - sample_block = source->getHeader(); - for (const auto & name_with_alias : joined_block_aliases) - { - if (sample_block.has(name_with_alias.first)) - { - auto pos = sample_block.getPositionByName(name_with_alias.first); - auto column = sample_block.getByPosition(pos); - sample_block.erase(pos); - column.name = name_with_alias.second; - sample_block.insert(std::move(column)); - } - } + renameColumns(sample_block); } void SubqueryForSet::renameColumns(Block & block) diff --git a/dbms/src/Interpreters/SubqueryForSet.h b/dbms/src/Interpreters/SubqueryForSet.h index 79d32d836c6..abba7a4ec2f 100644 --- a/dbms/src/Interpreters/SubqueryForSet.h +++ b/dbms/src/Interpreters/SubqueryForSet.h @@ -31,7 +31,7 @@ struct SubqueryForSet StoragePtr table; void makeSource(std::shared_ptr & interpreter, - const std::unordered_map & name_to_origin); + NamesWithAliases && joined_block_aliases_); Block renamedSampleBlock() const { return sample_block; } void renameColumns(Block & block); diff --git a/dbms/src/Interpreters/interpretSubquery.cpp b/dbms/src/Interpreters/interpretSubquery.cpp index 76b570ecdb9..82545d4b3be 100644 --- a/dbms/src/Interpreters/interpretSubquery.cpp +++ b/dbms/src/Interpreters/interpretSubquery.cpp @@ -18,6 +18,19 @@ namespace DB std::shared_ptr interpretSubquery( const ASTPtr & table_expression, const Context & context, size_t subquery_depth, const Names & required_source_columns) { + if (auto * expr = table_expression->as()) + { + ASTPtr table; + if (expr->subquery) + table = expr->subquery; + else if (expr->table_function) + table = expr->table_function; + else if (expr->database_and_table_name) + table = expr->database_and_table_name; + + return interpretSubquery(table, context, subquery_depth, required_source_columns); + } + /// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`. const auto * subquery = table_expression->as(); const auto * function = table_expression->as(); From a395be3e91ed3c91c874a56cbb1f9ce836d1913b Mon Sep 17 00:00:00 2001 From: chertus Date: Wed, 4 Sep 2019 19:37:05 +0300 Subject: [PATCH 57/75] minor changes --- dbms/src/Interpreters/ExpressionAnalyzer.cpp | 19 +++++++++---------- dbms/src/Interpreters/ExpressionAnalyzer.h | 1 - 2 files changed, 9 insertions(+), 11 deletions(-) diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 3b70ea519b1..9777e3d508d 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -456,6 +456,14 @@ static JoinPtr tryGetStorageJoin(const ASTTablesInSelectQueryElement & join_elem return {}; } +static ExpressionActionsPtr createJoinedBlockActions(const Context & context, const AnalyzedJoin & analyzed_join) +{ + ASTPtr expression_list = analyzed_join.rightKeysList(); + auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, + analyzed_join.columnsFromJoinedTable(), analyzed_join.requiredJoinedNames()); + return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false); +} + void SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQueryElement & join_element) { /// Two JOINs are not supported with the same subquery, but different USINGs. @@ -471,7 +479,7 @@ void SelectQueryExpressionAnalyzer::makeTableJoin(const ASTTablesInSelectQueryEl if (!subquery_for_set.join) { /// Actions which need to be calculated on joined block. - ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(); + ExpressionActionsPtr joined_block_actions = createJoinedBlockActions(context, analyzedJoin()); if (!subquery_for_set.source) makeSubqueryForJoin(join_element, joined_block_actions, subquery_for_set); @@ -510,15 +518,6 @@ void SelectQueryExpressionAnalyzer::makeSubqueryForJoin(const ASTTablesInSelectQ subquery_for_set.makeSource(interpreter, std::move(required_columns_with_aliases)); } -ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const -{ - ASTPtr expression_list = analyzedJoin().rightKeysList(); - Names required_columns = analyzedJoin().requiredJoinedNames(); - - auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columnsFromJoinedTable(), required_columns); - return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false); -} - bool SelectQueryExpressionAnalyzer::appendPrewhere( ExpressionActionsChain & chain, bool only_types, const Names & additional_required_columns) { diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index d80e7f68943..9356046aee3 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -222,7 +222,6 @@ private: void makeTableJoin(const ASTTablesInSelectQueryElement & join_element); void makeSubqueryForJoin(const ASTTablesInSelectQueryElement & join_element, const ExpressionActionsPtr & joined_block_actions, SubqueryForSet & subquery_for_set) const; - ExpressionActionsPtr createJoinedBlockActions() const; const ASTSelectQuery * getAggregatingQuery() const; }; From 0d1846baab5db83c8d77c1d4d3bbab4f9591b285 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 4 Sep 2019 19:44:25 +0300 Subject: [PATCH 58/75] Update test create table if not exists sample_prewhere --- .../00975_sample_prewhere.reference | 2 -- .../0_stateless/00975_sample_prewhere.sql | 20 +++++-------------- 2 files changed, 5 insertions(+), 17 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00975_sample_prewhere.reference b/dbms/tests/queries/0_stateless/00975_sample_prewhere.reference index 2559e5c49e7..e69de29bb2d 100644 --- a/dbms/tests/queries/0_stateless/00975_sample_prewhere.reference +++ b/dbms/tests/queries/0_stateless/00975_sample_prewhere.reference @@ -1,2 +0,0 @@ -3 -6 diff --git a/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql b/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql index 5d202a9d12c..3feb793c629 100644 --- a/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql +++ b/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql @@ -1,17 +1,7 @@ -drop table if exists test.sample_prewhere; -drop table if exists test.sample_prewhere_all; +create table if not exists sample_prewhere (date Date, id Int32, time Int64) engine = MergeTree partition by date order by (id, time, intHash64(time)) sample by intHash64(time); -create table if not exists test.sample_prewhere (date Date, id Int32, time Int64) engine = MergeTree partition by date order by (id, time, intHash64(time)) sample by intHash64(time); -insert into test.sample_prewhere values ('2019-01-01', 2, 1564028096); -insert into test.sample_prewhere values ('2019-01-01', 1, 1564028096); -insert into test.sample_prewhere values ('2019-01-02', 3, 1564028096); +insert into test.sample_prewhere values ('2019-01-01', 2, toDateTime('2019-07-20 00:00:01')); +insert into test.sample_prewhere values ('2019-01-01', 1, toDateTime('2019-07-20 00:00:02')); +insert into test.sample_prewhere values ('2019-01-02', 3, toDateTime('2019-07-20 00:00:03')); -create table if not exists test.sample_prewhere_all as test.sample_prewhere engine = Distributed('test_cluster_two_shards_localhost', 'test', 'sample_prewhere'); - -select id from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00' limit 0, 1; -select id from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) > '2019-07-20 00:00:00' limit 0, 1; -select id from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00'; -select count() from test.sample_prewhere_all SAMPLE 1 where toDateTime(time) > '2019-07-20 00:00:00' limit 0, 1; - -drop table if exists test.sample_prewhere; -drop table if exists test.sample_prewhere_all; +select id from remote('127.0.0.{1,3}', currentDatabase(), sample_prewhere) SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00'; From 5d6959173ca86975e4ffe3762d3344ef54137da4 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Wed, 4 Sep 2019 22:55:56 +0300 Subject: [PATCH 59/75] Improvement --- dbms/src/Common/parseGlobs.cpp | 39 ++++++++++++----------- dbms/src/Storages/StorageFile.cpp | 16 ++++++++-- dbms/src/Storages/StorageFile.h | 4 +-- dbms/src/Storages/StorageHDFS.cpp | 6 ++-- docs/en/interfaces/formats.md | 2 +- docs/en/operations/table_engines/hdfs.md | 10 ++++-- docs/en/operations/table_engines/index.md | 1 + docs/ru/interfaces/formats.md | 2 +- docs/ru/operations/table_engines/hdfs.md | 8 ++++- 9 files changed, 56 insertions(+), 32 deletions(-) diff --git a/dbms/src/Common/parseGlobs.cpp b/dbms/src/Common/parseGlobs.cpp index cdb1dc6cedd..b7595d4591c 100644 --- a/dbms/src/Common/parseGlobs.cpp +++ b/dbms/src/Common/parseGlobs.cpp @@ -16,60 +16,61 @@ namespace DB */ std::string makeRegexpPatternFromGlobs(const std::string & initial_str_with_globs) { - std::ostringstream oss; + std::ostringstream oss_for_escaping; /// Escaping only characters that not used in glob syntax for (const auto & letter : initial_str_with_globs) { if ((letter == '[') || (letter == ']') || (letter == '|') || (letter == '+') || (letter == '-') || (letter == '(') || (letter == ')')) - oss << '\\'; - oss << letter; + oss_for_escaping << '\\'; + oss_for_escaping << letter; } - std::string escaped_with_globs = oss.str(); - oss.str(""); + std::string escaped_with_globs = oss_for_escaping.str(); + static const re2::RE2 enum_or_range(R"({([\d]+\.\.[\d]+|[^{}*,]+,[^{}*]*[^{}*,])})"); /// regexp for {expr1,expr2,expr3} or {M..N}, where M and N - non-negative integers, expr's should be without {}*, re2::StringPiece input(escaped_with_globs); re2::StringPiece matched; + std::ostringstream oss_for_replacing; size_t current_index = 0; while (RE2::FindAndConsume(&input, enum_or_range, &matched)) { std::string buffer = matched.ToString(); - oss << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; + oss_for_replacing << escaped_with_globs.substr(current_index, matched.data() - escaped_with_globs.data() - current_index - 1) << '('; if (buffer.find(',') == std::string::npos) { size_t range_begin, range_end; char point; - std::istringstream iss(buffer); - iss >> range_begin >> point >> point >> range_end; - oss << range_begin; + std::istringstream iss_range(buffer); + iss_range >> range_begin >> point >> point >> range_end; + oss_for_replacing << range_begin; for (size_t i = range_begin + 1; i <= range_end; ++i) { - oss << '|' << i; + oss_for_replacing << '|' << i; } } else { std::replace(buffer.begin(), buffer.end(), ',', '|'); - oss << buffer; + oss_for_replacing << buffer; } - oss << ")"; + oss_for_replacing << ")"; current_index = input.data() - escaped_with_globs.data(); } - oss << escaped_with_globs.substr(current_index); - std::string almost_res = oss.str(); - oss.str(""); + oss_for_replacing << escaped_with_globs.substr(current_index); + std::string almost_res = oss_for_replacing.str(); + std::ostringstream oss_final_processing; for (const auto & letter : almost_res) { if ((letter == '?') || (letter == '*')) { - oss << "[^/]"; /// '?' is any symbol except '/' + oss_final_processing << "[^/]"; /// '?' is any symbol except '/' if (letter == '?') continue; } if ((letter == '.') || (letter == '{') || (letter == '}')) - oss << '\\'; - oss << letter; + oss_final_processing << '\\'; + oss_final_processing << letter; } - return oss.str(); + return oss_final_processing.str(); } } diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index bc745092e0b..ac8441f815c 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -152,13 +152,16 @@ StorageFile::StorageFile( if (db_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - paths[0] = getTablePath(db_dir_path, table_name, format_name); + paths.push_back(getTablePath(db_dir_path, table_name, format_name)); is_db_table = true; - Poco::File(Poco::Path(paths[0]).parent()).createDirectories(); + Poco::File(Poco::Path(paths.back()).parent()).createDirectories(); } } else /// Will use FD { + if (paths.size() != 1) + throw Exception("Table '" + table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + checkCreationIsAllowed(context_global, db_dir_path, paths[0], table_fd); is_db_table = false; @@ -266,6 +269,8 @@ public: explicit StorageFileBlockOutputStream(StorageFile & storage_) : storage(storage_), lock(storage.rwlock) { + if (storage.paths.size() != 1) + throw Exception("Table '" + storage.table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); if (storage.use_table_fd) { /** NOTE: Using real file binded to FD may be misleading: @@ -277,7 +282,6 @@ public: } else { - if (storage.paths.size() != 1) throw Exception("Table '" + storage.table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); write_buf = std::make_unique(storage.paths[0], DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_APPEND | O_CREAT); } @@ -320,6 +324,12 @@ BlockOutputStreamPtr StorageFile::write( return std::make_shared(*this); } +String StorageFile::getDataPath() const +{ + if (paths.size() != 1) + throw Exception("Table '" + table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); + return paths[0]; +} void StorageFile::drop() { diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index 589de3314fd..b0470069619 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -41,7 +41,7 @@ public: void rename(const String & new_path_to_db, const String & new_database_name, const String & new_table_name) override; - String getDataPath() const override { return paths[0]; } + String getDataPath() const override; protected: friend class StorageFileBlockInputStream; @@ -70,7 +70,7 @@ private: int table_fd = -1; - std::vector paths{""}; + std::vector paths; bool is_db_table = true; /// Table is stored in real database, not user's file bool use_table_fd = false; /// Use table_fd insted of path diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 04767c28cd2..480b0f2b068 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -155,15 +155,16 @@ Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, c const size_t last_slash = full_path.rfind('/'); const String file_name = full_path.substr(last_slash); const bool looking_for_directory = next_slash != std::string::npos; + const bool is_directory = ls.file_info[i].mKind == 'D'; /// Condition with type of current file_info means what kind of path is it in current iteration of ls - if ((ls.file_info[i].mKind == 'F') && !looking_for_directory) + if (!is_directory && !looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) { result.push_back(String(ls.file_info[i].mName)); } } - else if ((ls.file_info[i].mKind == 'D') && looking_for_directory) + else if (is_directory && looking_for_directory) { if (re2::RE2::FullMatch(file_name, matcher)) { @@ -187,7 +188,6 @@ BlockInputStreams StorageHDFS::read( size_t max_block_size, unsigned /*num_streams*/) { - Strings path_parts; const size_t begin_of_path = uri.find('/', uri.find("//") + 2); const String path_from_uri = uri.substr(begin_of_path); const String uri_without_path = uri.substr(0, begin_of_path); diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 46a553089d9..605a644c3aa 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -820,7 +820,7 @@ You can select data from a ClickHouse table and save them into some file in the clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_file.pq} ``` -To exchange data with the Hadoop, you can use `HDFS` table engine. +To exchange data with the Hadoop, you can use [`HDFS` table engine](../../operations/table_engines/hdfs.md). ## Format Schema {#formatschema} diff --git a/docs/en/operations/table_engines/hdfs.md b/docs/en/operations/table_engines/hdfs.md index 54d7cda27aa..a4c99c3fb9d 100644 --- a/docs/en/operations/table_engines/hdfs.md +++ b/docs/en/operations/table_engines/hdfs.md @@ -8,7 +8,7 @@ to the [File](file.md) and [URL](url.md) engine. ``` ENGINE = HDFS(URI, format) ``` - +The `URI` parameter is the whole file URI in HDFS. The `format` parameter specifies one of the available file formats. To perform `SELECT` queries, the format must be supported for input, and to perform `INSERT` queries -- for output. The available formats are listed in the @@ -21,7 +21,13 @@ The `format` parameter specifies one of the available file formats. To perform ``` sql CREATE TABLE hdfs_engine_table (name String, value UInt32) ENGINE=HDFS('hdfs://hdfs1:9000/other_storage', 'TSV') ``` -**2.** Query the data: + +**2.** Fill file: +``` sql +INSERT INTO hdfs_engine_table VALUES ('one', 1), ('two', 2), ('three', 3) +``` + +**3.** Query the data: ``` sql SELECT * FROM hdfs_engine_table LIMIT 2 diff --git a/docs/en/operations/table_engines/index.md b/docs/en/operations/table_engines/index.md index 41680a5b3af..6236b3d75f6 100644 --- a/docs/en/operations/table_engines/index.md +++ b/docs/en/operations/table_engines/index.md @@ -45,6 +45,7 @@ Engines of the family: - [MySQL](mysql.md) - [ODBC](odbc.md) - [JDBC](jdbc.md) +- [HDFS](hdfs.md) ### Special engines diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 20aa78630b7..74e7911dbef 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -752,7 +752,7 @@ cat {filename} | clickhouse-client --query="INSERT INTO {some_table} FORMAT Parq clickhouse-client --query="SELECT * FROM {some_table} FORMAT Parquet" > {some_file.pq} ``` -Для обмена данными с экосистемой Hadoop можно использовать движки таблиц `HDFS` и `URL`. +Для обмена данными с экосистемой Hadoop можно использовать движки таблиц [`HDFS`](../../operations/table_engines/hdfs.md) и `URL`. ## Схема формата {#formatschema} diff --git a/docs/ru/operations/table_engines/hdfs.md b/docs/ru/operations/table_engines/hdfs.md index a77d923985c..3f42c9ec447 100644 --- a/docs/ru/operations/table_engines/hdfs.md +++ b/docs/ru/operations/table_engines/hdfs.md @@ -8,6 +8,7 @@ ENGINE = HDFS(URI, format) ``` +В параметр `URI` нужно передавать полный URI файла в HDFS. Параметр `format` должен быть таким, который ClickHouse может использовать и в запросах `INSERT`, и в запросах `SELECT`. Полный список поддерживаемых форматов смотрите в разделе [Форматы](../../interfaces/formats.md#formats). **Пример:** @@ -18,7 +19,12 @@ ENGINE = HDFS(URI, format) CREATE TABLE hdfs_engine_table (name String, value UInt32) ENGINE=HDFS('hdfs://hdfs1:9000/other_storage', 'TSV') ``` -**2.** Запросим данные: +**2.** Заполним файл: +``` sql +INSERT INTO hdfs_engine_table VALUES ('one', 1), ('two', 2), ('three', 3) +``` + +**3.** Запросим данные: ``` sql SELECT * FROM hdfs_engine_table LIMIT 2 From e1e600b8b32784a79d9ee226da1d272083ca3d8b Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Sep 2019 10:42:14 +0300 Subject: [PATCH 60/75] Update test with sample and prewhere. --- dbms/tests/queries/0_stateless/00975_sample_prewhere.sql | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql b/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql index 3feb793c629..a64eb5d0edf 100644 --- a/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql +++ b/dbms/tests/queries/0_stateless/00975_sample_prewhere.sql @@ -1,7 +1,7 @@ create table if not exists sample_prewhere (date Date, id Int32, time Int64) engine = MergeTree partition by date order by (id, time, intHash64(time)) sample by intHash64(time); -insert into test.sample_prewhere values ('2019-01-01', 2, toDateTime('2019-07-20 00:00:01')); -insert into test.sample_prewhere values ('2019-01-01', 1, toDateTime('2019-07-20 00:00:02')); -insert into test.sample_prewhere values ('2019-01-02', 3, toDateTime('2019-07-20 00:00:03')); +insert into sample_prewhere values ('2019-01-01', 2, toDateTime('2019-07-20 00:00:01')); +insert into sample_prewhere values ('2019-01-01', 1, toDateTime('2019-07-20 00:00:02')); +insert into sample_prewhere values ('2019-01-02', 3, toDateTime('2019-07-20 00:00:03')); select id from remote('127.0.0.{1,3}', currentDatabase(), sample_prewhere) SAMPLE 1 where toDateTime(time) = '2019-07-20 00:00:00'; From 8a3841724d8b1bed09c93e45793416327c2232fa Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 5 Sep 2019 11:23:27 +0300 Subject: [PATCH 61/75] Add performance issue template --- .github/ISSUE_TEMPLATE/bug_report.md | 2 +- .github/ISSUE_TEMPLATE/performance-issue.md | 27 +++++++++++++++++++++ 2 files changed, 28 insertions(+), 1 deletion(-) create mode 100644 .github/ISSUE_TEMPLATE/performance-issue.md diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md index c3283d3d97d..2e472001fd9 100644 --- a/.github/ISSUE_TEMPLATE/bug_report.md +++ b/.github/ISSUE_TEMPLATE/bug_report.md @@ -2,7 +2,7 @@ name: Bug report about: Create a report to help us improve ClickHouse title: '' -labels: bug, issue +labels: bug assignees: '' --- diff --git a/.github/ISSUE_TEMPLATE/performance-issue.md b/.github/ISSUE_TEMPLATE/performance-issue.md new file mode 100644 index 00000000000..402617d00f7 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/performance-issue.md @@ -0,0 +1,27 @@ +--- +name: Performance issue +about: Report something working slower than expected +title: '' +labels: performance +assignees: '' + +--- + +(you don't have to strictly follow this form) + +**Describe the situation** +What exactly works slower than expected? + +**How to reproduce** +* Which ClickHouse server version to use +* Which interface to use, if matters +* Non-default settings, if any +* `CREATE TABLE` statements for all tables involved +* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/yandex/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary +* Queries to run that lead to slow performance + +**Expected performance** +What are your performance expectation, why do you think they are realistic? Has it been working faster in older ClickHouse releases? Is it working faster in some specific other system? + +**Additional context** +Add any other context about the problem here. From 6cf5a90c7b16bce1da9f593e90a2c13253c252af Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 5 Sep 2019 11:49:54 +0300 Subject: [PATCH 62/75] Add documentation issue template --- .github/ISSUE_TEMPLATE/documentation-issue.md | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/documentation-issue.md diff --git a/.github/ISSUE_TEMPLATE/documentation-issue.md b/.github/ISSUE_TEMPLATE/documentation-issue.md new file mode 100644 index 00000000000..a8f31eadc56 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/documentation-issue.md @@ -0,0 +1,16 @@ +--- +name: Documentation issue +about: Report something incorrect or missing in documentation +title: '' +labels: documentation +assignees: BayoNet + +--- + +(you don't have to strictly follow this form) + +**Describe the issue** +A clear and concise description of what's wrong in documentation. + +**Additional context** +Add any other context about the problem here. From c7e446e2fd62472fd3f0e1dce95f0f2291cfcaeb Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 5 Sep 2019 11:51:56 +0300 Subject: [PATCH 63/75] Update issue templates --- .../bug-report-or-unexpected-behaviour.md | 30 +++++++++++++++++++ 1 file changed, 30 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/bug-report-or-unexpected-behaviour.md diff --git a/.github/ISSUE_TEMPLATE/bug-report-or-unexpected-behaviour.md b/.github/ISSUE_TEMPLATE/bug-report-or-unexpected-behaviour.md new file mode 100644 index 00000000000..9526b99b22b --- /dev/null +++ b/.github/ISSUE_TEMPLATE/bug-report-or-unexpected-behaviour.md @@ -0,0 +1,30 @@ +--- +name: Bug report or unexpected behaviour +about: Create a report to help us improve ClickHouse +title: '' +labels: bug +assignees: '' + +--- + +(you don't have to strictly follow this form) + +**Describe the bug or unexpected behaviour** +A clear and concise description of what works not as it is supposed to. + +**How to reproduce** +* Which ClickHouse server version to use +* Which interface to use, if matters +* Non-default settings, if any +* `CREATE TABLE` statements for all tables involved +* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/yandex/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary +* Queries to run that lead to unexpected result + +**Expected behavior** +A clear and concise description of what you expected to happen. + +**Error message and/or stacktrace** +If applicable, add screenshots to help explain your problem. + +**Additional context** +Add any other context about the problem here. From 657c83219d9a73ea7c142c26ada28674652e43f5 Mon Sep 17 00:00:00 2001 From: Ivan Blinkov Date: Thu, 5 Sep 2019 11:52:45 +0300 Subject: [PATCH 64/75] Delete bug_report.md --- .github/ISSUE_TEMPLATE/bug_report.md | 30 ---------------------------- 1 file changed, 30 deletions(-) delete mode 100644 .github/ISSUE_TEMPLATE/bug_report.md diff --git a/.github/ISSUE_TEMPLATE/bug_report.md b/.github/ISSUE_TEMPLATE/bug_report.md deleted file mode 100644 index 2e472001fd9..00000000000 --- a/.github/ISSUE_TEMPLATE/bug_report.md +++ /dev/null @@ -1,30 +0,0 @@ ---- -name: Bug report -about: Create a report to help us improve ClickHouse -title: '' -labels: bug -assignees: '' - ---- - -(you don't have to strictly follow this form) - -**Describe the bug** -A clear and concise description of what the bug is. - -**How to reproduce** -* Which ClickHouse server version to use -* Which interface to use, if matters -* Non-default settings, if any -* `CREATE TABLE` statements for all tables involved -* Sample data for all these tables, use [clickhouse-obfuscator](https://github.com/yandex/ClickHouse/blob/master/dbms/programs/obfuscator/Obfuscator.cpp#L42-L80) if necessary -* Queries to run that lead to unexpected result - -**Expected behavior** -A clear and concise description of what you expected to happen. - -**Error message and/or stacktrace** -If applicable, add screenshots to help explain your problem. - -**Additional context** -Add any other context about the problem here. From 2cda8f1563e9b2d11f75c68db817c3f68ace1160 Mon Sep 17 00:00:00 2001 From: alesapin Date: Thu, 5 Sep 2019 13:10:01 +0300 Subject: [PATCH 65/75] Add query_masking_rules xml's into debug images --- docker/test/stateful_with_coverage/run.sh | 1 + docker/test/stateless_with_coverage/run.sh | 1 + 2 files changed, 2 insertions(+) diff --git a/docker/test/stateful_with_coverage/run.sh b/docker/test/stateful_with_coverage/run.sh index a6d2ba0e9e8..d521632f98a 100755 --- a/docker/test/stateful_with_coverage/run.sh +++ b/docker/test/stateful_with_coverage/run.sh @@ -47,6 +47,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ diff --git a/docker/test/stateless_with_coverage/run.sh b/docker/test/stateless_with_coverage/run.sh index ccf3e53f715..26e230573d5 100755 --- a/docker/test/stateless_with_coverage/run.sh +++ b/docker/test/stateless_with_coverage/run.sh @@ -49,6 +49,7 @@ ln -s /usr/share/clickhouse-test/config/zookeeper.xml /etc/clickhouse-server/con ln -s /usr/share/clickhouse-test/config/part_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/text_log.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/metric_log.xml /etc/clickhouse-server/config.d/; \ + ln -s /usr/share/clickhouse-test/config/query_masking_rules.xml /etc/clickhouse-server/config.d/; \ ln -s /usr/share/clickhouse-test/config/log_queries.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/readonly.xml /etc/clickhouse-server/users.d/; \ ln -s /usr/share/clickhouse-test/config/ints_dictionary.xml /etc/clickhouse-server/; \ From 587a780ba2d4fe087a0faff628a22002001d0962 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Thu, 5 Sep 2019 17:42:17 +0300 Subject: [PATCH 66/75] Better --- dbms/src/IO/HDFSCommon.cpp | 4 +++- dbms/src/IO/HDFSCommon.h | 3 +-- dbms/src/IO/ReadBufferFromHDFS.cpp | 9 ++++---- dbms/src/IO/WriteBufferFromHDFS.cpp | 13 ++++++----- dbms/src/Storages/StorageHDFS.cpp | 2 +- .../integration/test_storage_hdfs/test.py | 22 ++++++++++++++++--- 6 files changed, 36 insertions(+), 17 deletions(-) diff --git a/dbms/src/IO/HDFSCommon.cpp b/dbms/src/IO/HDFSCommon.cpp index 0f1a58942d6..0458ff27f1b 100644 --- a/dbms/src/IO/HDFSCommon.cpp +++ b/dbms/src/IO/HDFSCommon.cpp @@ -1,4 +1,5 @@ #include +#include #if USE_HDFS #include @@ -11,8 +12,9 @@ extern const int BAD_ARGUMENTS; extern const int NETWORK_ERROR; } -HDFSBuilderPtr createHDFSBuilder(const Poco::URI & uri) +HDFSBuilderPtr createHDFSBuilder(const std::string & uri_str) { + const Poco::URI uri(uri_str); auto & host = uri.getHost(); auto port = uri.getPort(); auto & path = uri.getPath(); diff --git a/dbms/src/IO/HDFSCommon.h b/dbms/src/IO/HDFSCommon.h index 0be00de08dd..c84990dfea1 100644 --- a/dbms/src/IO/HDFSCommon.h +++ b/dbms/src/IO/HDFSCommon.h @@ -1,7 +1,6 @@ #include #include #include -#include #if USE_HDFS #include @@ -52,7 +51,7 @@ using HDFSFSPtr = std::unique_ptr, detail::HDFSFsD // set read/connect timeout, default value in libhdfs3 is about 1 hour, and too large /// TODO Allow to tune from query Settings. -HDFSBuilderPtr createHDFSBuilder(const Poco::URI & hdfs_uri); +HDFSBuilderPtr createHDFSBuilder(const std::string & hdfs_uri); HDFSFSPtr createHDFSFS(hdfsBuilder * builder); } #endif diff --git a/dbms/src/IO/ReadBufferFromHDFS.cpp b/dbms/src/IO/ReadBufferFromHDFS.cpp index 9c44048d4ce..48409683799 100644 --- a/dbms/src/IO/ReadBufferFromHDFS.cpp +++ b/dbms/src/IO/ReadBufferFromHDFS.cpp @@ -2,7 +2,6 @@ #if USE_HDFS #include -#include #include @@ -16,7 +15,7 @@ namespace ErrorCodes struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl { - Poco::URI hdfs_uri; + std::string hdfs_uri; hdfsFile fin; HDFSBuilderPtr builder; HDFSFSPtr fs; @@ -26,8 +25,8 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl , builder(createHDFSBuilder(hdfs_uri)) , fs(createHDFSFS(builder.get())) { - - auto & path = hdfs_uri.getPath(); + const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); + const std::string path = hdfs_uri.substr(begin_of_path); fin = hdfsOpenFile(fs.get(), path.c_str(), O_RDONLY, 0, 0, 0); if (fin == nullptr) @@ -39,7 +38,7 @@ struct ReadBufferFromHDFS::ReadBufferFromHDFSImpl { int bytes_read = hdfsRead(fs.get(), fin, start, size); if (bytes_read < 0) - throw Exception("Fail to read HDFS file: " + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()), + throw Exception("Fail to read HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); return bytes_read; } diff --git a/dbms/src/IO/WriteBufferFromHDFS.cpp b/dbms/src/IO/WriteBufferFromHDFS.cpp index 698c58bc000..2cd83ee6479 100644 --- a/dbms/src/IO/WriteBufferFromHDFS.cpp +++ b/dbms/src/IO/WriteBufferFromHDFS.cpp @@ -2,7 +2,6 @@ #if USE_HDFS -#include #include #include #include @@ -21,7 +20,7 @@ extern const int CANNOT_FSYNC; struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { - Poco::URI hdfs_uri; + std::string hdfs_uri; hdfsFile fout; HDFSBuilderPtr builder; HDFSFSPtr fs; @@ -31,7 +30,11 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl , builder(createHDFSBuilder(hdfs_uri)) , fs(createHDFSFS(builder.get())) { - auto & path = hdfs_uri.getPath(); + const size_t begin_of_path = hdfs_uri.find('/', hdfs_uri.find("//") + 2); + const std::string path = hdfs_uri.substr(begin_of_path); + if (path.find("*?{") != std::string::npos) + throw Exception("URI '" + hdfs_uri + "' contains globs, so the table is in readonly mode", ErrorCodes::CANNOT_OPEN_FILE); + fout = hdfsOpenFile(fs.get(), path.c_str(), O_WRONLY, 0, 0, 0); if (fout == nullptr) @@ -52,7 +55,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { int bytes_written = hdfsWrite(fs.get(), fout, start, size); if (bytes_written < 0) - throw Exception("Fail to write HDFS file: " + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()), + throw Exception("Fail to write HDFS file: " + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::NETWORK_ERROR); return bytes_written; } @@ -61,7 +64,7 @@ struct WriteBufferFromHDFS::WriteBufferFromHDFSImpl { int result = hdfsSync(fs.get(), fout); if (result < 0) - throwFromErrno("Cannot HDFS sync" + hdfs_uri.toString() + " " + std::string(hdfsGetLastError()), + throwFromErrno("Cannot HDFS sync" + hdfs_uri + " " + std::string(hdfsGetLastError()), ErrorCodes::CANNOT_FSYNC); } }; diff --git a/dbms/src/Storages/StorageHDFS.cpp b/dbms/src/Storages/StorageHDFS.cpp index 480b0f2b068..cb8ef835612 100644 --- a/dbms/src/Storages/StorageHDFS.cpp +++ b/dbms/src/Storages/StorageHDFS.cpp @@ -192,7 +192,7 @@ BlockInputStreams StorageHDFS::read( const String path_from_uri = uri.substr(begin_of_path); const String uri_without_path = uri.substr(0, begin_of_path); - HDFSBuilderPtr builder = createHDFSBuilder(Poco::URI(uri_without_path + "/")); + HDFSBuilderPtr builder = createHDFSBuilder(uri_without_path + "/"); HDFSFSPtr fs = createHDFSFS(builder.get()); const Strings res_paths = LSWithRegexpMatching("/", fs, path_from_uri); diff --git a/dbms/tests/integration/test_storage_hdfs/test.py b/dbms/tests/integration/test_storage_hdfs/test.py index a9d34926346..55ef98f6fde 100644 --- a/dbms/tests/integration/test_storage_hdfs/test.py +++ b/dbms/tests/integration/test_storage_hdfs/test.py @@ -28,15 +28,31 @@ def started_cluster(): cluster.shutdown() def test_read_write_storage(started_cluster): - hdfs_api = HDFSApi("root") hdfs_api.write_data("/simple_storage", "1\tMark\t72.53\n") - assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" - node1.query("create table SimpleHDFSStorage (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/simple_storage', 'TSV')") + node1.query("insert into SimpleHDFSStorage values (1, 'Mark', 72.53)") + assert hdfs_api.read_data("/simple_storage") == "1\tMark\t72.53\n" assert node1.query("select * from SimpleHDFSStorage") == "1\tMark\t72.53\n" +def test_read_write_storage_with_globs(started_cluster): + hdfs_api = HDFSApi("root") + + for i in ["1", "2", "3"]: + hdfs_api.write_data("/storage" + i, i + "\tMark\t72.53\n") + assert hdfs_api.read_data("/storage" + i) == i + "\tMark\t72.53\n" + + node1.query("create table HDFSStorageWithRange (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage{1..5}', 'TSV')") + node1.query("create table HDFSStorageWithEnum (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage{1,2,3,4,5}', 'TSV')") + node1.query("create table HDFSStorageWithQuestionMark (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage?', 'TSV')") + node1.query("create table HDFSStorageWithAsterisk (id UInt32, name String, weight Float64) ENGINE = HDFS('hdfs://hdfs1:9000/storage*', 'TSV')") + + assert node1.query("select count(*) from HDFSStorageWithRange") == '3\n' + assert node1.query("select count(*) from HDFSStorageWithEnum") == '3\n' + assert node1.query("select count(*) from HDFSStorageWithQuestionMark") == '3\n' + assert node1.query("select count(*) from HDFSStorageWithAsterisk") == '3\n' + def test_read_write_table(started_cluster): hdfs_api = HDFSApi("root") data = "1\tSerialize\t555.222\n2\tData\t777.333\n" From 5e37510aae249a0f6f5281b58bfcbad695939fb8 Mon Sep 17 00:00:00 2001 From: stavrolia Date: Thu, 5 Sep 2019 21:09:19 +0300 Subject: [PATCH 67/75] Better --- dbms/src/Storages/StorageFile.cpp | 5 +++-- dbms/src/Storages/StorageFile.h | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/dbms/src/Storages/StorageFile.cpp b/dbms/src/Storages/StorageFile.cpp index ac8441f815c..57a886dd73a 100644 --- a/dbms/src/Storages/StorageFile.cpp +++ b/dbms/src/Storages/StorageFile.cpp @@ -152,13 +152,14 @@ StorageFile::StorageFile( if (db_dir_path.empty()) throw Exception("Storage " + getName() + " requires data path", ErrorCodes::INCORRECT_FILE_NAME); - paths.push_back(getTablePath(db_dir_path, table_name, format_name)); + paths = {getTablePath(db_dir_path, table_name, format_name)}; is_db_table = true; Poco::File(Poco::Path(paths.back()).parent()).createDirectories(); } } else /// Will use FD { + if (paths.size() != 1) throw Exception("Table '" + table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); @@ -326,7 +327,7 @@ BlockOutputStreamPtr StorageFile::write( String StorageFile::getDataPath() const { - if (paths.size() != 1) + if (paths.empty()) throw Exception("Table '" + table_name + "' is in readonly mode", ErrorCodes::DATABASE_ACCESS_DENIED); return paths[0]; } diff --git a/dbms/src/Storages/StorageFile.h b/dbms/src/Storages/StorageFile.h index b0470069619..0080ab224fb 100644 --- a/dbms/src/Storages/StorageFile.h +++ b/dbms/src/Storages/StorageFile.h @@ -70,7 +70,7 @@ private: int table_fd = -1; - std::vector paths; + std::vector paths{""}; bool is_db_table = true; /// Table is stored in real database, not user's file bool use_table_fd = false; /// Use table_fd insted of path From 3f40cd7eef2c1f6494951f97fc147dab723973cd Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 5 Sep 2019 19:33:36 -0300 Subject: [PATCH 68/75] Update array_functions.md added note about ArrayElement index 0 case. --- docs/ru/query_language/functions/array_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/ru/query_language/functions/array_functions.md b/docs/ru/query_language/functions/array_functions.md index 7945276434f..11d5c819b02 100644 --- a/docs/ru/query_language/functions/array_functions.md +++ b/docs/ru/query_language/functions/array_functions.md @@ -74,7 +74,7 @@ SELECT arrayConcat([1, 2], [3, 4], [5, 6]) AS res Индексы в массиве начинаются с единицы. Поддерживаются отрицательные индексы. В этом случае, будет выбран соответствующий по номеру элемент с конца. Например, arr\[-1\] - последний элемент массива. -Если индекс выходит за границы массива, то возвращается некоторое значение по умолчанию (0 для чисел, пустая строка для строк и т. п.). +Если индекс выходит за границы массива, то возвращается некоторое значение по умолчанию (0 для чисел, пустая строка для строк и т. п.), кроме случая с неконстантным массивом и константным индексом 0 (в этом случае будет ошибка `Array indices are 1-based`). ## has(arr, elem) From 037bd893fc2700ad3a74399db9d7812261ff681e Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 5 Sep 2019 19:36:34 -0300 Subject: [PATCH 69/75] Update array_functions.md added note about ArrayElement index 0 case. --- docs/en/query_language/functions/array_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 94d79a1898b..daef852cb70 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -73,7 +73,7 @@ Get the element with the index `n` from the array `arr`. `n` must be any integer Indexes in an array begin from one. Negative indexes are supported. In this case, it selects the corresponding element numbered from the end. For example, `arr[-1]` is the last item in the array. -If the index falls outside of the bounds of an array, it returns some default value (0 for numbers, an empty string for strings, etc.). +If the index falls outside of the bounds of an array, it returns some default value (0 for numbers, an empty string for strings, etc.), except for the case with a non-constant array and a constant index of 0 (in this case there will be an error `Array indices are 1-based`). ## has(arr, elem) From a2c95b3c4977c4d497dd42c03b8a902ee9cec074 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 5 Sep 2019 19:37:42 -0300 Subject: [PATCH 70/75] Update array_functions.md added note about ArrayElement index 0 case. --- docs/en/query_language/functions/array_functions.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index daef852cb70..41c6d9bae76 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -73,7 +73,7 @@ Get the element with the index `n` from the array `arr`. `n` must be any integer Indexes in an array begin from one. Negative indexes are supported. In this case, it selects the corresponding element numbered from the end. For example, `arr[-1]` is the last item in the array. -If the index falls outside of the bounds of an array, it returns some default value (0 for numbers, an empty string for strings, etc.), except for the case with a non-constant array and a constant index of 0 (in this case there will be an error `Array indices are 1-based`). +If the index falls outside of the bounds of an array, it returns some default value (0 for numbers, an empty string for strings, etc.), except for the case with a non-constant array and a constant index 0 (in this case there will be an error `Array indices are 1-based`). ## has(arr, elem) From 50c32490bef08dd785401ccaa5bc83c0ba362a36 Mon Sep 17 00:00:00 2001 From: Denis Zhuravlev Date: Thu, 5 Sep 2019 19:41:27 -0300 Subject: [PATCH 71/75] Update arrayElement.cpp Typo in error message ( is -> are ) --- dbms/src/Functions/array/arrayElement.cpp | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dbms/src/Functions/array/arrayElement.cpp b/dbms/src/Functions/array/arrayElement.cpp index 59f275692aa..d7132d0fe23 100644 --- a/dbms/src/Functions/array/arrayElement.cpp +++ b/dbms/src/Functions/array/arrayElement.cpp @@ -858,7 +858,7 @@ void FunctionArrayElement::perform(Block & block, const ColumnNumbers & argument builder.initSink(input_rows_count); if (index == 0u) - throw Exception("Array indices is 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); + throw Exception("Array indices are 1-based", ErrorCodes::ZERO_ARRAY_OR_TUPLE_INDEX); if (!(executeNumberConst(block, arguments, result, index, builder) || executeNumberConst(block, arguments, result, index, builder) From 34f393c17e3bd3f80941ee0f0c6cbadef57f1605 Mon Sep 17 00:00:00 2001 From: Mikhail Filimonov Date: Fri, 6 Sep 2019 01:35:25 +0200 Subject: [PATCH 72/75] Ability to change history path by changing env --- dbms/programs/client/Client.cpp | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/dbms/programs/client/Client.cpp b/dbms/programs/client/Client.cpp index df5e8568d21..854e226cb94 100644 --- a/dbms/programs/client/Client.cpp +++ b/dbms/programs/client/Client.cpp @@ -431,8 +431,14 @@ private: /// Load command history if present. if (config().has("history_file")) history_file = config().getString("history_file"); - else if (!home_path.empty()) - history_file = home_path + "/.clickhouse-client-history"; + else + { + auto history_file_from_env = getenv("CLICKHOUSE_HISTORY_FILE"); + if (history_file_from_env) + history_file = history_file_from_env; + else if (!home_path.empty()) + history_file = home_path + "/.clickhouse-client-history"; + } if (!history_file.empty()) { From 05a1a7ac7163d15ce2989bfaa9ba332cf1d63252 Mon Sep 17 00:00:00 2001 From: alexey-milovidov Date: Fri, 6 Sep 2019 03:18:06 +0300 Subject: [PATCH 73/75] Update InterpreterSelectQuery.cpp --- dbms/src/Interpreters/InterpreterSelectQuery.cpp | 1 - 1 file changed, 1 deletion(-) diff --git a/dbms/src/Interpreters/InterpreterSelectQuery.cpp b/dbms/src/Interpreters/InterpreterSelectQuery.cpp index aff7348a0c4..57ccce566be 100644 --- a/dbms/src/Interpreters/InterpreterSelectQuery.cpp +++ b/dbms/src/Interpreters/InterpreterSelectQuery.cpp @@ -1526,7 +1526,6 @@ void InterpreterSelectQuery::executeFetchColumns( streams.back() = std::make_shared(streams.back(), query_info.prewhere_info->remove_columns_actions); } } - } for (auto & stream : streams) From 76c84d1d638b421ae9bbccf1a0f8f22752178107 Mon Sep 17 00:00:00 2001 From: BayoNet Date: Fri, 6 Sep 2019 12:07:23 +0300 Subject: [PATCH 74/75] DOCAPI-7695: Functions support for indexes (#6784) --- docs/en/development/build.md | 10 +- docs/en/interfaces/formats.md | 4 +- docs/en/operations/settings/settings.md | 2 +- docs/en/operations/table_engines/mergetree.md | 42 +++++++ .../functions/array_functions.md | 4 +- .../functions/comparison_functions.md | 12 +- .../query_language/functions/in_functions.md | 2 +- .../functions/string_functions.md | 4 +- .../functions/string_search_functions.md | 6 +- docs/fa/interfaces/formats.md | 118 ++++++++++++++++++ docs/ru/interfaces/formats.md | 4 +- docs/ru/operations/settings/settings.md | 2 +- .../functions/comparison_functions.md | 12 +- .../query_language/functions/in_functions.md | 3 +- .../functions/string_functions.md | 8 ++ .../functions/string_search_functions.md | 6 +- docs/zh/interfaces/formats.md | 117 +++++++++++++++++ 17 files changed, 321 insertions(+), 35 deletions(-) diff --git a/docs/en/development/build.md b/docs/en/development/build.md index 854562e191d..02cea936c70 100644 --- a/docs/en/development/build.md +++ b/docs/en/development/build.md @@ -40,7 +40,7 @@ sudo apt-get install git cmake ninja-build Or cmake3 instead of cmake on older systems. -## Install GCC 8 +## Install GCC 9 There are several ways to do this. @@ -50,18 +50,18 @@ There are several ways to do this. sudo apt-get install software-properties-common sudo apt-add-repository ppa:ubuntu-toolchain-r/test sudo apt-get update -sudo apt-get install gcc-8 g++-8 +sudo apt-get install gcc-9 g++-9 ``` ### Install from Sources Look at [utils/ci/build-gcc-from-sources.sh](https://github.com/yandex/ClickHouse/blob/master/utils/ci/build-gcc-from-sources.sh) -## Use GCC 8 for Builds +## Use GCC 9 for Builds ```bash -export CC=gcc-8 -export CXX=g++-8 +export CC=gcc-9 +export CXX=g++-9 ``` ## Install Required Libraries from Packages diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 792e7bce020..ed36e79fbc0 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -11,7 +11,7 @@ The supported formats are: | [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ | | [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | | [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | -| [Template](#template) | ✔ | ✔ | +| [Template](#format-template) | ✔ | ✔ | | [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | | [CSV](#csv) | ✔ | ✔ | | [CSVWithNames](#csvwithnames) | ✔ | ✔ | @@ -121,7 +121,7 @@ During parsing, the first and second rows are completely ignored. This format is also available under the name `TSVWithNamesAndTypes`. -## Template {#template} +## Template {#format-template} This format allows to specify a custom format string with placeholders for values with specified escaping rule. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 9c711073177..6f6201feb24 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -74,7 +74,7 @@ If `force_primary_key=1`, ClickHouse checks to see if the query has a primary ke ## format_schema -This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) or [Template](https://clickhouse.yandex/docs/en/interfaces/formats/#template). The value depends on the format. +This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) or [Template](../../interfaces/formats.md#format-template). The value depends on the format. ## fsync_metadata diff --git a/docs/en/operations/table_engines/mergetree.md b/docs/en/operations/table_engines/mergetree.md index a3646d3d0e9..7c694a1612c 100644 --- a/docs/en/operations/table_engines/mergetree.md +++ b/docs/en/operations/table_engines/mergetree.md @@ -313,6 +313,48 @@ INDEX sample_index2 (u64 * length(str), i32 + f64 * 100, date, str) TYPE set(100 INDEX sample_index3 (lower(str), str) TYPE ngrambf_v1(3, 256, 2, 0) GRANULARITY 4 ``` +#### Functions Support + +Conditions in the `WHERE` clause contain calls of functions over the columns. If the column is a part of some index, ClickHouse tries to use this index when performing the functions. ClickHouse supports different subset of functions for using indexes. + +The `set` index can be used with all functions. Functions subsets for other indexes are in the table below. + +Function (operator) / Index | primary key | minmax | ngrambf_v1 | tokenbf_v1 | bloom_filter +----------------------------|-------------|--------|------------|------------|--------------- +[equals (=, ==)](../../query_language/functions/comparison_functions.md#function-equals) | ✔ | ✔ | ✔ | ✔ | ✔ +[notEquals(!=, <>)](../../query_language/functions/comparison_functions.md#function-notequals) | ✔ | ✔ | ✔ | ✔ | ✔ +[like](../../query_language/functions/string_search_functions.md#function-like) | ✔ | ✔ | ✔ | ✗ | ✗ +[notLike](../../query_language/functions/string_search_functions.md#function-notlike) | ✔ | ✔ | ✔ | ✔ | ✗ +[startsWith](../../query_language/functions/string_functions.md#function-startswith) | ✔ | ✔ | ✔ | ✔ | ✗ +[endsWith](../../query_language/functions/string_functions.md#function-endswith) | ✗ | ✗ | ✔ | ✔ | +[multiSearchAny](../../query_language/functions/string_search_functions.md#function-multisearchany) | ✗ | ✗ | ✔ | ✔ | ✗ +[in](../../query_language/functions/in_functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ +[notIn](../../query_language/functions/in_functions.md#in-functions) | ✔ | ✔ | ✔ | ✔ | ✔ +[less (<)](../../query_language/functions/comparison_functions.md#function-less) | ✔ | ✔ | ✗ | ✗ | ✗ +[greater (>)](../../query_language/functions/comparison_functions.md#function-greater) | ✔ | ✔ | ✗ | ✗ | ✗ +[lessOrEquals (<=)](../../query_language/functions/comparison_functions.md#function-lessorequals) | ✔ | ✔ | ✗ | ✗ | ✗ +[greaterOrEquals (>=)](../../query_language/functions/comparison_functions.md#function-greaterorequals) | ✔ | ✔ | ✗ | ✗ | ✗ +[empty](../../query_language/functions/array_functions.md#function-empty) | ✔ | ✔ | ✗ | ✗ | ✗ +[notEmpty](../../query_language/functions/array_functions.md#function-notempty) | ✔ | ✔ | ✗ | ✗ | ✗ +hasToken | ✗ | ✗ | ✗ | ✔ | ✗ + +Functions with a constant argument less than ngram size couldn't be used by `ngrambf_v1` for the query optimization. + +Bloom filters can have false positive matches, so the `ngrambf_v1`, `tokenbf_v1`, `bloom_filter` indexes couldn't be used for optimizing queries where the result of a function is expected to be false, for example: + +- Can be optimized: + - `s LIKE '%test%'` + - `NOT s NOT LIKE '%test%'` + - `s = 1` + - `NOT s != 1` + - `startsWith(s, 'test')` +- Can't be optimized: + - `NOT s LIKE '%test%'` + - `s NOT LIKE '%test%'` + - `NOT s = 1` + - `s != 1` + - `NOT startsWith(s, 'test')` + ## Concurrent Data Access For concurrent table access, we use multi-versioning. In other words, when a table is simultaneously read and updated, data is read from a set of parts that is current at the time of the query. There are no lengthy locks. Inserts do not get in the way of read operations. diff --git a/docs/en/query_language/functions/array_functions.md b/docs/en/query_language/functions/array_functions.md index 41c6d9bae76..5065d428994 100644 --- a/docs/en/query_language/functions/array_functions.md +++ b/docs/en/query_language/functions/array_functions.md @@ -1,12 +1,12 @@ # Functions for working with arrays -## empty +## empty {#function-empty} Returns 1 for an empty array, or 0 for a non-empty array. The result type is UInt8. The function also works for strings. -## notEmpty +## notEmpty {#function-notempty} Returns 0 for an empty array, or 1 for a non-empty array. The result type is UInt8. diff --git a/docs/en/query_language/functions/comparison_functions.md b/docs/en/query_language/functions/comparison_functions.md index 39987ef2893..337a213673d 100644 --- a/docs/en/query_language/functions/comparison_functions.md +++ b/docs/en/query_language/functions/comparison_functions.md @@ -17,17 +17,17 @@ Strings are compared by bytes. A shorter string is smaller than all strings that Note. Up until version 1.1.54134, signed and unsigned numbers were compared the same way as in C++. In other words, you could get an incorrect result in cases like SELECT 9223372036854775807 > -1. This behavior changed in version 1.1.54134 and is now mathematically correct. -## equals, a = b and a == b operator +## equals, a = b and a == b operator {#function-equals} -## notEquals, a ! operator= b and a `<>` b +## notEquals, a ! operator= b and a `<>` b {#function-notequals} -## less, `< operator` +## less, `< operator` {#function-less} -## greater, `> operator` +## greater, `> operator` {#function-greater} -## lessOrEquals, `<= operator` +## lessOrEquals, `<= operator` {#function-lessorequals} -## greaterOrEquals, `>= operator` +## greaterOrEquals, `>= operator` {#function-greaterorequals} [Original article](https://clickhouse.yandex/docs/en/query_language/functions/comparison_functions/) diff --git a/docs/en/query_language/functions/in_functions.md b/docs/en/query_language/functions/in_functions.md index e7b355bb76c..5886dcc3bc7 100644 --- a/docs/en/query_language/functions/in_functions.md +++ b/docs/en/query_language/functions/in_functions.md @@ -1,6 +1,6 @@ # Functions for implementing the IN operator -## in, notIn, globalIn, globalNotIn +## in, notIn, globalIn, globalNotIn {#in-functions} See the section [IN operators](../select.md#select-in-operators). diff --git a/docs/en/query_language/functions/string_functions.md b/docs/en/query_language/functions/string_functions.md index b2ef05d4c3e..1eca9c0e815 100644 --- a/docs/en/query_language/functions/string_functions.md +++ b/docs/en/query_language/functions/string_functions.md @@ -145,11 +145,11 @@ Decode base64-encoded string 's' into original string. In case of failure raises ## tryBase64Decode(s) Similar to base64Decode, but in case of error an empty string would be returned. -## endsWith(s, suffix) +## endsWith(s, suffix) {#function-endswith} Returns whether to end with the specified suffix. Returns 1 if the string ends with the specified suffix, otherwise it returns 0. -## startsWith(s, prefix) +## startsWith(s, prefix) {#function-startswith} Returns whether to start with the specified prefix. Returns 1 if the string starts with the specified prefix, otherwise it returns 0. diff --git a/docs/en/query_language/functions/string_search_functions.md b/docs/en/query_language/functions/string_search_functions.md index 86f038b6a1e..723b8edc154 100644 --- a/docs/en/query_language/functions/string_search_functions.md +++ b/docs/en/query_language/functions/string_search_functions.md @@ -33,7 +33,7 @@ Returns the index `i` (starting from 1) of the leftmost found needlei For a case-insensitive search or/and in UTF-8 format use functions `multiSearchFirstIndexCaseInsensitive, multiSearchFirstIndexUTF8, multiSearchFirstIndexCaseInsensitiveUTF8`. -## multiSearchAny(haystack, [needle1, needle2, ..., needlen]) +## multiSearchAny(haystack, [needle1, needle2, ..., needlen]) {#function-multisearchany} Returns 1, if at least one string needlei matches the string `haystack` and 0 otherwise. @@ -86,7 +86,7 @@ Extracts a fragment of a string using a regular expression. If 'haystack' doesn' Extracts all the fragments of a string using a regular expression. If 'haystack' doesn't match the 'pattern' regex, an empty string is returned. Returns an array of strings consisting of all matches to the regex. In general, the behavior is the same as the 'extract' function (it takes the first subpattern, or the entire expression if there isn't a subpattern). -## like(haystack, pattern), haystack LIKE pattern operator +## like(haystack, pattern), haystack LIKE pattern operator {#function-like} Checks whether a string matches a simple regular expression. The regular expression can contain the metasymbols `%` and `_`. @@ -100,7 +100,7 @@ Use the backslash (`\`) for escaping metasymbols. See the note on escaping in th For regular expressions like `%needle%`, the code is more optimal and works as fast as the `position` function. For other regular expressions, the code is the same as for the 'match' function. -## notLike(haystack, pattern), haystack NOT LIKE pattern operator +## notLike(haystack, pattern), haystack NOT LIKE pattern operator {#function-notlike} The same thing as 'like', but negative. diff --git a/docs/fa/interfaces/formats.md b/docs/fa/interfaces/formats.md index a8c91c73b8a..d01dc1d1be0 100644 --- a/docs/fa/interfaces/formats.md +++ b/docs/fa/interfaces/formats.md @@ -12,6 +12,8 @@ Format | INSERT | SELECT [TabSeparatedRaw](formats.md#tabseparatedraw) | ✗ | ✔ | [TabSeparatedWithNames](formats.md#tabseparatedwithnames) | ✔ | ✔ | [TabSeparatedWithNamesAndTypes](formats.md#tabseparatedwithnamesandtypes) | ✔ | ✔ | +[Template](#format-template) | ✔ | ✔ | +[TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | [CSV](formats.md#csv) | ✔ | ✔ | [CSVWithNames](formats.md#csvwithnames) | ✔ | ✔ | [Values](formats.md#data-format-values) | ✔ | ✔ | @@ -115,6 +117,122 @@ SELECT EventDate, count() AS c FROM test.hits GROUP BY EventDate WITH TOTALS ORD همچنین این فرمت تحت عنوان ` TSVWithNamesAndTypes`وجود دارد. + +## Template {#format-template} + +This format allows to specify a custom format string with placeholders for values with specified escaping rule. + +It uses settings `format_schema`, `format_schema_rows`, `format_schema_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` when using `JSON` escaping, see further) + +Format string `format_schema_rows` specifies rows format with the following syntax: + + `delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`, + + where `delimiter_i` is a delimiter between values (`$` symbol can be escaped as `$$`), + `column_i` is a name of a column whose values are to be selected or inserted (if empty, then column will be skipped), + `serializeAs_i` is an escaping rule for the column values. The following escaping rules are supported: + + - `CSV`, `JSON`, `XML` (similarly to the formats of the same names) + - `Escaped` (similarly to `TSV`) + - `Quoted` (similarly to `Values`) + - `Raw` (without escaping, similarly to `TSVRaw`) + - `None` (no escaping rule, see further) + + If escaping rule is omitted, then`None` will be used. `XML` and `Raw` are suitable only for output. + + So, for the following format string: + + `Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};` + + the values of `SearchPhrase`, `c` and `price` columns, which are escaped as `Quoted`, `Escaped` and `JSON` will be printed (for select) or will be expected (for insert) between `Search phrase: `, `, count: `, `, ad price: $` and `;` delimiters respectively. For example: + + `Search phrase: 'bathroom interior design', count: 2166, ad price: $3;` + + The `format_schema_rows_between_delimiter` setting specifies delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default) + +Format string `format_schema` has the same syntax as `format_schema_rows` and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names: + + - `data` is the rows with data in `format_schema_rows` format, separated by `format_schema_rows_between_delimiter`. This placeholder must be the first placeholder in the format string. + - `totals` is the row with total values in `format_schema_rows` format (when using WITH TOTALS) + - `min` is the row with minimum values in `format_schema_rows` format (when extremes is set to 1) + - `max` is the row with maximum values in `format_schema_rows` format (when extremes is set to 1) + - `rows` is the total number of output rows + - `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT. + - `time` is the request execution time in seconds + - `rows_read` is the number of rows have been read + - `bytes_read` is the number of bytes (uncompressed) have been read + + The placeholders `data`, `totals`, `min` and `max` must not have escaping rule specified (or `None` must be specified explicitly). The remaining placeholders may have any escaping rule specified. + If the `format_schema` setting is an empty string, `${data}` is used as default value. + For insert queries format allows to skip some columns or some fields if prefix or suffix (see example). + + `Select` example: +```sql +SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase ORDER BY c DESC LIMIT 5 +FORMAT Template +SETTINGS format_schema = ' + Search phrases + + + + ${data} +
Search phrases
Search phrase Count
+ + ${max} +
Max
+ Processed ${rows_read:XML} rows in ${time:XML} sec + +', +format_schema_rows = ' ${SearchPhrase:XML} ${с:XML} ', +format_schema_rows_between_delimiter = '\n ' +``` +```html + + Search phrases + + + + + + + + +
Search phrases
Search phrase Count
8267016
bathroom interior design 2166
yandex 1655
spring 2014 fashion 1549
freeform photos 1480
+ + +
Max
8873898
+ Processed 3095973 rows in 0.1569913 sec + + +``` + +`Insert` example: +``` +Some header +Page views: 5, User id: 4324182021466249494, Useless field: hello, Duration: 146, Sign: -1 +Page views: 6, User id: 4324182021466249494, Useless field: world, Duration: 185, Sign: 1 +Total rows: 2 +``` +```sql +INSERT INTO UserActivity FORMAT Template SETTINGS +format_schema = 'Some header\n${data}\nTotal rows: ${:CSV}\n', +format_schema_rows = 'Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Duration: ${Duration:CSV}, Sign: ${Sign:CSV}' +``` +`PageViews`, `UserID`, `Duration` and `Sign` inside placeholders are names of columns in the table. Values after `Useless field` in rows and after `\nTotal rows: ` in suffix will be ignored. +All delimiters in the input data must be strictly equal to delimiters in specified format strings. + +## TemplateIgnoreSpaces {#templateignorespaces} + +This format is suitable only for input. +Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows to specify empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters. +It's possible to read `JSON` using this format, if values of columns have the same order in all rows. For example, the following request can be used for inserting data from output example of format [JSON](#json): +```sql +INSERT INTO table_name FORMAT TemplateIgnoreSpaces SETTINGS +format_schema = '{${}"meta"${}:${:JSON},${}"data"${}:${}[${data}]${},${}"totals"${}:${:JSON},${}"extremes"${}:${:JSON},${}"rows"${}:${:JSON},${}"rows_before_limit_at_least"${}:${:JSON}${}}', +format_schema_rows = '{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}}', +format_schema_rows_between_delimiter = ',' +``` + ## TSKV مشابه فرمت TabSeparated، اما خروجی به صورت name=value می باشد. نام ها مشابه روش TabSeparated، escape می شوند، و همچنین = symbol هم escape می شود. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 8bfaf8186e0..15f7552f877 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -10,7 +10,7 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ | | [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | | [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | -| [Template](#template) | ✔ | ✔ | +| [Template](#format-template) | ✔ | ✔ | | [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | | [CSV](#csv) | ✔ | ✔ | | [CSVWithNames](#csvwithnames) | ✔ | ✔ | @@ -120,7 +120,7 @@ world Этот формат также доступен под именем `TSVWithNamesAndTypes`. -## Template {#template} +## Template {#format-template} Этот формат позволяет указать произвольную форматную строку, в которую подставляются значения, сериализованные выбранным способом. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 831f5958c29..c3518eb7f74 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -72,7 +72,7 @@ ClickHouse применяет настройку в тех случаях, ко ## format_schema -Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#template). Значение параметра зависит от формата. +Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#format-template). Значение параметра зависит от формата. ## fsync_metadata diff --git a/docs/ru/query_language/functions/comparison_functions.md b/docs/ru/query_language/functions/comparison_functions.md index 392c6c2573c..b69e272133b 100644 --- a/docs/ru/query_language/functions/comparison_functions.md +++ b/docs/ru/query_language/functions/comparison_functions.md @@ -18,16 +18,16 @@ Замечание. До версии 1.1.54134 сравнение знаковых и беззнаковых целых чисел производилось также, как в C++. То есть, вы могли получить неверный результат в таких случаях: SELECT 9223372036854775807 > -1. С версии 1.1.54134 поведение изменилось и стало математически корректным. -## equals, оператор a = b и a == b +## equals, оператор a = b и a == b {#function-equals} -## notEquals, оператор a != b и a `<>` b +## notEquals, оператор a != b и a `<>` b {#function-notequals} -## less, оператор `<` +## less, оператор `<` {#function-less} -## greater, оператор `>` +## greater, оператор `>` {#function-greater} -## lessOrEquals, оператор `<=` +## lessOrEquals, оператор `<=` {#function-lessorequals} -## greaterOrEquals, оператор `>=` +## greaterOrEquals, оператор `>=` {#function-greaterorequals} [Оригинальная статья](https://clickhouse.yandex/docs/ru/query_language/functions/comparison_functions/) diff --git a/docs/ru/query_language/functions/in_functions.md b/docs/ru/query_language/functions/in_functions.md index 7eb87e53a6e..8b4eccd0db6 100644 --- a/docs/ru/query_language/functions/in_functions.md +++ b/docs/ru/query_language/functions/in_functions.md @@ -1,6 +1,7 @@ # Функции для реализации оператора IN. -## in, notIn, globalIn, globalNotIn +## in, notIn, globalIn, globalNotIn {#in-functions} + Смотрите раздел [Операторы IN](../select.md#select-in-operators). ## tuple(x, y, ...), оператор (x, y, ...) diff --git a/docs/ru/query_language/functions/string_functions.md b/docs/ru/query_language/functions/string_functions.md index f427017ee59..cc6563dacd5 100644 --- a/docs/ru/query_language/functions/string_functions.md +++ b/docs/ru/query_language/functions/string_functions.md @@ -117,6 +117,14 @@ SELECT format('{} {}', 'Hello', 'World') ## tryBase64Decode(s) Функционал аналогичен base64Decode, но при невозможности декодирования возвращает пустую строку. +## endsWith(s, suffix) {#function-endswith} + +Возвращает 1, если строка завершается указанным суффиксом, и 0 в противном случае. + +## startsWith(s, prefix) {#function-startswith} + +Возвращает 1, если строка начинается указанным префиксом, и 0 в противном случае. + ## CRC32(s) Возвращает чексумму CRC32 данной строки. Тип результата - UInt32. diff --git a/docs/ru/query_language/functions/string_search_functions.md b/docs/ru/query_language/functions/string_search_functions.md index 3fc500bd203..0f86554b552 100644 --- a/docs/ru/query_language/functions/string_search_functions.md +++ b/docs/ru/query_language/functions/string_search_functions.md @@ -29,7 +29,7 @@ Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchFirstIndexCaseInsensitive, multiSearchFirstIndexUTF8, multiSearchFirstIndexCaseInsensitiveUTF8`. -## multiSearchAny(haystack, [needle1, needle2, ..., needlen]) +## multiSearchAny(haystack, [needle1, needle2, ..., needlen]) {#function-multisearchany} Возвращает 1, если хотя бы одна подстрока needlei нашлась в строке `haystack` и 0 иначе. Для поиска без учета регистра и/или в кодировке UTF-8 используйте функции `multiSearchAnyCaseInsensitive, multiSearchAnyUTF8, multiSearchAnyCaseInsensitiveUTF8`. @@ -77,7 +77,7 @@ ## extractAll(haystack, pattern) Извлечение всех фрагментов строки по регулярному выражению. Если haystack не соответствует регулярному выражению pattern, то возвращается пустая строка. Возвращается массив строк, состоящий из всех соответствий регулярному выражению. В остальном, поведение аналогично функции extract (по прежнему, вынимается первый subpattern, или всё выражение, если subpattern-а нет). -## like(haystack, pattern), оператор haystack LIKE pattern +## like(haystack, pattern), оператор haystack LIKE pattern {#function-like} Проверка строки на соответствие простому регулярному выражению. Регулярное выражение может содержать метасимволы `%` и `_`. @@ -90,7 +90,7 @@ Для регулярных выражений вида `%needle%` действует более оптимальный код, который работает также быстро, как функция `position`. Для остальных регулярных выражений, код аналогичен функции match. -## notLike(haystack, pattern), оператор haystack NOT LIKE pattern +## notLike(haystack, pattern), оператор haystack NOT LIKE pattern {#function-notlike} То же, что like, но с отрицанием. ## ngramDistance(haystack, needle) diff --git a/docs/zh/interfaces/formats.md b/docs/zh/interfaces/formats.md index 65358115295..e4663c2d418 100644 --- a/docs/zh/interfaces/formats.md +++ b/docs/zh/interfaces/formats.md @@ -10,6 +10,8 @@ ClickHouse 可以接受多种数据格式,可以在 (`INSERT`) 以及 (`SELECT | [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ | | [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | | [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [Template](#format-template) | ✔ | ✔ | +| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | | [CSV](#csv) | ✔ | ✔ | | [CSVWithNames](#csvwithnames) | ✔ | ✔ | | [Values](#data-format-values) | ✔ | ✔ | @@ -115,6 +117,121 @@ world 这种格式也可以使用名称 ` TSVWithNamesAndTypes` 来表示。 +## Template {#format-template} + +This format allows to specify a custom format string with placeholders for values with specified escaping rule. + +It uses settings `format_schema`, `format_schema_rows`, `format_schema_rows_between_delimiter` and some settings of other formats (e.g. `output_format_json_quote_64bit_integers` when using `JSON` escaping, see further) + +Format string `format_schema_rows` specifies rows format with the following syntax: + + `delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`, + + where `delimiter_i` is a delimiter between values (`$` symbol can be escaped as `$$`), + `column_i` is a name of a column whose values are to be selected or inserted (if empty, then column will be skipped), + `serializeAs_i` is an escaping rule for the column values. The following escaping rules are supported: + + - `CSV`, `JSON`, `XML` (similarly to the formats of the same names) + - `Escaped` (similarly to `TSV`) + - `Quoted` (similarly to `Values`) + - `Raw` (without escaping, similarly to `TSVRaw`) + - `None` (no escaping rule, see further) + + If escaping rule is omitted, then`None` will be used. `XML` and `Raw` are suitable only for output. + + So, for the following format string: + + `Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};` + + the values of `SearchPhrase`, `c` and `price` columns, which are escaped as `Quoted`, `Escaped` and `JSON` will be printed (for select) or will be expected (for insert) between `Search phrase: `, `, count: `, `, ad price: $` and `;` delimiters respectively. For example: + + `Search phrase: 'bathroom interior design', count: 2166, ad price: $3;` + + The `format_schema_rows_between_delimiter` setting specifies delimiter between rows, which is printed (or expected) after every row except the last one (`\n` by default) + +Format string `format_schema` has the same syntax as `format_schema_rows` and allows to specify a prefix, a suffix and a way to print some additional information. It contains the following placeholders instead of column names: + + - `data` is the rows with data in `format_schema_rows` format, separated by `format_schema_rows_between_delimiter`. This placeholder must be the first placeholder in the format string. + - `totals` is the row with total values in `format_schema_rows` format (when using WITH TOTALS) + - `min` is the row with minimum values in `format_schema_rows` format (when extremes is set to 1) + - `max` is the row with maximum values in `format_schema_rows` format (when extremes is set to 1) + - `rows` is the total number of output rows + - `rows_before_limit` is the minimal number of rows there would have been without LIMIT. Output only if the query contains LIMIT. If the query contains GROUP BY, rows_before_limit_at_least is the exact number of rows there would have been without a LIMIT. + - `time` is the request execution time in seconds + - `rows_read` is the number of rows have been read + - `bytes_read` is the number of bytes (uncompressed) have been read + + The placeholders `data`, `totals`, `min` and `max` must not have escaping rule specified (or `None` must be specified explicitly). The remaining placeholders may have any escaping rule specified. + If the `format_schema` setting is an empty string, `${data}` is used as default value. + For insert queries format allows to skip some columns or some fields if prefix or suffix (see example). + + `Select` example: +```sql +SELECT SearchPhrase, count() AS c FROM test.hits GROUP BY SearchPhrase ORDER BY c DESC LIMIT 5 +FORMAT Template +SETTINGS format_schema = ' + Search phrases + + + + ${data} +
Search phrases
Search phrase Count
+ + ${max} +
Max
+ Processed ${rows_read:XML} rows in ${time:XML} sec + +', +format_schema_rows = ' ${SearchPhrase:XML} ${с:XML} ', +format_schema_rows_between_delimiter = '\n ' +``` +```html + + Search phrases + + + + + + + + +
Search phrases
Search phrase Count
8267016
bathroom interior design 2166
yandex 1655
spring 2014 fashion 1549
freeform photos 1480
+ + +
Max
8873898
+ Processed 3095973 rows in 0.1569913 sec + + +``` + +`Insert` example: +``` +Some header +Page views: 5, User id: 4324182021466249494, Useless field: hello, Duration: 146, Sign: -1 +Page views: 6, User id: 4324182021466249494, Useless field: world, Duration: 185, Sign: 1 +Total rows: 2 +``` +```sql +INSERT INTO UserActivity FORMAT Template SETTINGS +format_schema = 'Some header\n${data}\nTotal rows: ${:CSV}\n', +format_schema_rows = 'Page views: ${PageViews:CSV}, User id: ${UserID:CSV}, Useless field: ${:CSV}, Duration: ${Duration:CSV}, Sign: ${Sign:CSV}' +``` +`PageViews`, `UserID`, `Duration` and `Sign` inside placeholders are names of columns in the table. Values after `Useless field` in rows and after `\nTotal rows: ` in suffix will be ignored. +All delimiters in the input data must be strictly equal to delimiters in specified format strings. + +## TemplateIgnoreSpaces {#templateignorespaces} + +This format is suitable only for input. +Similar to `Template`, but skips whitespace characters between delimiters and values in the input stream. However, if format strings contain whitespace characters, these characters will be expected in the input stream. Also allows to specify empty placeholders (`${}` or `${:None}`) to split some delimiter into separate parts to ignore spaces between them. Such placeholders are used only for skipping whitespace characters. +It's possible to read `JSON` using this format, if values of columns have the same order in all rows. For example, the following request can be used for inserting data from output example of format [JSON](#json): +```sql +INSERT INTO table_name FORMAT TemplateIgnoreSpaces SETTINGS +format_schema = '{${}"meta"${}:${:JSON},${}"data"${}:${}[${data}]${},${}"totals"${}:${:JSON},${}"extremes"${}:${:JSON},${}"rows"${}:${:JSON},${}"rows_before_limit_at_least"${}:${:JSON}${}}', +format_schema_rows = '{${}"SearchPhrase"${}:${}${phrase:JSON}${},${}"c"${}:${}${cnt:JSON}${}}', +format_schema_rows_between_delimiter = ',' +``` + ## TSKV {#tskv} 与 `TabSeparated` 格式类似,但它输出的是 `name=value` 的格式。名称会和 `TabSeparated` 格式一样被转义,`=` 字符也会被转义。 From 89a444c4a026c36421ed4bd414746f04c1be0af9 Mon Sep 17 00:00:00 2001 From: Ivan Lezhankin Date: Fri, 6 Sep 2019 13:36:15 +0300 Subject: [PATCH 75/75] Fix script for releases --- utils/github/__main__.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/utils/github/__main__.py b/utils/github/__main__.py index 836ad734fce..d5de241f25f 100644 --- a/utils/github/__main__.py +++ b/utils/github/__main__.py @@ -47,14 +47,14 @@ parser.add_argument('--login', type=str, args = parser.parse_args() -github = query.Query(args.token, 50) +github = query.Query(args.token, 30) repo = local.Local(args.repo, args.remote, github.get_default_branch()) stables = repo.get_stables()[-args.number:] # [(branch name, base)] if not stables: - sys.exit('No stable branches found!') + sys.exit('No release branches found!') else: - print('Found stable branches:') + print('Found release branches:') for stable in stables: print(f'{CHECK_MARK} {stable[0]} forked from {stable[1]}')