From 8d14f2ef8fd99d4f751f983f8500845902b9dfec Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Wed, 4 Aug 2021 20:58:18 +0300 Subject: [PATCH 1/4] Streams -> Processors for dicts, part 1. --- src/Dictionaries/CacheDictionary.cpp | 33 ++-- src/Dictionaries/CacheDictionary.h | 2 +- .../CassandraBlockInputStream.cpp | 26 ++-- src/Dictionaries/CassandraBlockInputStream.h | 10 +- .../CassandraDictionarySource.cpp | 22 ++- src/Dictionaries/CassandraDictionarySource.h | 8 +- .../ClickHouseDictionarySource.cpp | 49 +++--- src/Dictionaries/ClickHouseDictionarySource.h | 12 +- .../DictionaryBlockInputStream.cpp | 24 ++- src/Dictionaries/DictionaryBlockInputStream.h | 32 ++-- .../DictionaryBlockInputStreamBase.cpp | 21 +-- .../DictionaryBlockInputStreamBase.h | 10 +- src/Dictionaries/DictionaryHelpers.h | 19 ++- src/Dictionaries/DictionarySourceHelpers.cpp | 60 +++----- src/Dictionaries/DictionarySourceHelpers.h | 17 +-- src/Dictionaries/DirectDictionary.cpp | 37 ++--- src/Dictionaries/DirectDictionary.h | 6 +- .../ExecutableDictionarySource.cpp | 142 ++++++++++-------- src/Dictionaries/ExecutableDictionarySource.h | 12 +- .../ExecutablePoolDictionarySource.cpp | 95 ++++++------ .../ExecutablePoolDictionarySource.h | 12 +- src/Dictionaries/FileDictionarySource.cpp | 11 +- src/Dictionaries/FileDictionarySource.h | 10 +- src/Dictionaries/FlatDictionary.cpp | 35 +++-- src/Dictionaries/FlatDictionary.h | 4 +- src/Dictionaries/HashedDictionary.cpp | 8 +- src/Dictionaries/HashedDictionary.h | 4 +- src/Dictionaries/IDictionary.h | 2 +- src/Dictionaries/IDictionarySource.h | 29 ++-- src/Dictionaries/readInvalidateQuery.cpp | 19 ++- src/Dictionaries/readInvalidateQuery.h | 6 +- 31 files changed, 407 insertions(+), 370 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 8a8a64fab36..a8754691425 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -13,6 +13,9 @@ #include #include +#include +#include + namespace ProfileEvents { extern const Event DictCacheKeysRequested; @@ -481,24 +484,28 @@ MutableColumns CacheDictionary::aggregateColumns( } template -BlockInputStreamPtr CacheDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe CacheDictionary::read(const Names & column_names, size_t max_block_size) const { - std::shared_ptr stream; + Pipe pipe; { /// Write lock on storage const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; if constexpr (dictionary_key_type == DictionaryKeyType::simple) - stream = std::make_shared(shared_from_this(), max_block_size, cache_storage_ptr->getCachedSimpleKeys(), column_names); + pipe = Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names), + max_block_size)); else { auto keys = cache_storage_ptr->getCachedComplexKeys(); - stream = std::make_shared(shared_from_this(), max_block_size, keys, column_names); + pipe = Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), keys, column_names), + max_block_size)); } } - return stream; + return pipe; } template @@ -567,21 +574,21 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrloadIds(requested_keys_vector); + pipeline.init(current_source_ptr->loadIds(requested_keys_vector)); else - stream = current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows); - - stream->readPrefix(); + pipeline.init(current_source_ptr->loadKeys(update_unit_ptr->key_columns, requested_complex_key_rows)); size_t skip_keys_size_offset = dict_struct.getKeysSize(); PaddedPODArray found_keys_in_source; Columns fetched_columns_during_update = fetch_request.makeAttributesResultColumnsNonMutable(); - while (Block block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { Columns key_columns; key_columns.reserve(skip_keys_size_offset); @@ -625,8 +632,6 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtrassumeMutable()); - stream->readSuffix(); - { /// Lock for cache modification ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; @@ -686,4 +691,4 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr; template class CacheDictionary; -} +} \ No newline at end of file diff --git a/src/Dictionaries/CacheDictionary.h b/src/Dictionaries/CacheDictionary.h index baaf99d290b..613d73b0f83 100644 --- a/src/Dictionaries/CacheDictionary.h +++ b/src/Dictionaries/CacheDictionary.h @@ -137,7 +137,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; std::exception_ptr getLastException() const override; diff --git a/src/Dictionaries/CassandraBlockInputStream.cpp b/src/Dictionaries/CassandraBlockInputStream.cpp index 57a4555ea87..384717e2ba2 100644 --- a/src/Dictionaries/CassandraBlockInputStream.cpp +++ b/src/Dictionaries/CassandraBlockInputStream.cpp @@ -22,12 +22,13 @@ namespace ErrorCodes extern const int UNKNOWN_TYPE; } -CassandraBlockInputStream::CassandraBlockInputStream( +CassandraSource::CassandraSource( const CassSessionShared & session_, const String & query_str, const Block & sample_block, size_t max_block_size_) - : session(session_) + : SourceWithProgress(sample_block) + , session(session_) , statement(query_str.c_str(), /*parameters count*/ 0) , max_block_size(max_block_size_) , has_more_pages(cass_true) @@ -36,7 +37,7 @@ CassandraBlockInputStream::CassandraBlockInputStream( cassandraCheck(cass_statement_set_paging_size(statement, max_block_size)); } -void CassandraBlockInputStream::insertValue(IColumn & column, ValueType type, const CassValue * cass_value) +void CassandraSource::insertValue(IColumn & column, ValueType type, const CassValue * cass_value) { switch (type) { @@ -148,13 +149,15 @@ void CassandraBlockInputStream::insertValue(IColumn & column, ValueType type, co } } -void CassandraBlockInputStream::readPrefix() -{ - result_future = cass_session_execute(*session, statement); -} -Block CassandraBlockInputStream::readImpl() +Chunk CassandraSource::generate() { + if (!is_initialized) + { + result_future = cass_session_execute(*session, statement); + is_initialized = true; + } + if (!has_more_pages) return {}; @@ -194,12 +197,13 @@ Block CassandraBlockInputStream::readImpl() } } - assert(cass_result_row_count(result) == columns.front()->size()); + size_t num_rows = columns.front()->size(); + assert(cass_result_row_count(result) == num_rows); - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } -void CassandraBlockInputStream::assertTypes(const CassResultPtr & result) +void CassandraSource::assertTypes(const CassResultPtr & result) { if (!assert_types) return; diff --git a/src/Dictionaries/CassandraBlockInputStream.h b/src/Dictionaries/CassandraBlockInputStream.h index 3b0e583e3ad..98adb19fee6 100644 --- a/src/Dictionaries/CassandraBlockInputStream.h +++ b/src/Dictionaries/CassandraBlockInputStream.h @@ -4,17 +4,17 @@ #if USE_CASSANDRA #include -#include +#include #include namespace DB { -class CassandraBlockInputStream final : public IBlockInputStream +class CassandraSource final : public SourceWithProgress { public: - CassandraBlockInputStream( + CassandraSource( const CassSessionShared & session_, const String & query_str, const Block & sample_block, @@ -24,12 +24,11 @@ public: Block getHeader() const override { return description.sample_block.cloneEmpty(); } - void readPrefix() override; private: using ValueType = ExternalResultDescription::ValueType; - Block readImpl() override; + Chunk generate() override; static void insertValue(IColumn & column, ValueType type, const CassValue * cass_value); void assertTypes(const CassResultPtr & result); @@ -40,6 +39,7 @@ private: ExternalResultDescription description; cass_bool_t has_more_pages; bool assert_types = true; + bool is_initialized = false; }; } diff --git a/src/Dictionaries/CassandraDictionarySource.cpp b/src/Dictionaries/CassandraDictionarySource.cpp index 7605b86ef90..8b31b4d6fa2 100644 --- a/src/Dictionaries/CassandraDictionarySource.cpp +++ b/src/Dictionaries/CassandraDictionarySource.cpp @@ -40,7 +40,6 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory) #include #include "CassandraBlockInputStream.h" #include -#include namespace DB { @@ -132,12 +131,12 @@ void CassandraDictionarySource::maybeAllowFiltering(String & query) const query += " ALLOW FILTERING;"; } -BlockInputStreamPtr CassandraDictionarySource::loadAll() +Pipe CassandraDictionarySource::loadAll() { String query = query_builder.composeLoadAllQuery(); maybeAllowFiltering(query); LOG_INFO(log, "Loading all using query: {}", query); - return std::make_shared(getSession(), query, sample_block, max_block_size); + return Pipe(std::make_shared(getSession(), query, sample_block, max_block_size)); } std::string CassandraDictionarySource::toString() const @@ -145,15 +144,15 @@ std::string CassandraDictionarySource::toString() const return "Cassandra: " + settings.db + '.' + settings.table; } -BlockInputStreamPtr CassandraDictionarySource::loadIds(const std::vector & ids) +Pipe CassandraDictionarySource::loadIds(const std::vector & ids) { String query = query_builder.composeLoadIdsQuery(ids); maybeAllowFiltering(query); LOG_INFO(log, "Loading ids using query: {}", query); - return std::make_shared(getSession(), query, sample_block, max_block_size); + return Pipe(std::make_shared(getSession(), query, sample_block, max_block_size)); } -BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (requested_rows.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "No rows requested"); @@ -168,22 +167,19 @@ BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_colu partitions[partition_key.get64()].push_back(row); } - BlockInputStreams streams; + Pipes pipes; for (const auto & partition : partitions) { String query = query_builder.composeLoadKeysQuery(key_columns, partition.second, ExternalQueryBuilder::CASSANDRA_SEPARATE_PARTITION_KEY, settings.partition_key_prefix); maybeAllowFiltering(query); LOG_INFO(log, "Loading keys for partition hash {} using query: {}", partition.first, query); - streams.push_back(std::make_shared(getSession(), query, sample_block, max_block_size)); + pipes.push_back(Pipe(std::make_shared(getSession(), query, sample_block, max_block_size))); } - if (streams.size() == 1) - return streams.front(); - - return std::make_shared(streams, nullptr, settings.max_threads); + return Pipe::unitePipes(std::move(pipes)); } -BlockInputStreamPtr CassandraDictionarySource::loadUpdatedAll() +Pipe CassandraDictionarySource::loadUpdatedAll() { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for CassandraDictionarySource"); } diff --git a/src/Dictionaries/CassandraDictionarySource.h b/src/Dictionaries/CassandraDictionarySource.h index c0a4e774d23..871e3dc4857 100644 --- a/src/Dictionaries/CassandraDictionarySource.h +++ b/src/Dictionaries/CassandraDictionarySource.h @@ -49,7 +49,7 @@ public: const String & config_prefix, Block & sample_block); - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; bool supportsSelectiveLoad() const override { return true; } @@ -62,11 +62,11 @@ public: return std::make_unique(dict_struct, settings, sample_block); } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; String toString() const override; diff --git a/src/Dictionaries/ClickHouseDictionarySource.cpp b/src/Dictionaries/ClickHouseDictionarySource.cpp index 42ec73ee520..8b2373302c8 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.cpp +++ b/src/Dictionaries/ClickHouseDictionarySource.cpp @@ -1,8 +1,11 @@ #include "ClickHouseDictionarySource.h" #include #include -#include -#include +#include +#include +#include +#include +#include #include #include #include @@ -105,29 +108,29 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate() } } -BlockInputStreamPtr ClickHouseDictionarySource::loadAllWithSizeHint(std::atomic * result_size_hint) +Pipe ClickHouseDictionarySource::loadAllWithSizeHint(std::atomic * result_size_hint) { return createStreamForQuery(load_all_query, result_size_hint); } -BlockInputStreamPtr ClickHouseDictionarySource::loadAll() +Pipe ClickHouseDictionarySource::loadAll() { return createStreamForQuery(load_all_query); } -BlockInputStreamPtr ClickHouseDictionarySource::loadUpdatedAll() +Pipe ClickHouseDictionarySource::loadUpdatedAll() { String load_update_query = getUpdateFieldAndDate(); return createStreamForQuery(load_update_query); } -BlockInputStreamPtr ClickHouseDictionarySource::loadIds(const std::vector & ids) +Pipe ClickHouseDictionarySource::loadIds(const std::vector & ids) { return createStreamForQuery(query_builder.composeLoadIdsQuery(ids)); } -BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { String query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES); return createStreamForQuery(query); @@ -157,32 +160,41 @@ std::string ClickHouseDictionarySource::toString() const return "ClickHouse: " + configuration.db + '.' + configuration.table + (where.empty() ? "" : ", where: " + where); } -BlockInputStreamPtr ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic * result_size_hint) +Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic * result_size_hint) { - BlockInputStreamPtr stream; + QueryPipeline pipeline; /// Sample block should not contain first row default values auto empty_sample_block = sample_block.cloneEmpty(); if (configuration.is_local) { - stream = executeQuery(query, context, true).getInputStream(); - stream = std::make_shared(stream, empty_sample_block, ConvertingBlockInputStream::MatchColumnsMode::Position); + pipeline = executeQuery(query, context, true).pipeline; + auto converting = ActionsDAG::makeConvertingActions( + pipeline.getHeader().getColumnsWithTypeAndName(), + empty_sample_block.getColumnsWithTypeAndName(), + ActionsDAG::MatchColumnsMode::Position); + + pipeline.addSimpleTransform([&](const Block & header) + { + return std::make_shared(header, std::make_shared(converting)); + }); } else { - stream = std::make_shared(pool, query, empty_sample_block, context); + pipeline.init(Pipe(std::make_shared( + std::make_shared(pool, query, empty_sample_block, context), false, false))); } if (result_size_hint) { - stream->setProgressCallback([result_size_hint](const Progress & progress) + pipeline.setProgressCallback([result_size_hint](const Progress & progress) { *result_size_hint += progress.total_rows_to_read; }); } - return stream; + return QueryPipeline::getPipe(std::move(pipeline)); } std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & request) const @@ -191,15 +203,16 @@ std::string ClickHouseDictionarySource::doInvalidateQuery(const std::string & re if (configuration.is_local) { auto query_context = Context::createCopy(context); - auto input_block = executeQuery(request, query_context, true).getInputStream(); - return readInvalidateQuery(*input_block); + auto pipe = QueryPipeline::getPipe(executeQuery(request, query_context, true).pipeline); + return readInvalidateQuery(std::move(pipe)); } else { /// We pass empty block to RemoteBlockInputStream, because we don't know the structure of the result. Block invalidate_sample_block; - RemoteBlockInputStream invalidate_stream(pool, request, invalidate_sample_block, context); - return readInvalidateQuery(invalidate_stream); + Pipe pipe(std::make_shared( + std::make_shared(pool, request, invalidate_sample_block, context), false, false)); + return readInvalidateQuery(std::move(pipe)); } } diff --git a/src/Dictionaries/ClickHouseDictionarySource.h b/src/Dictionaries/ClickHouseDictionarySource.h index fe37610b9c4..f293c010ec3 100644 --- a/src/Dictionaries/ClickHouseDictionarySource.h +++ b/src/Dictionaries/ClickHouseDictionarySource.h @@ -44,15 +44,15 @@ public: ClickHouseDictionarySource(const ClickHouseDictionarySource & other); ClickHouseDictionarySource & operator=(const ClickHouseDictionarySource &) = delete; - BlockInputStreamPtr loadAllWithSizeHint(std::atomic * result_size_hint) override; + Pipe loadAllWithSizeHint(std::atomic * result_size_hint) override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; bool supportsSelectiveLoad() const override { return true; } @@ -70,7 +70,7 @@ public: private: std::string getUpdateFieldAndDate(); - BlockInputStreamPtr createStreamForQuery(const String & query, std::atomic * result_size_hint = nullptr); + Pipe createStreamForQuery(const String & query, std::atomic * result_size_hint = nullptr); std::string doInvalidateQuery(const std::string & request) const; diff --git a/src/Dictionaries/DictionaryBlockInputStream.cpp b/src/Dictionaries/DictionaryBlockInputStream.cpp index c902a87cfe3..fedde8bd886 100644 --- a/src/Dictionaries/DictionaryBlockInputStream.cpp +++ b/src/Dictionaries/DictionaryBlockInputStream.cpp @@ -8,9 +8,9 @@ namespace ErrorCodes extern const int LOGICAL_ERROR; } -DictionaryBlockInputStream::DictionaryBlockInputStream( - std::shared_ptr dictionary_, UInt64 max_block_size_, PaddedPODArray && ids_, const Names & column_names_) - : DictionaryBlockInputStreamBase(ids_.size(), max_block_size_) +DictionarySourceData::DictionarySourceData( + std::shared_ptr dictionary_, PaddedPODArray && ids_, const Names & column_names_) + : num_rows(ids_.size()) , dictionary(dictionary_) , column_names(column_names_) , ids(std::move(ids_)) @@ -18,12 +18,11 @@ DictionaryBlockInputStream::DictionaryBlockInputStream( { } -DictionaryBlockInputStream::DictionaryBlockInputStream( +DictionarySourceData::DictionarySourceData( std::shared_ptr dictionary_, - UInt64 max_block_size_, const PaddedPODArray & keys, const Names & column_names_) - : DictionaryBlockInputStreamBase(keys.size(), max_block_size_) + : num_rows(keys.size()) , dictionary(dictionary_) , column_names(column_names_) , key_type(DictionaryInputStreamKeyType::ComplexKey) @@ -32,14 +31,13 @@ DictionaryBlockInputStream::DictionaryBlockInputStream( fillKeyColumns(keys, 0, keys.size(), dictionary_structure, key_columns); } -DictionaryBlockInputStream::DictionaryBlockInputStream( +DictionarySourceData::DictionarySourceData( std::shared_ptr dictionary_, - UInt64 max_block_size_, const Columns & data_columns_, const Names & column_names_, GetColumnsFunction && get_key_columns_function_, GetColumnsFunction && get_view_columns_function_) - : DictionaryBlockInputStreamBase(data_columns_.front()->size(), max_block_size_) + : num_rows(data_columns_.front()->size()) , dictionary(dictionary_) , column_names(column_names_) , data_columns(data_columns_) @@ -49,7 +47,7 @@ DictionaryBlockInputStream::DictionaryBlockInputStream( { } -Block DictionaryBlockInputStream::getBlock(size_t start, size_t length) const +Block DictionarySourceData::getBlock(size_t start, size_t length) const { /// TODO: Rewrite switch (key_type) @@ -98,7 +96,7 @@ Block DictionaryBlockInputStream::getBlock(size_t start, size_t length) const throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected DictionaryInputStreamKeyType."); } -Block DictionaryBlockInputStream::fillBlock( +Block DictionarySourceData::fillBlock( const PaddedPODArray & ids_to_fill, const Columns & keys, const DataTypes & types, @@ -161,14 +159,14 @@ Block DictionaryBlockInputStream::fillBlock( return Block(block_columns); } -ColumnPtr DictionaryBlockInputStream::getColumnFromIds(const PaddedPODArray & ids_to_fill) +ColumnPtr DictionarySourceData::getColumnFromIds(const PaddedPODArray & ids_to_fill) { auto column_vector = ColumnVector::create(); column_vector->getData().assign(ids_to_fill); return column_vector; } -void DictionaryBlockInputStream::fillKeyColumns( +void DictionarySourceData::fillKeyColumns( const PaddedPODArray & keys, size_t start, size_t size, diff --git a/src/Dictionaries/DictionaryBlockInputStream.h b/src/Dictionaries/DictionaryBlockInputStream.h index 7692c910b94..c15406487e2 100644 --- a/src/Dictionaries/DictionaryBlockInputStream.h +++ b/src/Dictionaries/DictionaryBlockInputStream.h @@ -20,18 +20,16 @@ namespace DB /* BlockInputStream implementation for external dictionaries * read() returns blocks consisting of the in-memory contents of the dictionaries */ -class DictionaryBlockInputStream : public DictionaryBlockInputStreamBase +class DictionarySourceData { public: - DictionaryBlockInputStream( + DictionarySourceData( std::shared_ptr dictionary, - UInt64 max_block_size, PaddedPODArray && ids, const Names & column_names); - DictionaryBlockInputStream( + DictionarySourceData( std::shared_ptr dictionary, - UInt64 max_block_size, const PaddedPODArray & keys, const Names & column_names); @@ -41,18 +39,15 @@ public: // Calls get_key_columns_function to get key column for dictionary get function call // and get_view_columns_function to get key representation. // Now used in trie dictionary, where columns are stored as ip and mask, and are showed as string - DictionaryBlockInputStream( + DictionarySourceData( std::shared_ptr dictionary, - UInt64 max_block_size, const Columns & data_columns, const Names & column_names, GetColumnsFunction && get_key_columns_function, GetColumnsFunction && get_view_columns_function); - String getName() const override { return "Dictionary"; } - -protected: - Block getBlock(size_t start, size_t length) const override; + Block getBlock(size_t start, size_t length) const; + size_t getNumRows() const { return num_rows; } private: Block fillBlock( @@ -70,6 +65,7 @@ private: const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & result); + const size_t num_rows; std::shared_ptr dictionary; Names column_names; PaddedPODArray ids; @@ -89,4 +85,18 @@ private: DictionaryInputStreamKeyType key_type; }; +class DictionarySource final : public DictionarySourceBase +{ +public: + DictionarySource(DictionarySourceData data_, UInt64 max_block_size) + : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) + , data(std::move(data_)) + {} + + String getName() const override { return "DictionarySource"; } + Block getBlock(size_t start, size_t length) const override { return data.getBlock(start, length); } + + DictionarySourceData data; +}; + } diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp index 3a3fd09220f..d5e6e8a1cda 100644 --- a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp +++ b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp @@ -2,25 +2,20 @@ namespace DB { -DictionaryBlockInputStreamBase::DictionaryBlockInputStreamBase(size_t rows_count_, size_t max_block_size_) - : rows_count(rows_count_), max_block_size(max_block_size_) +DictionarySourceBase::DictionarySourceBase(const Block & header, size_t rows_count_, size_t max_block_size_) + : SourceWithProgress(header), rows_count(rows_count_), max_block_size(max_block_size_) { } -Block DictionaryBlockInputStreamBase::readImpl() +Chunk DictionarySourceBase::generate() { if (next_row == rows_count) - return Block(); + return {}; - size_t block_size = std::min(max_block_size, rows_count - next_row); - Block block = getBlock(next_row, block_size); - next_row += block_size; - return block; -} - -Block DictionaryBlockInputStreamBase::getHeader() const -{ - return getBlock(0, 0); + size_t size = std::min(max_block_size, rows_count - next_row); + auto chunk = getChunk(next_row, size); + next_row += size; + return chunk; } } diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.h b/src/Dictionaries/DictionaryBlockInputStreamBase.h index fb99918aed8..1533cba82f6 100644 --- a/src/Dictionaries/DictionaryBlockInputStreamBase.h +++ b/src/Dictionaries/DictionaryBlockInputStreamBase.h @@ -1,24 +1,22 @@ #pragma once -#include +#include namespace DB { -class DictionaryBlockInputStreamBase : public IBlockInputStream +class DictionarySourceBase : public SourceWithProgress { protected: - DictionaryBlockInputStreamBase(size_t rows_count_, size_t max_block_size_); + DictionarySourceBase(const Block & header, size_t rows_count_, size_t max_block_size_); virtual Block getBlock(size_t start, size_t length) const = 0; - Block getHeader() const override; - private: const size_t rows_count; const size_t max_block_size; size_t next_row = 0; - Block readImpl() override; + Chunk generate() override; }; } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index ed124ce1e0a..79459057822 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -15,7 +15,8 @@ #include #include #include -#include +#include +#include namespace DB @@ -501,10 +502,10 @@ private: * Note: readPrefix readImpl readSuffix will be called on stream object during function execution. */ template -void mergeBlockWithStream( +void mergeBlockWithPipe( size_t key_columns_size, Block & block_to_update, - BlockInputStreamPtr & stream) + Pipe pipe) { using KeyType = std::conditional_t; static_assert(dictionary_key_type != DictionaryKeyType::range, "Range key type is not supported by updatePreviousyLoadedBlockWithStream"); @@ -555,9 +556,13 @@ void mergeBlockWithStream( auto result_fetched_columns = block_to_update.cloneEmptyColumns(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(std::move(pipe)); - while (Block block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + + while (executor.pull(block)) { Columns block_key_columns; block_key_columns.reserve(key_columns_size); @@ -591,8 +596,6 @@ void mergeBlockWithStream( } } - stream->readSuffix(); - size_t result_fetched_rows = result_fetched_columns.front()->size(); size_t filter_hint = filter.size() - indexes_to_remove_count; @@ -645,4 +648,4 @@ static const PaddedPODArray & getColumnVectorData( } } -} +} \ No newline at end of file diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 54ed07092d3..79d4d2e5376 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -85,62 +85,44 @@ ContextMutablePtr copyContextAndApplySettings( return local_context; } - -BlockInputStreamWithAdditionalColumns::BlockInputStreamWithAdditionalColumns( - Block block_to_add_, std::unique_ptr && stream_) - : block_to_add(std::move(block_to_add_)) - , stream(std::move(stream_)) +static Block transformHeader(Block header, Block block_to_add) { -} - -Block BlockInputStreamWithAdditionalColumns::getHeader() const -{ - auto header = stream->getHeader(); - - if (header) - { - for (Int64 i = static_cast(block_to_add.columns() - 1); i >= 0; --i) - header.insert(0, block_to_add.getByPosition(i).cloneEmpty()); - } + for (Int64 i = static_cast(block_to_add.columns() - 1); i >= 0; --i) + header.insert(0, block_to_add.getByPosition(i).cloneEmpty()); return header; } -Block BlockInputStreamWithAdditionalColumns::readImpl() +TransformWithAdditionalColumns::TransformWithAdditionalColumns( + Block block_to_add_, const Block & header) + : ISimpleTransform(header, transformHeader(header, block_to_add_), true) + , block_to_add(std::move(block_to_add_)) { - auto block = stream->read(); +} - if (block) +void TransformWithAdditionalColumns::transform(Chunk & chunk) +{ + if (chunk) { - auto block_rows = block.rows(); + auto num_rows = chunk.getNumRows(); + auto columns = chunk.detachColumns(); - auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, block_rows); + auto cut_block = block_to_add.cloneWithCutColumns(current_range_index, num_rows); - if (cut_block.rows() != block_rows) + if (cut_block.rows() != num_rows) throw Exception(ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH, "Number of rows in block to add after cut must equal to number of rows in block from inner stream"); for (Int64 i = static_cast(cut_block.columns() - 1); i >= 0; --i) - block.insert(0, cut_block.getByPosition(i)); + columns.insert(columns.begin(), cut_block.getByPosition(i).column); - current_range_index += block_rows; + current_range_index += num_rows; + chunk.setColumns(std::move(columns), num_rows); } - - return block; } -void BlockInputStreamWithAdditionalColumns::readPrefix() +String TransformWithAdditionalColumns::getName() const { - stream->readPrefix(); -} - -void BlockInputStreamWithAdditionalColumns::readSuffix() -{ - stream->readSuffix(); -} - -String BlockInputStreamWithAdditionalColumns::getName() const -{ - return "BlockInputStreamWithAdditionalColumns"; -} + return "TransformWithAdditionalColumns"; } +} \ No newline at end of file diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index 6fed4c7181c..ba05cf9ebc9 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -4,7 +4,7 @@ #include #include -#include +#include #include #include #include @@ -38,25 +38,18 @@ ContextMutablePtr copyContextAndApplySettings( * * block_to_add rows size must be equal to final sum rows size of all inner stream blocks. */ -class BlockInputStreamWithAdditionalColumns final : public IBlockInputStream +class TransformWithAdditionalColumns final : public ISimpleTransform { public: - BlockInputStreamWithAdditionalColumns(Block block_to_add_, std::unique_ptr && stream_); + TransformWithAdditionalColumns(Block block_to_add_, const Block & header); - Block getHeader() const override; - - Block readImpl() override; - - void readPrefix() override; - - void readSuffix() override; + void transform(Chunk & chunk) override; String getName() const override; private: Block block_to_add; - std::unique_ptr stream; size_t current_range_index = 0; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index c9b38acfbb5..afdd9bee92c 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -8,6 +8,8 @@ #include #include +#include +#include namespace DB { @@ -66,11 +68,13 @@ Columns DirectDictionary::getColumns( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - BlockInputStreamPtr stream = getSourceBlockInputStream(key_columns, requested_keys); + QueryPipeline pipeline; + pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); - stream->readPrefix(); + PullingPipelineExecutor executor(pipeline); - while (const auto block = stream->read()) + Block block; + while (executor.pull(block)) { /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) @@ -98,8 +102,6 @@ Columns DirectDictionary::getColumns( block_key_columns.clear(); } - stream->readSuffix(); - Field value_to_insert; size_t requested_keys_size = requested_keys.size(); @@ -183,13 +185,14 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( size_t dictionary_keys_size = dict_struct.getKeysNames().size(); block_key_columns.reserve(dictionary_keys_size); - BlockInputStreamPtr stream = getSourceBlockInputStream(key_columns, requested_keys); + QueryPipeline pipeline; + pipeline.init(getSourceBlockInputStream(key_columns, requested_keys)); - stream->readPrefix(); + PullingPipelineExecutor executor(pipeline); size_t keys_found = 0; - - while (const auto block = stream->read()) + Block block; + while (executor.pull(block)) { /// Split into keys columns and attribute columns for (size_t i = 0; i < dictionary_keys_size; ++i) @@ -216,8 +219,6 @@ ColumnUInt8::Ptr DirectDictionary::hasKeys( block_key_columns.clear(); } - stream->readSuffix(); - query_count.fetch_add(requested_keys_size, std::memory_order_relaxed); found_count.fetch_add(keys_found, std::memory_order_relaxed); @@ -260,13 +261,13 @@ ColumnUInt8::Ptr DirectDictionary::isInHierarchy( } template -BlockInputStreamPtr DirectDictionary::getSourceBlockInputStream( +Pipe DirectDictionary::getSourceBlockInputStream( const Columns & key_columns [[maybe_unused]], const PaddedPODArray & requested_keys [[maybe_unused]]) const { size_t requested_keys_size = requested_keys.size(); - BlockInputStreamPtr stream; + Pipe pipe; if constexpr (dictionary_key_type == DictionaryKeyType::simple) { @@ -276,7 +277,7 @@ BlockInputStreamPtr DirectDictionary::getSourceBlockInputSt for (auto key : requested_keys) ids.emplace_back(key); - stream = source_ptr->loadIds(ids); + pipe = source_ptr->loadIds(ids); } else { @@ -285,14 +286,14 @@ BlockInputStreamPtr DirectDictionary::getSourceBlockInputSt for (size_t i = 0; i < requested_keys_size; ++i) requested_rows.emplace_back(i); - stream = source_ptr->loadKeys(key_columns, requested_rows); + pipe = source_ptr->loadKeys(key_columns, requested_rows); } - return stream; + return pipe; } template -BlockInputStreamPtr DirectDictionary::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const +Pipe DirectDictionary::read(const Names & /* column_names */, size_t /* max_block_size */) const { return source_ptr->loadAll(); } @@ -353,4 +354,4 @@ void registerDictionaryDirect(DictionaryFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index a4e8d5f82e6..841590164f6 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -97,10 +97,10 @@ public: ColumnPtr in_key_column, const DataTypePtr & key_type) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: - BlockInputStreamPtr getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray & requested_keys) const; + Pipe getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray & requested_keys) const; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; @@ -113,4 +113,4 @@ private: extern template class DirectDictionary; extern template class DirectDictionary; -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index daf79965428..82b74d8196e 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -2,13 +2,17 @@ #include #include -#include #include #include +#include +#include +#include +#include +#include #include +#include #include #include -#include #include #include #include @@ -34,26 +38,34 @@ namespace ErrorCodes namespace { /// Owns ShellCommand and calls wait for it. - class ShellCommandOwningBlockInputStream : public OwningBlockInputStream + class ShellCommandOwningTransform final : public ISimpleTransform { private: Poco::Logger * log; + std::unique_ptr command; public: - ShellCommandOwningBlockInputStream(Poco::Logger * log_, const BlockInputStreamPtr & impl, std::unique_ptr command_) - : OwningBlockInputStream(std::move(impl), std::move(command_)), log(log_) + ShellCommandOwningTransform(const Block & header, Poco::Logger * log_, std::unique_ptr command_) + : ISimpleTransform(header, header, true), log(log_), command(std::move(command_)) { } - void readSuffix() override + String getName() const override { return "ShellCommandOwningTransform"; } + void transform(Chunk &) override {} + + Status prepare() override { - OwningBlockInputStream::readSuffix(); + auto status = ISimpleTransform::prepare(); + if (status == Status::Finished) + { + std::string err; + readStringUntilEOF(err, command->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); - std::string err; - readStringUntilEOF(err, own->err); - if (!err.empty()) - LOG_ERROR(log, "Having stderr: {}", err); + command->wait(); + } - own->wait(); + return status; } }; @@ -94,18 +106,19 @@ ExecutableDictionarySource::ExecutableDictionarySource(const ExecutableDictionar { } -BlockInputStreamPtr ExecutableDictionarySource::loadAll() +Pipe ExecutableDictionarySource::loadAll() { if (configuration.implicit_key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadAll method"); LOG_TRACE(log, "loadAll {}", toString()); auto process = ShellCommand::execute(configuration.command); - auto input_stream = context->getInputFormat(configuration.format, process->out, sample_block, max_block_size); - return std::make_shared(log, input_stream, std::move(process)); + Pipe pipe(FormatFactory::instance().getInput(configuration.format, process->out, sample_block, context, max_block_size)); + pipe.addTransform(std::make_shared(pipe.getHeader(), log, std::move(process))); + return pipe; } -BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() +Pipe ExecutableDictionarySource::loadUpdatedAll() { if (configuration.implicit_key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutableDictionarySource with implicit_key does not support loadUpdatedAll method"); @@ -119,81 +132,86 @@ BlockInputStreamPtr ExecutableDictionarySource::loadUpdatedAll() LOG_TRACE(log, "loadUpdatedAll {}", command_with_update_field); auto process = ShellCommand::execute(command_with_update_field); - auto input_stream = context->getInputFormat(configuration.format, process->out, sample_block, max_block_size); - return std::make_shared(log, input_stream, std::move(process)); + + Pipe pipe(FormatFactory::instance().getInput(configuration.format, process->out, sample_block, context, max_block_size)); + pipe.addTransform(std::make_shared(pipe.getHeader(), log, std::move(process))); + return pipe; } namespace { /** A stream, that runs child process and sends data to its stdin in background thread, * and receives data from its stdout. + * + * TODO: implement without background thread. */ - class BlockInputStreamWithBackgroundThread final : public IBlockInputStream + class SourceWithBackgroundThread final : public SourceWithProgress { public: - BlockInputStreamWithBackgroundThread( + SourceWithBackgroundThread( ContextPtr context, const std::string & format, const Block & sample_block, const std::string & command_str, Poco::Logger * log_, std::function && send_data_) - : log(log_), - command(ShellCommand::execute(command_str)), - send_data(std::move(send_data_)), - thread([this] { send_data(command->in); }) + : SourceWithProgress(sample_block) + , log(log_) + , command(ShellCommand::execute(command_str)) + , send_data(std::move(send_data_)) + , thread([this] { send_data(command->in); }) { - stream = context->getInputFormat(format, command->out, sample_block, max_block_size); + pipeline.init(Pipe(FormatFactory::instance().getInput(format, command->out, sample_block, context, max_block_size))); + executor = std::make_unique(pipeline); } - ~BlockInputStreamWithBackgroundThread() override + ~SourceWithBackgroundThread() override { if (thread.joinable()) thread.join(); } - Block getHeader() const override + protected: + Chunk generate() override { - return stream->getHeader(); + Chunk chunk; + executor->pull(chunk); + return chunk; } - private: - Block readImpl() override + public: + Status prepare() override { - return stream->read(); + auto status = SourceWithProgress::prepare(); + + if (status == Status::Finished) + { + std::string err; + readStringUntilEOF(err, command->err); + if (!err.empty()) + LOG_ERROR(log, "Having stderr: {}", err); + + if (thread.joinable()) + thread.join(); + + command->wait(); + } + + return status; } - void readPrefix() override - { - stream->readPrefix(); - } - - void readSuffix() override - { - stream->readSuffix(); - - std::string err; - readStringUntilEOF(err, command->err); - if (!err.empty()) - LOG_ERROR(log, "Having stderr: {}", err); - - if (thread.joinable()) - thread.join(); - - command->wait(); - } - - String getName() const override { return "WithBackgroundThread"; } + String getName() const override { return "SourceWithBackgroundThread"; } Poco::Logger * log; - BlockInputStreamPtr stream; + QueryPipeline pipeline; + std::unique_ptr executor; std::unique_ptr command; std::function send_data; ThreadFromGlobalPool thread; }; } -BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector & ids) +Pipe ExecutableDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); @@ -201,7 +219,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector & requested_rows) +Pipe ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); @@ -209,21 +227,21 @@ BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_col return getStreamForBlock(block); } -BlockInputStreamPtr ExecutableDictionarySource::getStreamForBlock(const Block & block) +Pipe ExecutableDictionarySource::getStreamForBlock(const Block & block) { - auto stream = std::make_unique( + Pipe pipe(std::make_unique( context, configuration.format, sample_block, configuration.command, log, [block, this](WriteBufferFromFile & out) mutable { auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty()); formatBlock(output_stream, block); out.close(); - }); + })); if (configuration.implicit_key) - return std::make_shared(block, std::move(stream)); - else - return std::shared_ptr(stream.release()); + pipe.addTransform(std::make_shared(block, pipe.getHeader())); + + return pipe; } bool ExecutableDictionarySource::isModified() const @@ -289,4 +307,4 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) factory.registerSource("executable", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 0b92023df36..084b8c13c25 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -36,17 +36,17 @@ public: ExecutableDictionarySource(const ExecutableDictionarySource & other); ExecutableDictionarySource & operator=(const ExecutableDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; /** The logic of this method is flawed, absolutely incorrect and ignorant. * It may lead to skipping some values due to clock sync or timezone changes. * The intended usage of "update_field" is totally different. */ - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -58,7 +58,7 @@ public: std::string toString() const override; - BlockInputStreamPtr getStreamForBlock(const Block & block); + Pipe getStreamForBlock(const Block & block); private: Poco::Logger * log; @@ -69,4 +69,4 @@ private: ContextPtr context; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 9eacda343cf..2cd567bbc02 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -2,12 +2,15 @@ #include #include -#include +#include +#include +#include #include #include #include #include -#include +#include +#include #include #include #include @@ -69,12 +72,12 @@ ExecutablePoolDictionarySource::ExecutablePoolDictionarySource(const ExecutableP { } -BlockInputStreamPtr ExecutablePoolDictionarySource::loadAll() +Pipe ExecutablePoolDictionarySource::loadAll() { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource does not support loadAll method"); } -BlockInputStreamPtr ExecutablePoolDictionarySource::loadUpdatedAll() +Pipe ExecutablePoolDictionarySource::loadUpdatedAll() { throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "ExecutablePoolDictionarySource does not support loadUpdatedAll method"); } @@ -84,19 +87,19 @@ namespace /** A stream, that runs child process and sends data to its stdin in background thread, * and receives data from its stdout. */ - class PoolBlockInputStreamWithBackgroundThread final : public IBlockInputStream + class PoolSourceWithBackgroundThread final : public SourceWithProgress { public: - PoolBlockInputStreamWithBackgroundThread( + PoolSourceWithBackgroundThread( std::shared_ptr process_pool_, std::unique_ptr && command_, - BlockInputStreamPtr && stream_, + Pipe pipe, size_t read_rows_, Poco::Logger * log_, std::function && send_data_) - : process_pool(process_pool_) + : SourceWithProgress(pipe.getHeader()) + , process_pool(process_pool_) , command(std::move(command_)) - , stream(std::move(stream_)) , rows_to_read(read_rows_) , log(log_) , send_data(std::move(send_data_)) @@ -112,9 +115,12 @@ namespace exception_during_read = std::current_exception(); } }) - {} + { + pipeline.init(std::move(pipe)); + executor = std::make_unique(pipeline); + } - ~PoolBlockInputStreamWithBackgroundThread() override + ~PoolSourceWithBackgroundThread() override { if (thread.joinable()) thread.join(); @@ -123,25 +129,22 @@ namespace process_pool->returnObject(std::move(command)); } - Block getHeader() const override - { - return stream->getHeader(); - } - - private: - Block readImpl() override + protected: + Chunk generate() override { rethrowExceptionDuringReadIfNeeded(); if (current_read_rows == rows_to_read) - return Block(); + return {}; - Block block; + Chunk chunk; try { - block = stream->read(); - current_read_rows += block.rows(); + if (!executor->pull(chunk)) + return {}; + + current_read_rows += chunk.getNumRows(); } catch (...) { @@ -150,22 +153,23 @@ namespace throw; } - return block; + return chunk; } - void readPrefix() override + public: + Status prepare() override { - rethrowExceptionDuringReadIfNeeded(); - stream->readPrefix(); - } + auto status = SourceWithProgress::prepare(); - void readSuffix() override - { - if (thread.joinable()) - thread.join(); + if (status == Status::Finished) + { + if (thread.joinable()) + thread.join(); - rethrowExceptionDuringReadIfNeeded(); - stream->readSuffix(); + rethrowExceptionDuringReadIfNeeded(); + } + + return status; } void rethrowExceptionDuringReadIfNeeded() @@ -182,7 +186,8 @@ namespace std::shared_ptr process_pool; std::unique_ptr command; - BlockInputStreamPtr stream; + QueryPipeline pipeline; + std::unique_ptr executor; size_t rows_to_read; Poco::Logger * log; std::function send_data; @@ -194,7 +199,7 @@ namespace } -BlockInputStreamPtr ExecutablePoolDictionarySource::loadIds(const std::vector & ids) +Pipe ExecutablePoolDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); @@ -202,7 +207,7 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::loadIds(const std::vector & requested_rows) +Pipe ExecutablePoolDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); @@ -210,7 +215,7 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::loadKeys(const Columns & key return getStreamForBlock(block); } -BlockInputStreamPtr ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) +Pipe ExecutablePoolDictionarySource::getStreamForBlock(const Block & block) { std::unique_ptr process; bool result = process_pool->tryBorrowObject(process, [this]() @@ -227,20 +232,20 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::getStreamForBlock(const Bloc configuration.max_command_execution_time); size_t rows_to_read = block.rows(); - auto read_stream = context->getInputFormat(configuration.format, process->out, sample_block, rows_to_read); + auto format = FormatFactory::instance().getInput(configuration.format, process->out, sample_block, context, rows_to_read); - auto stream = std::make_unique( - process_pool, std::move(process), std::move(read_stream), rows_to_read, log, + Pipe pipe(std::make_unique( + process_pool, std::move(process), Pipe(std::move(format)), rows_to_read, log, [block, this](WriteBufferFromFile & out) mutable { auto output_stream = context->getOutputStream(configuration.format, out, block.cloneEmpty()); formatBlock(output_stream, block); - }); + })); if (configuration.implicit_key) - return std::make_shared(block, std::move(stream)); - else - return std::shared_ptr(stream.release()); + pipe.addTransform(std::make_shared(block, pipe.getHeader())); + + return pipe; } bool ExecutablePoolDictionarySource::isModified() const @@ -320,4 +325,4 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) factory.registerSource("executable_pool", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 02b0288a52e..9c8730632f7 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -48,17 +48,17 @@ public: ExecutablePoolDictionarySource(const ExecutablePoolDictionarySource & other); ExecutablePoolDictionarySource & operator=(const ExecutablePoolDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; /** The logic of this method is flawed, absolutely incorrect and ignorant. * It may lead to skipping some values due to clock sync or timezone changes. * The intended usage of "update_field" is totally different. */ - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -70,7 +70,7 @@ public: std::string toString() const override; - BlockInputStreamPtr getStreamForBlock(const Block & block); + Pipe getStreamForBlock(const Block & block); private: Poco::Logger * log; @@ -83,4 +83,4 @@ private: std::shared_ptr process_pool; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 239c13e71c2..3766da0a28d 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -5,6 +5,8 @@ #include #include #include +#include +#include #include "DictionarySourceFactory.h" #include "DictionaryStructure.h" #include "registerDictionaries.h" @@ -45,14 +47,15 @@ FileDictionarySource::FileDictionarySource(const FileDictionarySource & other) } -BlockInputStreamPtr FileDictionarySource::loadAll() +Pipe FileDictionarySource::loadAll() { LOG_TRACE(&Poco::Logger::get("FileDictionary"), "loadAll {}", toString()); auto in_ptr = std::make_unique(filepath); - auto stream = context->getInputFormat(format, *in_ptr, sample_block, max_block_size); + auto source = FormatFactory::instance().getInput(format, *in_ptr, sample_block, context, max_block_size); + source->addBuffer(std::move(in_ptr)); last_modification = getLastModification(); - return std::make_shared>(stream, std::move(in_ptr)); + return Pipe(std::move(source)); } @@ -92,4 +95,4 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) factory.registerSource("file", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index ffc29374f4f..efc86be8ba9 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -21,19 +21,19 @@ public: FileDictionarySource(const FileDictionarySource & other); - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for FileDictionarySource"); } - BlockInputStreamPtr loadIds(const std::vector & /*ids*/) override + Pipe loadIds(const std::vector & /*ids*/) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadIds is unsupported for FileDictionarySource"); } - BlockInputStreamPtr loadKeys(const Columns & /*key_columns*/, const std::vector & /*requested_rows*/) override + Pipe loadKeys(const Columns & /*key_columns*/, const std::vector & /*requested_rows*/) override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadKeys is unsupported for FileDictionarySource"); } @@ -65,4 +65,4 @@ private: Poco::Timestamp last_modification; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index abef2335ffd..40afe70f504 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -10,6 +10,9 @@ #include #include +#include +#include + #include #include #include @@ -319,10 +322,12 @@ void FlatDictionary::updateData() { if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - auto stream = source_ptr->loadUpdatedAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadUpdatedAll()); - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { /// We are using this to keep saved data if input stream consists of multiple blocks if (!update_field_loaded_block) @@ -335,15 +340,14 @@ void FlatDictionary::updateData() saved_column->insertRangeFrom(update_column, 0, update_column.size()); } } - stream->readSuffix(); } else { - auto stream = source_ptr->loadUpdatedAll(); - mergeBlockWithStream( + Pipe pipe(source_ptr->loadUpdatedAll()); + mergeBlockWithPipe( dict_struct.getKeysSize(), *update_field_loaded_block, - stream); + std::move(pipe)); } if (update_field_loaded_block) @@ -354,13 +358,13 @@ void FlatDictionary::loadData() { if (!source_ptr->hasUpdateField()) { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); + PullingPipelineExecutor executor(pipeline); - while (const auto block = stream->read()) + Block block; + while (executor.pull(block)) blockToAttributes(block); - - stream->readSuffix(); } else updateData(); @@ -531,7 +535,7 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const UInt64 key, callOnDictionaryAttributeType(attribute.type, type_call); } -BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe FlatDictionary::read(const Names & column_names, size_t max_block_size) const { const auto keys_count = loaded_keys.size(); @@ -542,7 +546,8 @@ BlockInputStreamPtr FlatDictionary::getBlockInputStream(const Names & column_nam if (loaded_keys[key_index]) keys.push_back(key_index); - return std::make_shared(shared_from_this(), max_block_size, std::move(keys), column_names); + return Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); } void registerDictionaryFlat(DictionaryFactory & factory) @@ -586,4 +591,4 @@ void registerDictionaryFlat(DictionaryFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index ccd3bf9d9eb..c16547bd3b4 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -97,7 +97,7 @@ public: const DataTypePtr & key_type, size_t level) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: template @@ -178,4 +178,4 @@ private: BlockPtr update_field_loaded_block; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index d65338b9a4b..33e06de23bf 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -637,7 +637,7 @@ void HashedDictionary::calculateBytesAllocated() } template -BlockInputStreamPtr HashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe HashedDictionary::read(const Names & column_names, size_t max_block_size) const { PaddedPODArray keys; @@ -667,9 +667,9 @@ BlockInputStreamPtr HashedDictionary::getBlockInput } if constexpr (dictionary_key_type == DictionaryKeyType::simple) - return std::make_shared(shared_from_this(), max_block_size, std::move(keys), column_names); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size)); else - return std::make_shared(shared_from_this(), max_block_size, keys, column_names); + return Pipe(std::make_shared(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size)); } template @@ -767,4 +767,4 @@ void registerDictionaryHashed(DictionaryFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 842e49aa8f0..82e8a91b603 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -116,7 +116,7 @@ public: const DataTypePtr & key_type, size_t level) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: template @@ -225,4 +225,4 @@ extern template class HashedDictionary; extern template class HashedDictionary; extern template class HashedDictionary; -} +} \ No newline at end of file diff --git a/src/Dictionaries/IDictionary.h b/src/Dictionaries/IDictionary.h index 5467a673503..f9e0223a698 100644 --- a/src/Dictionaries/IDictionary.h +++ b/src/Dictionaries/IDictionary.h @@ -191,7 +191,7 @@ struct IDictionary : public IExternalLoadable getDictionaryID().getNameForLogs()); } - virtual BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const = 0; + virtual Pipe read(const Names & column_names, size_t max_block_size) const = 0; bool supportUpdates() const override { return true; } diff --git a/src/Dictionaries/IDictionarySource.h b/src/Dictionaries/IDictionarySource.h index 42b35c95062..661f5b8eeb8 100644 --- a/src/Dictionaries/IDictionarySource.h +++ b/src/Dictionaries/IDictionarySource.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include #include @@ -21,28 +21,30 @@ class IDictionarySource { public: - /// Returns an input stream with all the data available from this source. - virtual BlockInputStreamPtr loadAll() = 0; + /// Returns a pipe with all the data available from this source. + virtual Pipe loadAll() = 0; - /// Returns an input stream with updated data available from this source. - virtual BlockInputStreamPtr loadUpdatedAll() = 0; + /// Returns a pipe with updated data available from this source. + virtual Pipe loadUpdatedAll() = 0; /** * result_size_hint - approx number of rows in the stream. - * Returns an input stream with all the data available from this source. + * Returns a pipe with all the data available from this source. * * NOTE: result_size_hint may be changed during you are reading (usually it * will be non zero for the first block and zero for others, since it uses - * Progress::total_rows_approx,) from the input stream, and may be called + * Progress::total_rows_approx,) from the pipe, and may be called * in parallel, so you should use something like this: * * ... * std::atomic new_size = 0; * - * auto stream = source->loadAll(&new_size); - * stream->readPrefix(); + * QueryPipeline pipeline; + * pipeline.init(source->loadAll(&new_size)); + * PullingPipelineExecutor executor; * - * while (const auto block = stream->read()) + * Block block; + * while (executor.pull(block)) * { * if (new_size) * { @@ -56,10 +58,9 @@ public: * } * } * - * stream->readSuffix(); * ... */ - virtual BlockInputStreamPtr loadAllWithSizeHint(std::atomic * /* result_size_hint */) + virtual Pipe loadAllWithSizeHint(std::atomic * /* result_size_hint */) { return loadAll(); } @@ -72,13 +73,13 @@ public: /** Returns an input stream with the data for a collection of identifiers. * It must be guaranteed, that 'ids' array will live at least until all data will be read from returned stream. */ - virtual BlockInputStreamPtr loadIds(const std::vector & ids) = 0; + virtual Pipe loadIds(const std::vector & ids) = 0; /** Returns an input stream with the data for a collection of composite keys. * `requested_rows` contains indices of all rows containing unique keys. * It must be guaranteed, that 'requested_rows' array will live at least until all data will be read from returned stream. */ - virtual BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) = 0; + virtual Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) = 0; /// indicates whether the source has been modified since last load* operation virtual bool isModified() const = 0; diff --git a/src/Dictionaries/readInvalidateQuery.cpp b/src/Dictionaries/readInvalidateQuery.cpp index 4664b61bfc4..bd1ec0e0983 100644 --- a/src/Dictionaries/readInvalidateQuery.cpp +++ b/src/Dictionaries/readInvalidateQuery.cpp @@ -1,5 +1,6 @@ #include "readInvalidateQuery.h" -#include +#include +#include #include #include @@ -14,11 +15,18 @@ namespace ErrorCodes extern const int RECEIVED_EMPTY_DATA; } -std::string readInvalidateQuery(IBlockInputStream & block_input_stream) +std::string readInvalidateQuery(Pipe pipe) { - block_input_stream.readPrefix(); + QueryPipeline pipeline; + pipeline.init(std::move(pipe)); + + PullingPipelineExecutor executor(pipeline); + + Block block; + while (executor.pull(block)) + if (block) + break; - Block block = block_input_stream.read(); if (!block) throw Exception(ErrorCodes::RECEIVED_EMPTY_DATA, "Empty response"); @@ -36,11 +44,10 @@ std::string readInvalidateQuery(IBlockInputStream & block_input_stream) auto & column_type = block.getByPosition(0); column_type.type->getDefaultSerialization()->serializeTextQuoted(*column_type.column->convertToFullColumnIfConst(), 0, out, FormatSettings()); - while ((block = block_input_stream.read())) + while (executor.pull(block)) if (block.rows() > 0) throw Exception(ErrorCodes::TOO_MANY_ROWS, "Expected single row in resultset, got at least {}", std::to_string(rows + 1)); - block_input_stream.readSuffix(); return out.str(); } diff --git a/src/Dictionaries/readInvalidateQuery.h b/src/Dictionaries/readInvalidateQuery.h index 0a259e1ec24..61d5b29dc89 100644 --- a/src/Dictionaries/readInvalidateQuery.h +++ b/src/Dictionaries/readInvalidateQuery.h @@ -1,13 +1,13 @@ #pragma once -#include - #include namespace DB { +class Pipe; + /// Using in MySQLDictionarySource and XDBCDictionarySource after processing invalidate_query. -std::string readInvalidateQuery(IBlockInputStream & block_input_stream); +std::string readInvalidateQuery(Pipe pipe); } From 8546df13c2ccaf19f3c8337cc5d1d64f7f0ed3c7 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Thu, 5 Aug 2021 21:08:52 +0300 Subject: [PATCH 2/4] Streams -> Processors for dicts, part 2. --- src/Bridge/LibraryBridgeHelper.cpp | 19 +++--- src/Bridge/LibraryBridgeHelper.h | 12 ++-- src/DataStreams/MongoDBBlockInputStream.cpp | 13 ++-- src/DataStreams/MongoDBBlockInputStream.h | 14 ++--- .../PostgreSQLBlockInputStream.cpp | 43 +++++++++---- src/DataStreams/PostgreSQLBlockInputStream.h | 22 ++++--- src/Dictionaries/HTTPDictionarySource.cpp | 19 +++--- src/Dictionaries/HTTPDictionarySource.h | 13 ++-- src/Dictionaries/HashedDictionary.cpp | 29 +++++---- src/Dictionaries/IPAddressDictionary.cpp | 24 +++---- src/Dictionaries/IPAddressDictionary.h | 4 +- src/Dictionaries/LibraryDictionarySource.cpp | 8 +-- src/Dictionaries/LibraryDictionarySource.h | 10 +-- src/Dictionaries/MongoDBDictionarySource.cpp | 14 ++--- src/Dictionaries/MongoDBDictionarySource.h | 10 +-- src/Dictionaries/MySQLDictionarySource.cpp | 17 ++--- src/Dictionaries/MySQLDictionarySource.h | 14 ++--- src/Dictionaries/PolygonDictionary.cpp | 16 ++--- src/Dictionaries/PolygonDictionary.h | 5 +- .../PostgreSQLDictionarySource.cpp | 12 ++-- src/Dictionaries/PostgreSQLDictionarySource.h | 12 ++-- .../RangeDictionaryBlockInputStream.h | 63 +++++++++++++------ src/Dictionaries/RangeHashedDictionary.cpp | 49 ++++++++------- src/Dictionaries/RangeHashedDictionary.h | 8 +-- src/Dictionaries/RedisBlockInputStream.cpp | 14 +++-- src/Dictionaries/RedisBlockInputStream.h | 14 ++--- src/Dictionaries/RedisDictionarySource.cpp | 16 ++--- src/Dictionaries/RedisDictionarySource.h | 10 +-- src/Dictionaries/XDBCDictionarySource.cpp | 61 ++++-------------- src/Dictionaries/XDBCDictionarySource.h | 12 ++-- src/Formats/MySQLBlockInputStream.cpp | 42 ++++++++----- src/Formats/MySQLBlockInputStream.h | 23 +++---- 32 files changed, 336 insertions(+), 306 deletions(-) diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index b13be0aba29..81484854367 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -1,11 +1,10 @@ #include "LibraryBridgeHelper.h" #include -#include -#include #include #include -#include +#include +#include #include #include #include @@ -117,7 +116,7 @@ bool LibraryBridgeHelper::supportsSelectiveLoad() } -BlockInputStreamPtr LibraryBridgeHelper::loadAll() +Pipe LibraryBridgeHelper::loadAll() { startBridgeSync(); auto uri = createRequestURI(LOAD_ALL_METHOD); @@ -125,7 +124,7 @@ BlockInputStreamPtr LibraryBridgeHelper::loadAll() } -BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::string ids_string) +Pipe LibraryBridgeHelper::loadIds(const std::string ids_string) { startBridgeSync(); auto uri = createRequestURI(LOAD_IDS_METHOD); @@ -133,7 +132,7 @@ BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::string ids_string) } -BlockInputStreamPtr LibraryBridgeHelper::loadKeys(const Block & requested_block) +Pipe LibraryBridgeHelper::loadKeys(const Block & requested_block) { startBridgeSync(); auto uri = createRequestURI(LOAD_KEYS_METHOD); @@ -163,7 +162,7 @@ bool LibraryBridgeHelper::executeRequest(const Poco::URI & uri, ReadWriteBufferF } -BlockInputStreamPtr LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) +Pipe LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback) { auto read_buf_ptr = std::make_unique( uri, @@ -176,7 +175,9 @@ BlockInputStreamPtr LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWri ReadWriteBufferFromHTTP::HTTPHeaderEntries{}); auto input_stream = getContext()->getInputFormat(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, DEFAULT_BLOCK_SIZE); - return std::make_shared>(input_stream, std::move(read_buf_ptr)); + auto source = FormatFactory::instance().getInput(LibraryBridgeHelper::DEFAULT_FORMAT, *read_buf_ptr, sample_block, getContext(), DEFAULT_BLOCK_SIZE); + source->addBuffer(std::move(read_buf_ptr)); + return Pipe(std::move(source)); } -} +} \ No newline at end of file diff --git a/src/Bridge/LibraryBridgeHelper.h b/src/Bridge/LibraryBridgeHelper.h index 12fe0c33363..18a9c18f2ec 100644 --- a/src/Bridge/LibraryBridgeHelper.h +++ b/src/Bridge/LibraryBridgeHelper.h @@ -11,6 +11,8 @@ namespace DB { +class Pipe; + class LibraryBridgeHelper : public IBridgeHelper { @@ -29,13 +31,13 @@ public: bool supportsSelectiveLoad(); - BlockInputStreamPtr loadAll(); + Pipe loadAll(); - BlockInputStreamPtr loadIds(std::string ids_string); + Pipe loadIds(std::string ids_string); - BlockInputStreamPtr loadKeys(const Block & requested_block); + Pipe loadKeys(const Block & requested_block); - BlockInputStreamPtr loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); + Pipe loadBase(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); bool executeRequest(const Poco::URI & uri, ReadWriteBufferFromHTTP::OutStreamCallback out_stream_callback = {}); @@ -83,4 +85,4 @@ private: size_t bridge_port; }; -} +} \ No newline at end of file diff --git a/src/DataStreams/MongoDBBlockInputStream.cpp b/src/DataStreams/MongoDBBlockInputStream.cpp index d583cb0d5b4..1bc77532eb9 100644 --- a/src/DataStreams/MongoDBBlockInputStream.cpp +++ b/src/DataStreams/MongoDBBlockInputStream.cpp @@ -150,13 +150,14 @@ std::unique_ptr createCursor(const std::string & database return cursor; } -MongoDBBlockInputStream::MongoDBBlockInputStream( +MongoDBSource::MongoDBSource( std::shared_ptr & connection_, std::unique_ptr cursor_, const Block & sample_block, UInt64 max_block_size_, bool strict_check_names_) - : connection(connection_) + : SourceWithProgress(sample_block.cloneEmpty()) + , connection(connection_) , cursor{std::move(cursor_)} , max_block_size{max_block_size_} , strict_check_names{strict_check_names_} @@ -165,7 +166,7 @@ MongoDBBlockInputStream::MongoDBBlockInputStream( } -MongoDBBlockInputStream::~MongoDBBlockInputStream() = default; +MongoDBSource::~MongoDBSource() = default; namespace @@ -307,7 +308,7 @@ namespace } -Block MongoDBBlockInputStream::readImpl() +Chunk MongoDBSource::generate() { if (all_read) return {}; @@ -362,7 +363,7 @@ Block MongoDBBlockInputStream::readImpl() if (num_rows == 0) return {}; - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } -} +} \ No newline at end of file diff --git a/src/DataStreams/MongoDBBlockInputStream.h b/src/DataStreams/MongoDBBlockInputStream.h index e1cbf9d570c..9100ba3240b 100644 --- a/src/DataStreams/MongoDBBlockInputStream.h +++ b/src/DataStreams/MongoDBBlockInputStream.h @@ -1,7 +1,7 @@ #pragma once #include -#include +#include #include @@ -22,24 +22,22 @@ void authenticate(Poco::MongoDB::Connection & connection, const std::string & da std::unique_ptr createCursor(const std::string & database, const std::string & collection, const Block & sample_block_to_select); /// Converts MongoDB Cursor to a stream of Blocks -class MongoDBBlockInputStream final : public IBlockInputStream +class MongoDBSource final : public SourceWithProgress { public: - MongoDBBlockInputStream( + MongoDBSource( std::shared_ptr & connection_, std::unique_ptr cursor_, const Block & sample_block, UInt64 max_block_size_, bool strict_check_names_ = false); - ~MongoDBBlockInputStream() override; + ~MongoDBSource() override; String getName() const override { return "MongoDB"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - private: - Block readImpl() override; + Chunk generate() override; std::shared_ptr connection; std::unique_ptr cursor; @@ -52,4 +50,4 @@ private: bool strict_check_names; }; -} +} \ No newline at end of file diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 3956ce16afa..477ede47258 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -24,12 +24,13 @@ namespace DB template -PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( +PostgreSQLSource::PostgreSQLSource( postgres::ConnectionHolderPtr connection_holder_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_) - : query_str(query_str_) + : SourceWithProgress(sample_block.cloneEmpty()) + , query_str(query_str_) , max_block_size(max_block_size_) , connection_holder(std::move(connection_holder_)) { @@ -38,13 +39,14 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( template -PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( +PostgreSQLSource::PostgreSQLSource( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, const UInt64 max_block_size_, bool auto_commit_) - : query_str(query_str_) + : SourceWithProgress(sample_block.cloneEmpty()) + , query_str(query_str_) , tx(std::move(tx_)) , max_block_size(max_block_size_) , auto_commit(auto_commit_) @@ -54,7 +56,7 @@ PostgreSQLBlockInputStream::PostgreSQLBlockInputStream( template -void PostgreSQLBlockInputStream::init(const Block & sample_block) +void PostgreSQLSource::init(const Block & sample_block) { description.init(sample_block); @@ -69,19 +71,34 @@ void PostgreSQLBlockInputStream::init(const Block & sample_block) template -void PostgreSQLBlockInputStream::readPrefix() +void PostgreSQLSource::onStart() { tx = std::make_shared(connection_holder->get()); stream = std::make_unique(*tx, pqxx::from_query, std::string_view(query_str)); } +template +IProcessor::Status PostgreSQLSource::prepare() +{ + if (!started) + { + onStart(); + started = true; + } + + auto status = SourceWithProgress::prepare(); + if (status == Status::Finished) + onFinish(); + + return status; +} template -Block PostgreSQLBlockInputStream::readImpl() +Chunk PostgreSQLSource::generate() { /// Check if pqxx::stream_from is finished if (!stream || !(*stream)) - return Block(); + return {}; MutableColumns columns = description.sample_block.cloneEmptyColumns(); size_t num_rows = 0; @@ -129,12 +146,12 @@ Block PostgreSQLBlockInputStream::readImpl() break; } - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } template -void PostgreSQLBlockInputStream::readSuffix() +void PostgreSQLSource::onFinish() { if (stream) { @@ -146,11 +163,11 @@ void PostgreSQLBlockInputStream::readSuffix() } template -class PostgreSQLBlockInputStream; +class PostgreSQLSource; template -class PostgreSQLBlockInputStream; +class PostgreSQLSource; } -#endif +#endif \ No newline at end of file diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 31bff0789bb..8b2ea3b62ef 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -6,7 +6,7 @@ #if USE_LIBPQXX #include -#include +#include #include #include #include @@ -18,23 +18,20 @@ namespace DB { template -class PostgreSQLBlockInputStream : public IBlockInputStream +class PostgreSQLSource : public SourceWithProgress { public: - PostgreSQLBlockInputStream( + PostgreSQLSource( postgres::ConnectionHolderPtr connection_holder_, const String & query_str_, const Block & sample_block, const UInt64 max_block_size_); String getName() const override { return "PostgreSQL"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - - void readPrefix() override; protected: - PostgreSQLBlockInputStream( + PostgreSQLSource( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block, @@ -45,9 +42,12 @@ protected: std::shared_ptr tx; std::unique_ptr stream; + Status prepare() override; + private: - Block readImpl() override; - void readSuffix() override; + void onStart(); + Chunk generate() override; + void onFinish(); void init(const Block & sample_block); @@ -55,6 +55,8 @@ private: bool auto_commit = true; ExternalResultDescription description; + bool started = false; + postgres::ConnectionHolderPtr connection_holder; std::unordered_map array_info; @@ -83,4 +85,4 @@ public: } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index ea26e9b7a2a..88aedd51395 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include #include #include #include @@ -62,14 +64,15 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other) credentials.setPassword(other.credentials.getPassword()); } -BlockInputStreamPtr HTTPDictionarySource::createWrappedBuffer(std::unique_ptr http_buffer_ptr) +Pipe HTTPDictionarySource::createWrappedBuffer(std::unique_ptr http_buffer_ptr) { Poco::URI uri(configuration.url); String http_request_compression_method_str = http_buffer_ptr->getCompressionMethod(); auto in_ptr_wrapped = wrapReadBufferWithCompressionMethod(std::move(http_buffer_ptr), chooseCompressionMethod(uri.getPath(), http_request_compression_method_str)); - auto input_stream = context->getInputFormat(configuration.format, *in_ptr_wrapped, sample_block, max_block_size); - return std::make_shared>(input_stream, std::move(in_ptr_wrapped)); + auto source = FormatFactory::instance().getInput(configuration.format, *in_ptr_wrapped, sample_block, context, max_block_size); + source->addBuffer(std::move(in_ptr_wrapped)); + return Pipe(std::move(source)); } void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri) @@ -89,7 +92,7 @@ void HTTPDictionarySource::getUpdateFieldAndDate(Poco::URI & uri) } } -BlockInputStreamPtr HTTPDictionarySource::loadAll() +Pipe HTTPDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); Poco::URI uri(configuration.url); @@ -106,7 +109,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadAll() return createWrappedBuffer(std::move(in_ptr)); } -BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() +Pipe HTTPDictionarySource::loadUpdatedAll() { Poco::URI uri(configuration.url); getUpdateFieldAndDate(uri); @@ -124,7 +127,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadUpdatedAll() return createWrappedBuffer(std::move(in_ptr)); } -BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & ids) +Pipe HTTPDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); @@ -151,7 +154,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector & id return createWrappedBuffer(std::move(in_ptr)); } -BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); @@ -260,4 +263,4 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) factory.registerSource("http", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index 638fcc876ed..1c7d67d3c23 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -43,13 +43,13 @@ public: HTTPDictionarySource(const HTTPDictionarySource & other); HTTPDictionarySource & operator=(const HTTPDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -65,7 +65,7 @@ private: void getUpdateFieldAndDate(Poco::URI & uri); // wrap buffer using encoding from made request - BlockInputStreamPtr createWrappedBuffer(std::unique_ptr http_buffer); + Pipe createWrappedBuffer(std::unique_ptr http_buffer); Poco::Logger * log; @@ -80,5 +80,4 @@ private: ConnectionTimeouts timeouts; }; -} - +} \ No newline at end of file diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 33e06de23bf..22a7ec87b07 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -367,10 +367,12 @@ void HashedDictionary::updateData() if (!update_field_loaded_block || update_field_loaded_block->rows() == 0) { - auto stream = source_ptr->loadUpdatedAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadUpdatedAll()); - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { /// We are using this to keep saved data if input stream consists of multiple blocks if (!update_field_loaded_block) @@ -383,15 +385,14 @@ void HashedDictionary::updateData() saved_column->insertRangeFrom(update_column, 0, update_column.size()); } } - stream->readSuffix(); } else { - auto stream = source_ptr->loadUpdatedAll(); - mergeBlockWithStream( + auto pipe = source_ptr->loadUpdatedAll(); + mergeBlockWithPipe( dict_struct.getKeysSize(), *update_field_loaded_block, - stream); + std::move(pipe)); } if (update_field_loaded_block) @@ -560,15 +561,15 @@ void HashedDictionary::loadData() { std::atomic new_size = 0; - BlockInputStreamPtr stream; + QueryPipeline pipeline; if (configuration.preallocate) - stream = source_ptr->loadAllWithSizeHint(&new_size); + pipeline.init(source_ptr->loadAllWithSizeHint(&new_size)); else - stream = source_ptr->loadAll(); + pipeline.init(source_ptr->loadAll()); - stream->readPrefix(); - - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { if (configuration.preallocate && new_size) { @@ -584,8 +585,6 @@ void HashedDictionary::loadData() blockToAttributes(block); } - - stream->readSuffix(); } else updateData(); diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index a36aeb59f01..53912f294b2 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -352,14 +352,16 @@ void IPAddressDictionary::createAttributes() void IPAddressDictionary::loadData() { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); std::vector ip_records; bool has_ipv6 = false; - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { const auto rows = block.rows(); element_count += rows; @@ -387,8 +389,6 @@ void IPAddressDictionary::loadData() } } - stream->readSuffix(); - if (access_to_key_from_attributes) { /// We format key attribute values here instead of filling with data from key_column @@ -835,7 +835,7 @@ static auto keyViewGetter() }; } -BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe IPAddressDictionary::read(const Names & column_names, size_t max_block_size) const { const bool is_ipv4 = std::get_if(&ip_column) != nullptr; @@ -857,13 +857,15 @@ BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & colum if (is_ipv4) { auto get_view = keyViewGetter, true>(); - return std::make_shared( - shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); + return Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)), + max_block_size)); } auto get_view = keyViewGetter(); - return std::make_shared( - shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)); + return Pipe(std::make_shared( + DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)), + max_block_size)); } IPAddressDictionary::RowIdxConstIter IPAddressDictionary::ipNotFound() const @@ -968,4 +970,4 @@ void registerDictionaryTrie(DictionaryFactory & factory) factory.registerLayout("ip_trie", create_layout, true); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index f44291cec6a..06eb480f825 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -78,7 +78,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; private: @@ -224,4 +224,4 @@ private: Poco::Logger * logger; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index 0b8b52a2d67..f75ca845f01 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -89,21 +89,21 @@ bool LibraryDictionarySource::supportsSelectiveLoad() const } -BlockInputStreamPtr LibraryDictionarySource::loadAll() +Pipe LibraryDictionarySource::loadAll() { LOG_TRACE(log, "loadAll {}", toString()); return bridge_helper->loadAll(); } -BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector & ids) +Pipe LibraryDictionarySource::loadIds(const std::vector & ids) { LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size()); return bridge_helper->loadIds(getDictIdsString(ids)); } -BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size()); auto block = blockForKeys(dict_struct, key_columns, requested_rows); @@ -183,4 +183,4 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) } -} +} \ No newline at end of file diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index 88e133666e6..11b9245f769 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -47,16 +47,16 @@ public: ~LibraryDictionarySource() override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for LibraryDictionarySource"); } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -92,4 +92,4 @@ private: ExternalResultDescription description; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 4a5207dc5d8..3afce312de1 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -142,12 +142,12 @@ MongoDBDictionarySource::MongoDBDictionarySource(const MongoDBDictionarySource & MongoDBDictionarySource::~MongoDBDictionarySource() = default; -BlockInputStreamPtr MongoDBDictionarySource::loadAll() +Pipe MongoDBDictionarySource::loadAll() { - return std::make_shared(connection, createCursor(db, collection, sample_block), sample_block, max_block_size); + return Pipe(std::make_shared(connection, createCursor(db, collection, sample_block), sample_block, max_block_size)); } -BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector & ids) +Pipe MongoDBDictionarySource::loadIds(const std::vector & ids) { if (!dict_struct.id) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'id' is required for selective loading"); @@ -164,11 +164,11 @@ BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector & cursor->query().selector().addNewDocument(dict_struct.id->name).add("$in", ids_array); - return std::make_shared(connection, std::move(cursor), sample_block, max_block_size); + return Pipe(std::make_shared(connection, std::move(cursor), sample_block, max_block_size)); } -BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (!dict_struct.key) throw Exception(ErrorCodes::UNSUPPORTED_METHOD, "'key' is required for selective loading"); @@ -230,7 +230,7 @@ BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_column /// If more than one key we should use $or cursor->query().selector().add("$or", keys_array); - return std::make_shared(connection, std::move(cursor), sample_block, max_block_size); + return Pipe(std::make_shared(connection, std::move(cursor), sample_block, max_block_size)); } std::string MongoDBDictionarySource::toString() const @@ -238,4 +238,4 @@ std::string MongoDBDictionarySource::toString() const return "MongoDB: " + db + '.' + collection + ',' + (user.empty() ? " " : " " + user + '@') + host + ':' + DB::toString(port); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index fef5749190f..811bec28fe7 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -46,18 +46,18 @@ public: ~MongoDBDictionarySource() override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for MongoDBDictionarySource"); } bool supportsSelectiveLoad() const override { return true; } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; /// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field bool isModified() const override { return true; } @@ -84,4 +84,4 @@ private: std::shared_ptr connection; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 77ad1e4bf29..6462158e882 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -11,6 +11,7 @@ #include "registerDictionaries.h" #include #include +#include namespace DB { @@ -131,13 +132,13 @@ std::string MySQLDictionarySource::getUpdateFieldAndDate() } } -BlockInputStreamPtr MySQLDictionarySource::loadFromQuery(const String & query) +Pipe MySQLDictionarySource::loadFromQuery(const String & query) { - return std::make_shared( - pool, query, sample_block, settings); + return Pipe(std::make_shared( + pool, query, sample_block, settings)); } -BlockInputStreamPtr MySQLDictionarySource::loadAll() +Pipe MySQLDictionarySource::loadAll() { auto connection = pool->get(); last_modification = getLastModification(connection, false); @@ -146,7 +147,7 @@ BlockInputStreamPtr MySQLDictionarySource::loadAll() return loadFromQuery(load_all_query); } -BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() +Pipe MySQLDictionarySource::loadUpdatedAll() { auto connection = pool->get(); last_modification = getLastModification(connection, false); @@ -156,14 +157,14 @@ BlockInputStreamPtr MySQLDictionarySource::loadUpdatedAll() return loadFromQuery(load_update_query); } -BlockInputStreamPtr MySQLDictionarySource::loadIds(const std::vector & ids) +Pipe MySQLDictionarySource::loadIds(const std::vector & ids) { /// We do not log in here and do not update the modification time, as the request can be large, and often called. const auto query = query_builder.composeLoadIdsQuery(ids); return loadFromQuery(query); } -BlockInputStreamPtr MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { /// We do not log in here and do not update the modification time, as the request can be large, and often called. const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); @@ -290,4 +291,4 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index afe0f0cfcc7..f3dbb89757d 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -6,7 +6,7 @@ # include "config_core.h" #endif -#if USE_MYSQL +//#if USE_MYSQL # include # include # include "DictionaryStructure.h" @@ -53,13 +53,13 @@ public: MySQLDictionarySource(const MySQLDictionarySource & other); MySQLDictionarySource & operator=(const MySQLDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -72,7 +72,7 @@ public: std::string toString() const override; private: - BlockInputStreamPtr loadFromQuery(const String & query); + Pipe loadFromQuery(const String & query); std::string getUpdateFieldAndDate(); @@ -99,4 +99,4 @@ private: } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index 73810d4b057..f1f337e8fcc 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -119,7 +119,7 @@ ColumnPtr IPolygonDictionary::getColumn( return result; } -BlockInputStreamPtr IPolygonDictionary::getBlockInputStream(const Names &, size_t) const +Pipe IPolygonDictionary::read(const Names &, size_t) const { // TODO: In order for this to work one would first have to support retrieving arrays from dictionaries. // I believe this is a separate task done by some other people. @@ -165,12 +165,13 @@ void IPolygonDictionary::blockToAttributes(const DB::Block & block) void IPolygonDictionary::loadData() { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); - while (const auto block = stream->read()) - blockToAttributes(block); - stream->readSuffix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) + blockToAttributes(block); /// Correct and sort polygons by area and update polygon_index_to_attribute_value_index after sort PaddedPODArray areas; @@ -515,5 +516,4 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column) } } -} - +} \ No newline at end of file diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index 14492df6469..c119b5a5cfe 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -97,7 +97,7 @@ public: ColumnUInt8::Ptr hasKeys(const Columns & key_columns, const DataTypes & key_types) const override; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; /** Single coordinate type. */ using Coord = Float32; @@ -166,5 +166,4 @@ private: static std::vector extractPoints(const Columns &key_columns); }; -} - +} \ No newline at end of file diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index f1ca3d4855b..1f256f3957f 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -77,35 +77,35 @@ PostgreSQLDictionarySource::PostgreSQLDictionarySource(const PostgreSQLDictionar } -BlockInputStreamPtr PostgreSQLDictionarySource::loadAll() +Pipe PostgreSQLDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); return loadBase(load_all_query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadUpdatedAll() +Pipe PostgreSQLDictionarySource::loadUpdatedAll() { auto load_update_query = getUpdateFieldAndDate(); LOG_TRACE(log, load_update_query); return loadBase(load_update_query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadIds(const std::vector & ids) +Pipe PostgreSQLDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); return loadBase(query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); return loadBase(query); } -BlockInputStreamPtr PostgreSQLDictionarySource::loadBase(const String & query) +Pipe PostgreSQLDictionarySource::loadBase(const String & query) { return std::make_shared>(pool->get(), query, sample_block, max_block_size); } @@ -220,4 +220,4 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) factory.registerSource("postgresql", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index 789f0d13966..e5a72f0f720 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -42,10 +42,10 @@ public: PostgreSQLDictionarySource(const PostgreSQLDictionarySource & other); PostgreSQLDictionarySource & operator=(const PostgreSQLDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadAll() override; + Pipe loadUpdatedAll() override; + Pipe loadIds(const std::vector & ids) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; bool supportsSelectiveLoad() const override; @@ -57,7 +57,7 @@ public: private: String getUpdateFieldAndDate(); String doInvalidateQuery(const std::string & request) const; - BlockInputStreamPtr loadBase(const String & query); + Pipe loadBase(const String & query); const DictionaryStructure dict_struct; const Configuration configuration; @@ -72,4 +72,4 @@ private: }; } -#endif +#endif \ No newline at end of file diff --git a/src/Dictionaries/RangeDictionaryBlockInputStream.h b/src/Dictionaries/RangeDictionaryBlockInputStream.h index 7d40531cfa5..1164683521c 100644 --- a/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -13,28 +13,22 @@ namespace DB { -/* - * BlockInputStream implementation for external dictionaries - * read() returns single block consisting of the in-memory contents of the dictionaries - */ + template -class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase +class RangeDictionarySourceData { public: using Key = UInt64; - RangeDictionaryBlockInputStream( + RangeDictionarySourceData( std::shared_ptr dictionary, - size_t max_block_size, const Names & column_names, PaddedPODArray && ids_to_fill, PaddedPODArray && start_dates, PaddedPODArray && end_dates); - String getName() const override { return "RangeDictionary"; } - -protected: - Block getBlock(size_t start, size_t length) const override; + Block getBlock(size_t start, size_t length) const; + size_t getNumRows() const { return ids.size(); } private: template @@ -58,15 +52,13 @@ private: template -RangeDictionaryBlockInputStream::RangeDictionaryBlockInputStream( +RangeDictionarySourceData::RangeDictionarySourceData( std::shared_ptr dictionary_, - size_t max_block_size_, const Names & column_names_, PaddedPODArray && ids_, PaddedPODArray && block_start_dates, PaddedPODArray && block_end_dates) - : DictionaryBlockInputStreamBase(ids_.size(), max_block_size_) - , dictionary(dictionary_) + : dictionary(dictionary_) , column_names(column_names_.begin(), column_names_.end()) , ids(std::move(ids_)) , start_dates(std::move(block_start_dates)) @@ -75,7 +67,7 @@ RangeDictionaryBlockInputStream::RangeDictionaryBlockInputStream( } template -Block RangeDictionaryBlockInputStream::getBlock(size_t start, size_t length) const +Block RangeDictionarySourceData::getBlock(size_t start, size_t length) const { PaddedPODArray block_ids; PaddedPODArray block_start_dates; @@ -96,7 +88,7 @@ Block RangeDictionaryBlockInputStream::getBlock(size_t start, size_t template template -ColumnPtr RangeDictionaryBlockInputStream::getColumnFromPODArray(const PaddedPODArray & array) const +ColumnPtr RangeDictionarySourceData::getColumnFromPODArray(const PaddedPODArray & array) const { auto column_vector = ColumnVector::create(); column_vector->getData().reserve(array.size()); @@ -106,7 +98,7 @@ ColumnPtr RangeDictionaryBlockInputStream::getColumnFromPODArray(cons } template -PaddedPODArray RangeDictionaryBlockInputStream::makeDateKey( +PaddedPODArray RangeDictionarySourceData::makeDateKey( const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const { PaddedPODArray key(block_start_dates.size()); @@ -123,7 +115,7 @@ PaddedPODArray RangeDictionaryBlockInputStream::makeDateKey( template -Block RangeDictionaryBlockInputStream::fillBlock( +Block RangeDictionarySourceData::fillBlock( const PaddedPODArray & ids_to_fill, const PaddedPODArray & block_start_dates, const PaddedPODArray & block_end_dates) const @@ -170,4 +162,37 @@ Block RangeDictionaryBlockInputStream::fillBlock( return Block(columns); } +/* + * BlockInputStream implementation for external dictionaries + * read() returns single block consisting of the in-memory contents of the dictionaries + */ +template +class RangeDictionarySource : public DictionarySourceBase +{ +public: + using Key = UInt64; + + RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size); + + String getName() const override { return "RangeDictionarySource"; } + +protected: + Block getBlock(size_t start, size_t length) const override; + + RangeDictionarySourceData data; +}; + +template +RangeDictionarySource::RangeDictionarySource(RangeDictionarySourceData data_, size_t max_block_size) + : DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size) + , data(std::move(data_)) +{ } + +template +Block RangeDictionarySource::getBlock(size_t start, size_t length) const +{ + return data.getBlock(start, length); +} + +} \ No newline at end of file diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index eadfc03ef33..bf930a7a526 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -298,10 +298,12 @@ void RangeHashedDictionary::createAttributes() void RangeHashedDictionary::loadData() { - auto stream = source_ptr->loadAll(); - stream->readPrefix(); + QueryPipeline pipeline; + pipeline.init(source_ptr->loadAll()); - while (const auto block = stream->read()) + PullingPipelineExecutor executor(pipeline); + Block block; + while (executor.pull(block)) { const auto & id_column = *block.safeGetByPosition(0).column; @@ -339,8 +341,6 @@ void RangeHashedDictionary::loadData() } } - stream->readSuffix(); - if (require_nonempty && 0 == element_count) throw Exception(ErrorCodes::DICTIONARY_IS_EMPTY, "{}: dictionary source is empty and 'require_nonempty' property is set."); @@ -594,29 +594,30 @@ void RangeHashedDictionary::getIdsAndDates( template -BlockInputStreamPtr RangeHashedDictionary::getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const +Pipe RangeHashedDictionary::readImpl(const Names & column_names, size_t max_block_size) const { PaddedPODArray ids; PaddedPODArray start_dates; PaddedPODArray end_dates; getIdsAndDates(ids, start_dates, end_dates); - using BlockInputStreamType = RangeDictionaryBlockInputStream; + using RangeDictionarySourceType = RangeDictionarySource; - auto stream = std::make_shared( - shared_from_this(), - max_block_size, - column_names, - std::move(ids), - std::move(start_dates), - std::move(end_dates)); + auto source = std::make_shared( + RangeDictionarySourceData( + shared_from_this(), + column_names, + std::move(ids), + std::move(start_dates), + std::move(end_dates)), + max_block_size); - return stream; + return Pipe(source); } -struct RangeHashedDictionaryCallGetBlockInputStreamImpl +struct RangeHashedDictionaryCallGetSourceImpl { - BlockInputStreamPtr stream; + Pipe pipe; const RangeHashedDictionary * dict; const Names * column_names; size_t max_block_size; @@ -625,28 +626,28 @@ struct RangeHashedDictionaryCallGetBlockInputStreamImpl void operator()() { const auto & type = dict->dict_struct.range_min->type; - if (!stream && dynamic_cast *>(type.get())) - stream = dict->getBlockInputStreamImpl(*column_names, max_block_size); + if (pipe.empty() && dynamic_cast *>(type.get())) + pipe = dict->readImpl(*column_names, max_block_size); } }; -BlockInputStreamPtr RangeHashedDictionary::getBlockInputStream(const Names & column_names, size_t max_block_size) const +Pipe RangeHashedDictionary::read(const Names & column_names, size_t max_block_size) const { using ListType = TypeList; - RangeHashedDictionaryCallGetBlockInputStreamImpl callable; + RangeHashedDictionaryCallGetSourceImpl callable; callable.dict = this; callable.column_names = &column_names; callable.max_block_size = max_block_size; ListType::forEach(callable); - if (!callable.stream) + if (callable.pipe.empty()) throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected range type for RangeHashed dictionary: {}", dict_struct.range_min->type->getName()); - return callable.stream; + return std::move(callable.pipe); } @@ -676,4 +677,4 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) factory.registerLayout("range_hashed", create_layout, false); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index 0dd9c41fd71..b0ce1e41bad 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -75,7 +75,7 @@ public: using RangeStorageType = Int64; - BlockInputStreamPtr getBlockInputStream(const Names & column_names, size_t max_block_size) const override; + Pipe read(const Names & column_names, size_t max_block_size) const override; struct Range { @@ -178,9 +178,9 @@ private: PaddedPODArray & end_dates) const; template - BlockInputStreamPtr getBlockInputStreamImpl(const Names & column_names, size_t max_block_size) const; + Pipe readImpl(const Names & column_names, size_t max_block_size) const; - friend struct RangeHashedDictionaryCallGetBlockInputStreamImpl; + friend struct RangeHashedDictionaryCallGetSourceImpl; const DictionaryStructure dict_struct; const DictionarySourcePtr source_ptr; @@ -197,4 +197,4 @@ private: mutable std::atomic found_count{0}; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisBlockInputStream.cpp b/src/Dictionaries/RedisBlockInputStream.cpp index 71d055e1d68..2d30e7fe167 100644 --- a/src/Dictionaries/RedisBlockInputStream.cpp +++ b/src/Dictionaries/RedisBlockInputStream.cpp @@ -29,18 +29,19 @@ namespace DB } - RedisBlockInputStream::RedisBlockInputStream( + RedisSource::RedisSource( const std::shared_ptr & client_, const RedisArray & keys_, const RedisStorageType & storage_type_, const DB::Block & sample_block, const size_t max_block_size_) - : client(client_), keys(keys_), storage_type(storage_type_), max_block_size{max_block_size_} + : SourceWithProgress(sample_block) + , client(client_), keys(keys_), storage_type(storage_type_), max_block_size{max_block_size_} { description.init(sample_block); } - RedisBlockInputStream::~RedisBlockInputStream() = default; + RedisSource::~RedisSource() = default; namespace @@ -121,7 +122,7 @@ namespace DB } - Block RedisBlockInputStream::readImpl() + Chunk RedisSource::generate() { if (keys.isNull() || description.sample_block.rows() == 0 || cursor >= keys.size()) all_read = true; @@ -218,6 +219,7 @@ namespace DB cursor += need_values; } - return description.sample_block.cloneWithColumns(std::move(columns)); + size_t num_rows = columns.at(0)->size(); + return Chunk(std::move(columns), num_rows); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisBlockInputStream.h index 185cfedb09d..feef6e5423e 100644 --- a/src/Dictionaries/RedisBlockInputStream.h +++ b/src/Dictionaries/RedisBlockInputStream.h @@ -3,7 +3,7 @@ #include #include -#include +#include #include #include #include "RedisDictionarySource.h" @@ -19,27 +19,25 @@ namespace Poco namespace DB { - class RedisBlockInputStream final : public IBlockInputStream + class RedisSource final : public SourceWithProgress { public: using RedisArray = Poco::Redis::Array; using RedisBulkString = Poco::Redis::BulkString; - RedisBlockInputStream( + RedisSource( const std::shared_ptr & client_, const Poco::Redis::Array & keys_, const RedisStorageType & storage_type_, const Block & sample_block, const size_t max_block_size); - ~RedisBlockInputStream() override; + ~RedisSource() override; String getName() const override { return "Redis"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - private: - Block readImpl() override; + Chunk generate() override; std::shared_ptr client; Poco::Redis::Array keys; @@ -50,4 +48,4 @@ namespace DB bool all_read = false; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 4e79a75c1d4..437c3fb0bf9 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -159,7 +159,7 @@ namespace DB __builtin_unreachable(); } - BlockInputStreamPtr RedisDictionarySource::loadAll() + Pipe RedisDictionarySource::loadAll() { if (!client->isConnected()) client->connect(host, port); @@ -170,7 +170,7 @@ namespace DB /// Get only keys for specified storage type. auto all_keys = client->execute(command_for_keys); if (all_keys.isNull()) - return std::make_shared(client, RedisArray{}, storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, RedisArray{}, storage_type, sample_block, max_block_size)); RedisArray keys; auto key_type = storageTypeToKeyType(storage_type); @@ -209,11 +209,11 @@ namespace DB keys = std::move(hkeys); } - return std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size)); } - BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector & ids) + Pipe RedisDictionarySource::loadIds(const std::vector & ids) { if (!client->isConnected()) client->connect(host, port); @@ -229,10 +229,10 @@ namespace DB for (UInt64 id : ids) keys << DB::toString(id); - return std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size)); } - BlockInputStreamPtr RedisDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) + Pipe RedisDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { if (!client->isConnected()) client->connect(host, port); @@ -258,7 +258,7 @@ namespace DB keys.add(key); } - return std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size); + return Pipe(std::make_shared(client, std::move(keys), storage_type, sample_block, max_block_size)); } @@ -276,4 +276,4 @@ namespace DB return RedisStorageType::SIMPLE; } -} +} \ No newline at end of file diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index d67403a7f1c..31c55fabef9 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -59,18 +59,18 @@ namespace ErrorCodes ~RedisDictionarySource() override; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override + Pipe loadUpdatedAll() override { throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadUpdatedAll is unsupported for RedisDictionarySource"); } bool supportsSelectiveLoad() const override { return true; } - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override { return true; } @@ -94,4 +94,4 @@ namespace ErrorCodes std::shared_ptr client; }; -} +} \ No newline at end of file diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 51984e9103a..3839e269158 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -1,7 +1,7 @@ #include "XDBCDictionarySource.h" #include -#include +#include #include #include #include @@ -30,37 +30,6 @@ namespace ErrorCodes namespace { - class XDBCBridgeBlockInputStream : public IBlockInputStream - { - public: - XDBCBridgeBlockInputStream( - const Poco::URI & uri, - std::function callback, - const Block & sample_block, - ContextPtr context, - UInt64 max_block_size, - const ConnectionTimeouts & timeouts, - const String name_) - : name(name_) - { - read_buf = std::make_unique(uri, Poco::Net::HTTPRequest::HTTP_POST, callback, timeouts); - auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, context, max_block_size); - reader = std::make_shared(format); - } - - Block getHeader() const override { return reader->getHeader(); } - - String getName() const override { return name; } - - private: - Block readImpl() override { return reader->read(); } - - String name; - std::unique_ptr read_buf; - BlockInputStreamPtr reader; - }; - - ExternalQueryBuilder makeExternalQueryBuilder(const DictionaryStructure & dict_struct_, const std::string & db_, const std::string & schema_, @@ -155,14 +124,14 @@ std::string XDBCDictionarySource::getUpdateFieldAndDate() } -BlockInputStreamPtr XDBCDictionarySource::loadAll() +Pipe XDBCDictionarySource::loadAll() { LOG_TRACE(log, load_all_query); return loadFromQuery(bridge_url, sample_block, load_all_query); } -BlockInputStreamPtr XDBCDictionarySource::loadUpdatedAll() +Pipe XDBCDictionarySource::loadUpdatedAll() { std::string load_query_update = getUpdateFieldAndDate(); @@ -171,14 +140,14 @@ BlockInputStreamPtr XDBCDictionarySource::loadUpdatedAll() } -BlockInputStreamPtr XDBCDictionarySource::loadIds(const std::vector & ids) +Pipe XDBCDictionarySource::loadIds(const std::vector & ids) { const auto query = query_builder.composeLoadIdsQuery(ids); return loadFromQuery(bridge_url, sample_block, query); } -BlockInputStreamPtr XDBCDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) +Pipe XDBCDictionarySource::loadKeys(const Columns & key_columns, const std::vector & requested_rows) { const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN); return loadFromQuery(bridge_url, sample_block, query); @@ -236,11 +205,11 @@ std::string XDBCDictionarySource::doInvalidateQuery(const std::string & request) for (const auto & [name, value] : url_params) invalidate_url.addQueryParameter(name, value); - return readInvalidateQuery(*loadFromQuery(invalidate_url, invalidate_sample_block, request)); + return readInvalidateQuery(loadFromQuery(invalidate_url, invalidate_sample_block, request)); } -BlockInputStreamPtr XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const +Pipe XDBCDictionarySource::loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const { bridge_helper->startBridgeSync(); @@ -251,16 +220,12 @@ BlockInputStreamPtr XDBCDictionarySource::loadFromQuery(const Poco::URI & url, c os << "query=" << escapeForFileName(query); }; - return std::make_shared( - url, - write_body_callback, - required_sample_block, - getContext(), - max_block_size, - timeouts, - bridge_helper->getName() + "BlockInputStream"); -} + auto read_buf = std::make_unique(url, Poco::Net::HTTPRequest::HTTP_POST, write_body_callback, timeouts); + auto format = FormatFactory::instance().getInput(IXDBCBridgeHelper::DEFAULT_FORMAT, *read_buf, sample_block, getContext(), max_block_size); + format->addBuffer(std::move(read_buf)); + return Pipe(std::move(format)); +} void registerDictionarySourceXDBC(DictionarySourceFactory & factory) { @@ -320,4 +285,4 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) factory.registerSource("jdbc", create_table_source); } -} +} \ No newline at end of file diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index ab68ab05804..82da3f22d0a 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -49,13 +49,13 @@ public: XDBCDictionarySource(const XDBCDictionarySource & other); XDBCDictionarySource & operator=(const XDBCDictionarySource &) = delete; - BlockInputStreamPtr loadAll() override; + Pipe loadAll() override; - BlockInputStreamPtr loadUpdatedAll() override; + Pipe loadUpdatedAll() override; - BlockInputStreamPtr loadIds(const std::vector & ids) override; + Pipe loadIds(const std::vector & ids) override; - BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; + Pipe loadKeys(const Columns & key_columns, const std::vector & requested_rows) override; bool isModified() const override; @@ -73,7 +73,7 @@ private: // execute invalidate_query. expects single cell in result std::string doInvalidateQuery(const std::string & request) const; - BlockInputStreamPtr loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const; + Pipe loadFromQuery(const Poco::URI & url, const Block & required_sample_block, const std::string & query) const; Poco::Logger * log; @@ -90,4 +90,4 @@ private: ConnectionTimeouts timeouts; }; -} +} \ No newline at end of file diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 79ebeacfad5..731afa5d1a6 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -40,7 +40,7 @@ StreamSettings::StreamSettings(const Settings & settings, bool auto_close_, bool { } -MySQLBlockInputStream::Connection::Connection( +MySQLSource::Connection::Connection( const mysqlxx::PoolWithFailover::Entry & entry_, const std::string & query_str) : entry(entry_) @@ -50,12 +50,13 @@ MySQLBlockInputStream::Connection::Connection( } /// Used in MaterializedMySQL and in doInvalidateQuery for dictionary source. -MySQLBlockInputStream::MySQLBlockInputStream( +MySQLSource::MySQLSource( const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, const StreamSettings & settings_) - : log(&Poco::Logger::get("MySQLBlockInputStream")) + : SourceWithProgress(sample_block.cloneEmpty()) + , log(&Poco::Logger::get("MySQLBlockInputStream")) , connection{std::make_unique(entry, query_str)} , settings{std::make_unique(settings_)} { @@ -64,26 +65,27 @@ MySQLBlockInputStream::MySQLBlockInputStream( } /// For descendant MySQLWithFailoverBlockInputStream - MySQLBlockInputStream::MySQLBlockInputStream(const Block &sample_block_, const StreamSettings & settings_) - : log(&Poco::Logger::get("MySQLBlockInputStream")) +MySQLSource::MySQLSource(const Block &sample_block_, const StreamSettings & settings_) + : SourceWithProgress(sample_block_.cloneEmpty()) + , log(&Poco::Logger::get("MySQLBlockInputStream")) , settings(std::make_unique(settings_)) { description.init(sample_block_); } /// Used by MySQL storage / table function and dictionary source. -MySQLWithFailoverBlockInputStream::MySQLWithFailoverBlockInputStream( +MySQLWithFailoverSource::MySQLWithFailoverSource( mysqlxx::PoolWithFailoverPtr pool_, const std::string & query_str_, const Block & sample_block_, const StreamSettings & settings_) -: MySQLBlockInputStream(sample_block_, settings_) -, pool(pool_) -, query_str(query_str_) + : MySQLSource(sample_block_, settings_) + , pool(pool_) + , query_str(query_str_) { } -void MySQLWithFailoverBlockInputStream::readPrefix() +void MySQLWithFailoverSource::onStart() { size_t count_connect_attempts = 0; @@ -110,6 +112,18 @@ void MySQLWithFailoverBlockInputStream::readPrefix() initPositionMappingFromQueryResultStructure(); } +Chunk MySQLWithFailoverSource::generate() +{ + if (!is_initialized) + { + onStart(); + is_initialized = true; + } + + return MySQLSource::generate(); +} + + namespace { using ValueType = ExternalResultDescription::ValueType; @@ -213,7 +227,7 @@ namespace } -Block MySQLBlockInputStream::readImpl() +Chunk MySQLSource::generate() { auto row = connection->result.fetch(); if (!row) @@ -272,10 +286,10 @@ Block MySQLBlockInputStream::readImpl() row = connection->result.fetch(); } - return description.sample_block.cloneWithColumns(std::move(columns)); + return Chunk(std::move(columns), num_rows); } -void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure() +void MySQLSource::initPositionMappingFromQueryResultStructure() { position_mapping.resize(description.sample_block.columns()); @@ -324,4 +338,4 @@ void MySQLBlockInputStream::initPositionMappingFromQueryResultStructure() } -#endif +#endif \ No newline at end of file diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index 12deb9c3146..a0ef9cfd4a8 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -2,7 +2,7 @@ #include #include -#include +#include #include #include #include @@ -25,10 +25,10 @@ struct StreamSettings }; /// Allows processing results of a MySQL query as a sequence of Blocks, simplifies chaining -class MySQLBlockInputStream : public IBlockInputStream +class MySQLSource : public SourceWithProgress { public: - MySQLBlockInputStream( + MySQLSource( const mysqlxx::PoolWithFailover::Entry & entry, const std::string & query_str, const Block & sample_block, @@ -36,11 +36,9 @@ public: String getName() const override { return "MySQL"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - protected: - MySQLBlockInputStream(const Block & sample_block_, const StreamSettings & settings); - Block readImpl() override; + MySQLSource(const Block & sample_block_, const StreamSettings & settings); + Chunk generate() override; void initPositionMappingFromQueryResultStructure(); struct Connection @@ -63,21 +61,24 @@ protected: /// Like MySQLBlockInputStream, but allocates connection only when reading is starting. /// It allows to create a lot of stream objects without occupation of all connection pool. /// Also makes attempts to reconnect in case of connection failures. -class MySQLWithFailoverBlockInputStream final : public MySQLBlockInputStream +class MySQLWithFailoverSource final : public MySQLSource { public: - MySQLWithFailoverBlockInputStream( + MySQLWithFailoverSource( mysqlxx::PoolWithFailoverPtr pool_, const std::string & query_str_, const Block & sample_block_, const StreamSettings & settings_); + Chunk generate() override; + private: - void readPrefix() override; + void onStart(); mysqlxx::PoolWithFailoverPtr pool; std::string query_str; + bool is_initialized = false; }; -} +} \ No newline at end of file From 13f95f3fdf7f71c0d9907384df4a86a41a027389 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 11:41:45 +0300 Subject: [PATCH 3/4] Streams -> Processors for dicts, part 3. --- src/Bridge/LibraryBridgeHelper.cpp | 2 +- src/Bridge/LibraryBridgeHelper.h | 2 +- src/DataStreams/MongoDBBlockInputStream.cpp | 2 +- src/DataStreams/MongoDBBlockInputStream.h | 2 +- .../PostgreSQLBlockInputStream.cpp | 2 +- src/DataStreams/PostgreSQLBlockInputStream.h | 26 ++++++--- src/Databases/MySQL/DatabaseMySQL.cpp | 10 +++- .../MySQL/FetchTablesColumnsList.cpp | 11 +++- src/Databases/MySQL/MaterializeMetadata.cpp | 55 +++++++++++++++---- .../MySQL/MaterializedMySQLSyncThread.cpp | 25 +++++++-- src/Dictionaries/CacheDictionary.cpp | 2 +- src/Dictionaries/CassandraBlockInputStream.h | 3 - .../DictionaryBlockInputStreamBase.cpp | 4 +- src/Dictionaries/DictionaryHelpers.h | 4 +- src/Dictionaries/DictionarySourceHelpers.cpp | 2 +- src/Dictionaries/DictionarySourceHelpers.h | 2 +- src/Dictionaries/DirectDictionary.cpp | 2 +- src/Dictionaries/DirectDictionary.h | 2 +- .../ExecutableDictionarySource.cpp | 2 +- src/Dictionaries/ExecutableDictionarySource.h | 2 +- .../ExecutablePoolDictionarySource.cpp | 2 +- .../ExecutablePoolDictionarySource.h | 2 +- src/Dictionaries/FileDictionarySource.cpp | 2 +- src/Dictionaries/FileDictionarySource.h | 2 +- src/Dictionaries/FlatDictionary.cpp | 2 +- src/Dictionaries/FlatDictionary.h | 2 +- src/Dictionaries/HTTPDictionarySource.cpp | 2 +- src/Dictionaries/HTTPDictionarySource.h | 2 +- src/Dictionaries/HashedDictionary.cpp | 2 +- src/Dictionaries/HashedDictionary.h | 2 +- src/Dictionaries/IPAddressDictionary.cpp | 2 +- src/Dictionaries/IPAddressDictionary.h | 2 +- src/Dictionaries/LibraryDictionarySource.cpp | 2 +- src/Dictionaries/LibraryDictionarySource.h | 2 +- src/Dictionaries/MongoDBDictionarySource.cpp | 2 +- src/Dictionaries/MongoDBDictionarySource.h | 2 +- src/Dictionaries/MySQLDictionarySource.cpp | 5 +- src/Dictionaries/MySQLDictionarySource.h | 4 +- src/Dictionaries/PolygonDictionary.cpp | 2 +- src/Dictionaries/PolygonDictionary.h | 2 +- .../PostgreSQLDictionarySource.cpp | 7 +-- src/Dictionaries/PostgreSQLDictionarySource.h | 2 +- .../RangeDictionaryBlockInputStream.h | 2 +- src/Dictionaries/RangeHashedDictionary.cpp | 2 +- src/Dictionaries/RangeHashedDictionary.h | 2 +- src/Dictionaries/RedisBlockInputStream.cpp | 2 +- src/Dictionaries/RedisBlockInputStream.h | 3 +- src/Dictionaries/RedisDictionarySource.cpp | 2 +- src/Dictionaries/RedisDictionarySource.h | 2 +- src/Dictionaries/XDBCDictionarySource.cpp | 2 +- src/Dictionaries/XDBCDictionarySource.h | 2 +- src/Formats/MySQLBlockInputStream.cpp | 2 +- src/Formats/MySQLBlockInputStream.h | 2 +- .../PostgreSQLReplicationHandler.cpp | 16 +++++- src/Storages/StorageDictionary.cpp | 4 +- src/Storages/StorageMongoDB.cpp | 3 +- src/Storages/StorageMySQL.cpp | 3 +- src/Storages/StoragePostgreSQL.cpp | 3 +- 58 files changed, 170 insertions(+), 98 deletions(-) diff --git a/src/Bridge/LibraryBridgeHelper.cpp b/src/Bridge/LibraryBridgeHelper.cpp index 81484854367..66f5cb7a600 100644 --- a/src/Bridge/LibraryBridgeHelper.cpp +++ b/src/Bridge/LibraryBridgeHelper.cpp @@ -180,4 +180,4 @@ Pipe LibraryBridgeHelper::loadBase(const Poco::URI & uri, ReadWriteBufferFromHTT return Pipe(std::move(source)); } -} \ No newline at end of file +} diff --git a/src/Bridge/LibraryBridgeHelper.h b/src/Bridge/LibraryBridgeHelper.h index 18a9c18f2ec..677c258d45d 100644 --- a/src/Bridge/LibraryBridgeHelper.h +++ b/src/Bridge/LibraryBridgeHelper.h @@ -85,4 +85,4 @@ private: size_t bridge_port; }; -} \ No newline at end of file +} diff --git a/src/DataStreams/MongoDBBlockInputStream.cpp b/src/DataStreams/MongoDBBlockInputStream.cpp index 1bc77532eb9..a0a8e3e40a5 100644 --- a/src/DataStreams/MongoDBBlockInputStream.cpp +++ b/src/DataStreams/MongoDBBlockInputStream.cpp @@ -366,4 +366,4 @@ Chunk MongoDBSource::generate() return Chunk(std::move(columns), num_rows); } -} \ No newline at end of file +} diff --git a/src/DataStreams/MongoDBBlockInputStream.h b/src/DataStreams/MongoDBBlockInputStream.h index 9100ba3240b..2e8858932f4 100644 --- a/src/DataStreams/MongoDBBlockInputStream.h +++ b/src/DataStreams/MongoDBBlockInputStream.h @@ -50,4 +50,4 @@ private: bool strict_check_names; }; -} \ No newline at end of file +} diff --git a/src/DataStreams/PostgreSQLBlockInputStream.cpp b/src/DataStreams/PostgreSQLBlockInputStream.cpp index 477ede47258..f4734ae0e3d 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.cpp +++ b/src/DataStreams/PostgreSQLBlockInputStream.cpp @@ -170,4 +170,4 @@ class PostgreSQLSource; } -#endif \ No newline at end of file +#endif diff --git a/src/DataStreams/PostgreSQLBlockInputStream.h b/src/DataStreams/PostgreSQLBlockInputStream.h index 8b2ea3b62ef..008da976619 100644 --- a/src/DataStreams/PostgreSQLBlockInputStream.h +++ b/src/DataStreams/PostgreSQLBlockInputStream.h @@ -44,11 +44,11 @@ protected: Status prepare() override; -private: void onStart(); Chunk generate() override; void onFinish(); +private: void init(const Block & sample_block); const UInt64 max_block_size; @@ -63,26 +63,34 @@ private: }; -/// Passes transaction object into PostgreSQLBlockInputStream and does not close transaction after read is finished. +/// Passes transaction object into PostgreSQLSource and does not close transaction after read is finished. template -class PostgreSQLTransactionBlockInputStream : public PostgreSQLBlockInputStream +class PostgreSQLTransactionSource : public PostgreSQLSource { public: - using Base = PostgreSQLBlockInputStream; + using Base = PostgreSQLSource; - PostgreSQLTransactionBlockInputStream( + PostgreSQLTransactionSource( std::shared_ptr tx_, const std::string & query_str_, const Block & sample_block_, const UInt64 max_block_size_) - : PostgreSQLBlockInputStream(tx_, query_str_, sample_block_, max_block_size_, false) {} + : PostgreSQLSource(tx_, query_str_, sample_block_, max_block_size_, false) {} - void readPrefix() override + Chunk generate() override { - Base::stream = std::make_unique(*Base::tx, pqxx::from_query, std::string_view(Base::query_str)); + if (!is_initialized) + { + Base::stream = std::make_unique(*Base::tx, pqxx::from_query, std::string_view(Base::query_str)); + is_initialized = true; + } + + return Base::generate(); } + + bool is_initialized = false; }; } -#endif \ No newline at end of file +#endif diff --git a/src/Databases/MySQL/DatabaseMySQL.cpp b/src/Databases/MySQL/DatabaseMySQL.cpp index 5f356348829..d4acd2af85e 100644 --- a/src/Databases/MySQL/DatabaseMySQL.cpp +++ b/src/Databases/MySQL/DatabaseMySQL.cpp @@ -12,6 +12,8 @@ # include # include # include +# include +# include # include # include # include @@ -281,9 +283,13 @@ std::map DatabaseMySQL::fetchTablesWithModificationTime(ContextP std::map tables_with_modification_time; StreamSettings mysql_input_stream_settings(local_context->getSettingsRef()); - MySQLBlockInputStream result(mysql_pool.get(), query.str(), tables_status_sample_block, mysql_input_stream_settings); + auto result = std::make_unique(mysql_pool.get(), query.str(), tables_status_sample_block, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(result))); - while (Block block = result.read()) + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { size_t rows = block.rows(); for (size_t index = 0; index < rows; ++index) diff --git a/src/Databases/MySQL/FetchTablesColumnsList.cpp b/src/Databases/MySQL/FetchTablesColumnsList.cpp index bbd187090df..353bcd877ee 100644 --- a/src/Databases/MySQL/FetchTablesColumnsList.cpp +++ b/src/Databases/MySQL/FetchTablesColumnsList.cpp @@ -8,6 +8,8 @@ #include #include #include +#include +#include #include #include #include @@ -85,8 +87,13 @@ std::map fetchTablesColumnsList( query << " TABLE_NAME IN " << toQueryStringWithQuote(tables_name) << " ORDER BY ORDINAL_POSITION"; StreamSettings mysql_input_stream_settings(settings); - MySQLBlockInputStream result(pool.get(), query.str(), tables_columns_sample_block, mysql_input_stream_settings); - while (Block block = result.read()) + auto result = std::make_unique(pool.get(), query.str(), tables_columns_sample_block, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(result))); + + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { const auto & table_name_col = *block.getByPosition(0).column; const auto & column_name_col = *block.getByPosition(1).column; diff --git a/src/Databases/MySQL/MaterializeMetadata.cpp b/src/Databases/MySQL/MaterializeMetadata.cpp index 4fd99ca3438..9f5100991aa 100644 --- a/src/Databases/MySQL/MaterializeMetadata.cpp +++ b/src/Databases/MySQL/MaterializeMetadata.cpp @@ -6,6 +6,8 @@ #include #include #include +#include +#include #include #include #include @@ -38,11 +40,16 @@ static std::unordered_map fetchTablesCreateQuery( }; StreamSettings mysql_input_stream_settings(global_settings, false, true); - MySQLBlockInputStream show_create_table( + auto show_create_table = std::make_unique( connection, "SHOW CREATE TABLE " + backQuoteIfNeed(database_name) + "." + backQuoteIfNeed(fetch_table_name), show_create_table_header, mysql_input_stream_settings); - Block create_query_block = show_create_table.read(); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(show_create_table))); + + Block create_query_block; + PullingPipelineExecutor executor(pipeline); + executor.pull(create_query_block); if (!create_query_block || create_query_block.rows() != 1) throw Exception("LOGICAL ERROR mysql show create return more rows.", ErrorCodes::LOGICAL_ERROR); @@ -60,9 +67,14 @@ static std::vector fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr std::vector tables_in_db; StreamSettings mysql_input_stream_settings(global_settings); - MySQLBlockInputStream input(connection, query, header, mysql_input_stream_settings); + auto input = std::make_unique(connection, query, header, mysql_input_stream_settings); - while (Block block = input.read()) + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { tables_in_db.reserve(tables_in_db.size() + block.rows()); for (size_t index = 0; index < block.rows(); ++index) @@ -83,8 +95,14 @@ void MaterializeMetadata::fetchMasterStatus(mysqlxx::PoolWithFailover::Entry & c }; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream input(connection, "SHOW MASTER STATUS;", header, mysql_input_stream_settings); - Block master_status = input.read(); + auto input = std::make_unique(connection, "SHOW MASTER STATUS;", header, mysql_input_stream_settings); + + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + + Block master_status; + PullingPipelineExecutor executor(pipeline); + executor.pull(master_status); if (!master_status || master_status.rows() != 1) throw Exception("Unable to get master status from MySQL.", ErrorCodes::LOGICAL_ERROR); @@ -106,9 +124,13 @@ void MaterializeMetadata::fetchMasterVariablesValue(const mysqlxx::PoolWithFailo const String & fetch_query = "SHOW VARIABLES WHERE Variable_name = 'binlog_checksum'"; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream variables_input(connection, fetch_query, variables_header, mysql_input_stream_settings); + auto variables_input = std::make_unique(connection, fetch_query, variables_header, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(variables_input))); - while (Block variables_block = variables_input.read()) + Block variables_block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(variables_block)) { ColumnPtr variables_name = variables_block.getByName("Variable_name").column; ColumnPtr variables_value = variables_block.getByName("Value").column; @@ -130,8 +152,13 @@ static bool checkSyncUserPrivImpl(const mysqlxx::PoolWithFailover::Entry & conne String grants_query, sub_privs; StreamSettings mysql_input_stream_settings(global_settings); - MySQLBlockInputStream input(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, mysql_input_stream_settings); - while (Block block = input.read()) + auto input = std::make_unique(connection, "SHOW GRANTS FOR CURRENT_USER();", sync_user_privs_header, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { for (size_t index = 0; index < block.rows(); ++index) { @@ -176,9 +203,13 @@ bool MaterializeMetadata::checkBinlogFileExists(const mysqlxx::PoolWithFailover: }; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream input(connection, "SHOW MASTER LOGS", logs_header, mysql_input_stream_settings); + auto input = std::make_unique(connection, "SHOW MASTER LOGS", logs_header, mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); - while (Block block = input.read()) + Block block; + PullingPipelineExecutor executor(pipeline); + while (executor.pull(block)) { for (size_t index = 0; index < block.rows(); ++index) { diff --git a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp index dcf77f56e18..5175e9d0467 100644 --- a/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp +++ b/src/Databases/MySQL/MaterializedMySQLSyncThread.cpp @@ -9,6 +9,8 @@ # include # include # include +# include +# include # include # include # include @@ -100,7 +102,7 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S const String & check_query = "SHOW VARIABLES;"; StreamSettings mysql_input_stream_settings(settings, false, true); - MySQLBlockInputStream variables_input(connection, check_query, variables_header, mysql_input_stream_settings); + auto variables_input = std::make_unique(connection, check_query, variables_header, mysql_input_stream_settings); std::unordered_map variables_error_message{ {"log_bin", "ON"}, @@ -110,7 +112,12 @@ static void checkMySQLVariables(const mysqlxx::Pool::Entry & connection, const S {"log_bin_use_v1_row_events", "OFF"} }; - while (Block variables_block = variables_input.read()) + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(variables_input))); + + PullingPipelineExecutor executor(pipeline); + Block variables_block; + while (executor.pull(variables_block)) { ColumnPtr variable_name_column = variables_block.getByName("Variable_name").column; ColumnPtr variable_value_column = variables_block.getByName("Value").column; @@ -327,12 +334,22 @@ static inline void dumpDataForTables( auto out = std::make_shared(getTableOutput(database_name, table_name, query_context)); StreamSettings mysql_input_stream_settings(context->getSettingsRef()); - MySQLBlockInputStream input( + auto input = std::make_unique( connection, "SELECT * FROM " + backQuoteIfNeed(mysql_database_name) + "." + backQuoteIfNeed(table_name), out->getHeader(), mysql_input_stream_settings); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + PullingPipelineExecutor executor(pipeline); + Stopwatch watch; - copyData(input, *out, is_cancelled); + + out->writePrefix(); + Block block; + while (executor.pull(block)) + out->write(block); + out->writeSuffix(); + const Progress & progress = out->getProgress(); LOG_INFO(&Poco::Logger::get("MaterializedMySQLSyncThread(" + database_name + ")"), "Materialize MySQL step 1: dump {}, {} rows, {} in {} sec., {} rows/sec., {}/sec." diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index a8754691425..39186f0d63f 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -691,4 +691,4 @@ void CacheDictionary::update(CacheDictionaryUpdateUnitPtr; template class CacheDictionary; -} \ No newline at end of file +} diff --git a/src/Dictionaries/CassandraBlockInputStream.h b/src/Dictionaries/CassandraBlockInputStream.h index 98adb19fee6..ba76e72c245 100644 --- a/src/Dictionaries/CassandraBlockInputStream.h +++ b/src/Dictionaries/CassandraBlockInputStream.h @@ -22,9 +22,6 @@ public: String getName() const override { return "Cassandra"; } - Block getHeader() const override { return description.sample_block.cloneEmpty(); } - - private: using ValueType = ExternalResultDescription::ValueType; diff --git a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp index d5e6e8a1cda..0eac8edac3d 100644 --- a/src/Dictionaries/DictionaryBlockInputStreamBase.cpp +++ b/src/Dictionaries/DictionaryBlockInputStreamBase.cpp @@ -13,9 +13,9 @@ Chunk DictionarySourceBase::generate() return {}; size_t size = std::min(max_block_size, rows_count - next_row); - auto chunk = getChunk(next_row, size); + auto block = getBlock(next_row, size); next_row += size; - return chunk; + return Chunk(block.getColumns(), size); } } diff --git a/src/Dictionaries/DictionaryHelpers.h b/src/Dictionaries/DictionaryHelpers.h index 79459057822..5a050d68326 100644 --- a/src/Dictionaries/DictionaryHelpers.h +++ b/src/Dictionaries/DictionaryHelpers.h @@ -648,4 +648,6 @@ static const PaddedPODArray & getColumnVectorData( } } -} \ No newline at end of file +} + + diff --git a/src/Dictionaries/DictionarySourceHelpers.cpp b/src/Dictionaries/DictionarySourceHelpers.cpp index 79d4d2e5376..092e7187e8f 100644 --- a/src/Dictionaries/DictionarySourceHelpers.cpp +++ b/src/Dictionaries/DictionarySourceHelpers.cpp @@ -125,4 +125,4 @@ String TransformWithAdditionalColumns::getName() const { return "TransformWithAdditionalColumns"; } -} \ No newline at end of file +} diff --git a/src/Dictionaries/DictionarySourceHelpers.h b/src/Dictionaries/DictionarySourceHelpers.h index ba05cf9ebc9..b955b6ffb66 100644 --- a/src/Dictionaries/DictionarySourceHelpers.h +++ b/src/Dictionaries/DictionarySourceHelpers.h @@ -52,4 +52,4 @@ private: size_t current_range_index = 0; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/DirectDictionary.cpp b/src/Dictionaries/DirectDictionary.cpp index afdd9bee92c..eb06701ab7a 100644 --- a/src/Dictionaries/DirectDictionary.cpp +++ b/src/Dictionaries/DirectDictionary.cpp @@ -354,4 +354,4 @@ void registerDictionaryDirect(DictionaryFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/DirectDictionary.h b/src/Dictionaries/DirectDictionary.h index 841590164f6..4700e71d94b 100644 --- a/src/Dictionaries/DirectDictionary.h +++ b/src/Dictionaries/DirectDictionary.h @@ -113,4 +113,4 @@ private: extern template class DirectDictionary; extern template class DirectDictionary; -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutableDictionarySource.cpp b/src/Dictionaries/ExecutableDictionarySource.cpp index 82b74d8196e..8802d04ff30 100644 --- a/src/Dictionaries/ExecutableDictionarySource.cpp +++ b/src/Dictionaries/ExecutableDictionarySource.cpp @@ -307,4 +307,4 @@ void registerDictionarySourceExecutable(DictionarySourceFactory & factory) factory.registerSource("executable", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutableDictionarySource.h b/src/Dictionaries/ExecutableDictionarySource.h index 084b8c13c25..f2d295b9b18 100644 --- a/src/Dictionaries/ExecutableDictionarySource.h +++ b/src/Dictionaries/ExecutableDictionarySource.h @@ -69,4 +69,4 @@ private: ContextPtr context; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.cpp b/src/Dictionaries/ExecutablePoolDictionarySource.cpp index 2cd567bbc02..e97b4253407 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.cpp +++ b/src/Dictionaries/ExecutablePoolDictionarySource.cpp @@ -325,4 +325,4 @@ void registerDictionarySourceExecutablePool(DictionarySourceFactory & factory) factory.registerSource("executable_pool", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/ExecutablePoolDictionarySource.h b/src/Dictionaries/ExecutablePoolDictionarySource.h index 9c8730632f7..22d7b713a24 100644 --- a/src/Dictionaries/ExecutablePoolDictionarySource.h +++ b/src/Dictionaries/ExecutablePoolDictionarySource.h @@ -83,4 +83,4 @@ private: std::shared_ptr process_pool; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/FileDictionarySource.cpp b/src/Dictionaries/FileDictionarySource.cpp index 3766da0a28d..bea14d88d1e 100644 --- a/src/Dictionaries/FileDictionarySource.cpp +++ b/src/Dictionaries/FileDictionarySource.cpp @@ -95,4 +95,4 @@ void registerDictionarySourceFile(DictionarySourceFactory & factory) factory.registerSource("file", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/FileDictionarySource.h b/src/Dictionaries/FileDictionarySource.h index efc86be8ba9..c8e37986b2f 100644 --- a/src/Dictionaries/FileDictionarySource.h +++ b/src/Dictionaries/FileDictionarySource.h @@ -65,4 +65,4 @@ private: Poco::Timestamp last_modification; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/FlatDictionary.cpp b/src/Dictionaries/FlatDictionary.cpp index 40afe70f504..58cb5048737 100644 --- a/src/Dictionaries/FlatDictionary.cpp +++ b/src/Dictionaries/FlatDictionary.cpp @@ -591,4 +591,4 @@ void registerDictionaryFlat(DictionaryFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/FlatDictionary.h b/src/Dictionaries/FlatDictionary.h index c16547bd3b4..e6a07200c05 100644 --- a/src/Dictionaries/FlatDictionary.h +++ b/src/Dictionaries/FlatDictionary.h @@ -178,4 +178,4 @@ private: BlockPtr update_field_loaded_block; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/HTTPDictionarySource.cpp b/src/Dictionaries/HTTPDictionarySource.cpp index 88aedd51395..4a80ebdf975 100644 --- a/src/Dictionaries/HTTPDictionarySource.cpp +++ b/src/Dictionaries/HTTPDictionarySource.cpp @@ -263,4 +263,4 @@ void registerDictionarySourceHTTP(DictionarySourceFactory & factory) factory.registerSource("http", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/HTTPDictionarySource.h b/src/Dictionaries/HTTPDictionarySource.h index 1c7d67d3c23..e4a76627d08 100644 --- a/src/Dictionaries/HTTPDictionarySource.h +++ b/src/Dictionaries/HTTPDictionarySource.h @@ -80,4 +80,4 @@ private: ConnectionTimeouts timeouts; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/HashedDictionary.cpp b/src/Dictionaries/HashedDictionary.cpp index 22a7ec87b07..b50b6a72707 100644 --- a/src/Dictionaries/HashedDictionary.cpp +++ b/src/Dictionaries/HashedDictionary.cpp @@ -766,4 +766,4 @@ void registerDictionaryHashed(DictionaryFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/HashedDictionary.h b/src/Dictionaries/HashedDictionary.h index 82e8a91b603..33c5fbf98bf 100644 --- a/src/Dictionaries/HashedDictionary.h +++ b/src/Dictionaries/HashedDictionary.h @@ -225,4 +225,4 @@ extern template class HashedDictionary; extern template class HashedDictionary; extern template class HashedDictionary; -} \ No newline at end of file +} diff --git a/src/Dictionaries/IPAddressDictionary.cpp b/src/Dictionaries/IPAddressDictionary.cpp index 53912f294b2..380ad460cba 100644 --- a/src/Dictionaries/IPAddressDictionary.cpp +++ b/src/Dictionaries/IPAddressDictionary.cpp @@ -970,4 +970,4 @@ void registerDictionaryTrie(DictionaryFactory & factory) factory.registerLayout("ip_trie", create_layout, true); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/IPAddressDictionary.h b/src/Dictionaries/IPAddressDictionary.h index 06eb480f825..94946e41ff8 100644 --- a/src/Dictionaries/IPAddressDictionary.h +++ b/src/Dictionaries/IPAddressDictionary.h @@ -224,4 +224,4 @@ private: Poco::Logger * logger; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/LibraryDictionarySource.cpp b/src/Dictionaries/LibraryDictionarySource.cpp index f75ca845f01..d923d114623 100644 --- a/src/Dictionaries/LibraryDictionarySource.cpp +++ b/src/Dictionaries/LibraryDictionarySource.cpp @@ -183,4 +183,4 @@ void registerDictionarySourceLibrary(DictionarySourceFactory & factory) } -} \ No newline at end of file +} diff --git a/src/Dictionaries/LibraryDictionarySource.h b/src/Dictionaries/LibraryDictionarySource.h index 11b9245f769..9c5de3df2e0 100644 --- a/src/Dictionaries/LibraryDictionarySource.h +++ b/src/Dictionaries/LibraryDictionarySource.h @@ -92,4 +92,4 @@ private: ExternalResultDescription description; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/MongoDBDictionarySource.cpp b/src/Dictionaries/MongoDBDictionarySource.cpp index 3afce312de1..a3c5119ade1 100644 --- a/src/Dictionaries/MongoDBDictionarySource.cpp +++ b/src/Dictionaries/MongoDBDictionarySource.cpp @@ -238,4 +238,4 @@ std::string MongoDBDictionarySource::toString() const return "MongoDB: " + db + '.' + collection + ',' + (user.empty() ? " " : " " + user + '@') + host + ':' + DB::toString(port); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/MongoDBDictionarySource.h b/src/Dictionaries/MongoDBDictionarySource.h index 811bec28fe7..3625deca9c6 100644 --- a/src/Dictionaries/MongoDBDictionarySource.h +++ b/src/Dictionaries/MongoDBDictionarySource.h @@ -84,4 +84,4 @@ private: std::shared_ptr connection; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/MySQLDictionarySource.cpp b/src/Dictionaries/MySQLDictionarySource.cpp index 6462158e882..c7309ddb950 100644 --- a/src/Dictionaries/MySQLDictionarySource.cpp +++ b/src/Dictionaries/MySQLDictionarySource.cpp @@ -285,10 +285,9 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - MySQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, settings); - return readInvalidateQuery(block_input_stream); + return readInvalidateQuery(Pipe(std::make_unique(pool->get(), request, invalidate_sample_block, settings))); } } -#endif \ No newline at end of file +#endif diff --git a/src/Dictionaries/MySQLDictionarySource.h b/src/Dictionaries/MySQLDictionarySource.h index f3dbb89757d..49ddc924a86 100644 --- a/src/Dictionaries/MySQLDictionarySource.h +++ b/src/Dictionaries/MySQLDictionarySource.h @@ -6,7 +6,7 @@ # include "config_core.h" #endif -//#if USE_MYSQL +#if USE_MYSQL # include # include # include "DictionaryStructure.h" @@ -99,4 +99,4 @@ private: } -#endif \ No newline at end of file +#endif diff --git a/src/Dictionaries/PolygonDictionary.cpp b/src/Dictionaries/PolygonDictionary.cpp index f1f337e8fcc..39152963ede 100644 --- a/src/Dictionaries/PolygonDictionary.cpp +++ b/src/Dictionaries/PolygonDictionary.cpp @@ -516,4 +516,4 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column) } } -} \ No newline at end of file +} diff --git a/src/Dictionaries/PolygonDictionary.h b/src/Dictionaries/PolygonDictionary.h index c119b5a5cfe..32771be4b16 100644 --- a/src/Dictionaries/PolygonDictionary.h +++ b/src/Dictionaries/PolygonDictionary.h @@ -166,4 +166,4 @@ private: static std::vector extractPoints(const Columns &key_columns); }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/PostgreSQLDictionarySource.cpp b/src/Dictionaries/PostgreSQLDictionarySource.cpp index 1f256f3957f..f226b7a9165 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.cpp +++ b/src/Dictionaries/PostgreSQLDictionarySource.cpp @@ -107,7 +107,7 @@ Pipe PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std Pipe PostgreSQLDictionarySource::loadBase(const String & query) { - return std::make_shared>(pool->get(), query, sample_block, max_block_size); + return Pipe(std::make_shared>(pool->get(), query, sample_block, max_block_size)); } @@ -129,8 +129,7 @@ std::string PostgreSQLDictionarySource::doInvalidateQuery(const std::string & re Block invalidate_sample_block; ColumnPtr column(ColumnString::create()); invalidate_sample_block.insert(ColumnWithTypeAndName(column, std::make_shared(), "Sample Block")); - PostgreSQLBlockInputStream<> block_input_stream(pool->get(), request, invalidate_sample_block, 1); - return readInvalidateQuery(block_input_stream); + return readInvalidateQuery(Pipe(std::make_unique>(pool->get(), request, invalidate_sample_block, 1))); } @@ -220,4 +219,4 @@ void registerDictionarySourcePostgreSQL(DictionarySourceFactory & factory) factory.registerSource("postgresql", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/PostgreSQLDictionarySource.h b/src/Dictionaries/PostgreSQLDictionarySource.h index e5a72f0f720..28ad28661ed 100644 --- a/src/Dictionaries/PostgreSQLDictionarySource.h +++ b/src/Dictionaries/PostgreSQLDictionarySource.h @@ -72,4 +72,4 @@ private: }; } -#endif \ No newline at end of file +#endif diff --git a/src/Dictionaries/RangeDictionaryBlockInputStream.h b/src/Dictionaries/RangeDictionaryBlockInputStream.h index 1164683521c..d17687b7164 100644 --- a/src/Dictionaries/RangeDictionaryBlockInputStream.h +++ b/src/Dictionaries/RangeDictionaryBlockInputStream.h @@ -195,4 +195,4 @@ Block RangeDictionarySource::getBlock(size_t start, size_t length) co return data.getBlock(start, length); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RangeHashedDictionary.cpp b/src/Dictionaries/RangeHashedDictionary.cpp index bf930a7a526..8b882b5a107 100644 --- a/src/Dictionaries/RangeHashedDictionary.cpp +++ b/src/Dictionaries/RangeHashedDictionary.cpp @@ -677,4 +677,4 @@ void registerDictionaryRangeHashed(DictionaryFactory & factory) factory.registerLayout("range_hashed", create_layout, false); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RangeHashedDictionary.h b/src/Dictionaries/RangeHashedDictionary.h index b0ce1e41bad..01ee2b3c773 100644 --- a/src/Dictionaries/RangeHashedDictionary.h +++ b/src/Dictionaries/RangeHashedDictionary.h @@ -197,4 +197,4 @@ private: mutable std::atomic found_count{0}; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/RedisBlockInputStream.cpp b/src/Dictionaries/RedisBlockInputStream.cpp index 2d30e7fe167..c6e2546cf1e 100644 --- a/src/Dictionaries/RedisBlockInputStream.cpp +++ b/src/Dictionaries/RedisBlockInputStream.cpp @@ -222,4 +222,4 @@ namespace DB size_t num_rows = columns.at(0)->size(); return Chunk(std::move(columns), num_rows); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RedisBlockInputStream.h b/src/Dictionaries/RedisBlockInputStream.h index feef6e5423e..db2e643eb4e 100644 --- a/src/Dictionaries/RedisBlockInputStream.h +++ b/src/Dictionaries/RedisBlockInputStream.h @@ -48,4 +48,5 @@ namespace DB bool all_read = false; }; -} \ No newline at end of file +} + diff --git a/src/Dictionaries/RedisDictionarySource.cpp b/src/Dictionaries/RedisDictionarySource.cpp index 437c3fb0bf9..bf309dd0e8a 100644 --- a/src/Dictionaries/RedisDictionarySource.cpp +++ b/src/Dictionaries/RedisDictionarySource.cpp @@ -276,4 +276,4 @@ namespace DB return RedisStorageType::SIMPLE; } -} \ No newline at end of file +} diff --git a/src/Dictionaries/RedisDictionarySource.h b/src/Dictionaries/RedisDictionarySource.h index 31c55fabef9..053094e2303 100644 --- a/src/Dictionaries/RedisDictionarySource.h +++ b/src/Dictionaries/RedisDictionarySource.h @@ -94,4 +94,4 @@ namespace ErrorCodes std::shared_ptr client; }; -} \ No newline at end of file +} diff --git a/src/Dictionaries/XDBCDictionarySource.cpp b/src/Dictionaries/XDBCDictionarySource.cpp index 3839e269158..26b6c24cd2d 100644 --- a/src/Dictionaries/XDBCDictionarySource.cpp +++ b/src/Dictionaries/XDBCDictionarySource.cpp @@ -285,4 +285,4 @@ void registerDictionarySourceJDBC(DictionarySourceFactory & factory) factory.registerSource("jdbc", create_table_source); } -} \ No newline at end of file +} diff --git a/src/Dictionaries/XDBCDictionarySource.h b/src/Dictionaries/XDBCDictionarySource.h index 82da3f22d0a..ebced022b62 100644 --- a/src/Dictionaries/XDBCDictionarySource.h +++ b/src/Dictionaries/XDBCDictionarySource.h @@ -90,4 +90,4 @@ private: ConnectionTimeouts timeouts; }; -} \ No newline at end of file +} diff --git a/src/Formats/MySQLBlockInputStream.cpp b/src/Formats/MySQLBlockInputStream.cpp index 731afa5d1a6..401d85f3d6b 100644 --- a/src/Formats/MySQLBlockInputStream.cpp +++ b/src/Formats/MySQLBlockInputStream.cpp @@ -338,4 +338,4 @@ void MySQLSource::initPositionMappingFromQueryResultStructure() } -#endif \ No newline at end of file +#endif diff --git a/src/Formats/MySQLBlockInputStream.h b/src/Formats/MySQLBlockInputStream.h index a0ef9cfd4a8..9c33b4404ae 100644 --- a/src/Formats/MySQLBlockInputStream.h +++ b/src/Formats/MySQLBlockInputStream.h @@ -81,4 +81,4 @@ private: bool is_initialized = false; }; -} \ No newline at end of file +} diff --git a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp index 4c614d8fd5a..3cc79756981 100644 --- a/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp +++ b/src/Storages/PostgreSQL/PostgreSQLReplicationHandler.cpp @@ -1,6 +1,8 @@ #include "PostgreSQLReplicationHandler.h" #include +#include +#include #include #include #include @@ -226,9 +228,17 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata(); auto sample_block = storage_metadata.getSampleBlockNonMaterialized(); - PostgreSQLTransactionBlockInputStream input(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); - assertBlocksHaveEqualStructure(input.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); - copyData(input, *block_io.out); + auto input = std::make_unique>(tx, query_str, sample_block, DEFAULT_BLOCK_SIZE); + QueryPipeline pipeline; + pipeline.init(Pipe(std::move(input))); + assertBlocksHaveEqualStructure(pipeline.getHeader(), block_io.out->getHeader(), "postgresql replica load from snapshot"); + + PullingPipelineExecutor executor(pipeline); + Block block; + block_io.out->writePrefix(); + while (executor.pull(block)) + block_io.out->write(block); + block_io.out->writeSuffix(); nested_storage = materialized_storage->prepare(); auto nested_table_id = nested_storage->getStorageID(); diff --git a/src/Storages/StorageDictionary.cpp b/src/Storages/StorageDictionary.cpp index 30a9dad8d91..c8bbb703999 100644 --- a/src/Storages/StorageDictionary.cpp +++ b/src/Storages/StorageDictionary.cpp @@ -169,9 +169,7 @@ Pipe StorageDictionary::read( { auto registered_dictionary_name = location == Location::SameDatabaseAndNameAsDictionary ? getStorageID().getInternalDictionaryName() : dictionary_name; auto dictionary = getContext()->getExternalDictionariesLoader().getDictionary(registered_dictionary_name, local_context); - auto stream = dictionary->getBlockInputStream(column_names, max_block_size); - /// TODO: update dictionary interface for processors. - return Pipe(std::make_shared(stream)); + return dictionary->read(column_names, max_block_size); } void StorageDictionary::shutdown() diff --git a/src/Storages/StorageMongoDB.cpp b/src/Storages/StorageMongoDB.cpp index 1fd58a293dc..a973efd7277 100644 --- a/src/Storages/StorageMongoDB.cpp +++ b/src/Storages/StorageMongoDB.cpp @@ -99,8 +99,7 @@ Pipe StorageMongoDB::read( sample_block.insert({ column_data.type, column_data.name }); } - return Pipe(std::make_shared( - std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true))); + return Pipe(std::make_shared(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true)); } void registerStorageMongoDB(StorageFactory & factory) diff --git a/src/Storages/StorageMySQL.cpp b/src/Storages/StorageMySQL.cpp index 99a930f37c4..431fda530f4 100644 --- a/src/Storages/StorageMySQL.cpp +++ b/src/Storages/StorageMySQL.cpp @@ -104,8 +104,7 @@ Pipe StorageMySQL::read( StreamSettings mysql_input_stream_settings(context_->getSettingsRef(), mysql_settings.connection_auto_close); - return Pipe(std::make_shared( - std::make_shared(pool, query, sample_block, mysql_input_stream_settings))); + return Pipe(std::make_shared(pool, query, sample_block, mysql_input_stream_settings)); } diff --git a/src/Storages/StoragePostgreSQL.cpp b/src/Storages/StoragePostgreSQL.cpp index 6072412af35..b71f2415fd8 100644 --- a/src/Storages/StoragePostgreSQL.cpp +++ b/src/Storages/StoragePostgreSQL.cpp @@ -90,8 +90,7 @@ Pipe StoragePostgreSQL::read( sample_block.insert({ column_data.type, column_data.name }); } - return Pipe(std::make_shared( - std::make_shared>(pool->get(), query, sample_block, max_block_size_))); + return Pipe(std::make_shared>(pool->get(), query, sample_block, max_block_size_)); } From a0f9df4d8711083c3f0bf1b4f38665b8566c6fe9 Mon Sep 17 00:00:00 2001 From: Nikolai Kochetov Date: Fri, 6 Aug 2021 13:06:05 +0300 Subject: [PATCH 4/4] Fix 01682_cache_dictionary_complex_key --- src/Dictionaries/CacheDictionary.cpp | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/src/Dictionaries/CacheDictionary.cpp b/src/Dictionaries/CacheDictionary.cpp index 39186f0d63f..4dfe802dd2b 100644 --- a/src/Dictionaries/CacheDictionary.cpp +++ b/src/Dictionaries/CacheDictionary.cpp @@ -487,25 +487,21 @@ template Pipe CacheDictionary::read(const Names & column_names, size_t max_block_size) const { Pipe pipe; - + std::optional data; { /// Write lock on storage const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs}; if constexpr (dictionary_key_type == DictionaryKeyType::simple) - pipe = Pipe(std::make_shared( - DictionarySourceData(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names), - max_block_size)); + data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names); else { auto keys = cache_storage_ptr->getCachedComplexKeys(); - pipe = Pipe(std::make_shared( - DictionarySourceData(shared_from_this(), keys, column_names), - max_block_size)); + data.emplace(shared_from_this(), keys, column_names); } } - return pipe; + return Pipe(std::make_shared(std::move(*data), max_block_size)); } template