Merge pull request #27273 from ClickHouse/remove-streams-from-dicts

Remove streams from dicts
This commit is contained in:
Maksim Kita 2021-08-09 15:58:37 +03:00 committed by GitHub
commit b8b24684b4
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
71 changed files with 813 additions and 678 deletions

View File

@ -1,10 +1,9 @@
#include "LibraryBridgeHelper.h"
#include <DataStreams/OneBlockInputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <DataStreams/formatBlock.h>
#include <Dictionaries/DictionarySourceHelpers.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <Processors/Pipe.h>
#include <Processors/Formats/IInputFormat.h>
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteBufferFromString.h>
#include <IO/ReadHelpers.h>
@ -200,7 +199,7 @@ bool LibraryBridgeHelper::supportsSelectiveLoad()
}
BlockInputStreamPtr LibraryBridgeHelper::loadAll()
Pipe LibraryBridgeHelper::loadAll()
{
startBridgeSync();
auto uri = createRequestURI(LOAD_ALL_METHOD);
@ -208,7 +207,7 @@ BlockInputStreamPtr LibraryBridgeHelper::loadAll()
}
BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::vector<uint64_t> & ids)
Pipe LibraryBridgeHelper::loadIds(const std::vector<uint64_t> & ids)
{
startBridgeSync();
auto uri = createRequestURI(LOAD_IDS_METHOD);
@ -218,7 +217,7 @@ BlockInputStreamPtr LibraryBridgeHelper::loadIds(const std::vector<uint64_t> & i
}
BlockInputStreamPtr LibraryBridgeHelper::loadKeys(const Block & requested_block)
Pipe LibraryBridgeHelper::loadKeys(const Block & requested_block)
{
startBridgeSync();
auto uri = createRequestURI(LOAD_KEYS_METHOD);
@ -248,7 +247,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<ReadWriteBufferFromHTTP>(
uri,
@ -261,7 +260,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<OwningBlockInputStream<ReadWriteBufferFromHTTP>>(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));
}

View File

@ -11,6 +11,8 @@
namespace DB
{
class Pipe;
class LibraryBridgeHelper : public IBridgeHelper
{
@ -36,13 +38,13 @@ public:
bool supportsSelectiveLoad();
BlockInputStreamPtr loadAll();
Pipe loadAll();
BlockInputStreamPtr loadIds(const std::vector<uint64_t> & ids);
Pipe loadIds(const std::vector<uint64_t> & ids);
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 = {}) const;

View File

@ -150,13 +150,14 @@ std::unique_ptr<Poco::MongoDB::Cursor> createCursor(const std::string & database
return cursor;
}
MongoDBBlockInputStream::MongoDBBlockInputStream(
MongoDBSource::MongoDBSource(
std::shared_ptr<Poco::MongoDB::Connection> & connection_,
std::unique_ptr<Poco::MongoDB::Cursor> 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);
}
}

View File

@ -1,7 +1,7 @@
#pragma once
#include <Core/Block.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Core/ExternalResultDescription.h>
@ -22,24 +22,22 @@ void authenticate(Poco::MongoDB::Connection & connection, const std::string & da
std::unique_ptr<Poco::MongoDB::Cursor> 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<Poco::MongoDB::Connection> & connection_,
std::unique_ptr<Poco::MongoDB::Cursor> 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<Poco::MongoDB::Connection> connection;
std::unique_ptr<Poco::MongoDB::Cursor> cursor;

View File

@ -24,12 +24,13 @@ namespace DB
template<typename T>
PostgreSQLBlockInputStream<T>::PostgreSQLBlockInputStream(
PostgreSQLSource<T>::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<T>::PostgreSQLBlockInputStream(
template<typename T>
PostgreSQLBlockInputStream<T>::PostgreSQLBlockInputStream(
PostgreSQLSource<T>::PostgreSQLSource(
std::shared_ptr<T> 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<T>::PostgreSQLBlockInputStream(
template<typename T>
void PostgreSQLBlockInputStream<T>::init(const Block & sample_block)
void PostgreSQLSource<T>::init(const Block & sample_block)
{
description.init(sample_block);
@ -69,19 +71,34 @@ void PostgreSQLBlockInputStream<T>::init(const Block & sample_block)
template<typename T>
void PostgreSQLBlockInputStream<T>::readPrefix()
void PostgreSQLSource<T>::onStart()
{
tx = std::make_shared<T>(connection_holder->get());
stream = std::make_unique<pqxx::stream_from>(*tx, pqxx::from_query, std::string_view(query_str));
}
template<typename T>
IProcessor::Status PostgreSQLSource<T>::prepare()
{
if (!started)
{
onStart();
started = true;
}
auto status = SourceWithProgress::prepare();
if (status == Status::Finished)
onFinish();
return status;
}
template<typename T>
Block PostgreSQLBlockInputStream<T>::readImpl()
Chunk PostgreSQLSource<T>::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<T>::readImpl()
break;
}
return description.sample_block.cloneWithColumns(std::move(columns));
return Chunk(std::move(columns), num_rows);
}
template<typename T>
void PostgreSQLBlockInputStream<T>::readSuffix()
void PostgreSQLSource<T>::onFinish()
{
if (stream)
{
@ -146,10 +163,10 @@ void PostgreSQLBlockInputStream<T>::readSuffix()
}
template
class PostgreSQLBlockInputStream<pqxx::ReplicationTransaction>;
class PostgreSQLSource<pqxx::ReplicationTransaction>;
template
class PostgreSQLBlockInputStream<pqxx::ReadTransaction>;
class PostgreSQLSource<pqxx::ReadTransaction>;
}

View File

@ -6,7 +6,7 @@
#if USE_LIBPQXX
#include <Core/Block.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Core/ExternalResultDescription.h>
#include <Core/Field.h>
#include <Core/PostgreSQL/insertPostgreSQLValue.h>
@ -18,23 +18,20 @@ namespace DB
{
template <typename T = pqxx::ReadTransaction>
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<T> tx_,
const std::string & query_str_,
const Block & sample_block,
@ -45,40 +42,53 @@ protected:
std::shared_ptr<T> tx;
std::unique_ptr<pqxx::stream_from> stream;
private:
Block readImpl() override;
void readSuffix() override;
Status prepare() override;
void onStart();
Chunk generate() override;
void onFinish();
private:
void init(const Block & sample_block);
const UInt64 max_block_size;
bool auto_commit = true;
ExternalResultDescription description;
bool started = false;
postgres::ConnectionHolderPtr connection_holder;
std::unordered_map<size_t, PostgreSQLArrayInfo> array_info;
};
/// 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 <typename T>
class PostgreSQLTransactionBlockInputStream : public PostgreSQLBlockInputStream<T>
class PostgreSQLTransactionSource : public PostgreSQLSource<T>
{
public:
using Base = PostgreSQLBlockInputStream<T>;
using Base = PostgreSQLSource<T>;
PostgreSQLTransactionBlockInputStream(
PostgreSQLTransactionSource(
std::shared_ptr<T> tx_,
const std::string & query_str_,
const Block & sample_block_,
const UInt64 max_block_size_)
: PostgreSQLBlockInputStream<T>(tx_, query_str_, sample_block_, max_block_size_, false) {}
: PostgreSQLSource<T>(tx_, query_str_, sample_block_, max_block_size_, false) {}
void readPrefix() override
Chunk generate() override
{
Base::stream = std::make_unique<pqxx::stream_from>(*Base::tx, pqxx::from_query, std::string_view(Base::query_str));
if (!is_initialized)
{
Base::stream = std::make_unique<pqxx::stream_from>(*Base::tx, pqxx::from_query, std::string_view(Base::query_str));
is_initialized = true;
}
return Base::generate();
}
bool is_initialized = false;
};
}

View File

@ -12,6 +12,8 @@
# include <Databases/MySQL/DatabaseMySQL.h>
# include <Databases/MySQL/FetchTablesColumnsList.h>
# include <Formats/MySQLBlockInputStream.h>
# include <Processors/Executors/PullingPipelineExecutor.h>
# include <Processors/QueryPipeline.h>
# include <IO/Operators.h>
# include <Interpreters/Context.h>
# include <Parsers/ASTCreateQuery.h>
@ -281,9 +283,13 @@ std::map<String, UInt64> DatabaseMySQL::fetchTablesWithModificationTime(ContextP
std::map<String, UInt64> 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<MySQLSource>(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)

View File

@ -8,6 +8,8 @@
#include <DataTypes/convertMySQLDataType.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/QueryPipeline.h>
#include <Formats/MySQLBlockInputStream.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
@ -85,8 +87,13 @@ std::map<String, ColumnsDescription> 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<MySQLSource>(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;

View File

@ -6,6 +6,8 @@
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <Formats/MySQLBlockInputStream.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/QueryPipeline.h>
#include <IO/ReadBufferFromFile.h>
#include <IO/WriteBufferFromFile.h>
#include <Common/quoteString.h>
@ -38,11 +40,16 @@ static std::unordered_map<String, String> fetchTablesCreateQuery(
};
StreamSettings mysql_input_stream_settings(global_settings, false, true);
MySQLBlockInputStream show_create_table(
auto show_create_table = std::make_unique<MySQLSource>(
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<String> fetchTablesInDB(const mysqlxx::PoolWithFailover::Entr
std::vector<String> tables_in_db;
StreamSettings mysql_input_stream_settings(global_settings);
MySQLBlockInputStream input(connection, query, header, mysql_input_stream_settings);
auto input = std::make_unique<MySQLSource>(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<MySQLSource>(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<MySQLSource>(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<MySQLSource>(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<MySQLSource>(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)
{

View File

@ -9,6 +9,8 @@
# include <random>
# include <Columns/ColumnTuple.h>
# include <Columns/ColumnDecimal.h>
# include <Processors/QueryPipeline.h>
# include <Processors/Executors/PullingPipelineExecutor.h>
# include <DataStreams/CountingBlockOutputStream.h>
# include <DataStreams/OneBlockInputStream.h>
# include <DataStreams/copyData.h>
@ -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<MySQLSource>(connection, check_query, variables_header, mysql_input_stream_settings);
std::unordered_map<String, String> 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<CountingBlockOutputStream>(getTableOutput(database_name, table_name, query_context));
StreamSettings mysql_input_stream_settings(context->getSettingsRef());
MySQLBlockInputStream input(
auto input = std::make_unique<MySQLSource>(
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."

View File

@ -13,6 +13,9 @@
#include <Dictionaries/DictionaryBlockInputStream.h>
#include <Dictionaries/HierarchyDictionariesUtils.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/QueryPipeline.h>
namespace ProfileEvents
{
extern const Event DictCacheKeysRequested;
@ -481,24 +484,24 @@ MutableColumns CacheDictionary<dictionary_key_type>::aggregateColumns(
}
template <DictionaryKeyType dictionary_key_type>
BlockInputStreamPtr CacheDictionary<dictionary_key_type>::getBlockInputStream(const Names & column_names, size_t max_block_size) const
Pipe CacheDictionary<dictionary_key_type>::read(const Names & column_names, size_t max_block_size) const
{
std::shared_ptr<DictionaryBlockInputStream> stream;
Pipe pipe;
std::optional<DictionarySourceData> data;
{
/// Write lock on storage
const ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
stream = std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, cache_storage_ptr->getCachedSimpleKeys(), column_names);
data.emplace(shared_from_this(), cache_storage_ptr->getCachedSimpleKeys(), column_names);
else
{
auto keys = cache_storage_ptr->getCachedComplexKeys();
stream = std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, keys, column_names);
data.emplace(shared_from_this(), keys, column_names);
}
}
return stream;
return Pipe(std::make_shared<DictionarySource>(std::move(*data), max_block_size));
}
template <DictionaryKeyType dictionary_key_type>
@ -567,21 +570,21 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
auto current_source_ptr = getSourceAndUpdateIfNeeded();
Stopwatch watch;
BlockInputStreamPtr stream;
QueryPipeline pipeline;
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
stream = current_source_ptr->loadIds(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<KeyType> 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 +628,6 @@ void CacheDictionary<dictionary_key_type>::update(CacheDictionaryUpdateUnitPtr<d
for (const auto & fetched_column : fetched_columns_during_update)
update_unit_ptr_mutable_columns.emplace_back(fetched_column->assumeMutable());
stream->readSuffix();
{
/// Lock for cache modification
ProfilingScopedWriteRWLock write_lock{rw_lock, ProfileEvents::DictCacheLockWriteNs};

View File

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

View File

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

View File

@ -4,17 +4,17 @@
#if USE_CASSANDRA
#include <Core/Block.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Core/ExternalResultDescription.h>
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,
@ -22,14 +22,10 @@ public:
String getName() const override { return "Cassandra"; }
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 +36,7 @@ private:
ExternalResultDescription description;
cass_bool_t has_more_pages;
bool assert_types = true;
bool is_initialized = false;
};
}

View File

@ -40,7 +40,6 @@ void registerDictionarySourceCassandra(DictionarySourceFactory & factory)
#include <Common/SipHash.h>
#include "CassandraBlockInputStream.h"
#include <common/logger_useful.h>
#include <DataStreams/UnionBlockInputStream.h>
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<CassandraBlockInputStream>(getSession(), query, sample_block, max_block_size);
return Pipe(std::make_shared<CassandraSource>(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<UInt64> & ids)
Pipe CassandraDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
String query = query_builder.composeLoadIdsQuery(ids);
maybeAllowFiltering(query);
LOG_INFO(log, "Loading ids using query: {}", query);
return std::make_shared<CassandraBlockInputStream>(getSession(), query, sample_block, max_block_size);
return Pipe(std::make_shared<CassandraSource>(getSession(), query, sample_block, max_block_size));
}
BlockInputStreamPtr CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe CassandraDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<CassandraBlockInputStream>(getSession(), query, sample_block, max_block_size));
pipes.push_back(Pipe(std::make_shared<CassandraSource>(getSession(), query, sample_block, max_block_size)));
}
if (streams.size() == 1)
return streams.front();
return std::make_shared<UnionBlockInputStream>(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");
}

View File

@ -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<CassandraDictionarySource>(dict_struct, settings, sample_block);
}
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
BlockInputStreamPtr loadUpdatedAll() override;
Pipe loadUpdatedAll() override;
String toString() const override;

View File

@ -1,8 +1,11 @@
#include "ClickHouseDictionarySource.h"
#include <memory>
#include <Client/ConnectionPool.h>
#include <DataStreams/RemoteBlockInputStream.h>
#include <DataStreams/ConvertingBlockInputStream.h>
#include <Processors/Sources/RemoteSource.h>
#include <DataStreams/RemoteQueryExecutor.h>
#include <Interpreters/ActionsDAG.h>
#include <Interpreters/ExpressionActions.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <IO/ConnectionTimeouts.h>
#include <Interpreters/executeQuery.h>
#include <Common/isLocalAddress.h>
@ -105,29 +108,29 @@ std::string ClickHouseDictionarySource::getUpdateFieldAndDate()
}
}
BlockInputStreamPtr ClickHouseDictionarySource::loadAllWithSizeHint(std::atomic<size_t> * result_size_hint)
Pipe ClickHouseDictionarySource::loadAllWithSizeHint(std::atomic<size_t> * 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<UInt64> & ids)
Pipe ClickHouseDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
return createStreamForQuery(query_builder.composeLoadIdsQuery(ids));
}
BlockInputStreamPtr ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe ClickHouseDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<size_t> * result_size_hint)
Pipe ClickHouseDictionarySource::createStreamForQuery(const String & query, std::atomic<size_t> * 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<ConvertingBlockInputStream>(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<ExpressionTransform>(header, std::make_shared<ExpressionActions>(converting));
});
}
else
{
stream = std::make_shared<RemoteBlockInputStream>(pool, query, empty_sample_block, context);
pipeline.init(Pipe(std::make_shared<RemoteSource>(
std::make_shared<RemoteQueryExecutor>(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<RemoteSource>(
std::make_shared<RemoteQueryExecutor>(pool, request, invalidate_sample_block, context), false, false));
return readInvalidateQuery(std::move(pipe));
}
}

View File

@ -44,15 +44,15 @@ public:
ClickHouseDictionarySource(const ClickHouseDictionarySource & other);
ClickHouseDictionarySource & operator=(const ClickHouseDictionarySource &) = delete;
BlockInputStreamPtr loadAllWithSizeHint(std::atomic<size_t> * result_size_hint) override;
Pipe loadAllWithSizeHint(std::atomic<size_t> * result_size_hint) override;
BlockInputStreamPtr loadAll() override;
Pipe loadAll() override;
BlockInputStreamPtr loadUpdatedAll() override;
Pipe loadUpdatedAll() override;
BlockInputStreamPtr loadIds(const std::vector<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<size_t> * result_size_hint = nullptr);
Pipe createStreamForQuery(const String & query, std::atomic<size_t> * result_size_hint = nullptr);
std::string doInvalidateQuery(const std::string & request) const;

View File

@ -8,9 +8,9 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
DictionaryBlockInputStream::DictionaryBlockInputStream(
std::shared_ptr<const IDictionary> dictionary_, UInt64 max_block_size_, PaddedPODArray<UInt64> && ids_, const Names & column_names_)
: DictionaryBlockInputStreamBase(ids_.size(), max_block_size_)
DictionarySourceData::DictionarySourceData(
std::shared_ptr<const IDictionary> dictionary_, PaddedPODArray<UInt64> && 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<const IDictionary> dictionary_,
UInt64 max_block_size_,
const PaddedPODArray<StringRef> & 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<const IDictionary> 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<UInt64> & 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<UInt64> & ids_to_fill)
ColumnPtr DictionarySourceData::getColumnFromIds(const PaddedPODArray<UInt64> & ids_to_fill)
{
auto column_vector = ColumnVector<UInt64>::create();
column_vector->getData().assign(ids_to_fill);
return column_vector;
}
void DictionaryBlockInputStream::fillKeyColumns(
void DictionarySourceData::fillKeyColumns(
const PaddedPODArray<StringRef> & keys,
size_t start,
size_t size,

View File

@ -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<const IDictionary> dictionary,
UInt64 max_block_size,
PaddedPODArray<UInt64> && ids,
const Names & column_names);
DictionaryBlockInputStream(
DictionarySourceData(
std::shared_ptr<const IDictionary> dictionary,
UInt64 max_block_size,
const PaddedPODArray<StringRef> & 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<const IDictionary> 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<const IDictionary> dictionary;
Names column_names;
PaddedPODArray<UInt64> 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;
};
}

View File

@ -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 block = getBlock(next_row, size);
next_row += size;
return Chunk(block.getColumns(), size);
}
}

View File

@ -1,24 +1,22 @@
#pragma once
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
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;
};
}

View File

@ -15,7 +15,8 @@
#include <Core/Block.h>
#include <Dictionaries/IDictionary.h>
#include <Dictionaries/DictionaryStructure.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/QueryPipeline.h>
namespace DB
@ -501,10 +502,10 @@ private:
* Note: readPrefix readImpl readSuffix will be called on stream object during function execution.
*/
template <DictionaryKeyType dictionary_key_type>
void mergeBlockWithStream(
void mergeBlockWithPipe(
size_t key_columns_size,
Block & block_to_update,
BlockInputStreamPtr & stream)
Pipe pipe)
{
using KeyType = std::conditional_t<dictionary_key_type == DictionaryKeyType::simple, UInt64, StringRef>;
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;
@ -646,3 +649,5 @@ static const PaddedPODArray<T> & getColumnVectorData(
}
}

View File

@ -85,62 +85,44 @@ ContextMutablePtr copyContextAndApplySettings(
return local_context;
}
BlockInputStreamWithAdditionalColumns::BlockInputStreamWithAdditionalColumns(
Block block_to_add_, std::unique_ptr<IBlockInputStream> && 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<Int64>(block_to_add.columns() - 1); i >= 0; --i)
header.insert(0, block_to_add.getByPosition(i).cloneEmpty());
}
for (Int64 i = static_cast<Int64>(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<Int64>(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";
}
}

View File

@ -4,7 +4,7 @@
#include <common/types.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/ISimpleTransform.h>
#include <Columns/IColumn.h>
#include <Core/Block.h>
#include <Interpreters/Context_fwd.h>
@ -38,24 +38,17 @@ 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<IBlockInputStream> && 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<IBlockInputStream> stream;
size_t current_range_index = 0;
};

View File

@ -8,6 +8,8 @@
#include <Dictionaries/DictionaryFactory.h>
#include <Dictionaries/HierarchyDictionariesUtils.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
namespace DB
{
@ -66,11 +68,13 @@ Columns DirectDictionary<dictionary_key_type>::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<dictionary_key_type>::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<dictionary_key_type>::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<dictionary_key_type>::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<dictionary_key_type>::isInHierarchy(
}
template <DictionaryKeyType dictionary_key_type>
BlockInputStreamPtr DirectDictionary<dictionary_key_type>::getSourceBlockInputStream(
Pipe DirectDictionary<dictionary_key_type>::getSourceBlockInputStream(
const Columns & key_columns [[maybe_unused]],
const PaddedPODArray<KeyType> & 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<dictionary_key_type>::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<dictionary_key_type>::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 <DictionaryKeyType dictionary_key_type>
BlockInputStreamPtr DirectDictionary<dictionary_key_type>::getBlockInputStream(const Names & /* column_names */, size_t /* max_block_size */) const
Pipe DirectDictionary<dictionary_key_type>::read(const Names & /* column_names */, size_t /* max_block_size */) const
{
return source_ptr->loadAll();
}

View File

@ -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<KeyType> & requested_keys) const;
Pipe getSourceBlockInputStream(const Columns & key_columns, const PaddedPODArray<KeyType> & requested_keys) const;
const DictionaryStructure dict_struct;
const DictionarySourcePtr source_ptr;

View File

@ -2,13 +2,17 @@
#include <functional>
#include <common/scope_guard.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/OwningBlockInputStream.h>
#include <DataStreams/formatBlock.h>
#include <Processors/ISimpleTransform.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Formats/IInputFormat.h>
#include <Interpreters/Context.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/copyData.h>
#include <Common/ShellCommand.h>
#include <Common/ThreadPool.h>
#include <common/logger_useful.h>
@ -34,26 +38,34 @@ namespace ErrorCodes
namespace
{
/// Owns ShellCommand and calls wait for it.
class ShellCommandOwningBlockInputStream : public OwningBlockInputStream<ShellCommand>
class ShellCommandOwningTransform final : public ISimpleTransform
{
private:
Poco::Logger * log;
std::unique_ptr<ShellCommand> command;
public:
ShellCommandOwningBlockInputStream(Poco::Logger * log_, const BlockInputStreamPtr & impl, std::unique_ptr<ShellCommand> command_)
: OwningBlockInputStream(std::move(impl), std::move(command_)), log(log_)
ShellCommandOwningTransform(const Block & header, Poco::Logger * log_, std::unique_ptr<ShellCommand> 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<ShellCommand>::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<ShellCommandOwningBlockInputStream>(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<ShellCommandOwningTransform>(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<ShellCommandOwningBlockInputStream>(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<ShellCommandOwningTransform>(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<void(WriteBufferFromFile &)> && 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<PullingPipelineExecutor>(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<PullingPipelineExecutor> executor;
std::unique_ptr<ShellCommand> command;
std::function<void(WriteBufferFromFile &)> send_data;
ThreadFromGlobalPool thread;
};
}
BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
Pipe ExecutableDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
@ -201,7 +219,7 @@ BlockInputStreamPtr ExecutableDictionarySource::loadIds(const std::vector<UInt64
return getStreamForBlock(block);
}
BlockInputStreamPtr ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe ExecutableDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<BlockInputStreamWithBackgroundThread>(
Pipe pipe(std::make_unique<SourceWithBackgroundThread>(
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<BlockInputStreamWithAdditionalColumns>(block, std::move(stream));
else
return std::shared_ptr<BlockInputStreamWithBackgroundThread>(stream.release());
pipe.addTransform(std::make_shared<TransformWithAdditionalColumns>(block, pipe.getHeader()));
return pipe;
}
bool ExecutableDictionarySource::isModified() const

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & 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;

View File

@ -2,12 +2,15 @@
#include <functional>
#include <common/scope_guard.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Processors/QueryPipeline.h>
#include <DataStreams/formatBlock.h>
#include <Interpreters/Context.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <IO/copyData.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IInputFormat.h>
#include <Common/ShellCommand.h>
#include <Common/ThreadPool.h>
#include <common/logger_useful.h>
@ -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<ProcessPool> process_pool_,
std::unique_ptr<ShellCommand> && command_,
BlockInputStreamPtr && stream_,
Pipe pipe,
size_t read_rows_,
Poco::Logger * log_,
std::function<void(WriteBufferFromFile &)> && 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<PullingPipelineExecutor>(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<ProcessPool> process_pool;
std::unique_ptr<ShellCommand> command;
BlockInputStreamPtr stream;
QueryPipeline pipeline;
std::unique_ptr<PullingPipelineExecutor> executor;
size_t rows_to_read;
Poco::Logger * log;
std::function<void(WriteBufferFromFile &)> send_data;
@ -194,7 +199,7 @@ namespace
}
BlockInputStreamPtr ExecutablePoolDictionarySource::loadIds(const std::vector<UInt64> & ids)
Pipe ExecutablePoolDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
@ -202,7 +207,7 @@ BlockInputStreamPtr ExecutablePoolDictionarySource::loadIds(const std::vector<UI
return getStreamForBlock(block);
}
BlockInputStreamPtr ExecutablePoolDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe ExecutablePoolDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<ShellCommand> 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<PoolBlockInputStreamWithBackgroundThread>(
process_pool, std::move(process), std::move(read_stream), rows_to_read, log,
Pipe pipe(std::make_unique<PoolSourceWithBackgroundThread>(
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<BlockInputStreamWithAdditionalColumns>(block, std::move(stream));
else
return std::shared_ptr<PoolBlockInputStreamWithBackgroundThread>(stream.release());
pipe.addTransform(std::make_shared<TransformWithAdditionalColumns>(block, pipe.getHeader()));
return pipe;
}
bool ExecutablePoolDictionarySource::isModified() const

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & 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;

View File

@ -5,6 +5,8 @@
#include <DataStreams/OwningBlockInputStream.h>
#include <IO/ReadBufferFromFile.h>
#include <Interpreters/Context.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IInputFormat.h>
#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<ReadBufferFromFile>(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<OwningBlockInputStream<ReadBuffer>>(stream, std::move(in_ptr));
return Pipe(std::move(source));
}

View File

@ -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<UInt64> & /*ids*/) override
Pipe loadIds(const std::vector<UInt64> & /*ids*/) override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadIds is unsupported for FileDictionarySource");
}
BlockInputStreamPtr loadKeys(const Columns & /*key_columns*/, const std::vector<size_t> & /*requested_rows*/) override
Pipe loadKeys(const Columns & /*key_columns*/, const std::vector<size_t> & /*requested_rows*/) override
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method loadKeys is unsupported for FileDictionarySource");
}

View File

@ -10,6 +10,9 @@
#include <Columns/ColumnNullable.h>
#include <Functions/FunctionHelpers.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Dictionaries/DictionaryBlockInputStream.h>
#include <Dictionaries/DictionaryFactory.h>
#include <Dictionaries/HierarchyDictionariesUtils.h>
@ -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<DictionaryKeyType::simple>(
Pipe pipe(source_ptr->loadUpdatedAll());
mergeBlockWithPipe<DictionaryKeyType::simple>(
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<DictionaryBlockInputStream>(shared_from_this(), max_block_size, std::move(keys), column_names);
return Pipe(std::make_shared<DictionarySource>(
DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size));
}
void registerDictionaryFlat(DictionaryFactory & factory)

View File

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

View File

@ -8,6 +8,8 @@
#include <IO/WriteBufferFromOStream.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IInputFormat.h>
#include <Interpreters/Context.h>
#include <Poco/Net/HTTPRequest.h>
#include <common/logger_useful.h>
@ -62,14 +64,15 @@ HTTPDictionarySource::HTTPDictionarySource(const HTTPDictionarySource & other)
credentials.setPassword(other.credentials.getPassword());
}
BlockInputStreamPtr HTTPDictionarySource::createWrappedBuffer(std::unique_ptr<ReadWriteBufferFromHTTP> http_buffer_ptr)
Pipe HTTPDictionarySource::createWrappedBuffer(std::unique_ptr<ReadWriteBufferFromHTTP> 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<OwningBlockInputStream<ReadBuffer>>(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<UInt64> & ids)
Pipe HTTPDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
@ -151,7 +154,7 @@ BlockInputStreamPtr HTTPDictionarySource::loadIds(const std::vector<UInt64> & id
return createWrappedBuffer(std::move(in_ptr));
}
BlockInputStreamPtr HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe HTTPDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<ReadWriteBufferFromHTTP> http_buffer);
Pipe createWrappedBuffer(std::unique_ptr<ReadWriteBufferFromHTTP> http_buffer);
Poco::Logger * log;
@ -81,4 +81,3 @@ private:
};
}

View File

@ -367,10 +367,12 @@ void HashedDictionary<dictionary_key_type, sparse>::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<dictionary_key_type, sparse>::updateData()
saved_column->insertRangeFrom(update_column, 0, update_column.size());
}
}
stream->readSuffix();
}
else
{
auto stream = source_ptr->loadUpdatedAll();
mergeBlockWithStream<dictionary_key_type>(
auto pipe = source_ptr->loadUpdatedAll();
mergeBlockWithPipe<dictionary_key_type>(
dict_struct.getKeysSize(),
*update_field_loaded_block,
stream);
std::move(pipe));
}
if (update_field_loaded_block)
@ -560,15 +561,15 @@ void HashedDictionary<dictionary_key_type, sparse>::loadData()
{
std::atomic<size_t> 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<dictionary_key_type, sparse>::loadData()
blockToAttributes(block);
}
stream->readSuffix();
}
else
updateData();
@ -637,7 +636,7 @@ void HashedDictionary<dictionary_key_type, sparse>::calculateBytesAllocated()
}
template <DictionaryKeyType dictionary_key_type, bool sparse>
BlockInputStreamPtr HashedDictionary<dictionary_key_type, sparse>::getBlockInputStream(const Names & column_names, size_t max_block_size) const
Pipe HashedDictionary<dictionary_key_type, sparse>::read(const Names & column_names, size_t max_block_size) const
{
PaddedPODArray<HashedDictionary::KeyType> keys;
@ -667,9 +666,9 @@ BlockInputStreamPtr HashedDictionary<dictionary_key_type, sparse>::getBlockInput
}
if constexpr (dictionary_key_type == DictionaryKeyType::simple)
return std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, std::move(keys), column_names);
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), std::move(keys), column_names), max_block_size));
else
return std::make_shared<DictionaryBlockInputStream>(shared_from_this(), max_block_size, keys, column_names);
return Pipe(std::make_shared<DictionarySource>(DictionarySourceData(shared_from_this(), keys, column_names), max_block_size));
}
template <DictionaryKeyType dictionary_key_type, bool sparse>

View File

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

View File

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

View File

@ -1,7 +1,7 @@
#pragma once
#include <Columns/IColumn.h>
#include <DataStreams/IBlockStream_fwd.h>
#include <Processors/Pipe.h>
#include <vector>
#include <atomic>
@ -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<uint64_t> 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<size_t> * /* result_size_hint */)
virtual Pipe loadAllWithSizeHint(std::atomic<size_t> * /* 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<UInt64> & ids) = 0;
virtual Pipe loadIds(const std::vector<UInt64> & 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<size_t> & requested_rows) = 0;
virtual Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) = 0;
/// indicates whether the source has been modified since last load* operation
virtual bool isModified() const = 0;

View File

@ -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<IPRecord> 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<IPv4Container>(&ip_column) != nullptr;
@ -857,13 +857,15 @@ BlockInputStreamPtr IPAddressDictionary::getBlockInputStream(const Names & colum
if (is_ipv4)
{
auto get_view = keyViewGetter<ColumnVector<UInt32>, true>();
return std::make_shared<DictionaryBlockInputStream>(
shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view));
return Pipe(std::make_shared<DictionarySource>(
DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)),
max_block_size));
}
auto get_view = keyViewGetter<ColumnFixedString, false>();
return std::make_shared<DictionaryBlockInputStream>(
shared_from_this(), max_block_size, getKeyColumns(), column_names, std::move(get_keys), std::move(get_view));
return Pipe(std::make_shared<DictionarySource>(
DictionarySourceData(shared_from_this(), getKeyColumns(), column_names, std::move(get_keys), std::move(get_view)),
max_block_size));
}
IPAddressDictionary::RowIdxConstIter IPAddressDictionary::ipNotFound() const

View File

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

View File

@ -108,21 +108,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<UInt64> & ids)
Pipe LibraryDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
LOG_TRACE(log, "loadIds {} size = {}", toString(), ids.size());
return bridge_helper->loadIds(ids);
}
BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
Pipe LibraryDictionarySource::loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows)
{
LOG_TRACE(log, "loadKeys {} size = {}", toString(), requested_rows.size());
auto block = blockForKeys(dict_struct, key_columns, requested_rows);

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<std::size_t> & requested_rows) override;
bool isModified() const override;

View File

@ -142,12 +142,12 @@ MongoDBDictionarySource::MongoDBDictionarySource(const MongoDBDictionarySource &
MongoDBDictionarySource::~MongoDBDictionarySource() = default;
BlockInputStreamPtr MongoDBDictionarySource::loadAll()
Pipe MongoDBDictionarySource::loadAll()
{
return std::make_shared<MongoDBBlockInputStream>(connection, createCursor(db, collection, sample_block), sample_block, max_block_size);
return Pipe(std::make_shared<MongoDBSource>(connection, createCursor(db, collection, sample_block), sample_block, max_block_size));
}
BlockInputStreamPtr MongoDBDictionarySource::loadIds(const std::vector<UInt64> & ids)
Pipe MongoDBDictionarySource::loadIds(const std::vector<UInt64> & 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<UInt64> &
cursor->query().selector().addNewDocument(dict_struct.id->name).add("$in", ids_array);
return std::make_shared<MongoDBBlockInputStream>(connection, std::move(cursor), sample_block, max_block_size);
return Pipe(std::make_shared<MongoDBSource>(connection, std::move(cursor), sample_block, max_block_size));
}
BlockInputStreamPtr MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe MongoDBDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<MongoDBBlockInputStream>(connection, std::move(cursor), sample_block, max_block_size);
return Pipe(std::make_shared<MongoDBSource>(connection, std::move(cursor), sample_block, max_block_size));
}
std::string MongoDBDictionarySource::toString() const

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
/// @todo: for MongoDB, modification date can somehow be determined from the `_id` object field
bool isModified() const override { return true; }

View File

@ -11,6 +11,7 @@
#include "registerDictionaries.h"
#include <Core/Settings.h>
#include <Interpreters/Context.h>
#include <Processors/Pipe.h>
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<MySQLWithFailoverBlockInputStream>(
pool, query, sample_block, settings);
return Pipe(std::make_shared<MySQLWithFailoverSource>(
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<UInt64> & ids)
Pipe MySQLDictionarySource::loadIds(const std::vector<UInt64> & 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<size_t> & requested_rows)
Pipe MySQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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);
@ -284,8 +285,7 @@ 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<DataTypeString>(), "Sample Block"));
MySQLBlockInputStream block_input_stream(pool->get(), request, invalidate_sample_block, settings);
return readInvalidateQuery(block_input_stream);
return readInvalidateQuery(Pipe(std::make_unique<MySQLSource>(pool->get(), request, invalidate_sample_block, settings)));
}
}

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & 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();

View File

@ -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<double> areas;
@ -516,4 +517,3 @@ void IPolygonDictionary::extractPolygons(const ColumnPtr & column)
}
}

View File

@ -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;
@ -167,4 +167,3 @@ private:
};
}

View File

@ -77,37 +77,37 @@ 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<UInt64> & ids)
Pipe PostgreSQLDictionarySource::loadIds(const std::vector<UInt64> & ids)
{
const auto query = query_builder.composeLoadIdsQuery(ids);
return loadBase(query);
}
BlockInputStreamPtr PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe PostgreSQLDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<PostgreSQLBlockInputStream<>>(pool->get(), query, sample_block, max_block_size);
return Pipe(std::make_shared<PostgreSQLSource<>>(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<DataTypeString>(), "Sample Block"));
PostgreSQLBlockInputStream<> block_input_stream(pool->get(), request, invalidate_sample_block, 1);
return readInvalidateQuery(block_input_stream);
return readInvalidateQuery(Pipe(std::make_unique<PostgreSQLSource<>>(pool->get(), request, invalidate_sample_block, 1)));
}

View File

@ -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<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadAll() override;
Pipe loadUpdatedAll() override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & 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;

View File

@ -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 <typename RangeType>
class RangeDictionaryBlockInputStream : public DictionaryBlockInputStreamBase
class RangeDictionarySourceData
{
public:
using Key = UInt64;
RangeDictionaryBlockInputStream(
RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary,
size_t max_block_size,
const Names & column_names,
PaddedPODArray<Key> && ids_to_fill,
PaddedPODArray<RangeType> && start_dates,
PaddedPODArray<RangeType> && 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 <typename T>
@ -58,15 +52,13 @@ private:
template <typename RangeType>
RangeDictionaryBlockInputStream<RangeType>::RangeDictionaryBlockInputStream(
RangeDictionarySourceData<RangeType>::RangeDictionarySourceData(
std::shared_ptr<const IDictionary> dictionary_,
size_t max_block_size_,
const Names & column_names_,
PaddedPODArray<Key> && ids_,
PaddedPODArray<RangeType> && block_start_dates,
PaddedPODArray<RangeType> && 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<RangeType>::RangeDictionaryBlockInputStream(
}
template <typename RangeType>
Block RangeDictionaryBlockInputStream<RangeType>::getBlock(size_t start, size_t length) const
Block RangeDictionarySourceData<RangeType>::getBlock(size_t start, size_t length) const
{
PaddedPODArray<Key> block_ids;
PaddedPODArray<RangeType> block_start_dates;
@ -96,7 +88,7 @@ Block RangeDictionaryBlockInputStream<RangeType>::getBlock(size_t start, size_t
template <typename RangeType>
template <typename T>
ColumnPtr RangeDictionaryBlockInputStream<RangeType>::getColumnFromPODArray(const PaddedPODArray<T> & array) const
ColumnPtr RangeDictionarySourceData<RangeType>::getColumnFromPODArray(const PaddedPODArray<T> & array) const
{
auto column_vector = ColumnVector<T>::create();
column_vector->getData().reserve(array.size());
@ -106,7 +98,7 @@ ColumnPtr RangeDictionaryBlockInputStream<RangeType>::getColumnFromPODArray(cons
}
template <typename RangeType>
PaddedPODArray<Int64> RangeDictionaryBlockInputStream<RangeType>::makeDateKey(
PaddedPODArray<Int64> RangeDictionarySourceData<RangeType>::makeDateKey(
const PaddedPODArray<RangeType> & block_start_dates, const PaddedPODArray<RangeType> & block_end_dates) const
{
PaddedPODArray<Int64> key(block_start_dates.size());
@ -123,7 +115,7 @@ PaddedPODArray<Int64> RangeDictionaryBlockInputStream<RangeType>::makeDateKey(
template <typename RangeType>
Block RangeDictionaryBlockInputStream<RangeType>::fillBlock(
Block RangeDictionarySourceData<RangeType>::fillBlock(
const PaddedPODArray<Key> & ids_to_fill,
const PaddedPODArray<RangeType> & block_start_dates,
const PaddedPODArray<RangeType> & block_end_dates) const
@ -170,4 +162,37 @@ Block RangeDictionaryBlockInputStream<RangeType>::fillBlock(
return Block(columns);
}
/*
* BlockInputStream implementation for external dictionaries
* read() returns single block consisting of the in-memory contents of the dictionaries
*/
template <typename RangeType>
class RangeDictionarySource : public DictionarySourceBase
{
public:
using Key = UInt64;
RangeDictionarySource(RangeDictionarySourceData<RangeType> data_, size_t max_block_size);
String getName() const override { return "RangeDictionarySource"; }
protected:
Block getBlock(size_t start, size_t length) const override;
RangeDictionarySourceData<RangeType> data;
};
template <typename RangeType>
RangeDictionarySource<RangeType>::RangeDictionarySource(RangeDictionarySourceData<RangeType> data_, size_t max_block_size)
: DictionarySourceBase(data_.getBlock(0, 0), data_.getNumRows(), max_block_size)
, data(std::move(data_))
{
}
template <typename RangeType>
Block RangeDictionarySource<RangeType>::getBlock(size_t start, size_t length) const
{
return data.getBlock(start, length);
}
}

View File

@ -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 <typename RangeType>
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<UInt64> ids;
PaddedPODArray<RangeType> start_dates;
PaddedPODArray<RangeType> end_dates;
getIdsAndDates(ids, start_dates, end_dates);
using BlockInputStreamType = RangeDictionaryBlockInputStream<RangeType>;
using RangeDictionarySourceType = RangeDictionarySource<RangeType>;
auto stream = std::make_shared<BlockInputStreamType>(
shared_from_this(),
max_block_size,
column_names,
std::move(ids),
std::move(start_dates),
std::move(end_dates));
auto source = std::make_shared<RangeDictionarySourceType>(
RangeDictionarySourceData<RangeType>(
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<const DataTypeNumberBase<RangeType> *>(type.get()))
stream = dict->getBlockInputStreamImpl<RangeType>(*column_names, max_block_size);
if (pipe.empty() && dynamic_cast<const DataTypeNumberBase<RangeType> *>(type.get()))
pipe = dict->readImpl<RangeType>(*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<UInt8, UInt16, UInt32, UInt64, Int8, Int16, Int32, Int64, Int128, Float32, Float64>;
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);
}

View File

@ -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<RangeType> & end_dates) const;
template <typename RangeType>
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;

View File

@ -29,18 +29,19 @@ namespace DB
}
RedisBlockInputStream::RedisBlockInputStream(
RedisSource::RedisSource(
const std::shared_ptr<Poco::Redis::Client> & 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);
}
}

View File

@ -3,7 +3,7 @@
#include <Core/Block.h>
#include <Core/ExternalResultDescription.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Poco/Redis/Array.h>
#include <Poco/Redis/Type.h>
#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<Poco::Redis::Client> & 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<Poco::Redis::Client> client;
Poco::Redis::Array keys;
@ -51,3 +49,4 @@ namespace DB
};
}

View File

@ -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<RedisArray>(command_for_keys);
if (all_keys.isNull())
return std::make_shared<RedisBlockInputStream>(client, RedisArray{}, storage_type, sample_block, max_block_size);
return Pipe(std::make_shared<RedisSource>(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<RedisBlockInputStream>(client, std::move(keys), storage_type, sample_block, max_block_size);
return Pipe(std::make_shared<RedisSource>(client, std::move(keys), storage_type, sample_block, max_block_size));
}
BlockInputStreamPtr RedisDictionarySource::loadIds(const std::vector<UInt64> & ids)
Pipe RedisDictionarySource::loadIds(const std::vector<UInt64> & 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<RedisBlockInputStream>(client, std::move(keys), storage_type, sample_block, max_block_size);
return Pipe(std::make_shared<RedisSource>(client, std::move(keys), storage_type, sample_block, max_block_size));
}
BlockInputStreamPtr RedisDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
Pipe RedisDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows)
{
if (!client->isConnected())
client->connect(host, port);
@ -258,7 +258,7 @@ namespace DB
keys.add(key);
}
return std::make_shared<RedisBlockInputStream>(client, std::move(keys), storage_type, sample_block, max_block_size);
return Pipe(std::make_shared<RedisSource>(client, std::move(keys), storage_type, sample_block, max_block_size));
}

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
bool isModified() const override { return true; }

View File

@ -1,7 +1,7 @@
#include "XDBCDictionarySource.h"
#include <Columns/ColumnString.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <DataTypes/DataTypeString.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
@ -30,37 +30,6 @@ namespace ErrorCodes
namespace
{
class XDBCBridgeBlockInputStream : public IBlockInputStream
{
public:
XDBCBridgeBlockInputStream(
const Poco::URI & uri,
std::function<void(std::ostream &)> callback,
const Block & sample_block,
ContextPtr context,
UInt64 max_block_size,
const ConnectionTimeouts & timeouts,
const String name_)
: name(name_)
{
read_buf = std::make_unique<ReadWriteBufferFromHTTP>(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<InputStreamFromInputFormat>(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<ReadWriteBufferFromHTTP> 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<UInt64> & ids)
Pipe XDBCDictionarySource::loadIds(const std::vector<UInt64> & 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<size_t> & requested_rows)
Pipe XDBCDictionarySource::loadKeys(const Columns & key_columns, const std::vector<size_t> & 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<XDBCBridgeBlockInputStream>(
url,
write_body_callback,
required_sample_block,
getContext(),
max_block_size,
timeouts,
bridge_helper->getName() + "BlockInputStream");
}
auto read_buf = std::make_unique<ReadWriteBufferFromHTTP>(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)
{

View File

@ -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<UInt64> & ids) override;
Pipe loadIds(const std::vector<UInt64> & ids) override;
BlockInputStreamPtr loadKeys(const Columns & key_columns, const std::vector<size_t> & requested_rows) override;
Pipe loadKeys(const Columns & key_columns, const std::vector<size_t> & 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;

View File

@ -1,5 +1,6 @@
#include "readInvalidateQuery.h"
#include <DataStreams/IBlockInputStream.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <IO/WriteBufferFromString.h>
#include <Formats/FormatSettings.h>
@ -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();
}

View File

@ -1,13 +1,13 @@
#pragma once
#include <DataStreams/IBlockStream_fwd.h>
#include <string>
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);
}

View File

@ -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<Connection>(entry, query_str)}
, settings{std::make_unique<StreamSettings>(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<StreamSettings>(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());

View File

@ -2,7 +2,7 @@
#include <string>
#include <Core/Block.h>
#include <DataStreams/IBlockInputStream.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <mysqlxx/PoolWithFailover.h>
#include <mysqlxx/Query.h>
#include <Core/ExternalResultDescription.h>
@ -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;
};
}

View File

@ -1,6 +1,8 @@
#include "PostgreSQLReplicationHandler.h"
#include <DataStreams/PostgreSQLBlockInputStream.h>
#include <Processors/QueryPipeline.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
#include <Databases/PostgreSQL/fetchPostgreSQLTableStructure.h>
#include <Storages/PostgreSQL/StorageMaterializedPostgreSQL.h>
#include <Interpreters/InterpreterDropQuery.h>
@ -226,9 +228,17 @@ StoragePtr PostgreSQLReplicationHandler::loadFromSnapshot(String & snapshot_name
const StorageInMemoryMetadata & storage_metadata = nested_storage->getInMemoryMetadata();
auto sample_block = storage_metadata.getSampleBlockNonMaterialized();
PostgreSQLTransactionBlockInputStream<pqxx::ReplicationTransaction> 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<PostgreSQLTransactionSource<pqxx::ReplicationTransaction>>(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();

View File

@ -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<SourceFromInputStream>(stream));
return dictionary->read(column_names, max_block_size);
}
void StorageDictionary::shutdown()

View File

@ -99,8 +99,7 @@ Pipe StorageMongoDB::read(
sample_block.insert({ column_data.type, column_data.name });
}
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<MongoDBBlockInputStream>(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true)));
return Pipe(std::make_shared<MongoDBSource>(connection, createCursor(database_name, collection_name, sample_block), sample_block, max_block_size, true));
}
void registerStorageMongoDB(StorageFactory & factory)

View File

@ -104,8 +104,7 @@ Pipe StorageMySQL::read(
StreamSettings mysql_input_stream_settings(context_->getSettingsRef(),
mysql_settings.connection_auto_close);
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<MySQLWithFailoverBlockInputStream>(pool, query, sample_block, mysql_input_stream_settings)));
return Pipe(std::make_shared<MySQLWithFailoverSource>(pool, query, sample_block, mysql_input_stream_settings));
}

View File

@ -90,8 +90,7 @@ Pipe StoragePostgreSQL::read(
sample_block.insert({ column_data.type, column_data.name });
}
return Pipe(std::make_shared<SourceFromInputStream>(
std::make_shared<PostgreSQLBlockInputStream<>>(pool->get(), query, sample_block, max_block_size_)));
return Pipe(std::make_shared<PostgreSQLSource<>>(pool->get(), query, sample_block, max_block_size_));
}