mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge pull request #27273 from ClickHouse/remove-streams-from-dicts
Remove streams from dicts
This commit is contained in:
commit
b8b24684b4
@ -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));
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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>;
|
||||
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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."
|
||||
|
@ -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};
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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");
|
||||
}
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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,
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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(
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -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";
|
||||
}
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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));
|
||||
}
|
||||
|
||||
|
||||
|
@ -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");
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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>
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -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>
|
||||
|
@ -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>
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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:
|
||||
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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; }
|
||||
|
@ -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)));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
|
||||
|
@ -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)
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -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:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -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)));
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -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));
|
||||
}
|
||||
|
||||
|
||||
|
@ -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; }
|
||||
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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()
|
||||
|
@ -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)
|
||||
|
@ -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));
|
||||
}
|
||||
|
||||
|
||||
|
@ -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_));
|
||||
}
|
||||
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user