diff --git a/.gitignore b/.gitignore index 5dc058c97c0..817e333d833 100644 --- a/.gitignore +++ b/.gitignore @@ -90,7 +90,6 @@ dbms/src/Core/tests/field dbms/src/Core/tests/rvo_test dbms/src/Core/tests/string_pool dbms/src/DataStreams/tests/aggregating_stream -dbms/src/DataStreams/tests/block_row_transforms dbms/src/DataStreams/tests/block_tab_separated_streams dbms/src/DataStreams/tests/collapsing_sorted_stream dbms/src/DataStreams/tests/expression_stream diff --git a/dbms/programs/main.cpp b/dbms/programs/main.cpp index 3fbbcee0f15..760eae4298b 100644 --- a/dbms/programs/main.cpp +++ b/dbms/programs/main.cpp @@ -21,6 +21,7 @@ #include #include +#include /// Universal executable for various clickhouse applications @@ -130,8 +131,19 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector & argv) } +/// This allows to implement assert to forbid initialization of a class in static constructors. +/// Usage: +/// +/// extern bool inside_main; +/// class C { C() { assert(inside_main); } }; +bool inside_main = false; + + int main(int argc_, char ** argv_) { + inside_main = true; + SCOPE_EXIT({ inside_main = false; }); + /// Reset new handler to default (that throws std::bad_alloc) /// It is needed because LLVM library clobbers it. std::set_new_handler(nullptr); diff --git a/dbms/src/Common/ErrorCodes.cpp b/dbms/src/Common/ErrorCodes.cpp index 76622cfa43b..fd058f1fbe0 100644 --- a/dbms/src/Common/ErrorCodes.cpp +++ b/dbms/src/Common/ErrorCodes.cpp @@ -447,6 +447,8 @@ namespace ErrorCodes extern const int QUERY_IS_NOT_SUPPORTED_IN_LIVE_VIEW = 470; extern const int SETTINGS_ARE_NOT_SUPPORTED = 471; extern const int READONLY_SETTING = 472; + extern const int DEADLOCK_AVOIDED = 473; + extern const int INVALID_TEMPLATE_FORMAT = 474; extern const int KEEPER_EXCEPTION = 999; extern const int POCO_EXCEPTION = 1000; diff --git a/dbms/src/Common/RWLock.cpp b/dbms/src/Common/RWLock.cpp index e343ce0b0cd..1b26b275528 100644 --- a/dbms/src/Common/RWLock.cpp +++ b/dbms/src/Common/RWLock.cpp @@ -4,6 +4,8 @@ #include #include +#include + namespace ProfileEvents { @@ -29,6 +31,7 @@ namespace DB namespace ErrorCodes { extern const int LOGICAL_ERROR; + extern const int DEADLOCK_AVOIDED; } @@ -37,7 +40,6 @@ class RWLockImpl::LockHolderImpl RWLock parent; GroupsContainer::iterator it_group; ClientsContainer::iterator it_client; - ThreadToHolder::key_type thread_id; QueryIdToHolder::key_type query_id; CurrentMetrics::Increment active_client_increment; @@ -53,6 +55,44 @@ public: }; +namespace +{ + /// Global information about all read locks that query has. It is needed to avoid some type of deadlocks. + + class QueryLockInfo + { + private: + std::mutex mutex; + std::map queries; + + public: + void add(const String & query_id) + { + std::lock_guard lock(mutex); + ++queries[query_id]; + } + + void remove(const String & query_id) + { + std::lock_guard lock(mutex); + auto it = queries.find(query_id); + assert(it != queries.end()); + if (--it->second == 0) + queries.erase(it); + } + + void check(const String & query_id) + { + std::lock_guard lock(mutex); + if (queries.count(query_id)) + throw Exception("Possible deadlock avoided. Client should retry.", ErrorCodes::DEADLOCK_AVOIDED); + } + }; + + QueryLockInfo all_read_locks; +} + + RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String & query_id) { Stopwatch watch(CLOCK_MONOTONIC_COARSE); @@ -69,34 +109,48 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String & GroupsContainer::iterator it_group; ClientsContainer::iterator it_client; + /// This object is placed above unique_lock, because it may lock in destructor. + LockHolder res; + std::unique_lock lock(mutex); /// Check if the same query is acquiring previously acquired lock - LockHolder existing_holder_ptr; - - auto this_thread_id = std::this_thread::get_id(); - auto it_thread = thread_to_holder.find(this_thread_id); - - auto it_query = query_id_to_holder.end(); if (query_id != RWLockImpl::NO_QUERY) - it_query = query_id_to_holder.find(query_id); + { + auto it_query = query_id_to_holder.find(query_id); + if (it_query != query_id_to_holder.end()) + res = it_query->second.lock(); + } - if (it_thread != thread_to_holder.end()) - existing_holder_ptr = it_thread->second.lock(); - else if (it_query != query_id_to_holder.end()) - existing_holder_ptr = it_query->second.lock(); - - if (existing_holder_ptr) + if (res) { /// XXX: it means we can't upgrade lock from read to write - with proper waiting! - if (type != Read || existing_holder_ptr->it_group->type != Read) + if (type != Read || res->it_group->type != Read) throw Exception("Attempt to acquire exclusive lock recursively", ErrorCodes::LOGICAL_ERROR); - - return existing_holder_ptr; + else + return res; } + /** If the query already has any active read lock and tries to acquire another read lock + * but it is not in front of the queue and has to wait, deadlock is possible: + * + * Example (four queries, two RWLocks - 'a' and 'b'): + * + * --> time --> + * + * q1: ra rb + * q2: wa + * q3: rb ra + * q4: wb + * + * We will throw an exception instead. + */ + if (type == Type::Write || queue.empty() || queue.back().type == Type::Write) { + if (type == Type::Read && !queue.empty() && queue.back().type == Type::Write && query_id != RWLockImpl::NO_QUERY) + all_read_locks.check(query_id); + /// Create new group of clients it_group = queue.emplace(queue.end(), type); } @@ -104,6 +158,9 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String & { /// Will append myself to last group it_group = std::prev(queue.end()); + + if (it_group != queue.begin() && query_id != RWLockImpl::NO_QUERY) + all_read_locks.check(query_id); } /// Append myself to the end of chosen group @@ -120,17 +177,19 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String & throw; } - LockHolder res(new LockHolderImpl(shared_from_this(), it_group, it_client)); + res.reset(new LockHolderImpl(shared_from_this(), it_group, it_client)); /// Wait a notification until we will be the only in the group. it_group->cv.wait(lock, [&] () { return it_group == queue.begin(); }); - /// Insert myself (weak_ptr to the holder) to threads set to implement recursive lock - thread_to_holder.emplace(this_thread_id, res); - res->thread_id = this_thread_id; - + /// Insert myself (weak_ptr to the holder) to queries set to implement recursive lock if (query_id != RWLockImpl::NO_QUERY) + { query_id_to_holder.emplace(query_id, res); + + if (type == Type::Read) + all_read_locks.add(query_id); + } res->query_id = query_id; finalize_metrics(); @@ -140,12 +199,14 @@ RWLockImpl::LockHolder RWLockImpl::getLock(RWLockImpl::Type type, const String & RWLockImpl::LockHolderImpl::~LockHolderImpl() { - std::unique_lock lock(parent->mutex); + std::lock_guard lock(parent->mutex); /// Remove weak_ptrs to the holder, since there are no owners of the current lock - parent->thread_to_holder.erase(thread_id); parent->query_id_to_holder.erase(query_id); + if (*it_client == RWLockImpl::Read && query_id != RWLockImpl::NO_QUERY) + all_read_locks.remove(query_id); + /// Removes myself from client list of our group it_group->clients.erase(it_client); @@ -166,6 +227,7 @@ RWLockImpl::LockHolderImpl::LockHolderImpl(RWLock && parent_, RWLockImpl::Groups : parent{std::move(parent_)}, it_group{it_group_}, it_client{it_client_}, active_client_increment{(*it_client == RWLockImpl::Read) ? CurrentMetrics::RWLockActiveReaders : CurrentMetrics::RWLockActiveWriters} -{} +{ +} } diff --git a/dbms/src/Common/RWLock.h b/dbms/src/Common/RWLock.h index 0467901fa27..c94dd3fe1a4 100644 --- a/dbms/src/Common/RWLock.h +++ b/dbms/src/Common/RWLock.h @@ -6,7 +6,6 @@ #include #include #include -#include #include #include @@ -19,7 +18,7 @@ using RWLock = std::shared_ptr; /// Implements shared lock with FIFO service -/// Can be acquired recursively (several calls for the same query or the same OS thread) in Read mode +/// Can be acquired recursively (several calls for the same query) in Read mode /// /// NOTE: it is important to allow acquiring the same lock in Read mode without waiting if it is already /// acquired by another thread of the same query. Otherwise the following deadlock is possible: @@ -55,7 +54,6 @@ private: struct Group; using GroupsContainer = std::list; using ClientsContainer = std::list; - using ThreadToHolder = std::map>; using QueryIdToHolder = std::map>; /// Group of clients that should be executed concurrently @@ -73,7 +71,6 @@ private: mutable std::mutex mutex; GroupsContainer queue; - ThreadToHolder thread_to_holder; QueryIdToHolder query_id_to_holder; }; diff --git a/dbms/src/Common/tests/gtest_rw_lock.cpp b/dbms/src/Common/tests/gtest_rw_lock.cpp index 68927c8bc4a..8533eaafaa3 100644 --- a/dbms/src/Common/tests/gtest_rw_lock.cpp +++ b/dbms/src/Common/tests/gtest_rw_lock.cpp @@ -13,6 +13,14 @@ using namespace DB; +namespace DB +{ + namespace ErrorCodes + { + extern const int DEADLOCK_AVOIDED; + } +} + TEST(Common, RWLock_1) { @@ -94,7 +102,7 @@ TEST(Common, RWLock_Recursive) { for (int i = 0; i < 2 * cycles; ++i) { - auto lock = fifo_lock->getLock(RWLockImpl::Write, RWLockImpl::NO_QUERY); + auto lock = fifo_lock->getLock(RWLockImpl::Write, "q1"); auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); std::this_thread::sleep_for(sleep_for); @@ -105,17 +113,17 @@ TEST(Common, RWLock_Recursive) { for (int i = 0; i < cycles; ++i) { - auto lock1 = fifo_lock->getLock(RWLockImpl::Read, RWLockImpl::NO_QUERY); + auto lock1 = fifo_lock->getLock(RWLockImpl::Read, "q2"); auto sleep_for = std::chrono::duration(std::uniform_int_distribution<>(1, 100)(gen)); std::this_thread::sleep_for(sleep_for); - auto lock2 = fifo_lock->getLock(RWLockImpl::Read, RWLockImpl::NO_QUERY); + auto lock2 = fifo_lock->getLock(RWLockImpl::Read, "q2"); - EXPECT_ANY_THROW({fifo_lock->getLock(RWLockImpl::Write, RWLockImpl::NO_QUERY);}); + EXPECT_ANY_THROW({fifo_lock->getLock(RWLockImpl::Write, "q2");}); } - fifo_lock->getLock(RWLockImpl::Write, RWLockImpl::NO_QUERY); + fifo_lock->getLock(RWLockImpl::Write, "q2"); }); t1.join(); @@ -123,6 +131,74 @@ TEST(Common, RWLock_Recursive) } +TEST(Common, RWLock_Deadlock) +{ + static auto lock1 = RWLockImpl::create(); + static auto lock2 = RWLockImpl::create(); + + /** + * q1: r1 r2 + * q2: w1 + * q3: r2 r1 + * q4: w2 + */ + + std::thread t1([&] () + { + auto holder1 = lock1->getLock(RWLockImpl::Read, "q1"); + usleep(100000); + usleep(100000); + usleep(100000); + try + { + auto holder2 = lock2->getLock(RWLockImpl::Read, "q1"); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::DEADLOCK_AVOIDED) + throw; + } + }); + + std::thread t2([&] () + { + usleep(100000); + auto holder1 = lock1->getLock(RWLockImpl::Write, "q2"); + }); + + std::thread t3([&] () + { + usleep(100000); + usleep(100000); + auto holder2 = lock2->getLock(RWLockImpl::Read, "q3"); + usleep(100000); + usleep(100000); + try + { + auto holder1 = lock1->getLock(RWLockImpl::Read, "q3"); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::DEADLOCK_AVOIDED) + throw; + } + }); + + std::thread t4([&] () + { + usleep(100000); + usleep(100000); + usleep(100000); + auto holder2 = lock2->getLock(RWLockImpl::Write, "q4"); + }); + + t1.join(); + t2.join(); + t3.join(); + t4.join(); +} + + TEST(Common, RWLock_PerfTest_Readers) { constexpr int cycles = 100000; // 100k diff --git a/dbms/src/Core/Settings.h b/dbms/src/Core/Settings.h index a000b2bae0b..5cb92038977 100644 --- a/dbms/src/Core/Settings.h +++ b/dbms/src/Core/Settings.h @@ -216,6 +216,8 @@ struct Settings : public SettingsCollection M(SettingMilliseconds, stream_flush_interval_ms, 7500, "Timeout for flushing data from streaming storages.") \ M(SettingMilliseconds, stream_poll_timeout_ms, 500, "Timeout for polling data from/to streaming storages.") \ M(SettingString, format_schema, "", "Schema identifier (used by schema-based formats)") \ + M(SettingString, format_schema_rows, "", "Row format string for Template format") \ + M(SettingString, format_schema_rows_between_delimiter, "\n", "Delimiter between rows for Template format") \ M(SettingBool, insert_allow_materialized_columns, 0, "If setting is enabled, Allow materialized columns in INSERT.") \ M(SettingSeconds, http_connection_timeout, DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT, "HTTP connection timeout.") \ M(SettingSeconds, http_send_timeout, DEFAULT_HTTP_READ_BUFFER_TIMEOUT, "HTTP send timeout") \ diff --git a/dbms/src/Formats/BlockInputStreamFromRowInputStream.cpp b/dbms/src/Formats/BlockInputStreamFromRowInputStream.cpp deleted file mode 100644 index fc38b476e0b..00000000000 --- a/dbms/src/Formats/BlockInputStreamFromRowInputStream.cpp +++ /dev/null @@ -1,178 +0,0 @@ -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED; - extern const int CANNOT_PARSE_QUOTED_STRING; - extern const int CANNOT_PARSE_DATE; - extern const int CANNOT_PARSE_DATETIME; - extern const int CANNOT_READ_ARRAY_FROM_TEXT; - extern const int CANNOT_PARSE_NUMBER; - extern const int CANNOT_PARSE_UUID; - extern const int TOO_LARGE_STRING_SIZE; - extern const int CANNOT_READ_ALL_DATA; - extern const int INCORRECT_DATA; - extern const int INCORRECT_NUMBER_OF_COLUMNS; -} - - -BlockInputStreamFromRowInputStream::BlockInputStreamFromRowInputStream( - const RowInputStreamPtr & row_input_, - const Block & sample_, - UInt64 max_block_size_, - UInt64 rows_portion_size_, - FormatFactory::ReadCallback callback, - const FormatSettings & settings) - : row_input(row_input_) - , sample(sample_) - , max_block_size(max_block_size_) - , rows_portion_size(rows_portion_size_) - , read_virtual_columns_callback(callback) - , allow_errors_num(settings.input_allow_errors_num) - , allow_errors_ratio(settings.input_allow_errors_ratio) -{ -} - - -static bool isParseError(int code) -{ - return code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED - || code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING - || code == ErrorCodes::CANNOT_PARSE_DATE - || code == ErrorCodes::CANNOT_PARSE_DATETIME - || code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT - || code == ErrorCodes::CANNOT_PARSE_NUMBER - || code == ErrorCodes::CANNOT_PARSE_UUID - || code == ErrorCodes::TOO_LARGE_STRING_SIZE - || code == ErrorCodes::CANNOT_READ_ALL_DATA - || code == ErrorCodes::INCORRECT_DATA; -} - - -Block BlockInputStreamFromRowInputStream::readImpl() -{ - size_t num_columns = sample.columns(); - MutableColumns columns = sample.cloneEmptyColumns(); - block_missing_values.clear(); - - try - { - for (size_t rows = 0, batch = 0; rows < max_block_size; ++rows, ++batch) - { - if (rows_portion_size && batch == rows_portion_size) - { - batch = 0; - if (!checkTimeLimit() || isCancelled()) - break; - } - - try - { - ++total_rows; - RowReadExtension info_; - if (!row_input->read(columns, info_)) - break; - if (read_virtual_columns_callback) - read_virtual_columns_callback(); - - for (size_t column_idx = 0; column_idx < info_.read_columns.size(); ++column_idx) - { - if (!info_.read_columns[column_idx]) - { - size_t column_size = columns[column_idx]->size(); - if (column_size == 0) - throw Exception("Unexpected empty column", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS); - block_missing_values.setBit(column_idx, column_size - 1); - } - } - } - catch (Exception & e) - { - /// Logic for possible skipping of errors. - - if (!isParseError(e.code())) - throw; - - if (allow_errors_num == 0 && allow_errors_ratio == 0) - throw; - - ++num_errors; - Float32 current_error_ratio = static_cast(num_errors) / total_rows; - - if (num_errors > allow_errors_num - && current_error_ratio > allow_errors_ratio) - { - e.addMessage("(Already have " + toString(num_errors) + " errors" - " out of " + toString(total_rows) + " rows" - ", which is " + toString(current_error_ratio) + " of all rows)"); - throw; - } - - if (!row_input->allowSyncAfterError()) - { - e.addMessage("(Input format doesn't allow to skip errors)"); - throw; - } - - row_input->syncAfterError(); - - /// Truncate all columns in block to minimal size (remove values, that was appended to only part of columns). - - size_t min_size = std::numeric_limits::max(); - for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) - min_size = std::min(min_size, columns[column_idx]->size()); - - for (size_t column_idx = 0; column_idx < num_columns; ++column_idx) - { - auto & column = columns[column_idx]; - if (column->size() > min_size) - column->popBack(column->size() - min_size); - } - } - } - } - catch (Exception & e) - { - if (!isParseError(e.code())) - throw; - - String verbose_diagnostic; - try - { - verbose_diagnostic = row_input->getDiagnosticInfo(); - } - catch (...) - { - /// Error while trying to obtain verbose diagnostic. Ok to ignore. - } - - e.addMessage("(at row " + toString(total_rows) + ")\n" + verbose_diagnostic); - throw; - } - - if (columns.empty() || columns[0]->empty()) - return {}; - - return sample.cloneWithColumns(std::move(columns)); -} - - -void BlockInputStreamFromRowInputStream::readSuffix() -{ - if (allow_errors_num > 0 || allow_errors_ratio > 0) - { - Logger * log = &Logger::get("BlockInputStreamFromRowInputStream"); - LOG_TRACE(log, "Skipped " << num_errors << " rows with errors while reading the input stream"); - } - - row_input->readSuffix(); -} - -} diff --git a/dbms/src/Formats/BlockInputStreamFromRowInputStream.h b/dbms/src/Formats/BlockInputStreamFromRowInputStream.h deleted file mode 100644 index 2338af3bf38..00000000000 --- a/dbms/src/Formats/BlockInputStreamFromRowInputStream.h +++ /dev/null @@ -1,62 +0,0 @@ -#pragma once - -#include -#include -#include -#include -#include - - -namespace DB -{ - -/** Makes block-oriented stream on top of row-oriented stream. - * It is used to read data from text formats. - * - * Also controls over parsing errors and prints diagnostic information about them. - */ -class BlockInputStreamFromRowInputStream : public IBlockInputStream -{ -public: - /// |sample| is a block with zero rows, that structure describes how to interpret values - /// |rows_portion_size| is a number of rows to read before break and check limits - BlockInputStreamFromRowInputStream( - const RowInputStreamPtr & row_input_, - const Block & sample_, - UInt64 max_block_size_, - UInt64 rows_portion_size_, - FormatFactory::ReadCallback callback, - const FormatSettings & settings); - - void readPrefix() override { row_input->readPrefix(); } - void readSuffix() override; - - String getName() const override { return "BlockInputStreamFromRowInputStream"; } - - RowInputStreamPtr & getRowInput() { return row_input; } - - Block getHeader() const override { return sample; } - - const BlockMissingValues & getMissingValues() const override { return block_missing_values; } - -protected: - Block readImpl() override; - -private: - RowInputStreamPtr row_input; - Block sample; - UInt64 max_block_size; - UInt64 rows_portion_size; - - /// Callback used to setup virtual columns after reading each row. - FormatFactory::ReadCallback read_virtual_columns_callback; - - BlockMissingValues block_missing_values; - - UInt64 allow_errors_num; - Float32 allow_errors_ratio; - - size_t total_rows = 0; - size_t num_errors = 0; -}; -} diff --git a/dbms/src/Formats/CSVRowInputStream.cpp b/dbms/src/Formats/CSVRowInputStream.cpp deleted file mode 100644 index 4b1b6198486..00000000000 --- a/dbms/src/Formats/CSVRowInputStream.cpp +++ /dev/null @@ -1,573 +0,0 @@ -#include - -#include -#include -#include - -#include -#include -#include -#include - -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; - extern const int LOGICAL_ERROR; -} - - -static inline void skipEndOfLine(ReadBuffer & istr) -{ - /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) - - if (*istr.position() == '\n') - { - ++istr.position(); - if (!istr.eof() && *istr.position() == '\r') - ++istr.position(); - } - else if (*istr.position() == '\r') - { - ++istr.position(); - if (!istr.eof() && *istr.position() == '\n') - ++istr.position(); - else - throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)." - " Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA); - } - else if (!istr.eof()) - throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA); -} - - -static inline void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column) -{ - if (is_last_column) - { - if (istr.eof()) - return; - - /// we support the extra delimiter at the end of the line - if (*istr.position() == delimiter) - { - ++istr.position(); - if (istr.eof()) - return; - } - - skipEndOfLine(istr); - } - else - assertChar(delimiter, istr); -} - - -/// Skip `whitespace` symbols allowed in CSV. -static inline void skipWhitespacesAndTabs(ReadBuffer & buf) -{ - while (!buf.eof() - && (*buf.position() == ' ' - || *buf.position() == '\t')) - ++buf.position(); -} - - -static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns) -{ - String tmp; - for (size_t i = 0; i < num_columns; ++i) - { - skipWhitespacesAndTabs(istr); - readCSVString(tmp, istr, settings); - skipWhitespacesAndTabs(istr); - - skipDelimiter(istr, settings.delimiter, i + 1 == num_columns); - } -} - - -CSVRowInputStream::CSVRowInputStream(ReadBuffer & istr_, const Block & header_, bool with_names_, const FormatSettings & format_settings_) - : istr(istr_), header(header_), with_names(with_names_), format_settings(format_settings_) -{ - const auto num_columns = header.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - column_idx_to_nullable_column_idx.resize(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = header.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - - /// If input_format_null_as_default=1 we need ColumnNullable of type DataTypeNullable(nested_type) - /// to parse value as nullable before inserting it in corresponding column of not-nullable type. - /// Constructing temporary column for each row is slow, so we prepare it here - if (format_settings.csv.null_as_default && !column_info.type->isNullable() && column_info.type->canBeInsideNullable()) - { - column_idx_to_nullable_column_idx[i] = nullable_columns.size(); - nullable_types.emplace_back(std::make_shared(column_info.type)); - nullable_columns.emplace_back(nullable_types.back()->createColumn()); - } - } -} - -/// Map an input file column to a table column, based on its name. -void CSVRowInputStream::addInputColumn(const String & column_name) -{ - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - - throw Exception( - "Unknown field found in CSV header: '" + column_name + "' " + - "at position " + std::to_string(column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (read_columns[column_index]) - throw Exception("Duplicate field found while parsing CSV header: " + column_name, ErrorCodes::INCORRECT_DATA); - - read_columns[column_index] = true; - column_indexes_for_input_fields.emplace_back(column_index); -} - -void CSVRowInputStream::readPrefix() -{ - /// In this format, we assume, that if first string field contain BOM as value, it will be written in quotes, - /// so BOM at beginning of stream cannot be confused with BOM in first string value, and it is safe to skip it. - skipBOMIfExists(istr); - - if (with_names) - { - /// This CSV file has a header row with column names. Depending on the - /// settings, use it or skip it. - if (format_settings.with_names_use_header) - { - /// Look at the file header to see which columns we have there. - /// The missing columns are filled with defaults. - read_columns.assign(header.columns(), false); - do - { - String column_name; - skipWhitespacesAndTabs(istr); - readCSVString(column_name, istr, format_settings.csv); - skipWhitespacesAndTabs(istr); - - addInputColumn(column_name); - } - while (checkChar(format_settings.csv.delimiter, istr)); - - skipDelimiter(istr, format_settings.csv.delimiter, true); - - for (size_t column = 0; column < read_columns.size(); column++) - { - if (!read_columns[column]) - { - have_always_default_columns = true; - break; - } - } - - return; - } - else - { - skipRow(istr, format_settings.csv, header.columns()); - } - } - - /// The default: map each column of the file to the column of the table with - /// the same index. - read_columns.assign(header.columns(), true); - column_indexes_for_input_fields.resize(header.columns()); - - for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) - { - column_indexes_for_input_fields[i] = i; - } -} - -/** If you change this function, don't forget to change its counterpart - * with extended error reporting: parseRowAndPrintDiagnosticInfo(). - */ -bool CSVRowInputStream::read(MutableColumns & columns, RowReadExtension & ext) -{ - if (istr.eof()) - return false; - - updateDiagnosticInfo(); - - /// Track whether we have to fill any columns in this row with default - /// values. If not, we return an empty column mask to the caller, so that - /// it doesn't have to check it. - bool have_default_columns = have_always_default_columns; - - const auto delimiter = format_settings.csv.delimiter; - for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) - { - const auto & table_column = column_indexes_for_input_fields[file_column]; - const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size(); - - if (table_column) - { - skipWhitespacesAndTabs(istr); - read_columns[*table_column] = readField(*columns[*table_column], data_types[*table_column], - is_last_file_column, *table_column); - if (!read_columns[*table_column]) - have_default_columns = true; - skipWhitespacesAndTabs(istr); - } - else - { - /// We never read this column from the file, just skip it. - String tmp; - readCSVString(tmp, istr, format_settings.csv); - } - - skipDelimiter(istr, delimiter, is_last_file_column); - } - - if (have_default_columns) - { - for (size_t i = 0; i < read_columns.size(); i++) - { - if (!read_columns[i]) - { - /// The column value for this row is going to be overwritten - /// with default by the caller, but the general assumption is - /// that the column size increases for each row, so we have - /// to insert something. Since we do not care about the exact - /// value, we do not have to use the default value specified by - /// the data type, and can just use IColumn::insertDefault(). - columns[i]->insertDefault(); - } - } - ext.read_columns = read_columns; - } - - return true; -} - - -String CSVRowInputStream::getDiagnosticInfo() -{ - if (istr.eof()) /// Buffer has gone, cannot extract information about what has been parsed. - return {}; - - WriteBufferFromOwnString out; - - MutableColumns columns = header.cloneEmptyColumns(); - - /// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer. - size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset(); - if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row) - { - out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n"; - return out.str(); - } - - size_t max_length_of_column_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).name.size() > max_length_of_column_name) - max_length_of_column_name = header.safeGetByPosition(i).name.size(); - - size_t max_length_of_data_type_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name) - max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size(); - - /// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information. - - if (pos_of_prev_row) - { - istr.position() = pos_of_prev_row; - - out << "\nRow " << (row_num - 1) << ":\n"; - if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name)) - return out.str(); - } - else - { - if (!pos_of_current_row) - { - out << "Could not print diagnostic info because parsing of data hasn't started.\n"; - return out.str(); - } - - istr.position() = pos_of_current_row; - } - - out << "\nRow " << row_num << ":\n"; - parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name); - out << "\n"; - - return out.str(); -} - - -/** gcc-7 generates wrong code with optimization level greater than 1. - * See tests: dbms/src/IO/tests/write_int.cpp - * and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh - * This is compiler bug. The bug does not present in gcc-8 and clang-8. - * Nevertheless, we don't need high optimization of this function. - */ -bool OPTIMIZE(1) CSVRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, - WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name) -{ - const char delimiter = format_settings.csv.delimiter; - - for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) - { - if (file_column == 0 && istr.eof()) - { - out << "\n"; - return false; - } - - if (column_indexes_for_input_fields[file_column].has_value()) - { - const auto & table_column = *column_indexes_for_input_fields[file_column]; - const auto & current_column_type = data_types[table_column]; - const bool is_last_file_column = - file_column + 1 == column_indexes_for_input_fields.size(); - const bool at_delimiter = !istr.eof() && *istr.position() == delimiter; - const bool at_last_column_line_end = is_last_file_column - && (istr.eof() || *istr.position() == '\n' || *istr.position() == '\r'); - - out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ') - << "name: " << header.safeGetByPosition(table_column).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(table_column).name.size(), ' ') - << "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' '); - - if (format_settings.csv.empty_as_default - && (at_delimiter || at_last_column_line_end)) - { - columns[table_column]->insertDefault(); - } - else - { - BufferBase::Position prev_position = istr.position(); - BufferBase::Position curr_position = istr.position(); - std::exception_ptr exception; - - try - { - skipWhitespacesAndTabs(istr); - prev_position = istr.position(); - readField(*columns[table_column], current_column_type, is_last_file_column, table_column); - curr_position = istr.position(); - skipWhitespacesAndTabs(istr); - } - catch (...) - { - exception = std::current_exception(); - } - - if (curr_position < prev_position) - throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); - - if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type)) - { - /// An empty string instead of a value. - if (curr_position == prev_position) - { - out << "ERROR: text "; - verbosePrintString(prev_position, std::min(prev_position + 10, istr.buffer().end()), out); - out << " is not like " << current_column_type->getName() << "\n"; - return false; - } - } - - out << "parsed text: "; - verbosePrintString(prev_position, curr_position, out); - - if (exception) - { - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - else - out << "ERROR\n"; - return false; - } - - out << "\n"; - - if (current_column_type->haveMaximumSizeOfValue() - && *curr_position != '\n' && *curr_position != '\r' - && *curr_position != delimiter) - { - out << "ERROR: garbage after " << current_column_type->getName() << ": "; - verbosePrintString(curr_position, std::min(curr_position + 10, istr.buffer().end()), out); - out << "\n"; - - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - - return false; - } - } - } - else - { - static const String skipped_column_str = ""; - out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ') - << "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ') - << "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' '); - - String tmp; - readCSVString(tmp, istr, format_settings.csv); - } - - /// Delimiters - if (file_column + 1 == column_indexes_for_input_fields.size()) - { - if (istr.eof()) - return false; - - /// we support the extra delimiter at the end of the line - if (*istr.position() == delimiter) - { - ++istr.position(); - if (istr.eof()) - break; - } - - if (!istr.eof() && *istr.position() != '\n' && *istr.position() != '\r') - { - out << "ERROR: There is no line feed. "; - verbosePrintString(istr.position(), istr.position() + 1, out); - out << " found instead.\n" - " It's like your file has more columns than expected.\n" - "And if your file have right number of columns, maybe it have unquoted string value with comma.\n"; - - return false; - } - - skipEndOfLine(istr); - } - else - { - try - { - assertChar(delimiter, istr); - } - catch (const DB::Exception &) - { - if (*istr.position() == '\n' || *istr.position() == '\r') - { - out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected." - " It's like your file has less columns than expected.\n" - "And if your file have right number of columns, maybe it have unescaped quotes in values.\n"; - } - else - { - out << "ERROR: There is no delimiter (" << delimiter << "). "; - verbosePrintString(istr.position(), istr.position() + 1, out); - out << " found instead.\n"; - } - return false; - } - } - } - - return true; -} - - -void CSVRowInputStream::syncAfterError() -{ - skipToNextLineOrEOF(istr); -} - -void CSVRowInputStream::updateDiagnosticInfo() -{ - ++row_num; - - bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row; - bytes_read_at_start_of_buffer_on_current_row = istr.count() - istr.offset(); - - pos_of_prev_row = pos_of_current_row; - pos_of_current_row = istr.position(); -} - -bool CSVRowInputStream::readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx) -{ - const bool at_delimiter = !istr.eof() || *istr.position() == format_settings.csv.delimiter; - const bool at_last_column_line_end = is_last_file_column - && (istr.eof() || *istr.position() == '\n' || *istr.position() == '\r'); - - if (format_settings.csv.empty_as_default - && (at_delimiter || at_last_column_line_end)) - { - /// Treat empty unquoted column value as default value, if - /// specified in the settings. Tuple columns might seem - /// problematic, because they are never quoted but still contain - /// commas, which might be also used as delimiters. However, - /// they do not contain empty unquoted fields, so this check - /// works for tuples as well. - return false; - } - else if (column_idx_to_nullable_column_idx[column_idx]) - { - /// If value is null but type is not nullable then use default value instead. - const size_t nullable_idx = *column_idx_to_nullable_column_idx[column_idx]; - auto & tmp_col = *nullable_columns[nullable_idx]; - nullable_types[nullable_idx]->deserializeAsTextCSV(tmp_col, istr, format_settings); - Field value = tmp_col[0]; - tmp_col.popBack(1); /// do not store copy of values in memory - if (value.isNull()) - return false; - column.insert(value); - return true; - } - else - { - /// Read the column normally. - type->deserializeAsTextCSV(column, istr, format_settings); - return true; - } -} - - -void registerInputFormatCSV(FormatFactory & factory) -{ - for (bool with_names : {false, true}) - { - factory.registerInputFormat(with_names ? "CSVWithNames" : "CSV", [=]( - ReadBuffer & buf, - const Block & sample, - const Context &, - UInt64 max_block_size, - UInt64 rows_portion_size, - FormatFactory::ReadCallback callback, - const FormatSettings & settings) - { - return std::make_shared( - std::make_shared(buf, sample, with_names, settings), - sample, max_block_size, rows_portion_size, callback, settings); - }); - } -} - -} diff --git a/dbms/src/Formats/CSVRowInputStream.h b/dbms/src/Formats/CSVRowInputStream.h deleted file mode 100644 index b398858ee78..00000000000 --- a/dbms/src/Formats/CSVRowInputStream.h +++ /dev/null @@ -1,83 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include - - -namespace DB -{ - -class ReadBuffer; - -/** A stream for inputting data in csv format. - * Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values. - */ -class CSVRowInputStream : public IRowInputStream -{ -public: - /** with_names - in the first line the header with column names - */ - CSVRowInputStream(ReadBuffer & istr_, const Block & header_, bool with_names_, const FormatSettings & format_settings_); - - bool read(MutableColumns & columns, RowReadExtension & ext) override; - void readPrefix() override; - bool allowSyncAfterError() const override { return true; } - void syncAfterError() override; - - std::string getDiagnosticInfo() override; - -private: - ReadBuffer & istr; - Block header; - bool with_names; - DataTypes data_types; - - const FormatSettings format_settings; - - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; - - /// Maps indexes of columns in the input file to indexes of table columns - using OptionalIndexes = std::vector>; - OptionalIndexes column_indexes_for_input_fields; - - /// Tracks which colums we have read in a single read() call. - /// For columns that are never read, it is initialized to false when we - /// read the file header, and never changed afterwards. - /// For other columns, it is updated on each read() call. - std::vector read_columns; - - /// Whether we have any columns that are not read from file at all, - /// and must be always initialized with defaults. - bool have_always_default_columns = false; - - void addInputColumn(const String & column_name); - - /// For convenient diagnostics in case of an error. - size_t row_num = 0; - - /// How many bytes were read, not counting those that are still in the buffer. - size_t bytes_read_at_start_of_buffer_on_current_row = 0; - size_t bytes_read_at_start_of_buffer_on_prev_row = 0; - - char * pos_of_current_row = nullptr; - char * pos_of_prev_row = nullptr; - - /// For setting input_format_null_as_default - DataTypes nullable_types; - MutableColumns nullable_columns; - OptionalIndexes column_idx_to_nullable_column_idx; - - void updateDiagnosticInfo(); - - bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, - WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name); - - bool readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx); -}; - -} diff --git a/dbms/src/Formats/FormatFactory.cpp b/dbms/src/Formats/FormatFactory.cpp index 491363e01b4..8a1ad3d7bd2 100644 --- a/dbms/src/Formats/FormatFactory.cpp +++ b/dbms/src/Formats/FormatFactory.cpp @@ -47,6 +47,9 @@ static FormatSettings getInputFormatSetting(const Settings & settings) format_settings.date_time_input_format = settings.date_time_input_format; format_settings.input_allow_errors_num = settings.input_format_allow_errors_num; format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio; + format_settings.template_settings.format = settings.format_schema; + format_settings.template_settings.row_format = settings.format_schema_rows; + format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter; return format_settings; } @@ -63,6 +66,9 @@ static FormatSettings getOutputFormatSetting(const Settings & settings) format_settings.pretty.max_rows = settings.output_format_pretty_max_rows; format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width; format_settings.pretty.color = settings.output_format_pretty_color; + format_settings.template_settings.format = settings.format_schema; + format_settings.template_settings.row_format = settings.format_schema_rows; + format_settings.template_settings.row_between_delimiter = settings.format_schema_rows_between_delimiter; format_settings.write_statistics = settings.output_format_write_statistics; format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size; @@ -220,8 +226,6 @@ void FormatFactory::registerOutputFormatProcessor(const String & name, OutputPro void registerInputFormatNative(FormatFactory & factory); void registerOutputFormatNative(FormatFactory & factory); -void registerInputFormatTabSeparated(FormatFactory & factory); -void registerInputFormatCSV(FormatFactory & factory); void registerInputFormatProcessorNative(FormatFactory & factory); void registerOutputFormatProcessorNative(FormatFactory & factory); @@ -242,6 +246,8 @@ void registerInputFormatProcessorORC(FormatFactory & factory); void registerOutputFormatProcessorParquet(FormatFactory & factory); void registerInputFormatProcessorProtobuf(FormatFactory & factory); void registerOutputFormatProcessorProtobuf(FormatFactory & factory); +void registerInputFormatProcessorTemplate(FormatFactory & factory); +void registerOutputFormatProcessorTemplate(FormatFactory &factory); /// Output only (presentational) formats. @@ -267,8 +273,6 @@ FormatFactory::FormatFactory() { registerInputFormatNative(*this); registerOutputFormatNative(*this); - registerInputFormatTabSeparated(*this); - registerInputFormatCSV(*this); registerOutputFormatProcessorJSONEachRowWithProgress(*this); @@ -292,6 +296,8 @@ FormatFactory::FormatFactory() registerInputFormatProcessorORC(*this); registerInputFormatProcessorParquet(*this); registerOutputFormatProcessorParquet(*this); + registerInputFormatProcessorTemplate(*this); + registerOutputFormatProcessorTemplate(*this); registerOutputFormatNull(*this); diff --git a/dbms/src/Formats/FormatSettings.h b/dbms/src/Formats/FormatSettings.h index f4dd8e6cb8a..43cf7c8f5e7 100644 --- a/dbms/src/Formats/FormatSettings.h +++ b/dbms/src/Formats/FormatSettings.h @@ -50,6 +50,15 @@ struct FormatSettings Values values; + struct Template + { + String format; + String row_format; + String row_between_delimiter; + }; + + Template template_settings; + bool skip_unknown_fields = false; bool with_names_use_header = false; bool write_statistics = true; diff --git a/dbms/src/Formats/ParsedTemplateFormatString.cpp b/dbms/src/Formats/ParsedTemplateFormatString.cpp new file mode 100644 index 00000000000..f89b1756693 --- /dev/null +++ b/dbms/src/Formats/ParsedTemplateFormatString.cpp @@ -0,0 +1,217 @@ +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int INVALID_TEMPLATE_FORMAT; +} + +ParsedTemplateFormatString::ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name) +{ + try + { + parse(format_string, idx_by_name); + } + catch (DB::Exception & e) + { + if (e.code() != ErrorCodes::INVALID_TEMPLATE_FORMAT) + throwInvalidFormat(e.message(), columnsCount()); + else + throw; + } +} + + +void ParsedTemplateFormatString::parse(const String & format_string, const ColumnIdxGetter & idx_by_name) +{ + enum ParserState + { + Delimiter, + Column, + Format + }; + + const char * pos = format_string.c_str(); + const char * end = format_string.c_str() + format_string.size(); + const char * token_begin = pos; + ParserState state = Delimiter; + delimiters.emplace_back(); + for (; *pos; ++pos) + { + switch (state) + { + case Delimiter: + if (*pos == '$') + { + delimiters.back().append(token_begin, pos - token_begin); + ++pos; + if (*pos == '{') + { + token_begin = pos + 1; + state = Column; + } + else if (*pos == '$') + { + token_begin = pos; + } + else + throwInvalidFormat("at pos " + std::to_string(pos - format_string.c_str()) + + ": expected '{' or '$' after '$'", columnsCount()); + } + break; + + case Column: + column_names.emplace_back(); + pos = readMayBeQuotedColumnNameInto(pos, end - pos, column_names.back()); + + if (*pos == ':') + state = Format; + else if (*pos == '}') + { + formats.push_back(ColumnFormat::None); + delimiters.emplace_back(); + state = Delimiter; + } + else + throwInvalidFormat("Expected ':' or '}' after column name: \"" + column_names.back() + "\"", columnsCount()); + + token_begin = pos + 1; + format_idx_to_column_idx.emplace_back(idx_by_name(column_names.back())); + break; + + case Format: + if (*pos == '}') + { + formats.push_back(stringToFormat(String(token_begin, pos - token_begin))); + token_begin = pos + 1; + delimiters.emplace_back(); + state = Delimiter; + } + } + } + if (state != Delimiter) + throwInvalidFormat("Unbalanced parentheses", columnsCount()); + delimiters.back().append(token_begin, pos - token_begin); +} + + +ParsedTemplateFormatString::ColumnFormat ParsedTemplateFormatString::stringToFormat(const String & col_format) const +{ + if (col_format.empty()) + return ColumnFormat::None; + else if (col_format == "None") + return ColumnFormat::None; + else if (col_format == "Escaped") + return ColumnFormat::Escaped; + else if (col_format == "Quoted") + return ColumnFormat::Quoted; + else if (col_format == "CSV") + return ColumnFormat::Csv; + else if (col_format == "JSON") + return ColumnFormat::Json; + else if (col_format == "XML") + return ColumnFormat::Xml; + else if (col_format == "Raw") + return ColumnFormat::Raw; + else + throwInvalidFormat("Unknown field format " + col_format, columnsCount()); +} + +size_t ParsedTemplateFormatString::columnsCount() const +{ + return format_idx_to_column_idx.size(); +} + +String ParsedTemplateFormatString::formatToString(ParsedTemplateFormatString::ColumnFormat format) +{ + switch (format) + { + case ColumnFormat::None: + return "None"; + case ColumnFormat::Escaped: + return "Escaped"; + case ColumnFormat::Quoted: + return "Quoted"; + case ColumnFormat::Csv: + return "CSV"; + case ColumnFormat::Json: + return "Json"; + case ColumnFormat::Xml: + return "Xml"; + case ColumnFormat::Raw: + return "Raw"; + } + __builtin_unreachable(); +} + +const char * ParsedTemplateFormatString::readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s) +{ + s.clear(); + if (!size) + return pos; + ReadBufferFromMemory buf{pos, size}; + if (*pos == '"') + readDoubleQuotedStringWithSQLStyle(s, buf); + else if (*pos == '`') + readBackQuotedStringWithSQLStyle(s, buf); + else if (isWordCharASCII(*pos)) + { + size_t name_size = 1; + while (name_size < size && isWordCharASCII(*(pos + name_size))) + ++name_size; + s = String{pos, name_size}; + return pos + name_size; + } + return pos + buf.count(); +} + +String ParsedTemplateFormatString::dump() const +{ + WriteBufferFromOwnString res; + res << "Delimiter " << 0 << ": "; + verbosePrintString(delimiters.front().c_str(), delimiters.front().c_str() + delimiters.front().size(), res); + + size_t num_columns = std::max(formats.size(), format_idx_to_column_idx.size()); + for (size_t i = 0; i < num_columns; ++i) + { + res << "\nColumn " << i << ": \""; + if (column_names.size() <= i) + res << ""; + else if (column_names[i].empty()) + res << ""; + else + res << column_names[i]; + + res << "\" (mapped to table column "; + if (format_idx_to_column_idx.size() <= i) + res << ""; + else if (!format_idx_to_column_idx[i]) + res << ""; + else + res << *format_idx_to_column_idx[i]; + + res << "), Format " << (i < formats.size() ? formatToString(formats[i]) : ""); + + res << "\nDelimiter " << i + 1 << ": "; + if (delimiters.size() <= i + 1) + res << ""; + else + verbosePrintString(delimiters[i + 1].c_str(), delimiters[i + 1].c_str() + delimiters[i + 1].size(), res); + } + + return res.str(); +} + +void ParsedTemplateFormatString::throwInvalidFormat(const String & message, size_t column) const +{ + throw Exception("Invalid format string for Template: " + message + " (near column " + std::to_string(column) + + ")" + ". Parsed format string:\n" + dump() + "\n", + ErrorCodes::INVALID_TEMPLATE_FORMAT); +} + +} diff --git a/dbms/src/Formats/ParsedTemplateFormatString.h b/dbms/src/Formats/ParsedTemplateFormatString.h new file mode 100644 index 00000000000..5353f336f64 --- /dev/null +++ b/dbms/src/Formats/ParsedTemplateFormatString.h @@ -0,0 +1,51 @@ +#pragma once + +#include +#include +#include + +namespace DB +{ + +struct ParsedTemplateFormatString +{ + enum class ColumnFormat + { + None, + Escaped, + Quoted, + Csv, + Json, + Xml, + Raw + }; + + /// Format string has syntax: "Delimiter0 ${ColumnName0:Format0} Delimiter1 ${ColumnName1:Format1} Delimiter2" + /// The following vectors is filled with corresponding values, delimiters.size() - 1 = formats.size() = format_idx_to_column_idx.size() + /// If format_idx_to_column_idx[i] has no value, then TemplateRowInputFormat will skip i-th column. + + std::vector delimiters; + std::vector formats; + std::vector> format_idx_to_column_idx; + + /// For diagnostic info + Strings column_names; + + typedef std::function(const String &)> ColumnIdxGetter; + + ParsedTemplateFormatString() = default; + ParsedTemplateFormatString(const String & format_string, const ColumnIdxGetter & idx_by_name); + + void parse(const String & format_string, const ColumnIdxGetter & idx_by_name); + + ColumnFormat stringToFormat(const String & format) const; + static String formatToString(ColumnFormat format); + static const char * readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s); + size_t columnsCount() const; + + String dump() const; + [[noreturn]] void throwInvalidFormat(const String & message, size_t column) const; +}; + +} + diff --git a/dbms/src/Formats/TabSeparatedRowInputStream.cpp b/dbms/src/Formats/TabSeparatedRowInputStream.cpp deleted file mode 100644 index 69850dbc455..00000000000 --- a/dbms/src/Formats/TabSeparatedRowInputStream.cpp +++ /dev/null @@ -1,504 +0,0 @@ -#include - -#include - -#include -#include -#include - -#include -#include -#include -#include - - -namespace DB -{ - -namespace ErrorCodes -{ - extern const int INCORRECT_DATA; - extern const int LOGICAL_ERROR; -} - - -static void skipTSVRow(ReadBuffer & istr, const size_t num_columns) -{ - NullSink null_sink; - - for (size_t i = 0; i < num_columns; ++i) - { - readEscapedStringInto(null_sink, istr); - assertChar(i == num_columns - 1 ? '\n' : '\t', istr); - } -} - - -/** Check for a common error case - usage of Windows line feed. - */ -static void checkForCarriageReturn(ReadBuffer & istr) -{ - if (istr.position()[0] == '\r' || (istr.position() != istr.buffer().begin() && istr.position()[-1] == '\r')) - throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row." - "\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format." - " You must transform your file to Unix format." - "\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\r.", - ErrorCodes::INCORRECT_DATA); -} - - -TabSeparatedRowInputStream::TabSeparatedRowInputStream( - ReadBuffer & istr_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_) - : istr(istr_), header(header_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) -{ - const auto num_columns = header.columns(); - - data_types.resize(num_columns); - column_indexes_by_names.reserve(num_columns); - - for (size_t i = 0; i < num_columns; ++i) - { - const auto & column_info = header.getByPosition(i); - - data_types[i] = column_info.type; - column_indexes_by_names.emplace(column_info.name, i); - } - - column_indexes_for_input_fields.reserve(num_columns); - read_columns.assign(num_columns, false); -} - - -void TabSeparatedRowInputStream::setupAllColumnsByTableSchema() -{ - read_columns.assign(header.columns(), true); - column_indexes_for_input_fields.resize(header.columns()); - - for (size_t i = 0; i < column_indexes_for_input_fields.size(); ++i) - column_indexes_for_input_fields[i] = i; -} - - -void TabSeparatedRowInputStream::addInputColumn(const String & column_name) -{ - const auto column_it = column_indexes_by_names.find(column_name); - if (column_it == column_indexes_by_names.end()) - { - if (format_settings.skip_unknown_fields) - { - column_indexes_for_input_fields.push_back(std::nullopt); - return; - } - - throw Exception( - "Unknown field found in TSV header: '" + column_name + "' " + - "at position " + std::to_string(column_indexes_for_input_fields.size()) + - "\nSet the 'input_format_skip_unknown_fields' parameter explicitly to ignore and proceed", - ErrorCodes::INCORRECT_DATA - ); - } - - const auto column_index = column_it->second; - - if (read_columns[column_index]) - throw Exception("Duplicate field found while parsing TSV header: " + column_name, ErrorCodes::INCORRECT_DATA); - - read_columns[column_index] = true; - column_indexes_for_input_fields.emplace_back(column_index); -} - - -void TabSeparatedRowInputStream::fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension & row_read_extension) -{ - /// It is safe to memorize this on the first run - the format guarantees this does not change - if (unlikely(row_num == 1)) - { - columns_to_fill_with_default_values.clear(); - for (size_t index = 0; index < read_columns.size(); ++index) - if (read_columns[index] == 0) - columns_to_fill_with_default_values.push_back(index); - } - - for (const auto column_index : columns_to_fill_with_default_values) - data_types[column_index]->insertDefaultInto(*columns[column_index]); - - row_read_extension.read_columns = read_columns; -} - - -void TabSeparatedRowInputStream::readPrefix() -{ - if (with_names || with_types) - { - /// In this format, we assume that column name or type cannot contain BOM, - /// so, if format has header, - /// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it. - skipBOMIfExists(istr); - } - - if (with_names) - { - if (format_settings.with_names_use_header) - { - String column_name; - do - { - readEscapedString(column_name, istr); - addInputColumn(column_name); - } - while (checkChar('\t', istr)); - - if (!istr.eof()) - { - checkForCarriageReturn(istr); - assertChar('\n', istr); - } - } - else - { - setupAllColumnsByTableSchema(); - skipTSVRow(istr, column_indexes_for_input_fields.size()); - } - } - else - setupAllColumnsByTableSchema(); - - if (with_types) - { - skipTSVRow(istr, column_indexes_for_input_fields.size()); - } -} - - -bool TabSeparatedRowInputStream::read(MutableColumns & columns, RowReadExtension & ext) -{ - if (istr.eof()) - return false; - - updateDiagnosticInfo(); - - for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position) - { - const auto & column_index = column_indexes_for_input_fields[input_position]; - if (column_index) - { - data_types[*column_index]->deserializeAsTextEscaped(*columns[*column_index], istr, format_settings); - } - else - { - NullSink null_sink; - readEscapedStringInto(null_sink, istr); - } - - /// skip separators - if (input_position + 1 < column_indexes_for_input_fields.size()) - { - assertChar('\t', istr); - } - else if (!istr.eof()) - { - if (unlikely(row_num == 1)) - checkForCarriageReturn(istr); - - assertChar('\n', istr); - } - } - - fillUnreadColumnsWithDefaults(columns, ext); - - return true; -} - - -String TabSeparatedRowInputStream::getDiagnosticInfo() -{ - if (istr.eof()) /// Buffer has gone, cannot extract information about what has been parsed. - return {}; - - WriteBufferFromOwnString out; - MutableColumns columns = header.cloneEmptyColumns(); - - /// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer. - size_t bytes_read_at_start_of_buffer = istr.count() - istr.offset(); - if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row) - { - out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n"; - return out.str(); - } - - size_t max_length_of_column_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).name.size() > max_length_of_column_name) - max_length_of_column_name = header.safeGetByPosition(i).name.size(); - - size_t max_length_of_data_type_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name) - max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size(); - - /// Roll back the cursor to the beginning of the previous or current line and parse all over again. But now we derive detailed information. - - if (pos_of_prev_row) - { - istr.position() = pos_of_prev_row; - - out << "\nRow " << (row_num - 1) << ":\n"; - if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name)) - return out.str(); - } - else - { - if (!pos_of_current_row) - { - out << "Could not print diagnostic info because parsing of data hasn't started.\n"; - return out.str(); - } - - istr.position() = pos_of_current_row; - } - - out << "\nRow " << row_num << ":\n"; - parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name); - out << "\n"; - - return out.str(); -} - - -/** gcc-7 generates wrong code with optimization level greater than 1. - * See tests: dbms/src/IO/tests/write_int.cpp - * and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh - * This is compiler bug. The bug does not present in gcc-8 and clang-8. - * Nevertheless, we don't need high optimization of this function. - */ -bool OPTIMIZE(1) TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo( - MutableColumns & columns, WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name) -{ - for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position) - { - if (input_position == 0 && istr.eof()) - { - out << "\n"; - return false; - } - - if (column_indexes_for_input_fields[input_position].has_value()) - { - const auto & column_index = *column_indexes_for_input_fields[input_position]; - const auto & current_column_type = data_types[column_index]; - - out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ') - << "name: " << header.safeGetByPosition(column_index).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(column_index).name.size(), ' ') - << "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' '); - - auto prev_position = istr.position(); - std::exception_ptr exception; - - try - { - current_column_type->deserializeAsTextEscaped(*columns[column_index], istr, format_settings); - } - catch (...) - { - exception = std::current_exception(); - } - - auto curr_position = istr.position(); - - if (curr_position < prev_position) - throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); - - if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type)) - { - /// An empty string instead of a value. - if (curr_position == prev_position) - { - out << "ERROR: text "; - verbosePrintString(prev_position, std::min(prev_position + 10, istr.buffer().end()), out); - out << " is not like " << current_column_type->getName() << "\n"; - return false; - } - } - - out << "parsed text: "; - verbosePrintString(prev_position, curr_position, out); - - if (exception) - { - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - else - out << "ERROR\n"; - return false; - } - - out << "\n"; - - if (current_column_type->haveMaximumSizeOfValue()) - { - if (*curr_position != '\n' && *curr_position != '\t') - { - out << "ERROR: garbage after " << current_column_type->getName() << ": "; - verbosePrintString(curr_position, std::min(curr_position + 10, istr.buffer().end()), out); - out << "\n"; - - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - - return false; - } - } - } - else - { - static const String skipped_column_str = ""; - out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ') - << "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ') - << "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' '); - - NullSink null_sink; - readEscapedStringInto(null_sink, istr); - } - - /// Delimiters - if (input_position + 1 == column_indexes_for_input_fields.size()) - { - if (!istr.eof()) - { - try - { - assertChar('\n', istr); - } - catch (const DB::Exception &) - { - if (*istr.position() == '\t') - { - out << "ERROR: Tab found where line feed is expected." - " It's like your file has more columns than expected.\n" - "And if your file have right number of columns, maybe it have unescaped tab in value.\n"; - } - else if (*istr.position() == '\r') - { - out << "ERROR: Carriage return found where line feed is expected." - " It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n"; - } - else - { - out << "ERROR: There is no line feed. "; - verbosePrintString(istr.position(), istr.position() + 1, out); - out << " found instead.\n"; - } - return false; - } - } - } - else - { - try - { - assertChar('\t', istr); - } - catch (const DB::Exception &) - { - if (*istr.position() == '\n') - { - out << "ERROR: Line feed found where tab is expected." - " It's like your file has less columns than expected.\n" - "And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n"; - } - else if (*istr.position() == '\r') - { - out << "ERROR: Carriage return found where tab is expected.\n"; - } - else - { - out << "ERROR: There is no tab. "; - verbosePrintString(istr.position(), istr.position() + 1, out); - out << " found instead.\n"; - } - return false; - } - } - } - - return true; -} - - -void TabSeparatedRowInputStream::syncAfterError() -{ - skipToUnescapedNextLineOrEOF(istr); -} - - -void TabSeparatedRowInputStream::updateDiagnosticInfo() -{ - ++row_num; - - bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row; - bytes_read_at_start_of_buffer_on_current_row = istr.count() - istr.offset(); - - pos_of_prev_row = pos_of_current_row; - pos_of_current_row = istr.position(); -} - - -void registerInputFormatTabSeparated(FormatFactory & factory) -{ - for (auto name : {"TabSeparated", "TSV"}) - { - factory.registerInputFormat(name, []( - ReadBuffer & buf, - const Block & sample, - const Context &, - UInt64 max_block_size, - UInt64 rows_portion_size, - FormatFactory::ReadCallback callback, - const FormatSettings & settings) - { - return std::make_shared( - std::make_shared(buf, sample, false, false, settings), - sample, max_block_size, rows_portion_size, callback, settings); - }); - } - - for (auto name : {"TabSeparatedWithNames", "TSVWithNames"}) - { - factory.registerInputFormat(name, []( - ReadBuffer & buf, - const Block & sample, - const Context &, - UInt64 max_block_size, - UInt64 rows_portion_size, - FormatFactory::ReadCallback callback, - const FormatSettings & settings) - { - return std::make_shared( - std::make_shared(buf, sample, true, false, settings), - sample, max_block_size, rows_portion_size, callback, settings); - }); - } - - for (auto name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"}) - { - factory.registerInputFormat(name, []( - ReadBuffer & buf, - const Block & sample, - const Context &, - UInt64 max_block_size, - UInt64 rows_portion_size, - FormatFactory::ReadCallback callback, - const FormatSettings & settings) - { - return std::make_shared( - std::make_shared(buf, sample, true, true, settings), - sample, max_block_size, rows_portion_size, callback, settings); - }); - } -} - -} diff --git a/dbms/src/Formats/TabSeparatedRowInputStream.h b/dbms/src/Formats/TabSeparatedRowInputStream.h deleted file mode 100644 index f8ebebbdfe4..00000000000 --- a/dbms/src/Formats/TabSeparatedRowInputStream.h +++ /dev/null @@ -1,73 +0,0 @@ -#pragma once - -#include -#include - -#include -#include -#include - - -namespace DB -{ - -class ReadBuffer; - - -/** A stream to input data in tsv format. - */ -class TabSeparatedRowInputStream : public IRowInputStream -{ -public: - /** with_names - the first line is the header with the names of the columns - * with_types - on the next line header with type names - */ - TabSeparatedRowInputStream( - ReadBuffer & istr_, const Block & header_, bool with_names_, bool with_types_, const FormatSettings & format_settings_); - - bool read(MutableColumns & columns, RowReadExtension & ext) override; - void readPrefix() override; - bool allowSyncAfterError() const override { return true; } - void syncAfterError() override; - - std::string getDiagnosticInfo() override; - -private: - ReadBuffer & istr; - Block header; - bool with_names; - bool with_types; - const FormatSettings format_settings; - DataTypes data_types; - - using IndexesMap = std::unordered_map; - IndexesMap column_indexes_by_names; - - using OptionalIndexes = std::vector>; - OptionalIndexes column_indexes_for_input_fields; - - std::vector read_columns; - std::vector columns_to_fill_with_default_values; - - void addInputColumn(const String & column_name); - void setupAllColumnsByTableSchema(); - void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension& ext); - - /// For convenient diagnostics in case of an error. - - size_t row_num = 0; - - /// How many bytes were read, not counting those still in the buffer. - size_t bytes_read_at_start_of_buffer_on_current_row = 0; - size_t bytes_read_at_start_of_buffer_on_prev_row = 0; - - char * pos_of_current_row = nullptr; - char * pos_of_prev_row = nullptr; - - void updateDiagnosticInfo(); - - bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, - WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name); -}; - -} diff --git a/dbms/src/Formats/tests/CMakeLists.txt b/dbms/src/Formats/tests/CMakeLists.txt index e12fa0f02fb..187700dff72 100644 --- a/dbms/src/Formats/tests/CMakeLists.txt +++ b/dbms/src/Formats/tests/CMakeLists.txt @@ -2,6 +2,3 @@ set(SRCS ) add_executable (tab_separated_streams tab_separated_streams.cpp ${SRCS}) target_link_libraries (tab_separated_streams PRIVATE dbms) - -add_executable (block_row_transforms block_row_transforms.cpp ${SRCS}) -target_link_libraries (block_row_transforms PRIVATE dbms) diff --git a/dbms/src/Formats/tests/block_row_transforms.cpp b/dbms/src/Formats/tests/block_row_transforms.cpp deleted file mode 100644 index 9edc520d85f..00000000000 --- a/dbms/src/Formats/tests/block_row_transforms.cpp +++ /dev/null @@ -1,57 +0,0 @@ -#include - -#include -#include - -#include -#include - -#include -#include - -#include -#include - -#include -#include - -#include -#include -#include - - -int main(int, char **) -try -{ - using namespace DB; - - Block sample; - - ColumnWithTypeAndName col1; - col1.name = "col1"; - col1.type = std::make_shared(); - col1.column = col1.type->createColumn(); - sample.insert(col1); - - ColumnWithTypeAndName col2; - col2.name = "col2"; - col2.type = std::make_shared(); - col2.column = col2.type->createColumn(); - sample.insert(col2); - - ReadBufferFromFile in_buf("test_in"); - WriteBufferFromFile out_buf("test_out"); - - FormatSettings format_settings; - - RowInputStreamPtr row_input = std::make_shared(in_buf, sample, false, false, format_settings); - BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings); - BlockOutputStreamPtr block_output = std::make_shared(std::make_shared(out_buf, sample, false, false, []{}, format_settings)); - - copyData(block_input, *block_output); -} -catch (const DB::Exception & e) -{ - std::cerr << e.what() << ", " << e.displayText() << std::endl; - return 1; -} diff --git a/dbms/src/Formats/tests/tab_separated_streams.cpp b/dbms/src/Formats/tests/tab_separated_streams.cpp index 2c44ccf2b43..f05a83bc751 100644 --- a/dbms/src/Formats/tests/tab_separated_streams.cpp +++ b/dbms/src/Formats/tests/tab_separated_streams.cpp @@ -9,12 +9,12 @@ #include #include -#include -#include +#include #include #include #include +#include using namespace DB; @@ -39,13 +39,15 @@ try FormatSettings format_settings; - RowInputStreamPtr row_input = std::make_shared(in_buf, sample, false, false, format_settings); - BlockInputStreamFromRowInputStream block_input(row_input, sample, DEFAULT_INSERT_BLOCK_SIZE, 0, []{}, format_settings); + RowInputFormatParams params{DEFAULT_INSERT_BLOCK_SIZE, 0, 0, 0, []{}}; + + InputFormatPtr input_format = std::make_shared(sample, in_buf, params, false, false, format_settings); + BlockInputStreamPtr block_input = std::make_shared(std::move(input_format)); BlockOutputStreamPtr block_output = std::make_shared( std::make_shared(out_buf, sample, false, false, [] {}, format_settings)); - copyData(block_input, *block_output); + copyData(*block_input, *block_output); return 0; } catch (...) diff --git a/dbms/src/Functions/GeoUtils.cpp b/dbms/src/Functions/GeoUtils.cpp index 847d934c6b4..b274f579aa3 100644 --- a/dbms/src/Functions/GeoUtils.cpp +++ b/dbms/src/Functions/GeoUtils.cpp @@ -332,7 +332,7 @@ UInt64 geohashesInBox(const GeohashesInBoxPreparedArgs & args, char * out) } } - if (items == 0 && args.items_count != 0) + if (items == 0) { size_t l = geohashEncodeImpl(args.longitude_min, args.latitude_min, args.precision, out); out += l; diff --git a/dbms/src/Functions/array/arrayEnumerateRanked.h b/dbms/src/Functions/array/arrayEnumerateRanked.h index ab46af2266d..a1019ba83bf 100644 --- a/dbms/src/Functions/array/arrayEnumerateRanked.h +++ b/dbms/src/Functions/array/arrayEnumerateRanked.h @@ -336,7 +336,9 @@ void FunctionArrayEnumerateRankedExtended::executeMethodImpl( /// Skipping offsets if no data in this array if (prev_off == off) { - want_clear = true; + + if (depth_to_look > 2) + want_clear = true; if (depth_to_look >= 2) { diff --git a/dbms/src/IO/PeekableReadBuffer.cpp b/dbms/src/IO/PeekableReadBuffer.cpp new file mode 100644 index 00000000000..07c815931b3 --- /dev/null +++ b/dbms/src/IO/PeekableReadBuffer.cpp @@ -0,0 +1,316 @@ +#include + +namespace DB +{ + +PeekableReadBuffer::PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ /*= DBMS_DEFAULT_BUFFER_SIZE*/, + size_t unread_limit_ /* = default_limit*/) + : BufferWithOwnMemory(start_size_), sub_buf(sub_buf_), unread_limit(unread_limit_) +{ + padded &= sub_buf.isPadded(); + /// Read from sub-buffer + Buffer & sub_working = sub_buf.buffer(); + BufferBase::set(sub_working.begin(), sub_working.size(), sub_buf.offset()); + + checkStateCorrect(); +} + +bool PeekableReadBuffer::peekNext() +{ + checkStateCorrect(); + + size_t bytes_read = 0; + Position copy_from = pos; + size_t bytes_to_copy = sub_buf.available(); + if (useSubbufferOnly()) + { + /// Don't have to copy all data from sub-buffer if there is no data in own memory (checkpoint and pos are in sub-buffer) + if (checkpoint) + copy_from = checkpoint; + bytes_read = copy_from - sub_buf.buffer().begin(); + bytes_to_copy = sub_buf.buffer().end() - copy_from; /// sub_buf.available(); + if (!bytes_to_copy) + { + bytes += bytes_read; + sub_buf.position() = copy_from; + + /// Both checkpoint and pos are at the end of sub-buffer. Just load next part of data. + bool res = sub_buf.next(); + BufferBase::set(sub_buf.buffer().begin(), sub_buf.buffer().size(), sub_buf.offset()); + if (checkpoint) + checkpoint = pos; + + checkStateCorrect(); + return res; + } + } + + /// May throw an exception + resizeOwnMemoryIfNecessary(bytes_to_copy); + + if (useSubbufferOnly()) + { + bytes += bytes_read; + sub_buf.position() = copy_from; + } + + /// Save unread data from sub-buffer to own memory + memcpy(memory.data() + peeked_size, sub_buf.position(), bytes_to_copy); + + /// If useSubbufferOnly() is false, then checkpoint is in own memory and it was updated in resizeOwnMemoryIfNecessary + /// Otherwise, checkpoint now at the beginning of own memory + if (checkpoint && useSubbufferOnly()) + { + checkpoint = memory.data(); + checkpoint_in_own_memory = true; + } + if (currentlyReadFromOwnMemory()) + { + /// Update buffer size + BufferBase::set(memory.data(), peeked_size + bytes_to_copy, offset()); + } + else + { + /// Switch to reading from own memory + size_t pos_offset = peeked_size + this->offset(); + if (useSubbufferOnly()) + { + if (checkpoint) + pos_offset = bytes_to_copy; + else + pos_offset = 0; + } + BufferBase::set(memory.data(), peeked_size + bytes_to_copy, pos_offset); + + } + + peeked_size += bytes_to_copy; + sub_buf.position() += bytes_to_copy; + + checkStateCorrect(); + return sub_buf.next(); +} + +void PeekableReadBuffer::setCheckpoint() +{ + checkStateCorrect(); +#ifndef NDEBUG + if (checkpoint) + throw DB::Exception("Does not support recursive checkpoints.", ErrorCodes::LOGICAL_ERROR); +#endif + checkpoint_in_own_memory = currentlyReadFromOwnMemory(); + if (!checkpoint_in_own_memory) + { + /// Don't need to store unread data anymore + peeked_size = 0; + } + checkpoint = pos; + checkStateCorrect(); +} + +void PeekableReadBuffer::dropCheckpoint() +{ + checkStateCorrect(); +#ifndef NDEBUG + if (!checkpoint) + throw DB::Exception("There is no checkpoint", ErrorCodes::LOGICAL_ERROR); +#endif + if (!currentlyReadFromOwnMemory()) + { + /// Don't need to store unread data anymore + peeked_size = 0; + } + checkpoint = nullptr; + checkpoint_in_own_memory = false; + checkStateCorrect(); +} + +void PeekableReadBuffer::rollbackToCheckpoint() +{ + checkStateCorrect(); + if (!checkpoint) + throw DB::Exception("There is no checkpoint", ErrorCodes::LOGICAL_ERROR); + else if (checkpointInOwnMemory() == currentlyReadFromOwnMemory()) + pos = checkpoint; + else /// Checkpoint is in own memory and pos is not. Switch to reading from own memory + BufferBase::set(memory.data(), peeked_size, checkpoint - memory.data()); + checkStateCorrect(); +} + +bool PeekableReadBuffer::nextImpl() +{ + /// FIXME wrong bytes count because it can read the same data again after rollbackToCheckpoint() + /// However, changing bytes count on every call of next() (even after rollback) allows to determine if some pointers were invalidated. + checkStateCorrect(); + bool res; + + if (!checkpoint) + { + if (!useSubbufferOnly()) + { + /// All copied data have been read from own memory, continue reading from sub_buf + peeked_size = 0; + res = sub_buf.hasPendingData() || sub_buf.next(); + } + else + { + /// Load next data to sub_buf + sub_buf.position() = pos; + res = sub_buf.next(); + } + + Buffer & sub_working = sub_buf.buffer(); + /// Switch to reading from sub_buf (or just update it if already switched) + BufferBase::set(sub_working.begin(), sub_working.size(), 0); + } + else + { + if (currentlyReadFromOwnMemory()) + res = sub_buf.hasPendingData() || sub_buf.next(); + else + res = peekNext(); + Buffer & sub_working = sub_buf.buffer(); + BufferBase::set(sub_working.begin(), sub_working.size(), 0); + } + + checkStateCorrect(); + return res; +} + +bool PeekableReadBuffer::useSubbufferOnly() const +{ + return !peeked_size; +} + +void PeekableReadBuffer::checkStateCorrect() const +{ +#ifndef NDEBUG + if (checkpoint) + { + if (checkpointInOwnMemory()) + { + if (!peeked_size) + throw DB::Exception("Checkpoint in empty own buffer", ErrorCodes::LOGICAL_ERROR); + if (currentlyReadFromOwnMemory() && pos < checkpoint) + throw DB::Exception("Current position in own buffer before checkpoint in own buffer", ErrorCodes::LOGICAL_ERROR); + } + else + { + if (peeked_size) + throw DB::Exception("Own buffer is not empty", ErrorCodes::LOGICAL_ERROR); + if (currentlyReadFromOwnMemory()) + throw DB::Exception("Current position in own buffer before checkpoint in subbuffer", ErrorCodes::LOGICAL_ERROR); + if (pos < checkpoint) + throw DB::Exception("Current position in subbuffer before checkpoint in subbuffer", ErrorCodes::LOGICAL_ERROR); + } + } + else + { + if (!currentlyReadFromOwnMemory() && peeked_size) + throw DB::Exception("Own buffer is not empty", ErrorCodes::LOGICAL_ERROR); + } + if (currentlyReadFromOwnMemory() && !peeked_size) + throw DB::Exception("Pos in empty own buffer", ErrorCodes::LOGICAL_ERROR); + if (unread_limit < memory.size()) + throw DB::Exception("Size limit exceed", ErrorCodes::LOGICAL_ERROR); +#endif +} + +size_t PeekableReadBuffer::resizeOwnMemoryIfNecessary(size_t bytes_to_append) +{ + checkStateCorrect(); + bool needUpdateCheckpoint = checkpointInOwnMemory(); + bool needUpdatePos = currentlyReadFromOwnMemory(); + size_t offset = 0; + if (needUpdateCheckpoint) + offset = checkpoint - memory.data(); + else if (needUpdatePos) + offset = this->offset(); + + size_t new_size = peeked_size + bytes_to_append; + if (memory.size() < new_size) + { + if (bytes_to_append < offset && 2 * (peeked_size - offset) <= memory.size()) + { + /// Move unread data to the beginning of own memory instead of resize own memory + peeked_size -= offset; + memmove(memory.data(), memory.data() + offset, peeked_size); + bytes += offset; + + if (needUpdateCheckpoint) + checkpoint -= offset; + if (needUpdatePos) + pos -= offset; + + checkStateCorrect(); + return 0; + } + else + { + if (unread_limit < new_size) + throw DB::Exception("PeekableReadBuffer: Memory limit exceed", ErrorCodes::MEMORY_LIMIT_EXCEEDED); + + size_t pos_offset = pos - memory.data(); + + size_t new_size_amortized = memory.size() * 2; + if (new_size_amortized < new_size) + new_size_amortized = new_size; + else if (unread_limit < new_size_amortized) + new_size_amortized = unread_limit; + memory.resize(new_size_amortized); + + if (needUpdateCheckpoint) + checkpoint = memory.data() + offset; + if (needUpdatePos) + { + BufferBase::set(memory.data(), peeked_size, pos_offset); + } + } + } + + checkStateCorrect(); + return offset; +} + +PeekableReadBuffer::~PeekableReadBuffer() +{ + if (!currentlyReadFromOwnMemory()) + sub_buf.position() = pos; +} + +std::shared_ptr> PeekableReadBuffer::takeUnreadData() +{ + checkStateCorrect(); + if (!currentlyReadFromOwnMemory()) + return std::make_shared>(0); + size_t unread_size = memory.data() + peeked_size - pos; + auto unread = std::make_shared>(unread_size); + memcpy(unread->buffer().begin(), pos, unread_size); + unread->BufferBase::set(unread->buffer().begin(), unread_size, 0); + peeked_size = 0; + checkpoint = nullptr; + checkpoint_in_own_memory = false; + BufferBase::set(sub_buf.buffer().begin(), sub_buf.buffer().size(), sub_buf.offset()); + checkStateCorrect(); + return unread; +} + +bool PeekableReadBuffer::currentlyReadFromOwnMemory() const +{ + return working_buffer.begin() != sub_buf.buffer().begin(); +} + +bool PeekableReadBuffer::checkpointInOwnMemory() const +{ + return checkpoint_in_own_memory; +} + +void PeekableReadBuffer::assertCanBeDestructed() const +{ + if (peeked_size && pos != memory.data() + peeked_size) + throw DB::Exception("There are data, which were extracted from sub-buffer, but not from peekable buffer. " + "Cannot destruct peekable buffer correctly because tha data will be lost." + "Most likely it's a bug.", ErrorCodes::LOGICAL_ERROR); +} + +} diff --git a/dbms/src/IO/PeekableReadBuffer.h b/dbms/src/IO/PeekableReadBuffer.h new file mode 100644 index 00000000000..30a38b69e5c --- /dev/null +++ b/dbms/src/IO/PeekableReadBuffer.h @@ -0,0 +1,96 @@ +#pragma once +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int MEMORY_LIMIT_EXCEEDED; + extern const int LOGICAL_ERROR; +} + +/// Allows to peek next part of data from sub-buffer without extracting it. +/// Also allows to set checkpoint at some position in stream and come back to this position later, +/// even if next() was called. +/// Sub-buffer should not be accessed directly during the lifelime of peekable buffer. +/// If position() of peekable buffer is explicitly set to some position before checkpoint +/// (e.g. by istr.position() = prev_pos), behavior is undefined. +class PeekableReadBuffer : public BufferWithOwnMemory +{ + friend class PeekableReadBufferCheckpoint; +public: + explicit PeekableReadBuffer(ReadBuffer & sub_buf_, size_t start_size_ = DBMS_DEFAULT_BUFFER_SIZE, + size_t unread_limit_ = 16 * DBMS_DEFAULT_BUFFER_SIZE); + + /// Use takeUnreadData() to extract unread data before destruct object + ~PeekableReadBuffer() override; + + /// Saves unread data to own memory, so it will be possible to read it later. Loads next data to sub-buffer. + /// Doesn't change checkpoint and position in stream, + /// but all pointers (such as this->buffer().end() and this->position()) may be invalidated + /// @returns false in case of EOF in sub-buffer, otherwise returns true + bool peekNext(); + + Buffer & lastPeeked() { return sub_buf.buffer(); } + + /// Sets checkpoint at current position + void setCheckpoint(); + + /// Forget checkpoint and all data between checkpoint and position + void dropCheckpoint(); + + /// Sets position at checkpoint. + /// All pointers (such as this->buffer().end()) may be invalidated + void rollbackToCheckpoint(); + + /// If position is in own memory, returns buffer with data, which were extracted from sub-buffer, + /// but not from this buffer, so the data will not be lost after destruction of this buffer. + /// If position is in sub-buffer, returns empty buffer. + std::shared_ptr> takeUnreadData(); + void assertCanBeDestructed() const; + +private: + + bool nextImpl() override; + + inline bool useSubbufferOnly() const; + inline bool currentlyReadFromOwnMemory() const; + inline bool checkpointInOwnMemory() const; + + void checkStateCorrect() const; + + /// Makes possible to append `bytes_to_append` bytes to data in own memory. + /// Updates all invalidated pointers and sizes. + /// @returns new offset of unread data in own memory + size_t resizeOwnMemoryIfNecessary(size_t bytes_to_append); + + + ReadBuffer & sub_buf; + const size_t unread_limit; + size_t peeked_size = 0; + Position checkpoint = nullptr; + bool checkpoint_in_own_memory = false; +}; + + +class PeekableReadBufferCheckpoint : boost::noncopyable +{ + PeekableReadBuffer & buf; + bool auto_rollback; +public: + explicit PeekableReadBufferCheckpoint(PeekableReadBuffer & buf_, bool auto_rollback_ = false) + : buf(buf_), auto_rollback(auto_rollback_) { buf.setCheckpoint(); } + ~PeekableReadBufferCheckpoint() + { + if (!buf.checkpoint) + return; + if (auto_rollback) + buf.rollbackToCheckpoint(); + buf.dropCheckpoint(); + } + +}; + +} diff --git a/dbms/src/IO/tests/gtest_peekable_read_buffer.cpp b/dbms/src/IO/tests/gtest_peekable_read_buffer.cpp new file mode 100644 index 00000000000..331184e701c --- /dev/null +++ b/dbms/src/IO/tests/gtest_peekable_read_buffer.cpp @@ -0,0 +1,131 @@ +#include + +#include +#include +#include +#include +#include + +void readAndAssert(DB::ReadBuffer & buf, const char * str) +{ + size_t n = strlen(str); + char tmp[n]; + buf.readStrict(tmp, n); + ASSERT_EQ(strncmp(tmp, str, n), 0); +} + +void assertAvailable(DB::ReadBuffer & buf, const char * str) +{ + size_t n = strlen(str); + ASSERT_EQ(buf.available(), n); + ASSERT_EQ(strncmp(buf.position(), str, n), 0); +} + +TEST(PeekableReadBuffer, CheckpointsWorkCorrectly) +try +{ + std::string s1 = "0123456789"; + std::string s2 = "qwertyuiop"; + std::string s3 = "asdfghjkl;"; + std::string s4 = "zxcvbnm,./"; + DB::ReadBufferFromString b1(s1); + DB::ReadBufferFromString b2(s2); + DB::ReadBufferFromString b3(s3); + DB::ReadBufferFromString b4(s4); + + DB::ConcatReadBuffer concat({&b1, &b2, &b3, &b4}); + DB::PeekableReadBuffer peekable(concat, 0, 16); + + ASSERT_TRUE(!peekable.eof()); + assertAvailable(peekable, "0123456789"); + { + DB::PeekableReadBufferCheckpoint checkpoint{peekable}; + readAndAssert(peekable, "01234"); + } + bool exception = false; + try + { + peekable.rollbackToCheckpoint(); + } + catch (DB::Exception & e) + { + if (e.code() != DB::ErrorCodes::LOGICAL_ERROR) + throw; + exception = true; + } + ASSERT_TRUE(exception); + assertAvailable(peekable, "56789"); + + readAndAssert(peekable, "56"); + + peekable.setCheckpoint(); + readAndAssert(peekable, "789qwertyu"); + peekable.rollbackToCheckpoint(); + peekable.dropCheckpoint(); + assertAvailable(peekable, "789"); + peekable.peekNext(); + assertAvailable(peekable, "789qwertyuiop"); + ASSERT_EQ(peekable.lastPeeked().size(), 10); + ASSERT_EQ(strncmp(peekable.lastPeeked().begin(), "asdfghjkl;", 10), 0); + + exception = false; + try + { + DB::PeekableReadBufferCheckpoint checkpoint{peekable, true}; + peekable.ignore(30); + } + catch (DB::Exception & e) + { + if (e.code() != DB::ErrorCodes::MEMORY_LIMIT_EXCEEDED) + throw; + exception = true; + } + ASSERT_TRUE(exception); + assertAvailable(peekable, "789qwertyuiop"); + ASSERT_EQ(peekable.lastPeeked().size(), 10); + ASSERT_EQ(strncmp(peekable.lastPeeked().begin(), "asdfghjkl;", 10), 0); + + readAndAssert(peekable, "789qwertyu"); + peekable.setCheckpoint(); + readAndAssert(peekable, "iopasdfghj"); + assertAvailable(peekable, "kl;"); + peekable.dropCheckpoint(); + + peekable.setCheckpoint(); + readAndAssert(peekable, "kl;zxcvbnm,./"); + ASSERT_TRUE(peekable.eof()); + ASSERT_TRUE(peekable.eof()); + ASSERT_TRUE(peekable.eof()); + peekable.rollbackToCheckpoint(); + readAndAssert(peekable, "kl;zxcvbnm"); + peekable.dropCheckpoint(); + + exception = false; + try + { + peekable.assertCanBeDestructed(); + } + catch (DB::Exception & e) + { + if (e.code() != DB::ErrorCodes::LOGICAL_ERROR) + throw; + exception = true; + } + ASSERT_TRUE(exception); + + auto buf_ptr = peekable.takeUnreadData(); + ASSERT_TRUE(peekable.eof()); + ASSERT_TRUE(peekable.eof()); + ASSERT_TRUE(peekable.eof()); + + readAndAssert(*buf_ptr, ",./"); + ASSERT_TRUE(buf_ptr->eof()); + + peekable.assertCanBeDestructed(); +} +catch (const DB::Exception & e) +{ + std::cerr << e.what() << ", " << e.displayText() << std::endl; + throw; +} + diff --git a/dbms/src/Interpreters/AnalyzedJoin.cpp b/dbms/src/Interpreters/AnalyzedJoin.cpp index 6a3b9b8ac1b..36b573c4093 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.cpp +++ b/dbms/src/Interpreters/AnalyzedJoin.cpp @@ -1,13 +1,18 @@ #include #include #include +#include #include #include #include +#include #include +#include + + namespace DB { @@ -26,7 +31,6 @@ void AnalyzedJoin::addUsingKey(const ASTPtr & ast) void AnalyzedJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) { - with_using = false; key_names_left.push_back(left_table_ast->getColumnName()); key_names_right.push_back(right_table_ast->getAliasOrColumnName()); @@ -37,7 +41,7 @@ void AnalyzedJoin::addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast) /// @return how many times right key appears in ON section. size_t AnalyzedJoin::rightKeyInclusion(const String & name) const { - if (with_using) + if (hasUsing()) return 0; size_t count = 0; @@ -101,6 +105,120 @@ std::unordered_map AnalyzedJoin::getOriginalColumnsMap(const Nam return out; } +ASTPtr AnalyzedJoin::leftKeysList() const +{ + ASTPtr keys_list = std::make_shared(); + keys_list->children = key_asts_left; + return keys_list; +} + +ASTPtr AnalyzedJoin::rightKeysList() const +{ + ASTPtr keys_list = std::make_shared(); + if (hasOn()) + keys_list->children = key_asts_right; + return keys_list; +} + +Names AnalyzedJoin::requiredJoinedNames() const +{ + NameSet required_columns_set(key_names_right.begin(), key_names_right.end()); + for (const auto & joined_column : columns_added_by_join) + required_columns_set.insert(joined_column.name); + + return Names(required_columns_set.begin(), required_columns_set.end()); +} + +void AnalyzedJoin::appendRequiredColumns(const Block & sample, NameSet & required_columns) const +{ + for (auto & column : key_names_right) + 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); +} + +void AnalyzedJoin::addJoinedColumn(const NameAndTypePair & joined_column) +{ + if (join_use_nulls && isLeftOrFull(table_join.kind)) + { + auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type; + columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type))); + } + else + columns_added_by_join.push_back(joined_column); +} + +void AnalyzedJoin::addJoinedColumnsAndCorrectNullability(Block & sample_block) const +{ + bool right_or_full_join = isRightOrFull(table_join.kind); + bool left_or_full_join = isLeftOrFull(table_join.kind); + + for (auto & col : sample_block) + { + /// Materialize column. + /// Column is not empty if it is constant, but after Join all constants will be materialized. + /// So, we need remove constants from header. + if (col.column) + col.column = nullptr; + + bool make_nullable = join_use_nulls && right_or_full_join; + + if (make_nullable && col.type->canBeInsideNullable()) + col.type = makeNullable(col.type); + } + + for (const auto & col : columns_added_by_join) + { + auto res_type = col.type; + + bool make_nullable = join_use_nulls && left_or_full_join; + + if (!make_nullable) + { + /// Keys from right table are usually not stored in Join, but copied from the left one. + /// So, if left key is nullable, let's make right key nullable too. + /// Note: for some join types it's not needed and, probably, may be removed. + /// Note: changing this code, take into account the implementation in Join.cpp. + auto it = std::find(key_names_right.begin(), key_names_right.end(), col.name); + if (it != key_names_right.end()) + { + auto pos = it - key_names_right.begin(); + const auto & left_key_name = key_names_left[pos]; + make_nullable = sample_block.getByName(left_key_name).type->isNullable(); + } + } + + if (make_nullable && res_type->canBeInsideNullable()) + res_type = makeNullable(res_type); + + sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name)); + } +} + +bool AnalyzedJoin::sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y) +{ + if (!x && !y) + return true; + if (!x || !y) + return false; + + return x->table_join.kind == y->table_join.kind + && x->table_join.strictness == y->table_join.strictness + && x->key_names_left == y->key_names_left + && x->key_names_right == y->key_names_right + && x->columns_added_by_join == y->columns_added_by_join; +} + +JoinPtr AnalyzedJoin::makeHashJoin(const Block & sample_block, const SizeLimits & size_limits_for_join) const +{ + auto join = std::make_shared(key_names_right, join_use_nulls, size_limits_for_join, table_join.kind, table_join.strictness); + join->setSampleBlock(sample_block); + return join; +} + NamesAndTypesList getNamesAndTypeListFromTableExpression(const ASTTableExpression & table_expression, const Context & context) { NamesAndTypesList names_and_type_list; diff --git a/dbms/src/Interpreters/AnalyzedJoin.h b/dbms/src/Interpreters/AnalyzedJoin.h index 1ce11da95e0..34fbede0d89 100644 --- a/dbms/src/Interpreters/AnalyzedJoin.h +++ b/dbms/src/Interpreters/AnalyzedJoin.h @@ -2,7 +2,8 @@ #include #include -#include +#include +#include #include #include @@ -13,6 +14,10 @@ namespace DB class Context; class ASTSelectQuery; struct DatabaseAndTableWithAlias; +class Block; + +class Join; +using JoinPtr = std::shared_ptr; struct AnalyzedJoin { @@ -30,18 +35,19 @@ struct AnalyzedJoin private: friend class SyntaxAnalyzer; - friend struct SyntaxAnalyzerResult; - friend class ExpressionAnalyzer; - friend class SelectQueryExpressionAnalyzer; Names key_names_left; Names key_names_right; /// Duplicating names are qualified. ASTs key_asts_left; ASTs key_asts_right; - bool with_using = true; + ASTTableJoin table_join; + bool join_use_nulls = false; /// All columns which can be read from joined table. Duplicating names are qualified. NamesAndTypesList columns_from_joined_table; + /// Columns will be added to block by JOIN. It's a subset of columns_from_joined_table with corrected Nullability + NamesAndTypesList columns_added_by_join; + /// Name -> original name. Names are the same as in columns_from_joined_table list. std::unordered_map original_names; /// Original name -> name. Only ranamed columns. @@ -51,8 +57,8 @@ public: void addUsingKey(const ASTPtr & ast); void addOnKeys(ASTPtr & left_table_ast, ASTPtr & right_table_ast); - bool hasUsing() const { return with_using; } - bool hasOn() const { return !with_using; } + bool hasUsing() const { return table_join.using_expression_list != nullptr; } + bool hasOn() const { return !hasUsing(); } NameSet getQualifiedColumnsSet() const; NameSet getOriginalColumnsSet() const; @@ -60,6 +66,22 @@ public: 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; + + ASTPtr leftKeysList() const; + ASTPtr rightKeysList() const; /// For ON syntax only + + Names requiredJoinedNames() const; + const Names & keyNamesLeft() const { return key_names_left; } + const NamesAndTypesList & columnsFromJoinedTable() const { return columns_from_joined_table; } + const NamesAndTypesList & columnsAddedByJoin() const { return columns_added_by_join; } + + JoinPtr makeHashJoin(const Block & sample_block, const SizeLimits & size_limits_for_join) const; + + static bool sameJoin(const AnalyzedJoin * x, const AnalyzedJoin * y); }; struct ASTTableExpression; diff --git a/dbms/src/Interpreters/ExpressionActions.cpp b/dbms/src/Interpreters/ExpressionActions.cpp index 5248350b99b..160f9d68672 100644 --- a/dbms/src/Interpreters/ExpressionActions.cpp +++ b/dbms/src/Interpreters/ExpressionActions.cpp @@ -12,7 +12,6 @@ #include #include #include -#include namespace ProfileEvents @@ -45,7 +44,8 @@ Names ExpressionAction::getNeededColumns() const res.insert(res.end(), array_joined_columns.begin(), array_joined_columns.end()); - res.insert(res.end(), join_key_names_left.begin(), join_key_names_left.end()); + if (join_params) + res.insert(res.end(), join_params->keyNamesLeft().begin(), join_params->keyNamesLeft().end()); for (const auto & column : projection) res.push_back(column.first); @@ -159,20 +159,12 @@ ExpressionAction ExpressionAction::arrayJoin(const NameSet & array_joined_column return a; } -ExpressionAction ExpressionAction::ordinaryJoin( - const ASTTableJoin & join_params, - std::shared_ptr join_, - const Names & join_key_names_left, - const Names & join_key_names_right, - const NamesAndTypesList & columns_added_by_join_) +ExpressionAction ExpressionAction::ordinaryJoin(std::shared_ptr join_params, std::shared_ptr hash_join) { ExpressionAction a; a.type = JOIN; - a.join = std::move(join_); - a.join_kind = join_params.kind; - a.join_key_names_left = join_key_names_left; - a.join_key_names_right = join_key_names_right; - a.columns_added_by_join = columns_added_by_join_; + a.join_params = join_params; + a.join = hash_join; return a; } @@ -277,51 +269,7 @@ void ExpressionAction::prepare(Block & sample_block, const Settings & settings, case JOIN: { - bool is_null_used_as_default = settings.join_use_nulls; - bool right_or_full_join = isRightOrFull(join_kind); - bool left_or_full_join = isLeftOrFull(join_kind); - - for (auto & col : sample_block) - { - /// Materialize column. - /// Column is not empty if it is constant, but after Join all constants will be materialized. - /// So, we need remove constants from header. - if (col.column) - col.column = nullptr; - - bool make_nullable = is_null_used_as_default && right_or_full_join; - - if (make_nullable && col.type->canBeInsideNullable()) - col.type = makeNullable(col.type); - } - - for (const auto & col : columns_added_by_join) - { - auto res_type = col.type; - - bool make_nullable = is_null_used_as_default && left_or_full_join; - - if (!make_nullable) - { - /// Keys from right table are usually not stored in Join, but copied from the left one. - /// So, if left key is nullable, let's make right key nullable too. - /// Note: for some join types it's not needed and, probably, may be removed. - /// Note: changing this code, take into account the implementation in Join.cpp. - auto it = std::find(join_key_names_right.begin(), join_key_names_right.end(), col.name); - if (it != join_key_names_right.end()) - { - auto pos = it - join_key_names_right.begin(); - const auto & left_key_name = join_key_names_left[pos]; - make_nullable = sample_block.getByName(left_key_name).type->isNullable(); - } - } - - if (make_nullable && res_type->canBeInsideNullable()) - res_type = makeNullable(res_type); - - sample_block.insert(ColumnWithTypeAndName(nullptr, res_type, col.name)); - } - + join_params->addJoinedColumnsAndCorrectNullability(sample_block); break; } @@ -527,7 +475,7 @@ void ExpressionAction::execute(Block & block, bool dry_run) const case JOIN: { - join->joinBlock(block, join_key_names_left, columns_added_by_join); + join->joinBlock(block, *join_params); break; } @@ -645,9 +593,10 @@ std::string ExpressionAction::toString() const case JOIN: ss << "JOIN "; - for (NamesAndTypesList::const_iterator it = columns_added_by_join.begin(); it != columns_added_by_join.end(); ++it) + for (NamesAndTypesList::const_iterator it = join_params->columnsAddedByJoin().begin(); + it != join_params->columnsAddedByJoin().end(); ++it) { - if (it != columns_added_by_join.begin()) + if (it != join_params->columnsAddedByJoin().begin()) ss << ", "; ss << it->name; } @@ -1220,7 +1169,7 @@ BlockInputStreamPtr ExpressionActions::createStreamWithNonJoinedDataIfFullOrRigh for (const auto & action : actions) if (action.join && isRightOrFull(action.join->getKind())) return action.join->createStreamWithNonJoinedRows( - source_header, action.join_key_names_left, action.columns_added_by_join, max_block_size); + source_header, *action.join_params, max_block_size); return {}; } @@ -1267,7 +1216,7 @@ UInt128 ExpressionAction::ActionHash::operator()(const ExpressionAction & action hash.update(col); break; case JOIN: - for (const auto & col : action.columns_added_by_join) + for (const auto & col : action.join_params->columnsAddedByJoin()) hash.update(col.name); break; case PROJECT: @@ -1326,9 +1275,7 @@ bool ExpressionAction::operator==(const ExpressionAction & other) const && array_joined_columns == other.array_joined_columns && array_join_is_left == other.array_join_is_left && join == other.join - && join_key_names_left == other.join_key_names_left - && join_key_names_right == other.join_key_names_right - && columns_added_by_join == other.columns_added_by_join + && AnalyzedJoin::sameJoin(join_params.get(), other.join_params.get()) && projection == other.projection && is_function_compiled == other.is_function_compiled; } diff --git a/dbms/src/Interpreters/ExpressionActions.h b/dbms/src/Interpreters/ExpressionActions.h index 62d50131b45..90638d86368 100644 --- a/dbms/src/Interpreters/ExpressionActions.h +++ b/dbms/src/Interpreters/ExpressionActions.h @@ -6,6 +6,7 @@ #include #include #include +#include #include #include "config_core.h" #include @@ -104,11 +105,8 @@ public: bool unaligned_array_join = false; /// For JOIN + std::shared_ptr join_params = nullptr; std::shared_ptr join; - ASTTableJoin::Kind join_kind; - Names join_key_names_left; - Names join_key_names_right; - NamesAndTypesList columns_added_by_join; /// For PROJECT. NamesWithAliases projection; @@ -124,9 +122,7 @@ public: static ExpressionAction project(const Names & projected_columns_); static ExpressionAction addAliases(const NamesWithAliases & aliased_columns_); static ExpressionAction arrayJoin(const NameSet & array_joined_columns, bool array_join_is_left, const Context & context); - static ExpressionAction ordinaryJoin(const ASTTableJoin & join_params, std::shared_ptr join_, - const Names & join_key_names_left, const Names & join_key_names_right, - const NamesAndTypesList & columns_added_by_join_); + static ExpressionAction ordinaryJoin(std::shared_ptr join_params, std::shared_ptr hash_join); /// Which columns necessary to perform this action. Names getNeededColumns() const; diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.cpp b/dbms/src/Interpreters/ExpressionAnalyzer.cpp index 2d0b3c2729a..e452d62ffca 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.cpp +++ b/dbms/src/Interpreters/ExpressionAnalyzer.cpp @@ -29,7 +29,6 @@ #include #include #include -#include #include #include @@ -134,14 +133,8 @@ void ExpressionAnalyzer::analyzeAggregation() const ASTTablesInSelectQueryElement * join = select_query->join(); if (join) { - const auto & table_join = join->table_join->as(); - if (table_join.using_expression_list) - getRootActions(table_join.using_expression_list, true, temp_actions); - if (table_join.on_expression) - for (const auto & key_ast : analyzedJoin().key_asts_left) - getRootActions(key_ast, true, temp_actions); - - addJoinAction(table_join, temp_actions); + getRootActions(analyzedJoin().leftKeysList(), true, temp_actions); + addJoinAction(temp_actions); } } @@ -298,7 +291,8 @@ void SelectQueryExpressionAnalyzer::makeSetsForIndex(const ASTPtr & node) { NamesAndTypesList temp_columns = sourceColumns(); temp_columns.insert(temp_columns.end(), array_join_columns.begin(), array_join_columns.end()); - temp_columns.insert(temp_columns.end(), columnsAddedByJoin().begin(), columnsAddedByJoin().end()); + temp_columns.insert(temp_columns.end(), + analyzedJoin().columnsAddedByJoin().begin(), analyzedJoin().columnsAddedByJoin().end()); ExpressionActionsPtr temp_actions = std::make_shared(temp_columns, context); getRootActions(left_in_operand, true, temp_actions); @@ -412,22 +406,10 @@ bool SelectQueryExpressionAnalyzer::appendArrayJoin(ExpressionActionsChain & cha return true; } -static void appendRequiredColumns( - NameSet & required_columns, const Block & sample, const Names & key_names_right, const NamesAndTypesList & columns_added_by_join) -{ - for (auto & column : key_names_right) - 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); -} - /// It's possible to set nullptr as join for only_types mode -void ExpressionAnalyzer::addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, JoinPtr join) const +void ExpressionAnalyzer::addJoinAction(ExpressionActionsPtr & actions, JoinPtr join) const { - actions->add(ExpressionAction::ordinaryJoin(join_params, std::move(join), analyzedJoin().key_names_left, analyzedJoin().key_names_right, columnsAddedByJoin())); + actions->add(ExpressionAction::ordinaryJoin(syntax->analyzed_join, join)); } bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_types) @@ -438,16 +420,11 @@ bool SelectQueryExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, b SubqueryForSet & subquery_for_set = getSubqueryForJoin(*ast_join); - ASTPtr left_keys_list = std::make_shared(); - left_keys_list->children = analyzedJoin().key_asts_left; - initChain(chain, sourceColumns()); ExpressionActionsChain::Step & step = chain.steps.back(); - auto & join_params = ast_join->table_join->as(); - - getRootActions(left_keys_list, only_types, step.actions); - addJoinAction(join_params, step.actions, subquery_for_set.join); + getRootActions(analyzedJoin().leftKeysList(), only_types, step.actions); + addJoinAction(step.actions, subquery_for_set.join); return true; } @@ -524,11 +501,9 @@ void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryEle Names action_columns = joined_block_actions->getRequiredColumns(); NameSet required_columns(action_columns.begin(), action_columns.end()); - auto & analyzed_join = analyzedJoin(); - appendRequiredColumns( - required_columns, joined_block_actions->getSampleBlock(), analyzed_join.key_names_right, columnsAddedByJoin()); + analyzedJoin().appendRequiredColumns(joined_block_actions->getSampleBlock(), required_columns); - auto original_map = analyzed_join.getOriginalColumnsMap(required_columns); + auto original_map = analyzedJoin().getOriginalColumnsMap(required_columns); Names original_columns; for (auto & pr : original_map) original_columns.push_back(pr.second); @@ -542,29 +517,16 @@ void SelectQueryExpressionAnalyzer::makeHashJoin(const ASTTablesInSelectQueryEle joined_block_actions->execute(sample_block); /// TODO You do not need to set this up when JOIN is only needed on remote servers. - auto & join_params = join_element.table_join->as(); - subquery_for_set.join = std::make_shared(analyzedJoin().key_names_right, settings.join_use_nulls, - settings.size_limits_for_join, join_params.kind, join_params.strictness); - subquery_for_set.join->setSampleBlock(sample_block); + subquery_for_set.join = analyzedJoin().makeHashJoin(sample_block, settings.size_limits_for_join); subquery_for_set.joined_block_actions = joined_block_actions; } ExpressionActionsPtr SelectQueryExpressionAnalyzer::createJoinedBlockActions() const { - /// Create custom expression list with join keys from right table. - ASTPtr expression_list = std::make_shared(); - ASTs & children = expression_list->children; + ASTPtr expression_list = analyzedJoin().rightKeysList(); + Names required_columns = analyzedJoin().requiredJoinedNames(); - if (analyzedJoin().hasOn()) - for (const auto & join_right_key : analyzedJoin().key_asts_right) - children.emplace_back(join_right_key); - - NameSet required_columns_set(analyzedJoin().key_names_right.begin(), analyzedJoin().key_names_right.end()); - for (const auto & joined_column : columnsAddedByJoin()) - required_columns_set.insert(joined_column.name); - Names required_columns(required_columns_set.begin(), required_columns_set.end()); - - auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columns_from_joined_table, required_columns); + auto syntax_result = SyntaxAnalyzer(context).analyze(expression_list, analyzedJoin().columnsFromJoinedTable(), required_columns); return ExpressionAnalyzer(expression_list, syntax_result, context).getActions(true, false); } diff --git a/dbms/src/Interpreters/ExpressionAnalyzer.h b/dbms/src/Interpreters/ExpressionAnalyzer.h index 2eafe4b85f0..a28f54210b2 100644 --- a/dbms/src/Interpreters/ExpressionAnalyzer.h +++ b/dbms/src/Interpreters/ExpressionAnalyzer.h @@ -121,9 +121,8 @@ protected: SyntaxAnalyzerResultPtr syntax; const StoragePtr & storage() const { return syntax->storage; } /// The main table in FROM clause, if exists. - const AnalyzedJoin & analyzedJoin() const { return syntax->analyzed_join; } + const AnalyzedJoin & analyzedJoin() const { return *syntax->analyzed_join; } const NamesAndTypesList & sourceColumns() const { return syntax->required_source_columns; } - const NamesAndTypesList & columnsAddedByJoin() const { return syntax->columns_added_by_join; } const std::vector & aggregates() const { return syntax->aggregates; } /// Find global subqueries in the GLOBAL IN/JOIN sections. Fills in external_tables. @@ -131,7 +130,7 @@ protected: void addMultipleArrayJoinAction(ExpressionActionsPtr & actions, bool is_left) const; - void addJoinAction(const ASTTableJoin & join_params, ExpressionActionsPtr & actions, JoinPtr join = {}) const; + void addJoinAction(ExpressionActionsPtr & actions, JoinPtr join = {}) const; void getRootActions(const ASTPtr & ast, bool no_subqueries, ExpressionActionsPtr & actions, bool only_consts = false); diff --git a/dbms/src/Interpreters/Join.cpp b/dbms/src/Interpreters/Join.cpp index 63bf88a8437..855b0d284e1 100644 --- a/dbms/src/Interpreters/Join.cpp +++ b/dbms/src/Interpreters/Join.cpp @@ -10,6 +10,7 @@ #include #include +#include #include #include @@ -1048,8 +1049,11 @@ void Join::joinGet(Block & block, const String & column_name) const } -void Join::joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const +void Join::joinBlock(Block & block, const AnalyzedJoin & join_params) const { + const Names & key_names_left = join_params.keyNamesLeft(); + const NamesAndTypesList & columns_added_by_join = join_params.columnsAddedByJoin(); + std::shared_lock lock(rwlock); checkTypesOfKeys(block, key_names_left, sample_block_with_keys); @@ -1457,10 +1461,11 @@ private: }; -BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, const Names & key_names_left, - const NamesAndTypesList & columns_added_by_join, UInt64 max_block_size) const +BlockInputStreamPtr Join::createStreamWithNonJoinedRows(const Block & left_sample_block, const AnalyzedJoin & join_params, + UInt64 max_block_size) const { - return std::make_shared(*this, left_sample_block, key_names_left, columns_added_by_join, max_block_size); + return std::make_shared(*this, left_sample_block, + join_params.keyNamesLeft(), join_params.columnsAddedByJoin(), max_block_size); } diff --git a/dbms/src/Interpreters/Join.h b/dbms/src/Interpreters/Join.h index 4756a5680ef..1a85481cf39 100644 --- a/dbms/src/Interpreters/Join.h +++ b/dbms/src/Interpreters/Join.h @@ -26,6 +26,8 @@ namespace DB { +struct AnalyzedJoin; + namespace JoinStuff { @@ -141,7 +143,7 @@ public: /** Join data from the map (that was previously built by calls to insertFromBlock) to the block with data from "left" table. * Could be called from different threads in parallel. */ - void joinBlock(Block & block, const Names & key_names_left, const NamesAndTypesList & columns_added_by_join) const; + void joinBlock(Block & block, const AnalyzedJoin & join_params) const; /// Infer the return type for joinGet function DataTypePtr joinGetReturnType(const String & column_name) const; @@ -161,8 +163,8 @@ public: * Use only after all calls to joinBlock was done. * left_sample_block is passed without account of 'use_nulls' setting (columns will be converted to Nullable inside). */ - BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, const Names & key_names_left, - const NamesAndTypesList & columns_added_by_join, UInt64 max_block_size) const; + BlockInputStreamPtr createStreamWithNonJoinedRows(const Block & left_sample_block, const AnalyzedJoin & join_params, + UInt64 max_block_size) const; /// Number of keys in all built JOIN maps. size_t getTotalRowCount() const; diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.cpp b/dbms/src/Interpreters/SyntaxAnalyzer.cpp index 0c73beeef16..1298d1733fb 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.cpp +++ b/dbms/src/Interpreters/SyntaxAnalyzer.cpp @@ -489,14 +489,13 @@ void getArrayJoinedColumns(ASTPtr & query, SyntaxAnalyzerResult & result, const } } -void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin::Kind & join_kind) +void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_default_strictness, ASTTableJoin & out_table_join) { const ASTTablesInSelectQueryElement * node = select_query.join(); if (!node) return; auto & table_join = const_cast(node)->table_join->as(); - join_kind = table_join.kind; if (table_join.strictness == ASTTableJoin::Strictness::Unspecified && table_join.kind != ASTTableJoin::Kind::Cross) @@ -509,6 +508,8 @@ void setJoinStrictness(ASTSelectQuery & select_query, JoinStrictness join_defaul throw Exception("Expected ANY or ALL in JOIN section, because setting (join_default_strictness) is empty", DB::ErrorCodes::EXPECTED_ALL_OR_ANY); } + + out_table_join = table_join; } /// Find the columns that are obtained by JOIN. @@ -609,8 +610,7 @@ std::vector getAggregates(const ASTPtr & query) /// Calculate which columns are required to execute the expression. /// Then, delete all other columns from the list of available columns. /// After execution, columns will only contain the list of columns needed to read from the table. -void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns, - bool make_joined_columns_nullable) +void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns) { /// We caclulate required_source_columns with source_columns modifications and swap them on exit required_source_columns = source_columns; @@ -637,8 +637,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA avaliable_columns.insert(name.name); /// Add columns obtained by JOIN (if needed). - columns_added_by_join.clear(); - for (const auto & joined_column : analyzed_join.columns_from_joined_table) + for (const auto & joined_column : analyzed_join->columnsFromJoinedTable()) { auto & name = joined_column.name; if (avaliable_columns.count(name)) @@ -647,16 +646,9 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA if (required.count(name)) { /// Optimisation: do not add columns needed only in JOIN ON section. - if (columns_context.nameInclusion(name) > analyzed_join.rightKeyInclusion(name)) - { - if (make_joined_columns_nullable) - { - auto type = joined_column.type->canBeInsideNullable() ? makeNullable(joined_column.type) : joined_column.type; - columns_added_by_join.emplace_back(NameAndTypePair(joined_column.name, std::move(type))); - } - else - columns_added_by_join.push_back(joined_column); - } + if (columns_context.nameInclusion(name) > analyzed_join->rightKeyInclusion(name)) + analyzed_join->addJoinedColumn(joined_column); + required.erase(name); } } @@ -766,7 +758,7 @@ void SyntaxAnalyzerResult::collectUsedColumns(const ASTPtr & query, const NamesA if (columns_context.has_table_join) { ss << ", joined columns:"; - for (const auto & column : analyzed_join.columns_from_joined_table) + for (const auto & column : analyzed_join->columnsFromJoinedTable()) ss << " '" << column.name << "'"; } @@ -798,15 +790,17 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( storage = context.tryGetTable(db_and_table->database, db_and_table->table); } + const auto & settings = context.getSettingsRef(); + SyntaxAnalyzerResult result; result.storage = storage; result.source_columns = source_columns_; + result.analyzed_join = std::make_shared(); /// TODO: move to select_query logic + result.analyzed_join->join_use_nulls = settings.join_use_nulls; collectSourceColumns(select_query, result.storage, result.source_columns); NameSet source_columns_set = removeDuplicateColumns(result.source_columns); - const auto & settings = context.getSettingsRef(); - Names source_columns_list; source_columns_list.reserve(result.source_columns.size()); for (const auto & type_name : result.source_columns) @@ -831,13 +825,13 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( const auto & joined_expression = node->table_expression->as(); DatabaseAndTableWithAlias table(joined_expression, context.getCurrentDatabase()); - result.analyzed_join.columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context); - result.analyzed_join.deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix()); + result.analyzed_join->columns_from_joined_table = getNamesAndTypeListFromTableExpression(joined_expression, context); + result.analyzed_join->deduplicateAndQualifyColumnNames(source_columns_set, table.getQualifiedNamePrefix()); } translateQualifiedNames(query, *select_query, context, (storage ? storage->getColumns().getOrdinary().getNames() : source_columns_list), source_columns_set, - result.analyzed_join.getQualifiedColumnsSet()); + result.analyzed_join->getQualifiedColumnsSet()); /// Rewrite IN and/or JOIN for distributed tables according to distributed_product_mode setting. InJoinSubqueriesPreprocessor(context).visit(query); @@ -872,7 +866,6 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Optimize if with constant condition after constants was substituted instead of scalar subqueries. OptimizeIfWithConstantConditionVisitor(result.aliases).visit(query); - bool make_joined_columns_nullable = false; if (select_query) { /// GROUP BY injective function elimination. @@ -893,15 +886,12 @@ SyntaxAnalyzerResultPtr SyntaxAnalyzer::analyze( /// Push the predicate expression down to the subqueries. result.rewrite_subqueries = PredicateExpressionsOptimizer(select_query, settings, context).optimize(); - ASTTableJoin::Kind join_kind = ASTTableJoin::Kind::Comma; - setJoinStrictness(*select_query, settings.join_default_strictness, join_kind); - make_joined_columns_nullable = settings.join_use_nulls && isLeftOrFull(join_kind); - - collectJoinedColumns(result.analyzed_join, *select_query, source_columns_set, result.aliases); + setJoinStrictness(*select_query, settings.join_default_strictness, result.analyzed_join->table_join); + collectJoinedColumns(*result.analyzed_join, *select_query, source_columns_set, result.aliases); } result.aggregates = getAggregates(query); - result.collectUsedColumns(query, additional_source_columns, make_joined_columns_nullable); + result.collectUsedColumns(query, additional_source_columns); return std::make_shared(result); } diff --git a/dbms/src/Interpreters/SyntaxAnalyzer.h b/dbms/src/Interpreters/SyntaxAnalyzer.h index e95d7354e8a..a2187ee2ef0 100644 --- a/dbms/src/Interpreters/SyntaxAnalyzer.h +++ b/dbms/src/Interpreters/SyntaxAnalyzer.h @@ -15,13 +15,11 @@ class ASTFunction; struct SyntaxAnalyzerResult { StoragePtr storage; - AnalyzedJoin analyzed_join; + std::shared_ptr analyzed_join; NamesAndTypesList source_columns; /// Set of columns that are enough to read from the table to evaluate the expression. It does not include joined columns. NamesAndTypesList required_source_columns; - /// Columns will be added to block by JOIN. It's a subset of analyzed_join.columns_from_joined_table with corrected Nullability - NamesAndTypesList columns_added_by_join; Aliases aliases; std::vector aggregates; @@ -42,7 +40,7 @@ struct SyntaxAnalyzerResult /// Predicate optimizer overrides the sub queries bool rewrite_subqueries = false; - void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns, bool make_joined_columns_nullable); + void collectUsedColumns(const ASTPtr & query, const NamesAndTypesList & additional_source_columns); Names requiredSourceColumns() const { return required_source_columns.getNames(); } }; diff --git a/dbms/src/Processors/Executors/PipelineExecutor.cpp b/dbms/src/Processors/Executors/PipelineExecutor.cpp index 6aad6f96b5c..a10a7d267ec 100644 --- a/dbms/src/Processors/Executors/PipelineExecutor.cpp +++ b/dbms/src/Processors/Executors/PipelineExecutor.cpp @@ -287,12 +287,6 @@ bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & pa switch (node.last_processor_status) { case IProcessor::Status::NeedData: - { - add_neighbours_to_prepare_queue(); - try_release_ownership(); - - break; - } case IProcessor::Status::PortFull: { add_neighbours_to_prepare_queue(); diff --git a/dbms/src/Processors/Formats/IRowInputFormat.cpp b/dbms/src/Processors/Formats/IRowInputFormat.cpp index 2860587cbf2..b45c714ea07 100644 --- a/dbms/src/Processors/Formats/IRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/IRowInputFormat.cpp @@ -174,7 +174,7 @@ Chunk IRowInputFormat::generate() { if (params.allow_errors_num > 0 || params.allow_errors_ratio > 0) { - Logger * log = &Logger::get("BlockInputStreamFromRowInputStream"); + Logger * log = &Logger::get("IRowInputFormat"); LOG_TRACE(log, "Skipped " << num_errors << " rows with errors while reading the input stream"); } diff --git a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp index 8fb6ab5a359..0cd5ffb03e0 100644 --- a/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp +++ b/dbms/src/Processors/Formats/Impl/ArrowColumnToCHColumn.cpp @@ -13,6 +13,8 @@ #include #include #include +#include + namespace DB { @@ -27,34 +29,28 @@ namespace DB extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN; extern const int THERE_IS_NO_COLUMN; } - const std::unordered_map> arrow_type_to_internal_type = { - //{arrow::Type::DECIMAL, std::make_shared()}, - {arrow::Type::UINT8, std::make_shared()}, - {arrow::Type::INT8, std::make_shared()}, - {arrow::Type::UINT16, std::make_shared()}, - {arrow::Type::INT16, std::make_shared()}, - {arrow::Type::UINT32, std::make_shared()}, - {arrow::Type::INT32, std::make_shared()}, - {arrow::Type::UINT64, std::make_shared()}, - {arrow::Type::INT64, std::make_shared()}, - {arrow::Type::HALF_FLOAT, std::make_shared()}, - {arrow::Type::FLOAT, std::make_shared()}, - {arrow::Type::DOUBLE, std::make_shared()}, - {arrow::Type::BOOL, std::make_shared()}, - //{arrow::Type::DATE32, std::make_shared()}, - {arrow::Type::DATE32, std::make_shared()}, - //{arrow::Type::DATE32, std::make_shared()}, - {arrow::Type::DATE64, std::make_shared()}, - {arrow::Type::TIMESTAMP, std::make_shared()}, - //{arrow::Type::TIME32, std::make_shared()}, + static const std::initializer_list> arrow_type_to_internal_type = + { + {arrow::Type::UINT8, "UInt8"}, + {arrow::Type::INT8, "Int8"}, + {arrow::Type::UINT16, "UInt16"}, + {arrow::Type::INT16, "Int16"}, + {arrow::Type::UINT32, "UInt32"}, + {arrow::Type::INT32, "Int32"}, + {arrow::Type::UINT64, "UInt64"}, + {arrow::Type::INT64, "Int64"}, + {arrow::Type::HALF_FLOAT, "Float32"}, + {arrow::Type::FLOAT, "Float32"}, + {arrow::Type::DOUBLE, "Float64"}, + {arrow::Type::BOOL, "UInt8"}, + {arrow::Type::DATE32, "Date"}, + {arrow::Type::DATE64, "DateTime"}, + {arrow::Type::TIMESTAMP, "DateTime"}, - {arrow::Type::STRING, std::make_shared()}, - {arrow::Type::BINARY, std::make_shared()}, - //{arrow::Type::FIXED_SIZE_BINARY, std::make_shared()}, - //{arrow::Type::UUID, std::make_shared()}, - + {arrow::Type::STRING, "String"}, + {arrow::Type::BINARY, "String"}, // TODO: add other types that are convertable to internal ones: // 0. ENUM? @@ -253,7 +249,7 @@ namespace DB void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr &table, arrow::Status &read_status, const Block &header, int &row_group_current, const Context &context, std::string format_name) - { + { Columns columns_list; UInt64 num_rows = 0; @@ -308,15 +304,16 @@ namespace DB const auto decimal_type = static_cast(arrow_column->type().get()); internal_nested_type = std::make_shared>(decimal_type->precision(), decimal_type->scale()); - } else if (arrow_type_to_internal_type.find(arrow_type) != arrow_type_to_internal_type.end()) + } + else if (auto internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(), + [=](auto && elem) { return elem.first == arrow_type; }); + internal_type_it != arrow_type_to_internal_type.end()) { - internal_nested_type = arrow_type_to_internal_type.at(arrow_type); + internal_nested_type = DataTypeFactory::instance().get(internal_type_it->second); } else { - throw Exception - { - "The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name() + throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name() + "\" is not supported for conversion from a " + format_name + " data format", ErrorCodes::CANNOT_CONVERT_TYPE}; } diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp index 2b8c1be6016..b5ee30fb7f8 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB @@ -17,9 +18,9 @@ namespace ErrorCodes } -CSVRowInputFormat::CSVRowInputFormat( - ReadBuffer & in_, Block header_, Params params_, bool with_names_, const FormatSettings & format_settings_) - : IRowInputFormat(std::move(header_), in_, std::move(params_)) +CSVRowInputFormat::CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, + bool with_names_, const FormatSettings & format_settings_) + : RowInputFormatWithDiagnosticInfo(header_, in_, params_) , with_names(with_names_) , format_settings(format_settings_) { @@ -79,72 +80,72 @@ void CSVRowInputFormat::addInputColumn(const String & column_name) column_indexes_for_input_fields.emplace_back(column_index); } -static void skipEndOfLine(ReadBuffer & istr) +static void skipEndOfLine(ReadBuffer & in) { /// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic) - if (*istr.position() == '\n') + if (*in.position() == '\n') { - ++istr.position(); - if (!istr.eof() && *istr.position() == '\r') - ++istr.position(); + ++in.position(); + if (!in.eof() && *in.position() == '\r') + ++in.position(); } - else if (*istr.position() == '\r') + else if (*in.position() == '\r') { - ++istr.position(); - if (!istr.eof() && *istr.position() == '\n') - ++istr.position(); + ++in.position(); + if (!in.eof() && *in.position() == '\n') + ++in.position(); else throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)." " Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA); } - else if (!istr.eof()) + else if (!in.eof()) throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA); } -static void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column) +static void skipDelimiter(ReadBuffer & in, const char delimiter, bool is_last_column) { if (is_last_column) { - if (istr.eof()) + if (in.eof()) return; /// we support the extra delimiter at the end of the line - if (*istr.position() == delimiter) + if (*in.position() == delimiter) { - ++istr.position(); - if (istr.eof()) + ++in.position(); + if (in.eof()) return; } - skipEndOfLine(istr); + skipEndOfLine(in); } else - assertChar(delimiter, istr); + assertChar(delimiter, in); } /// Skip `whitespace` symbols allowed in CSV. -static inline void skipWhitespacesAndTabs(ReadBuffer & buf) +static inline void skipWhitespacesAndTabs(ReadBuffer & in) { - while (!buf.eof() - && (*buf.position() == ' ' - || *buf.position() == '\t')) - ++buf.position(); + while (!in.eof() + && (*in.position() == ' ' + || *in.position() == '\t')) + ++in.position(); } -static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns) +static void skipRow(ReadBuffer & in, const FormatSettings::CSV & settings, size_t num_columns) { String tmp; for (size_t i = 0; i < num_columns; ++i) { - skipWhitespacesAndTabs(istr); - readCSVString(tmp, istr, settings); - skipWhitespacesAndTabs(istr); + skipWhitespacesAndTabs(in); + readCSVString(tmp, in, settings); + skipWhitespacesAndTabs(in); - skipDelimiter(istr, settings.delimiter, i + 1 == num_columns); + skipDelimiter(in, settings.delimiter, i + 1 == num_columns); } } @@ -156,7 +157,6 @@ void CSVRowInputFormat::readPrefix() skipBOMIfExists(in); size_t num_columns = data_types.size(); - String tmp; auto & header = getPort().getHeader(); if (with_names) @@ -224,8 +224,7 @@ bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) { const auto & table_column = column_indexes_for_input_fields[file_column]; - const bool is_last_file_column = - file_column + 1 == column_indexes_for_input_fields.size(); + const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size(); if (table_column) { @@ -267,71 +266,7 @@ bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext return true; } - -String CSVRowInputFormat::getDiagnosticInfo() -{ - if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed. - return {}; - - WriteBufferFromOwnString out; - - auto & header = getPort().getHeader(); - MutableColumns columns = header.cloneEmptyColumns(); - - /// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer. - size_t bytes_read_at_start_of_buffer = in.count() - in.offset(); - if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row) - { - out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n"; - return out.str(); - } - - size_t max_length_of_column_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).name.size() > max_length_of_column_name) - max_length_of_column_name = header.safeGetByPosition(i).name.size(); - - size_t max_length_of_data_type_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name) - max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size(); - - /// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information. - - if (pos_of_prev_row) - { - in.position() = pos_of_prev_row; - - out << "\nRow " << (row_num - 1) << ":\n"; - if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name)) - return out.str(); - } - else - { - if (!pos_of_current_row) - { - out << "Could not print diagnostic info because parsing of data hasn't started.\n"; - return out.str(); - } - - in.position() = pos_of_current_row; - } - - out << "\nRow " << row_num << ":\n"; - parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name); - out << "\n"; - - return out.str(); -} - -/** gcc-7 generates wrong code with optimization level greater than 1. - * See tests: dbms/src/IO/tests/write_int.cpp - * and dbms/tests/queries/0_stateless/00898_parsing_bad_diagnostic_message.sh - * This is compiler bug. The bug does not present in gcc-8 and clang-8. - * Nevertheless, we don't need high optimization of this function. - */ -bool OPTIMIZE(1) CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, - WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name) +bool CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) { const char delimiter = format_settings.csv.delimiter; @@ -345,100 +280,19 @@ bool OPTIMIZE(1) CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumn if (column_indexes_for_input_fields[file_column].has_value()) { - const auto & table_column = *column_indexes_for_input_fields[file_column]; - const auto & current_column_type = data_types[table_column]; - const bool is_last_file_column = - file_column + 1 == column_indexes_for_input_fields.size(); - const bool at_delimiter = !in.eof() && *in.position() == delimiter; - const bool at_last_column_line_end = is_last_file_column - && (in.eof() || *in.position() == '\n' || *in.position() == '\r'); - auto & header = getPort().getHeader(); - out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ') - << "name: " << header.safeGetByPosition(table_column).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(table_column).name.size(), ' ') - << "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' '); - - if (format_settings.csv.empty_as_default - && (at_delimiter || at_last_column_line_end)) - { - columns[table_column]->insertDefault(); - } - else - { - BufferBase::Position prev_position = in.position(); - BufferBase::Position curr_position = in.position(); - std::exception_ptr exception; - - try - { - skipWhitespacesAndTabs(in); - prev_position = in.position(); - readField(*columns[table_column], current_column_type, is_last_file_column, table_column); - curr_position = in.position(); - skipWhitespacesAndTabs(in); - } - catch (...) - { - exception = std::current_exception(); - } - - if (curr_position < prev_position) - throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); - - if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type)) - { - /// An empty string instead of a value. - if (curr_position == prev_position) - { - out << "ERROR: text "; - verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out); - out << " is not like " << current_column_type->getName() << "\n"; - return false; - } - } - - out << "parsed text: "; - verbosePrintString(prev_position, curr_position, out); - - if (exception) - { - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - else - out << "ERROR\n"; - return false; - } - - out << "\n"; - - if (current_column_type->haveMaximumSizeOfValue() - && *curr_position != '\n' && *curr_position != '\r' - && *curr_position != delimiter) - { - out << "ERROR: garbage after " << current_column_type->getName() << ": "; - verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out); - out << "\n"; - - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - - return false; - } - } + size_t col_idx = column_indexes_for_input_fields[file_column].value(); + if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx], + out, file_column)) + return false; } else { static const String skipped_column_str = ""; - out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ') - << "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ') - << "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' '); - - String tmp; - readCSVString(tmp, in, format_settings.csv); + static const DataTypePtr skipped_column_type = std::make_shared(); + static const MutableColumnPtr skipped_column = skipped_column_type->createColumn(); + if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column)) + return false; } /// Delimiters @@ -502,15 +356,26 @@ void CSVRowInputFormat::syncAfterError() skipToNextLineOrEOF(in); } -void CSVRowInputFormat::updateDiagnosticInfo() +void CSVRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, + ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) { - ++row_num; + skipWhitespacesAndTabs(in); + prev_pos = in.position(); - bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row; - bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset(); + if (column_indexes_for_input_fields[file_column]) + { + const bool is_last_file_column = file_column + 1 == column_indexes_for_input_fields.size(); + if (!readField(column, type, is_last_file_column, *column_indexes_for_input_fields[file_column])) + column.insertDefault(); + } + else + { + String tmp; + readCSVString(tmp, in, format_settings.csv); + } - pos_of_prev_row = pos_of_current_row; - pos_of_current_row = in.position(); + curr_pos = in.position(); + skipWhitespacesAndTabs(in); } bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx) @@ -563,7 +428,7 @@ void registerInputFormatProcessorCSV(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, std::move(params), with_names, settings); + return std::make_shared(sample, buf, params, with_names, settings); }); } } diff --git a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h index 59b24ae0140..b8a3a956e1e 100644 --- a/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/CSVRowInputFormat.h @@ -1,40 +1,38 @@ #pragma once +#include +#include + #include -#include +#include #include namespace DB { -class ReadBuffer; - /** A stream for inputting data in csv format. * Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values. */ -class CSVRowInputFormat : public IRowInputFormat +class CSVRowInputFormat : public RowInputFormatWithDiagnosticInfo { public: /** with_names - in the first line the header with column names - * with_types - on the next line header with type names */ - CSVRowInputFormat(ReadBuffer & in_, Block header_, Params params_, bool with_names_, const FormatSettings & format_settings_); + CSVRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, + bool with_names_, const FormatSettings & format_settings_); String getName() const override { return "CSVRowInputFormat"; } - bool readRow(MutableColumns & columns, RowReadExtension &) override; + bool readRow(MutableColumns & columns, RowReadExtension & ext) override; void readPrefix() override; bool allowSyncAfterError() const override { return true; } void syncAfterError() override; - std::string getDiagnosticInfo() override; - private: bool with_names; - DataTypes data_types; - const FormatSettings format_settings; + DataTypes data_types; using IndexesMap = std::unordered_map; IndexesMap column_indexes_by_names; @@ -43,7 +41,7 @@ private: using OptionalIndexes = std::vector>; OptionalIndexes column_indexes_for_input_fields; - /// Tracks which colums we have read in a single read() call. + /// Tracks which columns we have read in a single read() call. /// For columns that are never read, it is initialized to false when we /// read the file header, and never changed afterwards. /// For other columns, it is updated on each read() call. @@ -55,26 +53,19 @@ private: void addInputColumn(const String & column_name); - /// For convenient diagnostics in case of an error. - size_t row_num = 0; - - /// How many bytes were read, not counting those that are still in the buffer. - size_t bytes_read_at_start_of_buffer_on_current_row = 0; - size_t bytes_read_at_start_of_buffer_on_prev_row = 0; - - char * pos_of_current_row = nullptr; - char * pos_of_prev_row = nullptr; + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; + void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, + ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) override; + bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override + { + return *pos != '\n' && *pos != '\r' && *pos != format_settings.csv.delimiter; + } /// For setting input_format_null_as_default DataTypes nullable_types; MutableColumns nullable_columns; OptionalIndexes column_idx_to_nullable_column_idx; - void updateDiagnosticInfo(); - - bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, - WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name); - bool readField(IColumn & column, const DataTypePtr & type, bool is_last_file_column, size_t column_idx); }; diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp index 39c06c9441b..cb9ff5b53be 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.cpp @@ -5,6 +5,7 @@ #include #include #include +#include namespace DB { @@ -16,23 +17,23 @@ namespace ErrorCodes } -static void skipTSVRow(ReadBuffer & istr, const size_t num_columns) +static void skipTSVRow(ReadBuffer & in, const size_t num_columns) { NullSink null_sink; for (size_t i = 0; i < num_columns; ++i) { - readEscapedStringInto(null_sink, istr); - assertChar(i == num_columns - 1 ? '\n' : '\t', istr); + readEscapedStringInto(null_sink, in); + assertChar(i == num_columns - 1 ? '\n' : '\t', in); } } /** Check for a common error case - usage of Windows line feed. */ -static void checkForCarriageReturn(ReadBuffer & istr) +static void checkForCarriageReturn(ReadBuffer & in) { - if (istr.position()[0] == '\r' || (istr.position() != istr.buffer().begin() && istr.position()[-1] == '\r')) + if (in.position()[0] == '\r' || (in.position() != in.buffer().begin() && in.position()[-1] == '\r')) throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row." "\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format." " You must transform your file to Unix format." @@ -41,9 +42,9 @@ static void checkForCarriageReturn(ReadBuffer & istr) } -TabSeparatedRowInputFormat::TabSeparatedRowInputFormat( - ReadBuffer & in_, Block header_, bool with_names_, bool with_types_, Params params_, const FormatSettings & format_settings_) - : IRowInputFormat(std::move(header_), in_, std::move(params_)), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) +TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, + bool with_names_, bool with_types_, const FormatSettings & format_settings_) + : RowInputFormatWithDiagnosticInfo(header_, in_, params_), with_names(with_names_), with_types(with_types_), format_settings(format_settings_) { auto & sample = getPort().getHeader(); size_t num_columns = sample.columns(); @@ -173,9 +174,9 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens updateDiagnosticInfo(); - for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position) + for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) { - const auto & column_index = column_indexes_for_input_fields[input_position]; + const auto & column_index = column_indexes_for_input_fields[file_column]; if (column_index) { data_types[*column_index]->deserializeAsTextEscaped(*columns[*column_index], in, format_settings); @@ -187,7 +188,7 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens } /// skip separators - if (input_position + 1 < column_indexes_for_input_fields.size()) + if (file_column + 1 < column_indexes_for_input_fields.size()) { assertChar('\t', in); } @@ -205,160 +206,35 @@ bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtens return true; } - -String TabSeparatedRowInputFormat::getDiagnosticInfo() +bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) { - if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed. - return {}; - - auto & header = getPort().getHeader(); - WriteBufferFromOwnString out; - MutableColumns columns = header.cloneEmptyColumns(); - - /// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer. - size_t bytes_read_at_start_of_buffer = in.count() - in.offset(); - if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row) + for (size_t file_column = 0; file_column < column_indexes_for_input_fields.size(); ++file_column) { - out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n"; - return out.str(); - } - - size_t max_length_of_column_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).name.size() > max_length_of_column_name) - max_length_of_column_name = header.safeGetByPosition(i).name.size(); - - size_t max_length_of_data_type_name = 0; - for (size_t i = 0; i < header.columns(); ++i) - if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name) - max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size(); - - /// Roll back the cursor to the beginning of the previous or current line and pars all over again. But now we derive detailed information. - - if (pos_of_prev_row) - { - in.position() = pos_of_prev_row; - - out << "\nRow " << (row_num - 1) << ":\n"; - if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name)) - return out.str(); - } - else - { - if (!pos_of_current_row) - { - out << "Could not print diagnostic info because parsing of data hasn't started.\n"; - return out.str(); - } - - in.position() = pos_of_current_row; - } - - out << "\nRow " << row_num << ":\n"; - parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name); - out << "\n"; - - return out.str(); -} - - -bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, - WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name) -{ - for (size_t input_position = 0; input_position < column_indexes_for_input_fields.size(); ++input_position) - { - if (input_position == 0 && in.eof()) + if (file_column == 0 && in.eof()) { out << "\n"; return false; } - if (column_indexes_for_input_fields[input_position].has_value()) + if (column_indexes_for_input_fields[file_column].has_value()) { - const auto & column_index = *column_indexes_for_input_fields[input_position]; - const auto & current_column_type = data_types[column_index]; - - const auto & header = getPort().getHeader(); - - out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ') - << "name: " << header.safeGetByPosition(column_index).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(column_index).name.size(), ' ') - << "type: " << current_column_type->getName() << ", " << std::string(max_length_of_data_type_name - current_column_type->getName().size(), ' '); - - auto prev_position = in.position(); - std::exception_ptr exception; - - try - { - current_column_type->deserializeAsTextEscaped(*columns[column_index], in, format_settings); - } - catch (...) - { - exception = std::current_exception(); - } - - auto curr_position = in.position(); - - if (curr_position < prev_position) - throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); - - if (isNativeNumber(current_column_type) || isDateOrDateTime(current_column_type)) - { - /// An empty string instead of a value. - if (curr_position == prev_position) - { - out << "ERROR: text "; - verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out); - out << " is not like " << current_column_type->getName() << "\n"; - return false; - } - } - - out << "parsed text: "; - verbosePrintString(prev_position, curr_position, out); - - if (exception) - { - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - else - out << "ERROR\n"; + auto & header = getPort().getHeader(); + size_t col_idx = column_indexes_for_input_fields[file_column].value(); + if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], *columns[col_idx], + out, file_column)) return false; - } - - out << "\n"; - - if (current_column_type->haveMaximumSizeOfValue()) - { - if (*curr_position != '\n' && *curr_position != '\t') - { - out << "ERROR: garbage after " << current_column_type->getName() << ": "; - verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out); - out << "\n"; - - if (current_column_type->getName() == "DateTime") - out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; - else if (current_column_type->getName() == "Date") - out << "ERROR: Date must be in YYYY-MM-DD format.\n"; - - return false; - } - } } else { static const String skipped_column_str = ""; - out << "Column " << input_position << ", " << std::string((input_position < 10 ? 2 : input_position < 100 ? 1 : 0), ' ') - << "name: " << skipped_column_str << ", " << std::string(max_length_of_column_name - skipped_column_str.length(), ' ') - << "type: " << skipped_column_str << ", " << std::string(max_length_of_data_type_name - skipped_column_str.length(), ' '); - - NullSink null_sink; - readEscapedStringInto(null_sink, in); + static const DataTypePtr skipped_column_type = std::make_shared(); + static const MutableColumnPtr skipped_column = skipped_column_type->createColumn(); + if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, file_column)) + return false; } /// Delimiters - if (input_position + 1 == column_indexes_for_input_fields.size()) + if (file_column + 1 == column_indexes_for_input_fields.size()) { if (!in.eof()) { @@ -401,7 +277,8 @@ bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & { out << "ERROR: Line feed found where tab is expected." " It's like your file has less columns than expected.\n" - "And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n"; + "And if your file have right number of columns, " + "maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n"; } else if (*in.position() == '\r') { @@ -421,6 +298,19 @@ bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & return true; } +void TabSeparatedRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, + ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) +{ + prev_pos = in.position(); + if (column_indexes_for_input_fields[file_column]) + type->deserializeAsTextEscaped(column, in, format_settings); + else + { + NullSink null_sink; + readEscapedStringInto(null_sink, in); + } + curr_pos = in.position(); +} void TabSeparatedRowInputFormat::syncAfterError() { @@ -428,18 +318,6 @@ void TabSeparatedRowInputFormat::syncAfterError() } -void TabSeparatedRowInputFormat::updateDiagnosticInfo() -{ - ++row_num; - - bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row; - bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset(); - - pos_of_prev_row = pos_of_current_row; - pos_of_current_row = in.position(); -} - - void registerInputFormatProcessorTabSeparated(FormatFactory & factory) { for (auto name : {"TabSeparated", "TSV"}) @@ -451,7 +329,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, false, false, std::move(params), settings); + return std::make_shared(sample, buf, params, false, false, settings); }); } @@ -464,7 +342,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, true, false, std::move(params), settings); + return std::make_shared(sample, buf, params, true, false, settings); }); } @@ -477,7 +355,7 @@ void registerInputFormatProcessorTabSeparated(FormatFactory & factory) IRowInputFormat::Params params, const FormatSettings & settings) { - return std::make_shared(buf, sample, true, true, std::move(params), settings); + return std::make_shared(sample, buf, params, true, true, settings); }); } } diff --git a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h index 076cbb60152..a28ac62ed4f 100644 --- a/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h +++ b/dbms/src/Processors/Formats/Impl/TabSeparatedRowInputFormat.h @@ -2,25 +2,22 @@ #include #include -#include +#include namespace DB { -class ReadBuffer; - - /** A stream to input data in tsv format. */ -class TabSeparatedRowInputFormat : public IRowInputFormat +class TabSeparatedRowInputFormat : public RowInputFormatWithDiagnosticInfo { public: /** with_names - the first line is the header with the names of the columns * with_types - on the next line header with type names */ - TabSeparatedRowInputFormat( - ReadBuffer & in_, Block header_, bool with_names_, bool with_types_, Params params_, const FormatSettings & format_settings_); + TabSeparatedRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, + bool with_names_, bool with_types_, const FormatSettings & format_settings_); String getName() const override { return "TabSeparatedRowInputFormat"; } @@ -29,8 +26,6 @@ public: bool allowSyncAfterError() const override { return true; } void syncAfterError() override; - std::string getDiagnosticInfo() override; - private: bool with_names; bool with_types; @@ -50,21 +45,10 @@ private: void setupAllColumnsByTableSchema(); void fillUnreadColumnsWithDefaults(MutableColumns & columns, RowReadExtension& ext); - /// For convenient diagnostics in case of an error. - - size_t row_num = 0; - - /// How many bytes were read, not counting those still in the buffer. - size_t bytes_read_at_start_of_buffer_on_current_row = 0; - size_t bytes_read_at_start_of_buffer_on_prev_row = 0; - - char * pos_of_current_row = nullptr; - char * pos_of_prev_row = nullptr; - - void updateDiagnosticInfo(); - - bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, - WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name); + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; + void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, + ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) override; + bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; } }; } diff --git a/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp new file mode 100644 index 00000000000..cbaef1b0012 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.cpp @@ -0,0 +1,243 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int SYNTAX_ERROR; +} + +TemplateBlockOutputFormat::TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_) + : IOutputFormat(header_, out_), settings(settings_) +{ + auto & sample = getPort(PortKind::Main).getHeader(); + size_t columns = sample.columns(); + types.resize(columns); + for (size_t i = 0; i < columns; ++i) + types[i] = sample.safeGetByPosition(i).type; + + /// Parse format string for whole output + static const String default_format("${data}"); + const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format; + format = ParsedTemplateFormatString(format_str, [&](const String & partName) + { + return static_cast(stringToOutputPart(partName)); + }); + + /// Validate format string for whole output + size_t data_idx = format.format_idx_to_column_idx.size() + 1; + for (size_t i = 0; i < format.format_idx_to_column_idx.size(); ++i) + { + if (!format.format_idx_to_column_idx[i]) + format.throwInvalidFormat("Output part name cannot be empty, it's a bug.", i); + switch (static_cast(*format.format_idx_to_column_idx[i])) + { + case OutputPart::Data: + data_idx = i; + [[fallthrough]]; + case OutputPart::Totals: + case OutputPart::ExtremesMin: + case OutputPart::ExtremesMax: + if (format.formats[i] != ColumnFormat::None) + format.throwInvalidFormat("Serialization type for data, totals, min and max must be empty or None", i); + break; + default: + if (format.formats[i] == ColumnFormat::None) + format.throwInvalidFormat("Serialization type for output part rows, rows_before_limit, time, " + "rows_read or bytes_read is not specified", i); + break; + } + } + if (data_idx != 0) + format.throwInvalidFormat("${data} must be the first output part", 0); + + /// Parse format string for rows + row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName) + { + return sample.getPositionByName(colName); + }); + + /// Validate format string for rows + if (row_format.delimiters.size() == 1) + row_format.throwInvalidFormat("No columns specified", 0); + for (size_t i = 0; i < row_format.columnsCount(); ++i) + { + if (!row_format.format_idx_to_column_idx[i]) + row_format.throwInvalidFormat("Cannot skip format field for output, it's a bug.", i); + if (row_format.formats[i] == ColumnFormat::None) + row_format.throwInvalidFormat("Serialization type for file column is not specified", i); + } +} + +TemplateBlockOutputFormat::OutputPart TemplateBlockOutputFormat::stringToOutputPart(const String & part) +{ + if (part == "data") + return OutputPart::Data; + else if (part == "totals") + return OutputPart::Totals; + else if (part == "min") + return OutputPart::ExtremesMin; + else if (part == "max") + return OutputPart::ExtremesMax; + else if (part == "rows") + return OutputPart::Rows; + else if (part == "rows_before_limit") + return OutputPart::RowsBeforeLimit; + else if (part == "time") + return OutputPart::TimeElapsed; + else if (part == "rows_read") + return OutputPart::RowsRead; + else if (part == "bytes_read") + return OutputPart::BytesRead; + else + throw Exception("Unknown output part " + part, ErrorCodes::SYNTAX_ERROR); +} + +void TemplateBlockOutputFormat::writeRow(const Chunk & chunk, size_t row_num) +{ + size_t columns = row_format.format_idx_to_column_idx.size(); + for (size_t j = 0; j < columns; ++j) + { + writeString(row_format.delimiters[j], out); + + size_t col_idx = *row_format.format_idx_to_column_idx[j]; + serializeField(*chunk.getColumns()[col_idx], *types[col_idx], row_num, row_format.formats[j]); + } + writeString(row_format.delimiters[columns], out); +} + +void TemplateBlockOutputFormat::serializeField(const IColumn & column, const IDataType & type, size_t row_num, ColumnFormat col_format) +{ + switch (col_format) + { + case ColumnFormat::Escaped: + type.serializeAsTextEscaped(column, row_num, out, settings); + break; + case ColumnFormat::Quoted: + type.serializeAsTextQuoted(column, row_num, out, settings); + break; + case ColumnFormat::Csv: + type.serializeAsTextCSV(column, row_num, out, settings); + break; + case ColumnFormat::Json: + type.serializeAsTextJSON(column, row_num, out, settings); + break; + case ColumnFormat::Xml: + type.serializeAsTextXML(column, row_num, out, settings); + break; + case ColumnFormat::Raw: + type.serializeAsText(column, row_num, out, settings); + break; + default: + __builtin_unreachable(); + } +} + +template void TemplateBlockOutputFormat::writeValue(U value, ColumnFormat col_format) +{ + auto type = std::make_unique(); + auto col = type->createColumn(); + col->insert(value); + serializeField(*col, *type, 0, col_format); +} + +void TemplateBlockOutputFormat::consume(Chunk chunk) +{ + doWritePrefix(); + + size_t rows = chunk.getNumRows(); + + for (size_t i = 0; i < rows; ++i) + { + if (row_count) + writeString(settings.template_settings.row_between_delimiter, out); + + writeRow(chunk, i); + ++row_count; + } +} + +void TemplateBlockOutputFormat::doWritePrefix() +{ + if (need_write_prefix) + { + writeString(format.delimiters.front(), out); + need_write_prefix = false; + } +} + +void TemplateBlockOutputFormat::finalize() +{ + if (finalized) + return; + + doWritePrefix(); + + size_t parts = format.format_idx_to_column_idx.size(); + + for (size_t i = 0; i < parts; ++i) + { + auto type = std::make_shared(); + ColumnWithTypeAndName col(type->createColumnConst(1, row_count), type, String("tmp")); + switch (static_cast(*format.format_idx_to_column_idx[i])) + { + case OutputPart::Totals: + if (!totals) + format.throwInvalidFormat("Cannot print totals for this request", i); + writeRow(totals, 0); + break; + case OutputPart::ExtremesMin: + if (!extremes) + format.throwInvalidFormat("Cannot print extremes for this request", i); + writeRow(extremes, 0); + break; + case OutputPart::ExtremesMax: + if (!extremes) + format.throwInvalidFormat("Cannot print extremes for this request", i); + writeRow(extremes, 1); + break; + case OutputPart::Rows: + writeValue(row_count, format.formats[i]); + break; + case OutputPart::RowsBeforeLimit: + if (!rows_before_limit_set) + format.throwInvalidFormat("Cannot print rows_before_limit for this request", i); + writeValue(rows_before_limit, format.formats[i]); + break; + case OutputPart::TimeElapsed: + writeValue(watch.elapsedSeconds(), format.formats[i]); + break; + case OutputPart::RowsRead: + writeValue(progress.read_rows.load(), format.formats[i]); + break; + case OutputPart::BytesRead: + writeValue(progress.read_bytes.load(), format.formats[i]); + break; + default: + break; + } + writeString(format.delimiters[i + 1], out); + } + + finalized = true; +} + + +void registerOutputFormatProcessorTemplate(FormatFactory & factory) +{ + factory.registerOutputFormatProcessor("Template", []( + WriteBuffer & buf, + const Block & sample, + const Context &, + FormatFactory::WriteCallback, + const FormatSettings & settings) + { + return std::make_shared(sample, buf, settings); + }); +} +} diff --git a/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h new file mode 100644 index 00000000000..9eb5f61d4e7 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TemplateBlockOutputFormat.h @@ -0,0 +1,68 @@ +#pragma once + +#include +#include +#include +#include +#include + + +namespace DB +{ + +class TemplateBlockOutputFormat : public IOutputFormat +{ + using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; +public: + TemplateBlockOutputFormat(const Block & header_, WriteBuffer & out_, const FormatSettings & settings_); + + String getName() const override { return "TemplateBlockOutputFormat"; } + + void doWritePrefix() override; + + void setRowsBeforeLimit(size_t rows_before_limit_) override { rows_before_limit = rows_before_limit_; rows_before_limit_set = true; } + void onProgress(const Progress & progress_) override { progress.incrementPiecewiseAtomically(progress_); } + +protected: + void consume(Chunk chunk) override; + void consumeTotals(Chunk chunk) override { totals = std::move(chunk); } + void consumeExtremes(Chunk chunk) override { extremes = std::move(chunk); } + void finalize() override; + + enum class OutputPart : size_t + { + Data, + Totals, + ExtremesMin, + ExtremesMax, + Rows, + RowsBeforeLimit, + TimeElapsed, + RowsRead, + BytesRead + }; + + OutputPart stringToOutputPart(const String & part); + void writeRow(const Chunk & chunk, size_t row_num); + void serializeField(const IColumn & column, const IDataType & type, size_t row_num, ColumnFormat format); + template void writeValue(U value, ColumnFormat col_format); + +protected: + const FormatSettings settings; + DataTypes types; + + ParsedTemplateFormatString format; + ParsedTemplateFormatString row_format; + + size_t rows_before_limit = 0; + bool rows_before_limit_set = false; + Chunk totals; + Chunk extremes; + Progress progress; + Stopwatch watch; + + size_t row_count = 0; + bool need_write_prefix = true; +}; + +} diff --git a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp new file mode 100644 index 00000000000..c2145f88e69 --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.cpp @@ -0,0 +1,520 @@ +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ +extern const int ATTEMPT_TO_READ_AFTER_EOF; +extern const int CANNOT_READ_ALL_DATA; +extern const int CANNOT_PARSE_ESCAPE_SEQUENCE; +extern const int CANNOT_PARSE_QUOTED_STRING; +extern const int SYNTAX_ERROR; +} + + +TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, + const FormatSettings & settings_, bool ignore_spaces_) + : RowInputFormatWithDiagnosticInfo(header_, buf, params_), buf(in_), data_types(header_.getDataTypes()), + settings(settings_), ignore_spaces(ignore_spaces_) +{ + /// Parse format string for whole input + static const String default_format("${data}"); + const String & format_str = settings.template_settings.format.empty() ? default_format : settings.template_settings.format; + format = ParsedTemplateFormatString(format_str, [&](const String & partName) -> std::optional + { + if (partName == "data") + return 0; + else if (partName.empty()) /// For skipping some values in prefix and suffix +#if !__clang__ +#pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wmaybe-uninitialized" +#endif + /// Suppress false-positive warning (bug in GCC 9: https://gcc.gnu.org/bugzilla/show_bug.cgi?id=86465) + return {}; +#if !__clang__ +#pragma GCC diagnostic pop +#endif + throw Exception("Unknown input part " + partName, ErrorCodes::SYNTAX_ERROR); + }); + + /// Validate format string for whole input + bool has_data = false; + for (size_t i = 0; i < format.columnsCount(); ++i) + { + if (format.format_idx_to_column_idx[i]) + { + if (has_data) + format.throwInvalidFormat("${data} can occur only once", i); + if (format.formats[i] != ColumnFormat::None) + format.throwInvalidFormat("${data} must have empty or None deserialization type", i); + has_data = true; + format_data_idx = i; + } + else + { + if (format.formats[i] == ColumnFormat::Xml || format.formats[i] == ColumnFormat::Raw) + format.throwInvalidFormat("XML and Raw deserialization is not supported", i); + } + } + + /// Parse format string for rows + row_format = ParsedTemplateFormatString(settings.template_settings.row_format, [&](const String & colName) -> std::optional + { + if (colName.empty()) +#if !__clang__ + #pragma GCC diagnostic push +#pragma GCC diagnostic ignored "-Wmaybe-uninitialized" +#endif + return {}; +#if !__clang__ +#pragma GCC diagnostic pop +#endif + return header_.getPositionByName(colName); + }); + + /// Validate format string for rows + std::vector column_in_format(header_.columns(), false); + for (size_t i = 0; i < row_format.columnsCount(); ++i) + { + if (row_format.formats[i] == ColumnFormat::Xml || row_format.formats[i] == ColumnFormat::Raw) + row_format.throwInvalidFormat("XML and Raw deserialization is not supported", i); + + if (row_format.format_idx_to_column_idx[i]) + { + if (row_format.formats[i] == ColumnFormat::None) + row_format.throwInvalidFormat("Column is not skipped, but deserialization type is None", i); + + size_t col_idx = *row_format.format_idx_to_column_idx[i]; + if (column_in_format[col_idx]) + row_format.throwInvalidFormat("Duplicate column", i); + column_in_format[col_idx] = true; + } + } +} + +void TemplateRowInputFormat::readPrefix() +{ + size_t last_successfully_parsed_idx = 0; + try + { + tryReadPrefixOrSuffix(last_successfully_parsed_idx, format_data_idx); + } + catch (Exception & e) + { + format.throwInvalidFormat(e.message() + " While parsing prefix", last_successfully_parsed_idx); + } +} + +/// Asserts delimiters and skips fields in prefix or suffix. +/// tryReadPrefixOrSuffix(...) is used in checkForSuffix() to avoid throwing an exception after read of each row +/// (most likely false will be returned on first call of checkString(...)) +template +ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end) +{ + static constexpr bool throw_exception = std::is_same_v; + + skipSpaces(); + if constexpr (throw_exception) + assertString(format.delimiters[input_part_beg], buf); + else + { + if (likely(!checkString(format.delimiters[input_part_beg], buf))) + return ReturnType(false); + } + + while (input_part_beg < input_part_end) + { + skipSpaces(); + if constexpr (throw_exception) + skipField(format.formats[input_part_beg]); + else + { + try + { + skipField(format.formats[input_part_beg]); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && + e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE && + e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING) + throw; + /// If it's parsing error, then suffix is not found + return ReturnType(false); + } + } + ++input_part_beg; + + skipSpaces(); + if constexpr (throw_exception) + assertString(format.delimiters[input_part_beg], buf); + else + { + if (likely(!checkString(format.delimiters[input_part_beg], buf))) + return ReturnType(false); + } + } + + if constexpr (!throw_exception) + return ReturnType(true); +} + +bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & extra) +{ + /// This function can be called again after it returned false + if (unlikely(end_of_stream)) + return false; + + skipSpaces(); + + if (unlikely(checkForSuffix())) + { + end_of_stream = true; + return false; + } + + updateDiagnosticInfo(); + + if (likely(row_num != 1)) + assertString(settings.template_settings.row_between_delimiter, buf); + + extra.read_columns.assign(columns.size(), false); + + for (size_t i = 0; i < row_format.columnsCount(); ++i) + { + skipSpaces(); + assertString(row_format.delimiters[i], buf); + skipSpaces(); + if (row_format.format_idx_to_column_idx[i]) + { + size_t col_idx = *row_format.format_idx_to_column_idx[i]; + deserializeField(*data_types[col_idx], *columns[col_idx], row_format.formats[i]); + extra.read_columns[col_idx] = true; + } + else + skipField(row_format.formats[i]); + + } + + skipSpaces(); + assertString(row_format.delimiters.back(), buf); + + for (size_t i = 0; i < columns.size(); ++i) + if (!extra.read_columns[i]) + data_types[i]->insertDefaultInto(*columns[i]); + + return true; +} + +void TemplateRowInputFormat::deserializeField(const IDataType & type, IColumn & column, ColumnFormat col_format) +{ + try + { + switch (col_format) + { + case ColumnFormat::Escaped: + type.deserializeAsTextEscaped(column, buf, settings); + break; + case ColumnFormat::Quoted: + type.deserializeAsTextQuoted(column, buf, settings); + break; + case ColumnFormat::Csv: + type.deserializeAsTextCSV(column, buf, settings); + break; + case ColumnFormat::Json: + type.deserializeAsTextJSON(column, buf, settings); + break; + default: + __builtin_unreachable(); + } + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throwUnexpectedEof(); + throw; + } +} + +void TemplateRowInputFormat::skipField(TemplateRowInputFormat::ColumnFormat col_format) +{ + String tmp; + constexpr const char * field_name = ""; + constexpr size_t field_name_len = 16; + try + { + switch (col_format) + { + case ColumnFormat::None: + /// Empty field, just skip spaces + break; + case ColumnFormat::Escaped: + readEscapedString(tmp, buf); + break; + case ColumnFormat::Quoted: + readQuotedString(tmp, buf); + break; + case ColumnFormat::Csv: + readCSVString(tmp, buf, settings.csv); + break; + case ColumnFormat::Json: + skipJSONField(buf, StringRef(field_name, field_name_len)); + break; + default: + __builtin_unreachable(); + } + } + catch (Exception & e) + { + if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF) + throwUnexpectedEof(); + throw; + } +} + +/// Returns true if all rows have been read i.e. there are only suffix and spaces (if ignore_spaces == true) before EOF. +/// Otherwise returns false +bool TemplateRowInputFormat::checkForSuffix() +{ + PeekableReadBufferCheckpoint checkpoint{buf}; + bool suffix_found = false; + size_t last_successfully_parsed_idx = format_data_idx + 1; + try + { + suffix_found = tryReadPrefixOrSuffix(last_successfully_parsed_idx, format.columnsCount()); + } + catch (const Exception & e) + { + if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF && + e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE && + e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING) + throw; + } + + if (unlikely(suffix_found)) + { + skipSpaces(); + if (buf.eof()) + return true; + } + + buf.rollbackToCheckpoint(); + return false; +} + +bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) +{ + out << "Suffix does not match: "; + size_t last_successfully_parsed_idx = format_data_idx + 1; + const ReadBuffer::Position row_begin_pos = buf.position(); + bool caught = false; + try + { + PeekableReadBufferCheckpoint checkpoint{buf, true}; + tryReadPrefixOrSuffix(last_successfully_parsed_idx, format.columnsCount()); + } + catch (Exception & e) + { + out << e.message() << " Near column " << last_successfully_parsed_idx; + caught = true; + } + if (!caught) + { + out << " There is some data after suffix (EOF expected, got "; + verbosePrintString(buf.position(), std::min(buf.buffer().end(), buf.position() + 16), out); + out << "). "; + } + out << " Format string (from format_schema): \n" << format.dump() << "\n"; + + if (row_begin_pos != buf.position()) + { + /// Pointers to buffer memory were invalidated during checking for suffix + out << "\nCannot print more diagnostic info."; + return false; + } + + out << "\nUsing format string (from format_schema_rows): " << row_format.dump() << "\n"; + out << "\nTrying to parse next row, because suffix does not match:\n"; + try + { + if (likely(row_num != 1)) + assertString(settings.template_settings.row_between_delimiter, buf); + } + catch (const DB::Exception &) + { + writeErrorStringForWrongDelimiter(out, "delimiter between rows", settings.template_settings.row_between_delimiter); + + return false; + } + for (size_t i = 0; i < row_format.columnsCount(); ++i) + { + skipSpaces(); + try + { + assertString(row_format.delimiters[i], buf); + } + catch (const DB::Exception &) + { + writeErrorStringForWrongDelimiter(out, "delimiter before field " + std::to_string(i), row_format.delimiters[i]); + return false; + } + + skipSpaces(); + if (row_format.format_idx_to_column_idx[i]) + { + auto & header = getPort().getHeader(); + size_t col_idx = *row_format.format_idx_to_column_idx[i]; + if (!deserializeFieldAndPrintDiagnosticInfo(header.getByPosition(col_idx).name, data_types[col_idx], + *columns[col_idx], out, i)) + { + out << "Maybe it's not possible to deserialize field " + std::to_string(i) + + " as " + ParsedTemplateFormatString::formatToString(row_format.formats[i]); + return false; + } + } + else + { + static const String skipped_column_str = ""; + static const DataTypePtr skipped_column_type = std::make_shared(); + static const MutableColumnPtr skipped_column = skipped_column_type->createColumn(); + if (!deserializeFieldAndPrintDiagnosticInfo(skipped_column_str, skipped_column_type, *skipped_column, out, i)) + return false; + } + } + + skipSpaces(); + try + { + assertString(row_format.delimiters.back(), buf); + } + catch (const DB::Exception &) + { + writeErrorStringForWrongDelimiter(out, "delimiter after last field", row_format.delimiters.back()); + return false; + } + + return true; +} + +void TemplateRowInputFormat::writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim) +{ + out << "ERROR: There is no " << description << ": expected "; + verbosePrintString(delim.data(), delim.data() + delim.size(), out); + out << ", got "; + if (buf.eof()) + out << ""; + else + verbosePrintString(buf.position(), std::min(buf.position() + delim.size() + 10, buf.buffer().end()), out); + out << '\n'; +} + +void TemplateRowInputFormat::tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, + ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) +{ + prev_pos = buf.position(); + if (row_format.format_idx_to_column_idx[file_column]) + deserializeField(*type, column, row_format.formats[file_column]); + else + skipField(row_format.formats[file_column]); + curr_pos = buf.position(); +} + +bool TemplateRowInputFormat::isGarbageAfterField(size_t, ReadBuffer::Position) +{ + /// Garbage will be considered as wrong delimiter + return false; +} + +bool TemplateRowInputFormat::allowSyncAfterError() const +{ + return !row_format.delimiters.back().empty() || !settings.template_settings.row_between_delimiter.empty(); +} + +void TemplateRowInputFormat::syncAfterError() +{ + bool at_beginning_of_row_or_eof = false; + while (!at_beginning_of_row_or_eof) + { + skipToNextDelimiterOrEof(row_format.delimiters.back()); + if (buf.eof()) + { + end_of_stream = true; + return; + } + buf.ignore(row_format.delimiters.back().size()); + + skipSpaces(); + if (checkForSuffix()) + return; + + bool last_delimiter_in_row_found = !row_format.delimiters.back().empty(); + + if (last_delimiter_in_row_found && checkString(settings.template_settings.row_between_delimiter, buf)) + at_beginning_of_row_or_eof = true; + else + skipToNextDelimiterOrEof(settings.template_settings.row_between_delimiter); + + if (buf.eof()) + at_beginning_of_row_or_eof = end_of_stream = true; + } + /// It can happen that buf.position() is not at the beginning of row + /// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter. + /// It will cause another parsing error. +} + +/// Searches for delimiter in input stream and sets buffer position to the beginning of delimiter (if found) or EOF (if not) +void TemplateRowInputFormat::skipToNextDelimiterOrEof(const String & delimiter) +{ + if (delimiter.empty()) + return; + + while (!buf.eof()) + { + void * pos = memchr(buf.position(), delimiter[0], buf.available()); + if (!pos) + { + buf.position() += buf.available(); + continue; + } + + buf.position() = static_cast(pos); + + PeekableReadBufferCheckpoint checkpoint{buf}; + if (checkString(delimiter, buf)) + return; + + buf.rollbackToCheckpoint(); + ++buf.position(); + } +} + +void TemplateRowInputFormat::throwUnexpectedEof() +{ + throw Exception("Unexpected EOF while parsing row " + std::to_string(row_num) + ". " + "Maybe last row has wrong format or input doesn't contain specified suffix before EOF.", + ErrorCodes::CANNOT_READ_ALL_DATA); +} + + +void registerInputFormatProcessorTemplate(FormatFactory & factory) +{ + for (bool ignore_spaces : {false, true}) + { + factory.registerInputFormatProcessor(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=]( + ReadBuffer & buf, + const Block & sample, + const Context &, + IRowInputFormat::Params params, + const FormatSettings & settings) + { + return std::make_shared(sample, buf, params, settings, ignore_spaces); + }); + } +} + +} diff --git a/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h new file mode 100644 index 00000000000..ff7b2adc34a --- /dev/null +++ b/dbms/src/Processors/Formats/Impl/TemplateRowInputFormat.h @@ -0,0 +1,61 @@ +#pragma once + +#include +#include +#include +#include +#include +#include + + +namespace DB +{ + +class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo +{ + using ColumnFormat = ParsedTemplateFormatString::ColumnFormat; +public: + TemplateRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, + const FormatSettings & settings_, bool ignore_spaces_); + + String getName() const override { return "TemplateRowInputFormat"; } + + bool readRow(MutableColumns & columns, RowReadExtension & extra) override; + + void readPrefix() override; + + bool allowSyncAfterError() const override; + void syncAfterError() override; + +private: + void deserializeField(const IDataType & type, IColumn & column, ColumnFormat col_format); + void skipField(ColumnFormat col_format); + inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(buf); } + + template + ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end); + bool checkForSuffix(); + [[noreturn]] void throwUnexpectedEof(); + + bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override; + void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, ReadBuffer::Position & prev_pos, + ReadBuffer::Position & curr_pos) override; + bool isGarbageAfterField(size_t after_col_idx, ReadBuffer::Position pos) override; + void writeErrorStringForWrongDelimiter(WriteBuffer & out, const String & description, const String & delim); + + void skipToNextDelimiterOrEof(const String & delimiter); + +private: + PeekableReadBuffer buf; + DataTypes data_types; + + FormatSettings settings; + ParsedTemplateFormatString format; + ParsedTemplateFormatString row_format; + const bool ignore_spaces; + + size_t format_data_idx; + bool end_of_stream = false; +}; + +} diff --git a/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp new file mode 100644 index 00000000000..452cfa46acf --- /dev/null +++ b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.cpp @@ -0,0 +1,167 @@ +#include +#include +#include +#include + + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int LOGICAL_ERROR; +} + +DB::RowInputFormatWithDiagnosticInfo::RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_) + : IRowInputFormat(header_, in_, params_) +{ +} + +void DB::RowInputFormatWithDiagnosticInfo::updateDiagnosticInfo() +{ + ++row_num; + + bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row; + bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset(); + + offset_of_prev_row = offset_of_current_row; + offset_of_current_row = in.offset(); +} + +String DB::RowInputFormatWithDiagnosticInfo::getDiagnosticInfo() +{ + if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed. + return {}; + + WriteBufferFromOwnString out; + + auto & header = getPort().getHeader(); + MutableColumns columns = header.cloneEmptyColumns(); + + /// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer. + size_t bytes_read_at_start_of_buffer = in.count() - in.offset(); + if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row) + { + out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n"; + return out.str(); + } + + max_length_of_column_name = 0; + for (size_t i = 0; i < header.columns(); ++i) + if (header.safeGetByPosition(i).name.size() > max_length_of_column_name) + max_length_of_column_name = header.safeGetByPosition(i).name.size(); + + max_length_of_data_type_name = 0; + for (size_t i = 0; i < header.columns(); ++i) + if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name) + max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size(); + + /// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information. + + if (offset_of_prev_row <= in.buffer().size()) + { + in.position() = in.buffer().begin() + offset_of_prev_row; + + out << "\nRow " << (row_num - 1) << ":\n"; + if (!parseRowAndPrintDiagnosticInfo(columns, out)) + return out.str(); + } + else + { + if (in.buffer().size() < offset_of_current_row) + { + out << "Could not print diagnostic info because parsing of data hasn't started.\n"; + return out.str(); + } + + in.position() = in.buffer().begin() + offset_of_current_row; + } + + out << "\nRow " << row_num << ":\n"; + parseRowAndPrintDiagnosticInfo(columns, out); + out << "\n"; + + return out.str(); +} + +bool RowInputFormatWithDiagnosticInfo::deserializeFieldAndPrintDiagnosticInfo(const String & col_name, + const DataTypePtr & type, + IColumn & column, + WriteBuffer & out, + size_t file_column) +{ + out << "Column " << file_column << ", " << std::string((file_column < 10 ? 2 : file_column < 100 ? 1 : 0), ' ') + << "name: " << alignedName(col_name, max_length_of_column_name) + << "type: " << alignedName(type->getName(), max_length_of_data_type_name); + + auto prev_position = in.position(); + auto curr_position = in.position(); + std::exception_ptr exception; + + try + { + tryDeserializeFiled(type, column, file_column, prev_position, curr_position); + } + catch (...) + { + exception = std::current_exception(); + } + + if (curr_position < prev_position) + throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR); + + if (isNativeNumber(type) || isDateOrDateTime(type)) + { + /// An empty string instead of a value. + if (curr_position == prev_position) + { + out << "ERROR: text "; + verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out); + out << " is not like " << type->getName() << "\n"; + return false; + } + } + + out << "parsed text: "; + verbosePrintString(prev_position, curr_position, out); + + if (exception) + { + if (type->getName() == "DateTime") + out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; + else if (type->getName() == "Date") + out << "ERROR: Date must be in YYYY-MM-DD format.\n"; + else + out << "ERROR\n"; + return false; + } + + out << "\n"; + + if (type->haveMaximumSizeOfValue()) + { + if (isGarbageAfterField(file_column, curr_position)) + { + out << "ERROR: garbage after " << type->getName() << ": "; + verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out); + out << "\n"; + + if (type->getName() == "DateTime") + out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n"; + else if (type->getName() == "Date") + out << "ERROR: Date must be in YYYY-MM-DD format.\n"; + + return false; + } + } + + return true; +} + +String RowInputFormatWithDiagnosticInfo::alignedName(const String & name, size_t max_length) const +{ + size_t spaces_count = max_length >= name.size() ? max_length - name.size() : 0; + return name + ", " + std::string(spaces_count, ' '); +} + +} diff --git a/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.h b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.h new file mode 100644 index 00000000000..98dea066436 --- /dev/null +++ b/dbms/src/Processors/Formats/RowInputFormatWithDiagnosticInfo.h @@ -0,0 +1,46 @@ +#pragma once + +#include +#include +#include +#include + + +namespace DB +{ + +class RowInputFormatWithDiagnosticInfo : public IRowInputFormat +{ +public: + RowInputFormatWithDiagnosticInfo(const Block & header_, ReadBuffer & in_, const Params & params_); + + String getDiagnosticInfo() override; + +protected: + void updateDiagnosticInfo(); + bool deserializeFieldAndPrintDiagnosticInfo(const String & col_name, const DataTypePtr & type, IColumn & column, + WriteBuffer & out, size_t file_column); + String alignedName(const String & name, size_t max_length) const; + + virtual bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) = 0; + virtual void tryDeserializeFiled(const DataTypePtr & type, IColumn & column, size_t file_column, + ReadBuffer::Position & prev_pos, ReadBuffer::Position & curr_pos) = 0; + virtual bool isGarbageAfterField(size_t after_input_pos_idx, ReadBuffer::Position pos) = 0; + + /// For convenient diagnostics in case of an error. + size_t row_num = 0; + +private: + /// How many bytes were read, not counting those still in the buffer. + size_t bytes_read_at_start_of_buffer_on_current_row = 0; + size_t bytes_read_at_start_of_buffer_on_prev_row = 0; + + size_t offset_of_current_row = std::numeric_limits::max(); + size_t offset_of_prev_row = std::numeric_limits::max(); + + /// For alignment of diagnostic info. + size_t max_length_of_column_name = 0; + size_t max_length_of_data_type_name = 0; +}; + +} diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp index ba0f78bf3e6..a9e8f24d588 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.cpp @@ -21,34 +21,26 @@ MergedColumnOnlyOutputStream::MergedColumnOnlyOutputStream( header(header_), sync(sync_), skip_offsets(skip_offsets_), already_written_offset_columns(already_written_offset_columns_) { + serialization_states.reserve(header.columns()); + WrittenOffsetColumns tmp_offset_columns; + IDataType::SerializeBinaryBulkSettings settings; + + for (const auto & column_name : header.getNames()) + { + const auto & col = header.getByName(column_name); + + const auto columns = storage.getColumns(); + addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets); + serialization_states.emplace_back(nullptr); + settings.getter = createStreamGetter(col.name, tmp_offset_columns, false); + col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); + } + + initSkipIndices(); } void MergedColumnOnlyOutputStream::write(const Block & block) { - if (!initialized) - { - column_streams.clear(); - serialization_states.clear(); - serialization_states.reserve(header.columns()); - WrittenOffsetColumns tmp_offset_columns; - IDataType::SerializeBinaryBulkSettings settings; - - for (const auto & column_name : header.getNames()) - { - const auto & col = block.getByName(column_name); - - const auto columns = storage.getColumns(); - addStreams(part_path, col.name, *col.type, columns.getCodecOrDefault(col.name, codec), 0, skip_offsets); - serialization_states.emplace_back(nullptr); - settings.getter = createStreamGetter(col.name, tmp_offset_columns, false); - col.type->serializeBinaryBulkStatePrefix(settings, serialization_states.back()); - } - - initSkipIndices(); - - initialized = true; - } - std::set skip_indexes_column_names_set; for (const auto & index : skip_indices) std::copy(index->columns.cbegin(), index->columns.cend(), @@ -68,7 +60,6 @@ void MergedColumnOnlyOutputStream::write(const Block & block) if (!rows) return; - size_t new_index_offset = 0; size_t new_current_mark = 0; WrittenOffsetColumns offset_columns = already_written_offset_columns; @@ -106,7 +97,8 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG serialize_settings.getter = createStreamGetter(column.name, already_written_offset_columns, skip_offsets); column.type->serializeBinaryBulkStateSuffix(serialize_settings, serialization_states[i]); - if (with_final_mark) + /// We wrote at least one row + if (with_final_mark && (index_offset != 0 || current_mark != 0)) writeFinalMark(column.name, column.type, offset_columns, skip_offsets, serialize_settings.path); } @@ -125,7 +117,6 @@ MergeTreeData::DataPart::Checksums MergedColumnOnlyOutputStream::writeSuffixAndG column_streams.clear(); serialization_states.clear(); - initialized = false; return checksums; } diff --git a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h index 8970bf19565..f19c970ac41 100644 --- a/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h +++ b/dbms/src/Storages/MergeTree/MergedColumnOnlyOutputStream.h @@ -28,7 +28,6 @@ public: private: Block header; - bool initialized = false; bool sync; bool skip_offsets; diff --git a/dbms/tests/integration/README.md b/dbms/tests/integration/README.md index 06819af7668..a0a5322e5a6 100644 --- a/dbms/tests/integration/README.md +++ b/dbms/tests/integration/README.md @@ -34,7 +34,8 @@ set the following environment variables: ### Running with runner script -The only requirement is fresh docker configured docker. +The only requirement is fresh configured docker and +docker pull yandex/clickhouse-integration-tests-runner Notes: * If you want to run integration tests without `sudo` you have to add your user to docker group `sudo usermod -aG docker $USER`. [More information](https://docs.docker.com/install/linux/linux-postinstall/) about docker configuration. diff --git a/dbms/tests/integration/helpers/common_instance_config.xml b/dbms/tests/integration/helpers/0_common_instance_config.xml similarity index 100% rename from dbms/tests/integration/helpers/common_instance_config.xml rename to dbms/tests/integration/helpers/0_common_instance_config.xml diff --git a/dbms/tests/integration/helpers/cluster.py b/dbms/tests/integration/helpers/cluster.py index 1288aaa23f2..aadd2e70a52 100644 --- a/dbms/tests/integration/helpers/cluster.py +++ b/dbms/tests/integration/helpers/cluster.py @@ -723,7 +723,8 @@ class ClickHouseInstance: os.mkdir(config_d_dir) os.mkdir(users_d_dir) - shutil.copy(p.join(HELPERS_DIR, 'common_instance_config.xml'), config_d_dir) + # The file is named with 0_ prefix to be processed before other configuration overloads. + shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), config_d_dir) # Generate and write macros file macros = self.macros.copy() diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/.gitignore b/dbms/tests/integration/test_dictionaries/configs/dictionaries/.gitignore deleted file mode 100644 index 8f1b0e23a85..00000000000 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/.gitignore +++ /dev/null @@ -1,4 +0,0 @@ -* -!.gitignore -!source.tsv -!dictionary_preset* \ No newline at end of file diff --git a/dbms/tests/integration/test_dictionaries/test.py b/dbms/tests/integration/test_dictionaries/test.py deleted file mode 100644 index 95f82f65c0d..00000000000 --- a/dbms/tests/integration/test_dictionaries/test.py +++ /dev/null @@ -1,411 +0,0 @@ -import pytest -import os -import time - -from helpers.cluster import ClickHouseCluster -from helpers.test_tools import TSV, assert_eq_with_retry -from generate_dictionaries import generate_structure, generate_dictionaries, DictionaryTestTable - -SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) - -cluster = None -instance = None -test_table = None - - -def get_status(dictionary_name): - return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n") - - -def get_last_exception(dictionary_name): - return instance.query("SELECT last_exception FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n").replace("\\'", "'") - - -def get_loading_start_time(dictionary_name): - s = instance.query("SELECT loading_start_time FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n") - if s == "0000-00-00 00:00:00": - return None - return time.strptime(s, "%Y-%m-%d %H:%M:%S") - - -def get_loading_duration(dictionary_name): - return float(instance.query("SELECT loading_duration FROM system.dictionaries WHERE name='" + dictionary_name + "'")) - - -def replace_in_file_in_container(file_name, what, replace_with): - instance.exec_in_container('sed -i "s/' + what + '/' + replace_with + '/g" ' + file_name) - - -def setup_module(module): - global cluster - global instance - global test_table - - structure = generate_structure() - dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure) - - cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) - instance = cluster.add_instance('instance', main_configs=dictionary_files) - test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv')) - - -@pytest.fixture(scope="module") -def started_cluster(): - try: - cluster.start() - instance.query("CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary") - test_table.create_clickhouse_source(instance) - for line in TSV(instance.query('select name from system.dictionaries')).lines: - print line, - - # Create table `test.small_dict_source` - instance.query(''' - drop table if exists test.small_dict_source; - create table test.small_dict_source (id UInt64, a String, b Int32, c Float64) engine=Log; - insert into test.small_dict_source values (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7); - ''') - - yield cluster - - finally: - cluster.shutdown() - - -@pytest.fixture(params=[ - # name, keys, use_parent - ('clickhouse_hashed', ('id',), True), - ('clickhouse_flat', ('id',), True), - ('clickhouse_complex_integers_key_hashed', ('key0', 'key1'), False), - ('clickhouse_complex_mixed_key_hashed', ('key0_str', 'key1'), False), - ('clickhouse_range_hashed', ('id', 'StartDate', 'EndDate'), False), -], - ids=['clickhouse_hashed', 'clickhouse_flat', - 'clickhouse_complex_integers_key_hashed', - 'clickhouse_complex_mixed_key_hashed', - 'clickhouse_range_hashed'] -) -def dictionary_structure(started_cluster, request): - return request.param - - -def test_select_all(dictionary_structure): - name, keys, use_parent = dictionary_structure - query = instance.query - - structure = test_table.get_structure_for_keys(keys, use_parent) - query(''' - DROP TABLE IF EXISTS test.{0} - '''.format(name)) - - create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure) - TSV(query(create_query)) - - result = TSV(query('select * from test.{0}'.format(name))) - - diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True) - print test_table.process_diff(diff) - assert not diff - - -@pytest.fixture(params=[ - # name, keys, use_parent - ('clickhouse_cache', ('id',), True), - ('clickhouse_complex_integers_key_cache', ('key0', 'key1'), False), - ('clickhouse_complex_mixed_key_cache', ('key0_str', 'key1'), False) -], - ids=['clickhouse_cache', 'clickhouse_complex_integers_key_cache', 'clickhouse_complex_mixed_key_cache'] -) -def cached_dictionary_structure(started_cluster, request): - return request.param - - -def test_select_all_from_cached(cached_dictionary_structure): - name, keys, use_parent = cached_dictionary_structure - query = instance.query - - structure = test_table.get_structure_for_keys(keys, use_parent) - query(''' - DROP TABLE IF EXISTS test.{0} - '''.format(name)) - - create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure) - TSV(query(create_query)) - - for i in range(4): - result = TSV(query('select * from test.{0}'.format(name))) - diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=False) - print test_table.process_diff(diff) - assert not diff - - key = [] - for key_name in keys: - if key_name.endswith('str'): - key.append("'" + str(i) + "'") - else: - key.append(str(i)) - if len(key) == 1: - key = 'toUInt64(' + str(i) + ')' - else: - key = str('(' + ','.join(key) + ')') - query("select dictGetUInt8('{0}', 'UInt8_', {1})".format(name, key)) - - result = TSV(query('select * from test.{0}'.format(name))) - diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True) - print test_table.process_diff(diff) - assert not diff - - -def test_null_value(started_cluster): - query = instance.query - - assert TSV(query("select dictGetUInt8('clickhouse_cache', 'UInt8_', toUInt64(12121212))")) == TSV("1") - assert TSV(query("select dictGetString('clickhouse_cache', 'String_', toUInt64(12121212))")) == TSV("implicit-default") - assert TSV(query("select dictGetDate('clickhouse_cache', 'Date_', toUInt64(12121212))")) == TSV("2015-11-25") - - # Check, that empty null_value interprets as default value - assert TSV(query("select dictGetUInt64('clickhouse_cache', 'UInt64_', toUInt64(12121212))")) == TSV("0") - assert TSV(query("select dictGetDateTime('clickhouse_cache', 'DateTime_', toUInt64(12121212))")) == TSV("0000-00-00 00:00:00") - - -def test_dictionary_dependency(started_cluster): - query = instance.query - - # dictionaries_lazy_load == false, so these dictionary are not loaded. - assert get_status('dep_x') == 'NOT_LOADED' - assert get_status('dep_y') == 'NOT_LOADED' - assert get_status('dep_z') == 'NOT_LOADED' - - # Dictionary 'dep_x' depends on 'dep_z', which depends on 'dep_y'. - # So they all should be loaded at once. - assert query("SELECT dictGetString('dep_x', 'a', toUInt64(1))") == "air\n" - assert get_status('dep_x') == 'LOADED' - assert get_status('dep_y') == 'LOADED' - assert get_status('dep_z') == 'LOADED' - - # Other dictionaries should work too. - assert query("SELECT dictGetString('dep_y', 'a', toUInt64(1))") == "air\n" - assert query("SELECT dictGetString('dep_z', 'a', toUInt64(1))") == "air\n" - - assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n" - assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "YY\n" - assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n" - - # Update the source table. - query("insert into test.small_dict_source values (3, 'fire', 30, 8)") - - # Wait for dictionaries to be reloaded. - assert_eq_with_retry(instance, "SELECT dictHas('dep_y', toUInt64(3))", "1", sleep_time = 2, retry_count = 10) - assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n" - assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n" - assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n" - - # dep_x and dep_z are updated only when there `intDiv(count(), 4)` is changed. - query("insert into test.small_dict_source values (4, 'ether', 404, 0.001)") - assert_eq_with_retry(instance, "SELECT dictHas('dep_x', toUInt64(4))", "1", sleep_time = 2, retry_count = 10) - assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "fire\n" - assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n" - assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "fire\n" - assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "ether\n" - assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n" - assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ether\n" - - -def test_reload_while_loading(started_cluster): - query = instance.query - - # dictionaries_lazy_load == false, so this dictionary is not loaded. - assert get_status('longload') == "NOT_LOADED" - assert get_loading_duration('longload') == 0 - - # It's not possible to get a value from the dictionary within 1.0 second, so the following query fails by timeout. - assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))", timeout = 1, ignore_error = True) == "" - - # The dictionary is now loading. - assert get_status('longload') == "LOADING" - start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload') - assert duration > 0 - - time.sleep(0.5) # Still loading. - assert get_status('longload') == "LOADING" - prev_start_time, prev_duration = start_time, duration - start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload') - assert start_time == prev_start_time - assert duration >= prev_duration - - # SYSTEM RELOAD DICTIONARY should restart loading. - query("SYSTEM RELOAD DICTIONARY 'longload'") - assert get_status('longload') == "LOADING" - prev_start_time, prev_duration = start_time, duration - start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload') - assert start_time > prev_start_time - assert duration < prev_duration - - time.sleep(0.5) # Still loading. - assert get_status('longload') == "LOADING" - prev_start_time, prev_duration = start_time, duration - start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload') - assert start_time == prev_start_time - assert duration >= prev_duration - - # SYSTEM RELOAD DICTIONARIES should restart loading again. - query("SYSTEM RELOAD DICTIONARIES") - assert get_status('longload') == "LOADING" - prev_start_time, prev_duration = start_time, duration - start_time, duration = get_loading_start_time('longload'), get_loading_duration('longload') - assert start_time > prev_start_time - assert duration < prev_duration - - # Changing the configuration file should restart loading one more time. - replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_longload.xml', 'sleep 100', 'sleep 0') - time.sleep(5) # Configuration files are reloaded once in 5 seconds. - - # This time loading should finish quickly. - assert get_status('longload') == "LOADED" - assert query("SELECT dictGetInt32('longload', 'a', toUInt64(5))") == "6\n" - - -def test_reload_after_loading(started_cluster): - query = instance.query - - assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "8\n" - assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n" - - # Change the dictionaries' data. - replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '8', '81') - replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '10', '101') - - # SYSTEM RELOAD 'name' reloads only the specified dictionary. - query("SYSTEM RELOAD DICTIONARY 'cmd'") - assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n" - assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n" - - query("SYSTEM RELOAD DICTIONARY 'file'") - assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "81\n" - assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "101\n" - - # SYSTEM RELOAD DICTIONARIES reloads all loaded dictionaries. - replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '81', '82') - replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '101', '102') - query("SYSTEM RELOAD DICTIONARIES") - assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "82\n" - assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "102\n" - - # Configuration files are reloaded and lifetimes are checked automatically once in 5 seconds. - replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_cmd.xml', '82', '83') - replace_in_file_in_container('/etc/clickhouse-server/config.d/dictionary_preset_file.txt', '102', '103') - time.sleep(5) - assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "103\n" - assert query("SELECT dictGetInt32('cmd', 'a', toUInt64(7))") == "83\n" - - -def test_reload_after_fail_by_system_reload(started_cluster): - query = instance.query - - # dictionaries_lazy_load == false, so this dictionary is not loaded. - assert get_status("no_file") == "NOT_LOADED" - - # We expect an error because the file source doesn't exist. - expected_error = "No such file" - assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") - assert get_status("no_file") == "FAILED" - - # SYSTEM RELOAD should not change anything now, the status is still FAILED. - query("SYSTEM RELOAD DICTIONARY 'no_file'") - assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") - assert get_status("no_file") == "FAILED" - - # Creating the file source makes the dictionary able to load. - instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file.txt") - query("SYSTEM RELOAD DICTIONARY 'no_file'") - query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n" - assert get_status("no_file") == "LOADED" - - # Removing the file source should not spoil the loaded dictionary. - instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file.txt") - query("SYSTEM RELOAD DICTIONARY 'no_file'") - query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n" - assert get_status("no_file") == "LOADED" - - -def test_reload_after_fail_by_timer(started_cluster): - query = instance.query - - # dictionaries_lazy_load == false, so this dictionary is not loaded. - assert get_status("no_file_2") == "NOT_LOADED" - - # We expect an error because the file source doesn't exist. - expected_error = "No such file" - assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") - assert get_status("no_file_2") == "FAILED" - - # Passed time should not change anything now, the status is still FAILED. - time.sleep(6); - assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") - assert get_status("no_file_2") == "FAILED" - - # Creating the file source makes the dictionary able to load. - instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/dictionary_preset_file.txt"), "/etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt") - time.sleep(6); - query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" - assert get_status("no_file_2") == "LOADED" - - # Removing the file source should not spoil the loaded dictionary. - instance.exec_in_container("rm /etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt") - time.sleep(6); - query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" - assert get_status("no_file_2") == "LOADED" - - -def test_reload_after_fail_in_cache_dictionary(started_cluster): - query = instance.query - query_and_get_error = instance.query_and_get_error - - # Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond. - expected_error = "Table test.xypairs doesn't exist" - assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))") - assert get_status("cache_xypairs") == "LOADED" - assert expected_error in get_last_exception("cache_xypairs") - - # Create table `test.xypairs`. - query(''' - drop table if exists test.xypairs; - create table test.xypairs (x UInt64, y UInt64) engine=Log; - insert into test.xypairs values (1, 56), (3, 78); - ''') - - # Cache dictionary now works. - assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "56", ignore_error=True) - query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" - assert get_last_exception("cache_xypairs") == "" - - # Drop table `test.xypairs`. - query('drop table if exists test.xypairs') - - # Values are cached so we can get them. - query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56" - query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" - assert get_last_exception("cache_xypairs") == "" - - # But we can't get a value from the source table which isn't cached. - assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))") - assert expected_error in get_last_exception("cache_xypairs") - - # Passed time should not spoil the cache. - time.sleep(5); - query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56" - query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" - assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))") - assert expected_error in get_last_exception("cache_xypairs") - - # Create table `test.xypairs` again with changed values. - query(''' - drop table if exists test.xypairs; - create table test.xypairs (x UInt64, y UInt64) engine=Log; - insert into test.xypairs values (1, 57), (3, 79); - ''') - - # The cache dictionary returns new values now. - assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "57") - query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" - query("SELECT dictGet('cache_xypairs', 'y', toUInt64(3))") == "79" - assert get_last_exception("cache_xypairs") == "" diff --git a/dbms/tests/integration/test_cached_dictionary_string/__init__.py b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/__init__.py similarity index 100% rename from dbms/tests/integration/test_cached_dictionary_string/__init__.py rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/__init__.py diff --git a/dbms/tests/integration/test_external_dictionaries/configs/config.xml b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/configs/config.xml similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/configs/config.xml rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/configs/config.xml diff --git a/dbms/tests/integration/test_external_dictionaries/configs/dictionaries/.gitkeep b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/configs/dictionaries/.gitkeep similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/configs/dictionaries/.gitkeep rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/configs/dictionaries/.gitkeep diff --git a/dbms/tests/integration/test_cached_dictionary_string/configs/users.xml b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/configs/users.xml similarity index 100% rename from dbms/tests/integration/test_cached_dictionary_string/configs/users.xml rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/configs/users.xml diff --git a/dbms/tests/integration/test_external_dictionaries/dictionary.py b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/dictionary.py rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/dictionary.py diff --git a/dbms/tests/integration/test_external_dictionaries/external_sources.py b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/external_sources.py rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/external_sources.py diff --git a/dbms/tests/integration/test_external_dictionaries/fake_cert.pem b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/fake_cert.pem similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/fake_cert.pem rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/fake_cert.pem diff --git a/dbms/tests/integration/test_external_dictionaries/http_server.py b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/http_server.py similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/http_server.py rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/http_server.py diff --git a/dbms/tests/integration/test_external_dictionaries/test.py b/dbms/tests/integration/test_dictionaries_all_layouts_and_sources/test.py similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/test.py rename to dbms/tests/integration/test_dictionaries_all_layouts_and_sources/test.py diff --git a/dbms/tests/integration/test_dictionaries/__init__.py b/dbms/tests/integration/test_dictionaries_complex_key_cache_string/__init__.py similarity index 100% rename from dbms/tests/integration/test_dictionaries/__init__.py rename to dbms/tests/integration/test_dictionaries_complex_key_cache_string/__init__.py diff --git a/dbms/tests/integration/test_cached_dictionary_string/configs/config.xml b/dbms/tests/integration/test_dictionaries_complex_key_cache_string/configs/config.xml similarity index 100% rename from dbms/tests/integration/test_cached_dictionary_string/configs/config.xml rename to dbms/tests/integration/test_dictionaries_complex_key_cache_string/configs/config.xml diff --git a/dbms/tests/integration/test_cached_dictionary_string/configs/dictionaries/complex_key_cache_string.xml b/dbms/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/complex_key_cache_string.xml similarity index 100% rename from dbms/tests/integration/test_cached_dictionary_string/configs/dictionaries/complex_key_cache_string.xml rename to dbms/tests/integration/test_dictionaries_complex_key_cache_string/configs/dictionaries/complex_key_cache_string.xml diff --git a/dbms/tests/integration/test_dictionaries/configs/users.xml b/dbms/tests/integration/test_dictionaries_complex_key_cache_string/configs/users.xml similarity index 100% rename from dbms/tests/integration/test_dictionaries/configs/users.xml rename to dbms/tests/integration/test_dictionaries_complex_key_cache_string/configs/users.xml diff --git a/dbms/tests/integration/test_cached_dictionary_string/test.py b/dbms/tests/integration/test_dictionaries_complex_key_cache_string/test.py similarity index 100% rename from dbms/tests/integration/test_cached_dictionary_string/test.py rename to dbms/tests/integration/test_dictionaries_complex_key_cache_string/test.py diff --git a/dbms/tests/integration/test_external_dictionaries/__init__.py b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/__init__.py similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/__init__.py rename to dbms/tests/integration/test_dictionaries_depend_on_dictionaries/__init__.py diff --git a/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/config.xml b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/config.xml new file mode 100644 index 00000000000..b60daf72dcf --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/config.xml @@ -0,0 +1,30 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + /etc/clickhouse-server/config.d/*.xml + diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_x.xml b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/dictionaries/dep_x.xml similarity index 100% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_x.xml rename to dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/dictionaries/dep_x.xml diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_y.xml b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/dictionaries/dep_y.xml similarity index 95% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_y.xml rename to dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/dictionaries/dep_y.xml index 8806c724111..227d87ca92a 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_y.xml +++ b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/dictionaries/dep_y.xml @@ -8,7 +8,7 @@ default test - small_dict_source
+ elements
5 diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_z.xml b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/dictionaries/dep_z.xml similarity index 100% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_dep_z.xml rename to dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/dictionaries/dep_z.xml diff --git a/dbms/tests/integration/test_external_dictionaries/configs/users.xml b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/users.xml similarity index 100% rename from dbms/tests/integration/test_external_dictionaries/configs/users.xml rename to dbms/tests/integration/test_dictionaries_depend_on_dictionaries/configs/users.xml diff --git a/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/test.py b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/test.py new file mode 100644 index 00000000000..c0ce0af0313 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_depend_on_dictionaries/test.py @@ -0,0 +1,76 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DICTIONARY_FILES = ['configs/dictionaries/dep_x.xml', 'configs/dictionaries/dep_y.xml', 'configs/dictionaries/dep_z.xml'] + +cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + instance.query(''' + CREATE DATABASE IF NOT EXISTS dict ENGINE=Dictionary; + CREATE DATABASE IF NOT EXISTS test; + DROP TABLE IF EXISTS test.elements; + CREATE TABLE test.elements (id UInt64, a String, b Int32, c Float64) ENGINE=Log; + INSERT INTO test.elements VALUES (0, 'water', 10, 1), (1, 'air', 40, 0.01), (2, 'earth', 100, 1.7); + ''') + + yield cluster + + finally: + cluster.shutdown() + + +def get_status(dictionary_name): + return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n") + + +def test_get_data(started_cluster): + query = instance.query + + # dictionaries_lazy_load == false, so these dictionary are not loaded. + assert get_status('dep_x') == 'NOT_LOADED' + assert get_status('dep_y') == 'NOT_LOADED' + assert get_status('dep_z') == 'NOT_LOADED' + + # Dictionary 'dep_x' depends on 'dep_z', which depends on 'dep_y'. + # So they all should be loaded at once. + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(1))") == "air\n" + assert get_status('dep_x') == 'LOADED' + assert get_status('dep_y') == 'LOADED' + assert get_status('dep_z') == 'LOADED' + + # Other dictionaries should work too. + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(1))") == "air\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(1))") == "air\n" + + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n" + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "YY\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n" + + # Update the source table. + query("INSERT INTO test.elements VALUES (3, 'fire', 30, 8)") + + # Wait for dictionaries to be reloaded. + assert_eq_with_retry(instance, "SELECT dictHas('dep_y', toUInt64(3))", "1", sleep_time = 2, retry_count = 10) + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "XX\n" + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "ZZ\n" + + # dep_x and dep_z are updated only when there `intDiv(count(), 4)` is changed. + query("INSERT INTO test.elements VALUES (4, 'ether', 404, 0.001)") + assert_eq_with_retry(instance, "SELECT dictHas('dep_x', toUInt64(4))", "1", sleep_time = 2, retry_count = 10) + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(3))") == "fire\n" + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(3))") == "fire\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(3))") == "fire\n" + assert query("SELECT dictGetString('dep_x', 'a', toUInt64(4))") == "ether\n" + assert query("SELECT dictGetString('dep_y', 'a', toUInt64(4))") == "ether\n" + assert query("SELECT dictGetString('dep_z', 'a', toUInt64(4))") == "ether\n" diff --git a/dbms/tests/integration/test_dictionaries_null_value/__init__.py b/dbms/tests/integration/test_dictionaries_null_value/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_dictionaries/configs/config.xml b/dbms/tests/integration/test_dictionaries_null_value/configs/config.xml similarity index 100% rename from dbms/tests/integration/test_dictionaries/configs/config.xml rename to dbms/tests/integration/test_dictionaries_null_value/configs/config.xml diff --git a/dbms/tests/integration/test_dictionaries_null_value/configs/dictionaries/cache.xml b/dbms/tests/integration/test_dictionaries_null_value/configs/dictionaries/cache.xml new file mode 100644 index 00000000000..9a1ae0732db --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_null_value/configs/dictionaries/cache.xml @@ -0,0 +1,113 @@ + + + cache + + + + localhost + 9000 + default + + test + source
+
+ + + 0 + + + 128 + + + + + id + + + + UInt8_ + UInt8 + 1 + + + + UInt16_ + UInt16 + 1 + + + + UInt32_ + UInt32 + 1 + + + + UInt64_ + UInt64 + + + + + Int8_ + Int8 + -1 + + + + Int16_ + Int16 + -1 + + + + Int32_ + Int32 + -1 + + + + Int64_ + Int64 + -1 + + + + Float32_ + Float32 + 2.71828 + + + + Float64_ + Float64 + 2.71828 + + + + String_ + String + implicit-default + + + + Date_ + Date + 2015-11-25 + + + + DateTime_ + DateTime + + + + + Parent + UInt64 + true + 0 + + +
+
diff --git a/dbms/tests/integration/test_dictionaries_null_value/configs/users.xml b/dbms/tests/integration/test_dictionaries_null_value/configs/users.xml new file mode 100644 index 00000000000..6061af8e33d --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_null_value/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_dictionaries_null_value/test.py b/dbms/tests/integration/test_dictionaries_null_value/test.py new file mode 100644 index 00000000000..e31f397c246 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_null_value/test.py @@ -0,0 +1,45 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV, assert_eq_with_retry + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DICTIONARY_FILES = ['configs/dictionaries/cache.xml'] + +cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + + instance.query(''' + CREATE DATABASE IF NOT EXISTS test; + DROP TABLE IF EXISTS test.source; + CREATE TABLE test.source (id UInt64, key0 UInt8, key0_str String, key1 UInt8, + StartDate Date, EndDate Date, + UInt8_ UInt8, UInt16_ UInt16, UInt32_ UInt32, UInt64_ UInt64, + Int8_ Int8, Int16_ Int16, Int32_ Int32, Int64_ Int64, + Float32_ Float32, Float64_ Float64, + String_ String, + Date_ Date, DateTime_ DateTime, Parent UInt64) ENGINE=Log; + ''') + + yield cluster + + finally: + cluster.shutdown() + + +def test_null_value(started_cluster): + query = instance.query + + assert query("select dictGetUInt8('cache', 'UInt8_', toUInt64(12121212))") == "1\n" + assert query("select dictGetString('cache', 'String_', toUInt64(12121212))") == "implicit-default\n" + assert query("select dictGetDate('cache', 'Date_', toUInt64(12121212))") == "2015-11-25\n" + + # Check, that empty null_value interprets as default value + assert query("select dictGetUInt64('cache', 'UInt64_', toUInt64(12121212))") == "0\n" + assert query("select dictGetDateTime('cache', 'DateTime_', toUInt64(12121212))") == "0000-00-00 00:00:00\n" diff --git a/dbms/tests/integration/test_dictionaries_select_all/__init__.py b/dbms/tests/integration/test_dictionaries_select_all/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_dictionaries_select_all/configs/config.xml b/dbms/tests/integration/test_dictionaries_select_all/configs/config.xml new file mode 100644 index 00000000000..1e4c14585a9 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_select_all/configs/config.xml @@ -0,0 +1,30 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + /etc/clickhouse-server/config.d/*.xml + diff --git a/dbms/tests/integration/test_dictionaries_select_all/configs/dictionaries/.gitignore b/dbms/tests/integration/test_dictionaries_select_all/configs/dictionaries/.gitignore new file mode 100644 index 00000000000..cc461064a39 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_select_all/configs/dictionaries/.gitignore @@ -0,0 +1,3 @@ +* +!.gitignore +!source.tsv \ No newline at end of file diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/source.tsv b/dbms/tests/integration/test_dictionaries_select_all/configs/dictionaries/source.tsv similarity index 100% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/source.tsv rename to dbms/tests/integration/test_dictionaries_select_all/configs/dictionaries/source.tsv diff --git a/dbms/tests/integration/test_dictionaries_select_all/configs/users.xml b/dbms/tests/integration/test_dictionaries_select_all/configs/users.xml new file mode 100644 index 00000000000..6061af8e33d --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_select_all/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_dictionaries/generate_dictionaries.py b/dbms/tests/integration/test_dictionaries_select_all/generate_dictionaries.py similarity index 96% rename from dbms/tests/integration/test_dictionaries/generate_dictionaries.py rename to dbms/tests/integration/test_dictionaries_select_all/generate_dictionaries.py index c644bd8f644..30a5648fdbe 100644 --- a/dbms/tests/integration/test_dictionaries/generate_dictionaries.py +++ b/dbms/tests/integration/test_dictionaries_select_all/generate_dictionaries.py @@ -12,13 +12,6 @@ types = [ 'Date', 'DateTime' ] -explicit_defaults = [ - '42', '42', '42', '42', - '-42', '-42', '-42', '-42', - '1.5', '1.6', - "'explicit-default'", - "'2015-01-01'", "'2015-01-01 00:00:00'" -] implicit_defaults = [ '1', '1', '1', '', @@ -182,9 +175,6 @@ def generate_dictionaries(path, structure): file_names = [] - # Add ready dictionaries. - file_names.extend(glob.glob(os.path.join(path, "*dictionary_preset*"))) - # Generate dictionaries. for (name, key_idx, has_parent), (source, layout) in zip(structure, sources_and_layouts): filename = os.path.join(path, 'dictionary_%s.xml' % name) diff --git a/dbms/tests/integration/test_dictionaries_select_all/test.py b/dbms/tests/integration/test_dictionaries_select_all/test.py new file mode 100644 index 00000000000..8bad8a9b214 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_select_all/test.py @@ -0,0 +1,122 @@ +import pytest +import os +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import TSV, assert_eq_with_retry +from generate_dictionaries import generate_structure, generate_dictionaries, DictionaryTestTable + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) + +cluster = None +instance = None +test_table = None + + +def setup_module(module): + global cluster + global instance + global test_table + + structure = generate_structure() + dictionary_files = generate_dictionaries(os.path.join(SCRIPT_DIR, 'configs/dictionaries'), structure) + + cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) + instance = cluster.add_instance('instance', main_configs=dictionary_files) + test_table = DictionaryTestTable(os.path.join(SCRIPT_DIR, 'configs/dictionaries/source.tsv')) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + test_table.create_clickhouse_source(instance) + for line in TSV(instance.query('select name from system.dictionaries')).lines: + print line, + + yield cluster + + finally: + cluster.shutdown() + + +@pytest.fixture(params=[ + # name, keys, use_parent + ('clickhouse_hashed', ('id',), True), + ('clickhouse_flat', ('id',), True), + ('clickhouse_complex_integers_key_hashed', ('key0', 'key1'), False), + ('clickhouse_complex_mixed_key_hashed', ('key0_str', 'key1'), False), + ('clickhouse_range_hashed', ('id', 'StartDate', 'EndDate'), False), +], + ids=['clickhouse_hashed', 'clickhouse_flat', + 'clickhouse_complex_integers_key_hashed', + 'clickhouse_complex_mixed_key_hashed', + 'clickhouse_range_hashed'] +) +def dictionary_structure(started_cluster, request): + return request.param + + +def test_select_all(dictionary_structure): + name, keys, use_parent = dictionary_structure + query = instance.query + + structure = test_table.get_structure_for_keys(keys, use_parent) + query(''' + DROP TABLE IF EXISTS test.{0} + '''.format(name)) + + create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure) + TSV(query(create_query)) + + result = TSV(query('select * from test.{0}'.format(name))) + + diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True) + print test_table.process_diff(diff) + assert not diff + + +@pytest.fixture(params=[ + # name, keys, use_parent + ('clickhouse_cache', ('id',), True), + ('clickhouse_complex_integers_key_cache', ('key0', 'key1'), False), + ('clickhouse_complex_mixed_key_cache', ('key0_str', 'key1'), False) +], + ids=['clickhouse_cache', 'clickhouse_complex_integers_key_cache', 'clickhouse_complex_mixed_key_cache'] +) +def cached_dictionary_structure(started_cluster, request): + return request.param + + +def test_select_all_from_cached(cached_dictionary_structure): + name, keys, use_parent = cached_dictionary_structure + query = instance.query + + structure = test_table.get_structure_for_keys(keys, use_parent) + query(''' + DROP TABLE IF EXISTS test.{0} + '''.format(name)) + + create_query = "CREATE TABLE test.{0} ({1}) engine = Dictionary({0})".format(name, structure) + TSV(query(create_query)) + + for i in range(4): + result = TSV(query('select * from test.{0}'.format(name))) + diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=False) + print test_table.process_diff(diff) + assert not diff + + key = [] + for key_name in keys: + if key_name.endswith('str'): + key.append("'" + str(i) + "'") + else: + key.append(str(i)) + if len(key) == 1: + key = 'toUInt64(' + str(i) + ')' + else: + key = str('(' + ','.join(key) + ')') + query("select dictGetUInt8('{0}', 'UInt8_', {1})".format(name, key)) + + result = TSV(query('select * from test.{0}'.format(name))) + diff = test_table.compare_by_keys(keys, result.lines, use_parent, add_not_found_rows=True) + print test_table.process_diff(diff) + assert not diff diff --git a/dbms/tests/integration/test_dictionaries_update_and_reload/__init__.py b/dbms/tests/integration/test_dictionaries_update_and_reload/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_dictionaries_update_and_reload/configs/config.xml b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/config.xml new file mode 100644 index 00000000000..b60daf72dcf --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/config.xml @@ -0,0 +1,30 @@ + + + + trace + /var/log/clickhouse-server/clickhouse-server.log + /var/log/clickhouse-server/clickhouse-server.err.log + 1000M + 10 + + + 9000 + 127.0.0.1 + + + + true + none + + AcceptCertificateHandler + + + + + 500 + 5368709120 + ./clickhouse/ + users.xml + + /etc/clickhouse-server/config.d/*.xml + diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cache_xypairs.xml b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/cache_xypairs.xml similarity index 100% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cache_xypairs.xml rename to dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/cache_xypairs.xml diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cmd.xml b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/executable.xml similarity index 93% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cmd.xml rename to dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/executable.xml index 9f1e259e2d7..519a2915a59 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_cmd.xml +++ b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/executable.xml @@ -1,7 +1,7 @@ - cmd + executable echo '7\t8'; diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.txt b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.txt similarity index 100% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.txt rename to dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.txt diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.xml b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.xml similarity index 82% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.xml rename to dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.xml index 0e6db1f1637..2a937b5444d 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_file.xml +++ b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/file.xml @@ -4,7 +4,7 @@ file - /etc/clickhouse-server/config.d/dictionary_preset_file.txt + /etc/clickhouse-server/config.d/file.txt TabSeparated @@ -21,7 +21,7 @@ no_file - /etc/clickhouse-server/config.d/dictionary_preset_no_file.txt + /etc/clickhouse-server/config.d/no_file.txt TabSeparated @@ -38,7 +38,7 @@ no_file_2 - /etc/clickhouse-server/config.d/dictionary_preset_no_file_2.txt + /etc/clickhouse-server/config.d/no_file_2.txt TabSeparated diff --git a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_longload.xml b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/slow.xml similarity index 94% rename from dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_longload.xml rename to dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/slow.xml index f5d4cdec583..c6814c5fe9c 100644 --- a/dbms/tests/integration/test_dictionaries/configs/dictionaries/dictionary_preset_longload.xml +++ b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/dictionaries/slow.xml @@ -1,7 +1,7 @@ - longload + slow sleep 100 && echo '5\t6'; diff --git a/dbms/tests/integration/test_dictionaries_update_and_reload/configs/users.xml b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/users.xml new file mode 100644 index 00000000000..6061af8e33d --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_update_and_reload/configs/users.xml @@ -0,0 +1,23 @@ + + + + + + + + + + + + ::/0 + + default + default + + + + + + + + diff --git a/dbms/tests/integration/test_dictionaries_update_and_reload/test.py b/dbms/tests/integration/test_dictionaries_update_and_reload/test.py new file mode 100644 index 00000000000..b972dc6c918 --- /dev/null +++ b/dbms/tests/integration/test_dictionaries_update_and_reload/test.py @@ -0,0 +1,246 @@ +import pytest +import os +import time +from helpers.cluster import ClickHouseCluster +from helpers.test_tools import assert_eq_with_retry + +SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__)) +DICTIONARY_FILES = ['configs/dictionaries/cache_xypairs.xml', 'configs/dictionaries/executable.xml', 'configs/dictionaries/file.xml', 'configs/dictionaries/file.txt', 'configs/dictionaries/slow.xml'] + +cluster = ClickHouseCluster(__file__, base_configs_dir=os.path.join(SCRIPT_DIR, 'configs')) +instance = cluster.add_instance('instance', main_configs=DICTIONARY_FILES) + + +@pytest.fixture(scope="module") +def started_cluster(): + try: + cluster.start() + instance.query("CREATE DATABASE IF NOT EXISTS test") + + yield cluster + + finally: + cluster.shutdown() + + +def get_status(dictionary_name): + return instance.query("SELECT status FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n") + + +def get_last_exception(dictionary_name): + return instance.query("SELECT last_exception FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n").replace("\\'", "'") + + +def get_loading_start_time(dictionary_name): + s = instance.query("SELECT loading_start_time FROM system.dictionaries WHERE name='" + dictionary_name + "'").rstrip("\n") + if s == "0000-00-00 00:00:00": + return None + return time.strptime(s, "%Y-%m-%d %H:%M:%S") + + +def get_loading_duration(dictionary_name): + return float(instance.query("SELECT loading_duration FROM system.dictionaries WHERE name='" + dictionary_name + "'")) + + +def replace_in_file_in_container(file_name, what, replace_with): + instance.exec_in_container('sed -i "s/' + what + '/' + replace_with + '/g" ' + file_name) + + +def test_reload_while_loading(started_cluster): + query = instance.query + + # dictionaries_lazy_load == false, so this dictionary is not loaded. + assert get_status('slow') == "NOT_LOADED" + assert get_loading_duration('slow') == 0 + + # It's not possible to get a value from the dictionary within 1.0 second, so the following query fails by timeout. + assert query("SELECT dictGetInt32('slow', 'a', toUInt64(5))", timeout = 1, ignore_error = True) == "" + + # The dictionary is now loading. + assert get_status('slow') == "LOADING" + start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow') + assert duration > 0 + + time.sleep(0.5) # Still loading. + assert get_status('slow') == "LOADING" + prev_start_time, prev_duration = start_time, duration + start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow') + assert start_time == prev_start_time + assert duration >= prev_duration + + # SYSTEM RELOAD DICTIONARY should restart loading. + query("SYSTEM RELOAD DICTIONARY 'slow'") + assert get_status('slow') == "LOADING" + prev_start_time, prev_duration = start_time, duration + start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow') + assert start_time > prev_start_time + assert duration < prev_duration + + time.sleep(0.5) # Still loading. + assert get_status('slow') == "LOADING" + prev_start_time, prev_duration = start_time, duration + start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow') + assert start_time == prev_start_time + assert duration >= prev_duration + + # SYSTEM RELOAD DICTIONARIES should restart loading again. + query("SYSTEM RELOAD DICTIONARIES") + assert get_status('slow') == "LOADING" + prev_start_time, prev_duration = start_time, duration + start_time, duration = get_loading_start_time('slow'), get_loading_duration('slow') + assert start_time > prev_start_time + assert duration < prev_duration + + # Changing the configuration file should restart loading one more time. + replace_in_file_in_container('/etc/clickhouse-server/config.d/slow.xml', 'sleep 100', 'sleep 0') + time.sleep(5) # Configuration files are reloaded once in 5 seconds. + + # This time loading should finish quickly. + assert get_status('slow') == "LOADED" + assert query("SELECT dictGetInt32('slow', 'a', toUInt64(5))") == "6\n" + + +def test_reload_after_loading(started_cluster): + query = instance.query + + assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "8\n" + assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n" + + # Change the dictionaries' data. + replace_in_file_in_container('/etc/clickhouse-server/config.d/executable.xml', '8', '81') + replace_in_file_in_container('/etc/clickhouse-server/config.d/file.txt', '10', '101') + + # SYSTEM RELOAD 'name' reloads only the specified dictionary. + query("SYSTEM RELOAD DICTIONARY 'executable'") + assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "81\n" + assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "10\n" + + query("SYSTEM RELOAD DICTIONARY 'file'") + assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "81\n" + assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "101\n" + + # SYSTEM RELOAD DICTIONARIES reloads all loaded dictionaries. + replace_in_file_in_container('/etc/clickhouse-server/config.d/executable.xml', '81', '82') + replace_in_file_in_container('/etc/clickhouse-server/config.d/file.txt', '101', '102') + query("SYSTEM RELOAD DICTIONARIES") + assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "82\n" + assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "102\n" + + # Configuration files are reloaded and lifetimes are checked automatically once in 5 seconds. + replace_in_file_in_container('/etc/clickhouse-server/config.d/executable.xml', '82', '83') + replace_in_file_in_container('/etc/clickhouse-server/config.d/file.txt', '102', '103') + time.sleep(5) + assert query("SELECT dictGetInt32('file', 'a', toUInt64(9))") == "103\n" + assert query("SELECT dictGetInt32('executable', 'a', toUInt64(7))") == "83\n" + + +def test_reload_after_fail_by_system_reload(started_cluster): + query = instance.query + + # dictionaries_lazy_load == false, so this dictionary is not loaded. + assert get_status("no_file") == "NOT_LOADED" + + # We expect an error because the file source doesn't exist. + expected_error = "No such file" + assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") + assert get_status("no_file") == "FAILED" + + # SYSTEM RELOAD should not change anything now, the status is still FAILED. + query("SYSTEM RELOAD DICTIONARY 'no_file'") + assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") + assert get_status("no_file") == "FAILED" + + # Creating the file source makes the dictionary able to load. + instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/file.txt"), "/etc/clickhouse-server/config.d/no_file.txt") + query("SYSTEM RELOAD DICTIONARY 'no_file'") + query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n" + assert get_status("no_file") == "LOADED" + + # Removing the file source should not spoil the loaded dictionary. + instance.exec_in_container("rm /etc/clickhouse-server/config.d/no_file.txt") + query("SYSTEM RELOAD DICTIONARY 'no_file'") + query("SELECT dictGetInt32('no_file', 'a', toUInt64(9))") == "10\n" + assert get_status("no_file") == "LOADED" + + +def test_reload_after_fail_by_timer(started_cluster): + query = instance.query + + # dictionaries_lazy_load == false, so this dictionary is not loaded. + assert get_status("no_file_2") == "NOT_LOADED" + + # We expect an error because the file source doesn't exist. + expected_error = "No such file" + assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") + assert get_status("no_file_2") == "FAILED" + + # Passed time should not change anything now, the status is still FAILED. + time.sleep(6); + assert expected_error in instance.query_and_get_error("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") + assert get_status("no_file_2") == "FAILED" + + # Creating the file source makes the dictionary able to load. + instance.copy_file_to_container(os.path.join(SCRIPT_DIR, "configs/dictionaries/file.txt"), "/etc/clickhouse-server/config.d/no_file_2.txt") + time.sleep(6); + query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" + assert get_status("no_file_2") == "LOADED" + + # Removing the file source should not spoil the loaded dictionary. + instance.exec_in_container("rm /etc/clickhouse-server/config.d/no_file_2.txt") + time.sleep(6); + query("SELECT dictGetInt32('no_file_2', 'a', toUInt64(9))") == "10\n" + assert get_status("no_file_2") == "LOADED" + + +def test_reload_after_fail_in_cache_dictionary(started_cluster): + query = instance.query + query_and_get_error = instance.query_and_get_error + + # Can't get a value from the cache dictionary because the source (table `test.xypairs`) doesn't respond. + expected_error = "Table test.xypairs doesn't exist" + assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(1))") + assert get_status("cache_xypairs") == "LOADED" + assert expected_error in get_last_exception("cache_xypairs") + + # Create table `test.xypairs`. + query(''' + DROP TABLE IF EXISTS test.xypairs; + CREATE TABLE test.xypairs (x UInt64, y UInt64) ENGINE=Log; + INSERT INTO test.xypairs VALUES (1, 56), (3, 78); + ''') + + # Cache dictionary now works. + assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "56", ignore_error=True) + query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" + assert get_last_exception("cache_xypairs") == "" + + # Drop table `test.xypairs`. + query('DROP TABLE test.xypairs') + + # Values are cached so we can get them. + query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56" + query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" + assert get_last_exception("cache_xypairs") == "" + + # But we can't get a value from the source table which isn't cached. + assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))") + assert expected_error in get_last_exception("cache_xypairs") + + # Passed time should not spoil the cache. + time.sleep(5); + query("SELECT dictGet('cache_xypairs', 'y', toUInt64(1))") == "56" + query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" + assert expected_error in query_and_get_error("SELECT dictGetUInt64('cache_xypairs', 'y', toUInt64(3))") + assert expected_error in get_last_exception("cache_xypairs") + + # Create table `test.xypairs` again with changed values. + query(''' + CREATE TABLE test.xypairs (x UInt64, y UInt64) ENGINE=Log; + INSERT INTO test.xypairs VALUES (1, 57), (3, 79); + ''') + + # The cache dictionary returns new values now. + assert_eq_with_retry(instance, "SELECT dictGet('cache_xypairs', 'y', toUInt64(1))", "57") + query("SELECT dictGet('cache_xypairs', 'y', toUInt64(2))") == "0" + query("SELECT dictGet('cache_xypairs', 'y', toUInt64(3))") == "79" + assert get_last_exception("cache_xypairs") == "" diff --git a/dbms/tests/integration/test_timezone_config/__init__.py b/dbms/tests/integration/test_timezone_config/__init__.py new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/integration/test_timezone_config/configs/config.xml b/dbms/tests/integration/test_timezone_config/configs/config.xml new file mode 100644 index 00000000000..c601a1d09ef --- /dev/null +++ b/dbms/tests/integration/test_timezone_config/configs/config.xml @@ -0,0 +1,4 @@ + + + America/Los_Angeles + diff --git a/dbms/tests/integration/test_timezone_config/test.py b/dbms/tests/integration/test_timezone_config/test.py new file mode 100644 index 00000000000..22e11daa72e --- /dev/null +++ b/dbms/tests/integration/test_timezone_config/test.py @@ -0,0 +1,17 @@ +import pytest + +from helpers.cluster import ClickHouseCluster + +cluster = ClickHouseCluster(__file__) +node = cluster.add_instance('node', main_configs=['configs/config.xml']) + +@pytest.fixture(scope="module") +def start_cluster(): + try: + cluster.start() + yield cluster + finally: + cluster.shutdown() + +def test_check_timezone_config(start_cluster): + assert node.query("SELECT toDateTime(1111111111)") == "2005-03-17 17:58:31\n" diff --git a/dbms/tests/queries/0_stateless/00763_lock_buffer.sh b/dbms/tests/queries/0_stateless/00763_lock_buffer.sh index 4ec4875e3e2..3146ce96132 100755 --- a/dbms/tests/queries/0_stateless/00763_lock_buffer.sh +++ b/dbms/tests/queries/0_stateless/00763_lock_buffer.sh @@ -17,7 +17,7 @@ function thread1() function thread2() { - seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218' + seq 1 1000 | sed -r -e 's/.+/SELECT count() FROM test.buffer_00763_2;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^0$|^10$|^Received exception|^Code: 60|^Code: 218|^Code: 473' } thread1 & diff --git a/dbms/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh b/dbms/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh index ba50d4e9f04..059d70253b2 100755 --- a/dbms/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh +++ b/dbms/tests/queries/0_stateless/00763_long_lock_buffer_alter_destination_table.sh @@ -1,6 +1,8 @@ #!/usr/bin/env bash set -e +CLICKHOUSE_CLIENT_SERVER_LOGS_LEVEL=none + CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) . $CURDIR/../shell_config.sh @@ -18,7 +20,7 @@ function thread1() function thread2() { - seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM test.buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --server_logs_file='/dev/null' --ignore-error 2>&1 | grep -vP '^3$' + seq 1 2000 | sed -r -e 's/.+/SELECT sum(length(s)) FROM test.buffer_00763_1;/' | ${CLICKHOUSE_CLIENT} --multiquery --ignore-error 2>&1 | grep -vP '(^3$|^Received exception from server|^Code: 473)' } thread1 & diff --git a/dbms/tests/queries/0_stateless/00838_system_tables_drop_table_race.sh b/dbms/tests/queries/0_stateless/00838_system_tables_drop_table_race.sh index 06a6b8f7341..9fd84ab1034 100755 --- a/dbms/tests/queries/0_stateless/00838_system_tables_drop_table_race.sh +++ b/dbms/tests/queries/0_stateless/00838_system_tables_drop_table_race.sh @@ -8,6 +8,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS table" seq 1 100 | sed -r -e "s/.+/CREATE TABLE table (x UInt8) ENGINE = MergeTree ORDER BY x; DROP TABLE table;/" | $CLICKHOUSE_CLIENT -n & -seq 1 100 | sed -r -e "s/.+/SELECT * FROM system.tables WHERE database = '${CLICKHOUSE_DATABASE}' LIMIT 1000000, 1;/" | $CLICKHOUSE_CLIENT -n & +seq 1 100 | sed -r -e "s/.+/SELECT * FROM system.tables WHERE database = '${CLICKHOUSE_DATABASE}' LIMIT 1000000, 1;/" | $CLICKHOUSE_CLIENT -n 2>/dev/null & wait diff --git a/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.reference b/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.reference index f97d393cc32..595dcdf3803 100644 --- a/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.reference +++ b/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.reference @@ -278,3 +278,9 @@ a1,a2 12 [1,2] 1 2019-06-06 1 4 2 1 5 1 [1,2] [1001,1002] [1,1] 1 2019-06-06 1 4 2 1 5 0 [1,2] [1002,1003] [1,1] 1 2019-06-06 1 4 2 1 6 0 [3] [2001] [1] +-- empty +[[1],[],[2]] +[[1],[],[2]] +[[1],[],[2],[],[3],[],[4],[],[5],[],[6],[],[7],[],[8],[],[9]] +[[],[1],[],[2],[],[3],[],[4],[],[5],[],[6],[],[7],[],[8]] +[[1],[2],[],[3]] diff --git a/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql b/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql index 5f4b12e1988..9cf82a368d6 100644 --- a/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql +++ b/dbms/tests/queries/0_stateless/00909_arrayEnumerateUniq.sql @@ -305,3 +305,11 @@ ARRAY JOIN Test.PuidVal AS PuidValArr; DROP TABLE arr_tests_visits; + + +select '-- empty'; +SELECT arrayEnumerateUniqRanked([['a'], [], ['a']]); +SELECT arrayEnumerateUniqRanked([[1], [], [1]]); +SELECT arrayEnumerateUniqRanked([[1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1]]); +SELECT arrayEnumerateUniqRanked([[], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1], [], [1]]); +SELECT arrayEnumerateUniqRanked([[1], [1], [], [1]]); diff --git a/dbms/tests/queries/0_stateless/00933_ttl_with_default.sql b/dbms/tests/queries/0_stateless/00933_ttl_with_default.sql index e6c0a6e700c..d3f3b62126c 100644 --- a/dbms/tests/queries/0_stateless/00933_ttl_with_default.sql +++ b/dbms/tests/queries/0_stateless/00933_ttl_with_default.sql @@ -5,6 +5,7 @@ insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 1); insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2); insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3); insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4); +select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_2 final; select a from ttl_00933_2 order by a; @@ -15,6 +16,7 @@ insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 1, 100); insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2, 200); insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3, 300); insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4, 400); +select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_2 final; select a, b from ttl_00933_2 order by a; @@ -25,6 +27,7 @@ insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 1, 5); insert into ttl_00933_2 values (toDateTime('2000-10-10 00:00:00'), 2, 10); insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 3, 15); insert into ttl_00933_2 values (toDateTime('2100-10-10 00:00:00'), 4, 20); +select sleep(0.7) format Null; -- wait if very fast merge happen optimize table ttl_00933_2 final; select a, b from ttl_00933_2 order by a; diff --git a/dbms/tests/queries/0_stateless/00937_template_output_format.reference b/dbms/tests/queries/0_stateless/00937_template_output_format.reference new file mode 100644 index 00000000000..c4cfb4ed3a4 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00937_template_output_format.reference @@ -0,0 +1,16 @@ +{prefix} +n: "123", s1: qwe,rty, s2: 'as"df\'gh', s3: "", s4: "zx +cv bn m", d: 2016-01-01, n: 123 ; +n: "456", s1: as"df\'gh, s2: '', s3: "zx\ncv\tbn m", s4: "qwe,rty", d: 2016-01-02, n: 456 ; +n: "9876543210", s1: , s2: 'zx\ncv\tbn m', s3: "qwe,rty", s4: "as""df'gh", d: 2016-01-03, n: 9876543210 ; +n: "789", s1: zx\ncv\tbn m, s2: 'qwe,rty', s3: "as\"df'gh", s4: "", d: 2016-01-04, n: 789 +------ +n: "0", s1: , s2: '', s3: "", s4: "", d: 0000-00-00, n: 0 +------ +n: "123", s1: , s2: '', s3: "", s4: "", d: 2016-01-01, n: 123 +------ +n: "9876543210", s1: zx\ncv\tbn m, s2: 'zx\ncv\tbn m', s3: "zx\ncv\tbn m", s4: "zx +cv bn m", d: 2016-01-04, n: 9876543210 +4 rows +before limit 4 +read 4 $ suffix $ \ No newline at end of file diff --git a/dbms/tests/queries/0_stateless/00937_template_output_format.sql b/dbms/tests/queries/0_stateless/00937_template_output_format.sql new file mode 100644 index 00000000000..7a981c641da --- /dev/null +++ b/dbms/tests/queries/0_stateless/00937_template_output_format.sql @@ -0,0 +1,12 @@ +DROP TABLE IF EXISTS template; +CREATE TABLE template (s1 String, s2 String, `s 3` String, "s 4" String, n UInt64, d Date) ENGINE = Memory; +INSERT INTO template VALUES +('qwe,rty', 'as"df''gh', '', 'zx\ncv\tbn m', 123, '2016-01-01'),('as"df''gh', '', 'zx\ncv\tbn m', 'qwe,rty', 456, '2016-01-02'),('', 'zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', 9876543210, '2016-01-03'),('zx\ncv\tbn m', 'qwe,rty', 'as"df''gh', '', 789, '2016-01-04'); + +SELECT * FROM template WITH TOTALS LIMIT 4 FORMAT Template SETTINGS +extremes = 1, +format_schema = '{prefix} \n${data:None}\n------\n${totals:}\n------\n${min}\n------\n${max}\n${rows:Escaped} rows\nbefore limit ${rows_before_limit:XML}\nread ${rows_read:Escaped} $$ suffix $$', +format_schema_rows = 'n:\t${n:JSON}, s1:\t${s1:Escaped}, s2:\t${s2:Quoted}, s3:\t${`s 3`:JSON}, s4:\t${"s 4":CSV}, d:\t${d:Escaped}, n:\t${n:Raw}\t', +format_schema_rows_between_delimiter = ';\n'; + +DROP TABLE template; diff --git a/dbms/tests/queries/0_stateless/00938_template_input_format.reference b/dbms/tests/queries/0_stateless/00938_template_input_format.reference new file mode 100644 index 00000000000..ce89532886d --- /dev/null +++ b/dbms/tests/queries/0_stateless/00938_template_input_format.reference @@ -0,0 +1,25 @@ +==== check escaping ==== +"qwe,rty","as""df'gh","","zx +cv bn m",123,"2016-01-01" +"as""df'gh","","zx +cv bn m","qwe,rty",456,"2016-01-02" +"zx +cv bn m","qwe,rty","as""df'gh","",789,"2016-01-04" +"","zx +cv bn m","qwe,rty","as""df'gh",9876543210,"2016-01-03" +==== parse json (sophisticated template) ==== +"qwe,rty","as""df'gh","","zx +cv bn m",123,"2016-01-01" +"as""df'gh","","zx +cv bn m","qwe,rty",456,"2016-01-02" +"zx +cv bn m","qwe,rty","as""df'gh","",789,"2016-01-04" +"","zx +cv bn m","qwe,rty","as""df'gh",9876543210,"2016-01-03" +==== parse json ==== +"","","qwe,rty","",123,"2016-01-01" +"zx +cv bn m","","as""df'gh","",456,"2016-01-02" +"as""df'gh","","zx +cv bn m","",789,"2016-01-04" +"qwe,rty","","","",9876543210,"2016-01-03" diff --git a/dbms/tests/queries/0_stateless/00938_template_input_format.sh b/dbms/tests/queries/0_stateless/00938_template_input_format.sh new file mode 100755 index 00000000000..c33741543e9 --- /dev/null +++ b/dbms/tests/queries/0_stateless/00938_template_input_format.sh @@ -0,0 +1,46 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS template1"; +$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS template2"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE template1 (s1 String, s2 String, s3 String, s4 String, n UInt64, d Date) ENGINE = Memory"; +$CLICKHOUSE_CLIENT --query="CREATE TABLE template2 (s1 String, s2 String, s3 String, s4 String, n UInt64, d Date) ENGINE = Memory"; + +echo "==== check escaping ====" + +echo "{prefix} +n: 123, s1: qwe,rty , s2: 'as\"df\\'gh', s3: \"\", s4: \"zx +cv bn m\", d: 2016-01-01 ; +n: 456, s1: as\"df\\'gh , s2: '', s3: \"zx\\ncv\\tbn m\", s4: \"qwe,rty\", d: 2016-01-02 ; +n: 9876543210, s1: , s2: 'zx\\ncv\\tbn m', s3: \"qwe,rty\", s4: \"as\"\"df'gh\", d: 2016-01-03 ; +n: 789, s1: zx\\ncv\\tbn m , s2: 'qwe,rty', s3: \"as\\\"df'gh\", s4: \"\", d: 2016-01-04 + $ suffix $" | $CLICKHOUSE_CLIENT --query="INSERT INTO template1 FORMAT Template SETTINGS \ +format_schema = '{prefix} \n\${data}\n \$\$ suffix \$\$\n', \ +format_schema_rows = 'n:\t\${n:Escaped}, s1:\t\${s1:Escaped}\t, s2:\t\${s2:Quoted}, s3:\t\${s3:JSON}, s4:\t\${s4:CSV}, d:\t\${d:Escaped}\t', \ +format_schema_rows_between_delimiter = ';\n'"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT CSV"; + +echo "==== parse json (sophisticated template) ====" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \ +format_schema = '{\${:}\"meta\"\${:}:\${:}[\${:}{\${:}\"name\"\${:}:\${:}\"s1\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s2\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s3\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"s4\"\${:},\${:}\"type\"\${:}:\${:}\"String\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"n\"\${:},\${:}\"type\"\${:}:\${:}\"UInt64\"\${:}}\${:},\${:}{\${:}\"name\"\${:}:\${:}\"d\"\${:},\${:}\"type\"\${:}:\${:}\"Date\"\${:}}\${:}]\${:},\${:}\"data\"\${:}:\${:}[\${data}]\${:},\${:}\"rows\"\${:}:\${:}\${:CSV}\${:},\${:}\"statistics\"\${:}:\${:}{\${:}\"elapsed\"\${:}:\${:}\${:CSV}\${:},\${:}\"rows_read\"\${:}:\${:}\${:CSV}\${:},\${:}\"bytes_read\"\${:}:\${:}\${:CSV}\${:}}\${:}}', \ +format_schema_rows = '{\${:}\"s1\"\${:}:\${:}\${s1:JSON}\${:},\${:}\"s2\"\${:}:\${:}\${s2:JSON}\${:},\${:}\"s3\"\${:}:\${:}\${s3:JSON}\${:},\${:}\"s4\"\${:}:\${:}\${s4:JSON}\${:},\${:}\"n\"\${:}:\${:}\${n:JSON}\${:},\${:}\"d\"\${:}:\${:}\${d:JSON}\${:}\${:}}', \ +format_schema_rows_between_delimiter = ','"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV"; +$CLICKHOUSE_CLIENT --query="TRUNCATE TABLE template2"; + +echo "==== parse json ====" + +$CLICKHOUSE_CLIENT --query="SELECT * FROM template1 ORDER BY n FORMAT JSON" | $CLICKHOUSE_CLIENT --query="INSERT INTO template2 FORMAT TemplateIgnoreSpaces SETTINGS \ +format_schema = '{\${:}\"meta\"\${:}:\${:JSON},\${:}\"data\"\${:}:\${:}[\${data}]\${:},\${:}\"rows\"\${:}:\${:JSON},\${:}\"statistics\"\${:}:\${:JSON}\${:}}', \ +format_schema_rows = '{\${:}\"s1\"\${:}:\${:}\${s3:JSON}\${:},\${:}\"s2\"\${:}:\${:}\${:JSON}\${:},\${:}\"s3\"\${:}:\${:}\${s1:JSON}\${:},\${:}\"s4\"\${:}:\${:}\${:JSON}\${:},\${:}\"n\"\${:}:\${:}\${n:JSON}\${:},\${:}\"d\"\${:}:\${:}\${d:JSON}\${:}\${:}}', \ +format_schema_rows_between_delimiter = ','"; + +$CLICKHOUSE_CLIENT --query="SELECT * FROM template2 ORDER BY n FORMAT CSV"; + +$CLICKHOUSE_CLIENT --query="DROP TABLE template1"; +$CLICKHOUSE_CLIENT --query="DROP TABLE template2"; diff --git a/dbms/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.reference b/dbms/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.reference new file mode 100644 index 00000000000..6ed281c757a --- /dev/null +++ b/dbms/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.reference @@ -0,0 +1,2 @@ +1 +1 diff --git a/dbms/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.sh b/dbms/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.sh new file mode 100755 index 00000000000..58efd0f8bb8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01006_simpod_empty_part_single_column_write.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +. $CURDIR/mergetree_mutations.lib + + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS table_with_empty_part" + +${CLICKHOUSE_CLIENT} --query="CREATE TABLE table_with_empty_part +( + id UInt64, + value UInt64 +) +ENGINE = MergeTree() +ORDER BY id +PARTITION BY id +SETTINGS vertical_merge_algorithm_min_rows_to_activate=0, vertical_merge_algorithm_min_columns_to_activate=0 +" + + +${CLICKHOUSE_CLIENT} --query="INSERT INTO table_with_empty_part VALUES (1, 1)" + +${CLICKHOUSE_CLIENT} --query="INSERT INTO table_with_empty_part VALUES (2, 2)" + +${CLICKHOUSE_CLIENT} --query="ALTER TABLE table_with_empty_part DELETE WHERE id % 2 == 0" + +sleep 0.5 + +mutation_id=`${CLICKHOUSE_CLIENT} --query="SELECT max(mutation_id) FROM system.mutations WHERE table='table_with_empty_part'"` + +wait_for_mutation "table_with_empty_part" "$mutation_id" + +${CLICKHOUSE_CLIENT} --query="SELECT COUNT(DISTINCT value) FROM table_with_empty_part" + +${CLICKHOUSE_CLIENT} --query="ALTER TABLE table_with_empty_part MODIFY COLUMN value Nullable(UInt64)" + +${CLICKHOUSE_CLIENT} --query="SELECT COUNT(distinct value) FROM table_with_empty_part" + +${CLICKHOUSE_CLIENT} --query="OPTIMIZE TABLE table_with_empty_part FINAL" + +${CLICKHOUSE_CLIENT} --query="DROP TABLE IF EXISTS table_with_empty_part" diff --git a/dbms/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.reference b/dbms/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.reference new file mode 100644 index 00000000000..e69de29bb2d diff --git a/dbms/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh b/dbms/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh new file mode 100755 index 00000000000..e28cf5d9f7b --- /dev/null +++ b/dbms/tests/queries/0_stateless/01007_r1r2_w_r2r1_deadlock.sh @@ -0,0 +1,61 @@ +#!/usr/bin/env bash + +CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd) +. $CURDIR/../shell_config.sh + +set -e + +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS a" +$CLICKHOUSE_CLIENT --query "DROP TABLE IF EXISTS b" + +$CLICKHOUSE_CLIENT --query "CREATE TABLE a (x UInt8) ENGINE = MergeTree ORDER BY tuple()" +$CLICKHOUSE_CLIENT --query "CREATE TABLE b (x UInt8) ENGINE = MergeTree ORDER BY tuple()" + + +function thread1() +{ + while true; do + seq 1 100 | awk '{ print "SELECT x FROM a WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n + done +} + +function thread2() +{ + while true; do + seq 1 100 | awk '{ print "SELECT x FROM b WHERE x IN (SELECT toUInt8(count()) FROM system.tables);" }' | $CLICKHOUSE_CLIENT -n + done +} + +function thread3() +{ + while true; do + $CLICKHOUSE_CLIENT --query "ALTER TABLE a MODIFY COLUMN x Nullable(UInt8)" + $CLICKHOUSE_CLIENT --query "ALTER TABLE a MODIFY COLUMN x UInt8" + done +} + +function thread4() +{ + while true; do + $CLICKHOUSE_CLIENT --query "ALTER TABLE b MODIFY COLUMN x Nullable(UInt8)" + $CLICKHOUSE_CLIENT --query "ALTER TABLE b MODIFY COLUMN x UInt8" + done +} + +# https://stackoverflow.com/questions/9954794/execute-a-shell-function-with-timeout +export -f thread1; +export -f thread2; +export -f thread3; +export -f thread4; + +TIMEOUT=10 + +timeout $TIMEOUT bash -c thread1 2> /dev/null & +timeout $TIMEOUT bash -c thread2 2> /dev/null & +timeout $TIMEOUT bash -c thread3 2> /dev/null & +timeout $TIMEOUT bash -c thread4 2> /dev/null & + +wait + +$CLICKHOUSE_CLIENT --query "DROP TABLE a" +$CLICKHOUSE_CLIENT --query "DROP TABLE b" diff --git a/dbms/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.reference b/dbms/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.reference new file mode 100644 index 00000000000..2f3b99134b8 --- /dev/null +++ b/dbms/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.reference @@ -0,0 +1 @@ +0000-00-00 1 bar_n_1 1 diff --git a/dbms/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.sql b/dbms/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.sql new file mode 100644 index 00000000000..8deec159dca --- /dev/null +++ b/dbms/tests/queries/0_stateless/01008_materialized_view_henyihanwobushi.sql @@ -0,0 +1,14 @@ +DROP TABLE IF EXISTS foo; +DROP TABLE IF EXISTS bar; +DROP TABLE IF EXISTS view_foo_bar; + +create table foo (ddate Date, id Int64, n String) ENGINE = ReplacingMergeTree(ddate, (id), 8192); +create table bar (ddate Date, id Int64, n String, foo_id Int64) ENGINE = ReplacingMergeTree(ddate, (id), 8192); +insert into bar (id, n, foo_id) values (1, 'bar_n_1', 1); +create MATERIALIZED view view_foo_bar ENGINE = ReplacingMergeTree(ddate, (bar_id), 8192) as select ddate, bar_id, bar_n, foo_id, foo_n from (select ddate, id as bar_id, n as bar_n, foo_id from bar) any left join (select id as foo_id, n as foo_n from foo) using foo_id; +insert into bar (id, n, foo_id) values (1, 'bar_n_1', 1); +SELECT * FROM view_foo_bar; + +DROP TABLE foo; +DROP TABLE bar; +DROP TABLE view_foo_bar; diff --git a/docs/en/interfaces/formats.md b/docs/en/interfaces/formats.md index 2f409706c61..9590665df54 100644 --- a/docs/en/interfaces/formats.md +++ b/docs/en/interfaces/formats.md @@ -11,6 +11,8 @@ The supported formats are: | [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ | | [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | | [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [Template](#template) | ✔ | ✔ | +| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | | [CSV](#csv) | ✔ | ✔ | | [CSVWithNames](#csvwithnames) | ✔ | ✔ | | [Values](#data-format-values) | ✔ | ✔ | @@ -119,6 +121,121 @@ During parsing, the first and second rows are completely ignored. This format is also available under the name `TSVWithNamesAndTypes`. +## Template {#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} Similar to TabSeparated, but outputs a value in name=value format. Names are escaped the same way as in TabSeparated format, and the = symbol is also escaped. diff --git a/docs/en/operations/settings/settings.md b/docs/en/operations/settings/settings.md index 02d5871894b..9c711073177 100644 --- a/docs/en/operations/settings/settings.md +++ b/docs/en/operations/settings/settings.md @@ -72,6 +72,9 @@ Works with tables in the MergeTree family. If `force_primary_key=1`, ClickHouse checks to see if the query has a primary key condition that can be used for restricting data ranges. If there is no suitable condition, it throws an exception. However, it does not check whether the condition actually reduces the amount of data to read. For more information about data ranges in MergeTree tables, see "[MergeTree](../../operations/table_engines/mergetree.md)". +## 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. ## fsync_metadata @@ -573,10 +576,6 @@ If a query from the same user with the same 'query_id' already exists at this ti Yandex.Metrica uses this parameter set to 1 for implementing suggestions for segmentation conditions. After entering the next character, if the old query hasn't finished yet, it should be canceled. -## schema - -This parameter is useful when you are using formats that require a schema definition, such as [Cap'n Proto](https://capnproto.org/). The value depends on the format. - ## stream_flush_interval_ms diff --git a/docs/en/operations/table_engines/collapsingmergetree.md b/docs/en/operations/table_engines/collapsingmergetree.md index b4f7bd8e6cf..67cad64e950 100644 --- a/docs/en/operations/table_engines/collapsingmergetree.md +++ b/docs/en/operations/table_engines/collapsingmergetree.md @@ -214,4 +214,66 @@ SELECT * FROM UAct FINAL This way of selecting the data is very inefficient. Don't use it for big tables. +## Example of another approach + +Example data: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ -5 │ -146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + +The idea is that merges take into account only key fields. And in the "Cancel" line we can specify negative values that equalize the previous version of the row when summing without using the Sign column. For this approach, it is necessary to change the data type `PageViews`,` Duration` to store negative values of UInt8 -> Int16. + +```sql +CREATE TABLE UAct +( + UserID UInt64, + PageViews Int16, + Duration Int16, + Sign Int8 +) +ENGINE = CollapsingMergeTree(Sign) +ORDER BY UserID +``` + +Let's test the approach: +```sql +insert into UAct values(4324182021466249494, 5, 146, 1); +insert into UAct values(4324182021466249494, -5, -146, -1); +insert into UAct values(4324182021466249494, 6, 185, 1); + +select * from UAct final; // avoid using final in production (just for a test or small tables) +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ + +SELECT + UserID, + sum(PageViews) AS PageViews, + sum(Duration) AS Duration +FROM UAct +GROUP BY UserID +┌──────────────UserID─┬─PageViews─┬─Duration─┐ +│ 4324182021466249494 │ 6 │ 185 │ +└─────────────────────┴───────────┴──────────┘ + +select count() FROM UAct +┌─count()─┐ +│ 3 │ +└─────────┘ + +optimize table UAct final; + +select * FROM UAct +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + + + [Original article](https://clickhouse.yandex/docs/en/operations/table_engines/collapsingmergetree/) diff --git a/docs/en/query_language/functions/other_functions.md b/docs/en/query_language/functions/other_functions.md index 349397059af..7a883bb0464 100644 --- a/docs/en/query_language/functions/other_functions.md +++ b/docs/en/query_language/functions/other_functions.md @@ -394,6 +394,33 @@ FROM └─────────┴─────────────────────┴───────┘ ``` +Please note - block size affects the result. With each new block, the `runningDifference` state is reset. + +``` sql +SELECT + number, + runningDifference(number + 1) AS diff +FROM numbers(100000) +WHERE diff != 1 +┌─number─┬─diff─┐ +│ 0 │ 0 │ +└────────┴──────┘ +┌─number─┬─diff─┐ +│ 65536 │ 0 │ +└────────┴──────┘ + +set max_block_size=100000 // default value is 65536! + +SELECT + number, + runningDifference(number + 1) AS diff +FROM numbers(100000) +WHERE diff != 1 +┌─number─┬─diff─┐ +│ 0 │ 0 │ +└────────┴──────┘ +``` + ## runningDifferenceStartingWithFirstValue Same as for [runningDifference](./other_functions.md#other_functions-runningdifference), the difference is the value of the first row, returned the value of the first row, and each subsequent row returns the difference from the previous row. diff --git a/docs/ru/interfaces/formats.md b/docs/ru/interfaces/formats.md index 130a32d63fa..8f9d100ad91 100644 --- a/docs/ru/interfaces/formats.md +++ b/docs/ru/interfaces/formats.md @@ -10,6 +10,8 @@ ClickHouse может принимать (`INSERT`) и отдавать (`SELECT | [TabSeparatedRaw](#tabseparatedraw) | ✗ | ✔ | | [TabSeparatedWithNames](#tabseparatedwithnames) | ✔ | ✔ | | [TabSeparatedWithNamesAndTypes](#tabseparatedwithnamesandtypes) | ✔ | ✔ | +| [Template](#template) | ✔ | ✔ | +| [TemplateIgnoreSpaces](#templateignorespaces) | ✔ | ✗ | | [CSV](#csv) | ✔ | ✔ | | [CSVWithNames](#csvwithnames) | ✔ | ✔ | | [Values](#data-format-values) | ✔ | ✔ | @@ -118,6 +120,120 @@ world Этот формат также доступен под именем `TSVWithNamesAndTypes`. +## Template {#template} + +Этот формат позволяет указать произвольную форматную строку, в которую подставляются значения, сериализованные выбранным способом. + +Для этого используются настройки `format_schema`, `format_schema_rows`, `format_schema_rows_between_delimiter` и настройки экранирования других форматов (например, `output_format_json_quote_64bit_integers` при экранировании как в `JSON`, см. далее) + +Форматная строка `format_schema_rows` задаёт формат для строк таблицы и должна иметь вид: + + `delimiter_1${column_1:serializeAs_1}delimiter_2${column_2:serializeAs_2} ... delimiter_N`, + + где `delimiter_i` - разделители между значениями (символ `$` в разделителе экранируется как `$$`), + `column_i` - имена столбцов, значения которых должны быть выведены или считаны (если имя не указано - столбец пропускается), + `serializeAs_i` - тип экранирования для значений соответствующего столбца. Поддерживаются следующие типы экранирования: + + - `CSV`, `JSON`, `XML` (как в одноимённых форматах) + - `Escaped` (как в `TSV`) + - `Quoted` (как в `Values`) + - `Raw` (без экранирования, как в `TSVRaw`) + - `None` (тип экранирования отсутствует, см. далее) + + Если для столбца не указан тип экранирования, используется `None`. `XML` и `Raw` поддерживаются только для вывода. + + Так, в форматной строке + + `Search phrase: ${SearchPhrase:Quoted}, count: ${c:Escaped}, ad price: $$${price:JSON};` + + между разделителями `Search phrase: `, `, count: `, `, ad price: $` и `;` при выводе будут подставлены (при вводе - будут ожидаться) значения столбцов `SearchPhrase`, `c` и `price`, сериализованные как `Quoted`, `Escaped` и `JSON` соответственно, например: + + `Search phrase: 'bathroom interior design', count: 2166, ad price: $3;` + + Настройка `format_schema_rows_between_delimiter` задаёт разделитель между строками, который выводится (или ожмдается при вводе) после каждой строки, кроме последней. По умолчанию `\n`. + +Форматная строка `format_schema` имеет аналогичный `format_schema_rows` синтаксис и позволяет указать префикс, суффикс и способ вывода дополнительной информации. Вместо имён столбцов в ней указываются следующие имена подстановок: + + - `data` - строки с данными в формате `format_schema_rows`, разделённые `format_schema_rows_between_delimiter`. Эта подстановка должна быть первой подстановкой в форматной строке. + - `totals` - строка с тотальными значениями в формате `format_schema_rows` (при использовании WITH TOTALS) + - `min` - строка с минимальными значениями в формате `format_schema_rows` (при настройке extremes, выставленной в 1) + - `max` - строка с максимальными значениями в формате `format_schema_rows` (при настройке extremes, выставленной в 1) + - `rows` - общее количество выведенных стрчек + - `rows_before_limit` - не менее скольких строчек получилось бы, если бы не было LIMIT-а. Выводится только если запрос содержит LIMIT. В случае, если запрос содержит GROUP BY, `rows_before_limit` - точное число строк, которое получилось бы, если бы не было LIMIT-а. + - `time` - время выполнения запроса в секундах + - `rows_read` - сколько строк было прочитано при выполнении запроса + - `bytes_read` - сколько байт (несжатых) было прочитано при выполнении запроса + + У подстановок `data`, `totals`, `min` и `max` не должны быть указаны типы экранирования (или должен быть указан `None`). Остальные подстановки - это отдельные значения, для них может быть указан любой тип экранирования. + Если строка `format_schema` пустая, то по-умолчанию используется `${data}`. + Из всех перечисленных подстановок форматная строка `format_schema` для ввода может содержать только `data`. + Также при вводе формат поддерживает пропуск значений столбцов и пропуск значений в префиксе и суффиксе (см. пример). + + Пример вывода: +```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 + + +``` + +Пример ввода: +``` +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` и `Sign` внутри подстановок - имена столбцов в таблице, в которую вставляются данные. Значения после `Useless field` в строках и значение после `\nTotal rows: ` в суффиксе будут проигнорированы. +Все разделители во входных данных должны строго соответствовать разделителям в форматных строках. + +## TemplateIgnoreSpaces {#templateignorespaces} + +Подходит только для ввода. Отличается от формата `Template` тем, что пропускает пробельные символы между разделителями и значениями во входном потоке. Также в этом формате можно указать пустые подстановки с типом экранирования `None` (`${}` или `${:None}`), чтобы разбить разделители на несколько частей, пробелы между которыми должны игнорироваться. Такие подстановки используются только для пропуска пробелов. С помощью этого формата можно считывать `JSON`, если значения столбцов в нём всегда идут в одном порядке в каждой строке. Например, для вставки данных из примера вывода формата [JSON](#json) в таблицу со столбцами `phrase` и `cnt` можно использовать следующий запрос: +```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 и, дополнительно, экранируется также символ =. diff --git a/docs/ru/operations/settings/settings.md b/docs/ru/operations/settings/settings.md index 07c1e5ae977..831f5958c29 100644 --- a/docs/ru/operations/settings/settings.md +++ b/docs/ru/operations/settings/settings.md @@ -70,6 +70,10 @@ ClickHouse применяет настройку в тех случаях, ко При `force_primary_key=1` ClickHouse проверяет, есть ли в запросе условие на первичный ключ, которое может использоваться для отсечения диапазонов данных. Если подходящего условия нет - кидается исключение. При этом не проверяется, действительно ли условие уменьшает объём данных для чтения. Подробнее про диапазоны данных в таблицах MergeTree читайте в разделе "[MergeTree](../../operations/table_engines/mergetree.md)". +## format_schema + +Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/), [Protobuf](https://developers.google.com/protocol-buffers/) или [Template](../../interfaces/formats.md#template). Значение параметра зависит от формата. + ## fsync_metadata Включает или отключает [fsync](http://pubs.opengroup.org/onlinepubs/9699919799/functions/fsync.html) при записи `.sql` файлов. По умолчанию включено. @@ -579,10 +583,6 @@ ClickHouse использует этот параметр при чтении д Эта настройка, выставленная в 1, используется в Яндекс.Метрике для реализации suggest-а значений для условий сегментации. После ввода очередного символа, если старый запрос ещё не выполнился, его следует отменить. -## schema - -Параметр применяется в том случае, когда используются форматы, требующие определения схемы, например [Cap'n Proto](https://capnproto.org/). Значение параметра зависит от формата. - ## stream_flush_interval_ms Работает для таблиц со стриммингом в случае тайм-аута, или когда поток генерирует [max_insert_block_size](#settings-max_insert_block_size) строк. diff --git a/docs/ru/operations/table_engines/collapsingmergetree.md b/docs/ru/operations/table_engines/collapsingmergetree.md index 59fdf66324f..17e85bcca0c 100644 --- a/docs/ru/operations/table_engines/collapsingmergetree.md +++ b/docs/ru/operations/table_engines/collapsingmergetree.md @@ -220,5 +220,65 @@ SELECT * FROM UAct FINAL Такой способ выбора данных очень неэффективен. Не используйте его для больших таблиц. +## Пример другого подхода + +Исходные данные: + +``` +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 5 │ 146 │ 1 │ +│ 4324182021466249494 │ -5 │ -146 │ -1 │ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` +Идея состоит в том, что слияния при сворачивании учитывают только ключевые поля, поэтому в отменяющей строке можно указать отрицательные значения, которые нивелируют предыдущую версию записи при суммировании без учета поля Sign. +Для этого подхода необходимо изменить тип данных `PageViews`, `Duration` для хранения отрицательных значений UInt8 -> Int16. + +```sql +CREATE TABLE UAct +( + UserID UInt64, + PageViews Int16, + Duration Int16, + Sign Int8 +) +ENGINE = CollapsingMergeTree(Sign) +ORDER BY UserID +``` + +Тестируем подход: +```sql +insert into UAct values(4324182021466249494, 5, 146, 1); +insert into UAct values(4324182021466249494, -5, -146, -1); +insert into UAct values(4324182021466249494, 6, 185, 1); + +select * from UAct final; // старайтесь не использовать final (он подходит только для тестов и маленьких таблиц) +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ + +SELECT + UserID, + sum(PageViews) AS PageViews, + sum(Duration) AS Duration +FROM UAct +GROUP BY UserID +┌──────────────UserID─┬─PageViews─┬─Duration─┐ +│ 4324182021466249494 │ 6 │ 185 │ +└─────────────────────┴───────────┴──────────┘ + +select count() FROM UAct +┌─count()─┐ +│ 3 │ +└─────────┘ + +optimize table UAct final; + +select * FROM UAct +┌──────────────UserID─┬─PageViews─┬─Duration─┬─Sign─┐ +│ 4324182021466249494 │ 6 │ 185 │ 1 │ +└─────────────────────┴───────────┴──────────┴──────┘ +``` + [Оригинальная статья](https://clickhouse.yandex/docs/ru/operations/table_engines/collapsingmergetree/) diff --git a/docs/ru/query_language/functions/other_functions.md b/docs/ru/query_language/functions/other_functions.md index fdc46a0d4ee..44bf76b7047 100644 --- a/docs/ru/query_language/functions/other_functions.md +++ b/docs/ru/query_language/functions/other_functions.md @@ -369,6 +369,39 @@ FROM └─────────┴─────────────────────┴───────┘ ``` +Обратите внимание — размер блока влияет на результат. С каждым новым блоком состояние `runningDifference` сбрасывается. + +``` sql +SELECT + number, + runningDifference(number + 1) AS diff +FROM numbers(100000) +WHERE diff != 1 + +┌─number─┬─diff─┐ +│ 0 │ 0 │ +└────────┴──────┘ +┌─number─┬─diff─┐ +│ 65536 │ 0 │ +└────────┴──────┘ + + +set max_block_size=100000 // по умолчанию 65536! + +SELECT + number, + runningDifference(number + 1) AS diff +FROM numbers(100000) +WHERE diff != 1 + +┌─number─┬─diff─┐ +│ 0 │ 0 │ +└────────┴──────┘ +``` + +## runningDifferenceStartingWithFirstValue +То же, что и [runningDifference] (./other_functions.md # other_functions-runningdifference), но в первой строке возвращается значение первой строки, а не ноль. + ## MACNumToString(num) Принимает число типа UInt64. Интерпретирует его, как MAC-адрес в big endian. Возвращает строку, содержащую соответствующий MAC-адрес в формате AA:BB:CC:DD:EE:FF (числа в шестнадцатеричной форме через двоеточие). diff --git a/libs/libcommon/src/DateLUTImpl.cpp b/libs/libcommon/src/DateLUTImpl.cpp index 3f812accb48..51f5ceb759c 100644 --- a/libs/libcommon/src/DateLUTImpl.cpp +++ b/libs/libcommon/src/DateLUTImpl.cpp @@ -16,6 +16,7 @@ #include #include #include +#include #include #define DATE_LUT_MIN 0 @@ -44,9 +45,16 @@ UInt8 getDayOfWeek(const cctz::civil_day & date) } +__attribute__((__weak__)) extern bool inside_main; + DateLUTImpl::DateLUTImpl(const std::string & time_zone_) : time_zone(time_zone_) { + /// DateLUT should not be initialized in global constructors for the following reasons: + /// 1. It is too heavy. + if (&inside_main) + assert(inside_main); + size_t i = 0; time_t start_of_day = DATE_LUT_MIN; diff --git a/libs/libdaemon/src/BaseDaemon.cpp b/libs/libdaemon/src/BaseDaemon.cpp index 5c51b357f8a..807506775ae 100644 --- a/libs/libdaemon/src/BaseDaemon.cpp +++ b/libs/libdaemon/src/BaseDaemon.cpp @@ -597,10 +597,12 @@ void BaseDaemon::initialize(Application & self) /// This must be done before any usage of DateLUT. In particular, before any logging. if (config().has("timezone")) { - if (0 != setenv("TZ", config().getString("timezone").data(), 1)) + const std::string timezone = config().getString("timezone"); + if (0 != setenv("TZ", timezone.data(), 1)) throw Poco::Exception("Cannot setenv TZ variable"); tzset(); + DateLUT::setDefaultTimezone(timezone); } std::string log_path = config().getString("logger.log", "");