Added method "getHeader" in IBlockInputStream [#CLICKHOUSE-2]

This commit is contained in:
Alexey Milovidov 2018-01-06 21:10:44 +03:00
parent e2d70943b8
commit 9996e181e1
141 changed files with 747 additions and 969 deletions

View File

@ -338,7 +338,8 @@ protected:
std::stack<backtrack_info> back_stack;
/// backtrack if possible
const auto do_backtrack = [&] {
const auto do_backtrack = [&]
{
while (!back_stack.empty())
{
auto & top = back_stack.top();

View File

@ -210,7 +210,8 @@ private:
if (sorted)
return;
sorted = true;
std::sort(samples.begin(), samples.end(), [] (const std::pair<T, UInt32> & lhs, const std::pair<T, UInt32> & rhs) {
std::sort(samples.begin(), samples.end(), [] (const std::pair<T, UInt32> & lhs, const std::pair<T, UInt32> & rhs)
{
return lhs.first < rhs.first;
});
}

View File

@ -309,7 +309,7 @@ public:
{
/// Normalizing factor for harmonic mean.
static constexpr double alpha_m =
bucket_count == 2 ? 0.351 :
bucket_count == 2 ? 0.351 :
bucket_count == 4 ? 0.532 :
bucket_count == 8 ? 0.626 :
bucket_count == 16 ? 0.673 :
@ -516,10 +516,8 @@ private:
}
private:
/// Maximum rank.
static constexpr int max_rank = sizeof(HashValueType) * 8 - precision + 1;
/// Rank storage.
RankStore rank_store;
/// Expression's denominator for HyperLogLog algorithm.

View File

@ -85,7 +85,7 @@ public:
unsigned getNumberOfSubpatterns() const { return number_of_subpatterns; }
/// Get the regexp re2 or nullptr if the pattern is trivial (for output to the log).
const std::unique_ptr<RegexType>& getRE2() const { return re2; }
const std::unique_ptr<RegexType> & getRE2() const { return re2; }
static void analyze(const std::string & regexp_, std::string & required_substring, bool & is_trivial, bool & required_substring_is_prefix);

View File

@ -62,7 +62,8 @@ public:
void free(const void * ptr)
{
union {
union
{
const void * p_v;
Block * block;
};

View File

@ -74,7 +74,8 @@ void formatIPv6(const unsigned char * src, char *& dst, UInt8 zeroed_tail_bytes_
for (const auto i : ext::range(0, words.size()))
{
if (words[i] == 0) {
if (words[i] == 0)
{
if (cur.base == -1)
cur.base = i, cur.len = 1;
else

View File

@ -30,7 +30,8 @@ TEST(Common, RWLockFIFO_1)
static thread_local std::random_device rd;
static thread_local pcg64 gen(rd());
auto func = [&] (size_t threads, int round) {
auto func = [&] (size_t threads, int round)
{
for (int i = 0; i < cycles; ++i)
{
auto type = (std::uniform_int_distribution<>(0, 9)(gen) >= round) ? RWLockFIFO::Read : RWLockFIFO::Write;
@ -92,7 +93,8 @@ TEST(Common, RWLockFIFO_Recursive)
static thread_local std::random_device rd;
static thread_local pcg64 gen(rd());
std::thread t1([&] () {
std::thread t1([&] ()
{
for (int i = 0; i < 2 * cycles; ++i)
{
auto lock = fifo_lock->getLock(RWLockFIFO::Write);
@ -102,7 +104,8 @@ TEST(Common, RWLockFIFO_Recursive)
}
});
std::thread t2([&] () {
std::thread t2([&] ()
{
for (int i = 0; i < cycles; ++i)
{
auto lock1 = fifo_lock->getLock(RWLockFIFO::Read);

View File

@ -286,7 +286,7 @@ namespace Hashes
template <template <typename...> class Map, typename Hash>
void NO_INLINE test(const Key * data, size_t size, std::function<void(Map<Key, Value, Hash>&)> init = {})
void NO_INLINE test(const Key * data, size_t size, std::function<void(Map<Key, Value, Hash> &)> init = {})
{
Stopwatch watch;

View File

@ -47,7 +47,8 @@ try
}
// <defunct> hunting:
for (int i = 0; i < 1000; ++i) {
for (int i = 0; i < 1000; ++i)
{
auto command = ShellCommand::execute("echo " + std::to_string(i));
//command->wait(); // now automatic
}

View File

@ -31,6 +31,13 @@ public:
return res.str();
}
Block getHeader() override
{
Block res = children.back()->getHeader();
res.insert({nullptr, data_type, column_name});
return res;
}
protected:
Block readImpl() override
{

View File

@ -14,6 +14,11 @@ namespace ProfileEvents
namespace DB
{
Block AggregatingBlockInputStream::getHeader()
{
return aggregator.getHeader(final);
}
Block AggregatingBlockInputStream::readImpl()
{
@ -42,7 +47,7 @@ Block AggregatingBlockInputStream::readImpl()
if (!isCancelled())
{
/// Flush data in the RAM to disk also. It's easier.
/// Flush data in the RAM to disk also. It's easier than merging on-disk and RAM data.
if (data_variants->size())
aggregator.writeToTemporaryFile(*data_variants);
}

View File

@ -37,6 +37,8 @@ public:
return res.str();
}
Block getHeader() override;
protected:
Block readImpl() override;

View File

@ -49,6 +49,8 @@ public:
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
/// Can return 1 more records than max_block_size.
Block readImpl() override;

View File

@ -80,6 +80,9 @@ public:
}
Block getHeader() override { return children.at(0)->getHeader(); }
~AsynchronousBlockInputStream() override
{
if (started)

View File

@ -5,7 +5,7 @@
namespace DB
{
/** Adds to one thread additional block information that is specified
/** Adds to one stream additional block information that is specified
* as the constructor parameter.
*/
class BlockExtraInfoInputStream : public IProfilingBlockInputStream
@ -31,6 +31,8 @@ public:
return res.str();
}
Block getHeader() override { return children.back()->getHeader(); }
protected:
Block readImpl() override
{

View File

@ -38,6 +38,8 @@ public:
RowInputStreamPtr & getRowInput() { return row_input; }
Block getHeader() override { return sample; }
protected:
Block readImpl() override;

View File

@ -36,45 +36,46 @@ CapnProtoRowInputStream::NestedField split(const Block & header, size_t i)
Field convertNodeToField(capnp::DynamicValue::Reader value)
{
switch (value.getType()) {
case capnp::DynamicValue::UNKNOWN:
throw Exception("Unknown field type");
case capnp::DynamicValue::VOID:
return Field();
case capnp::DynamicValue::BOOL:
return UInt64(value.as<bool>() ? 1 : 0);
case capnp::DynamicValue::INT:
return Int64((value.as<int64_t>()));
case capnp::DynamicValue::UINT:
return UInt64(value.as<uint64_t>());
case capnp::DynamicValue::FLOAT:
return Float64(value.as<double>());
case capnp::DynamicValue::TEXT:
switch (value.getType())
{
auto arr = value.as<capnp::Text>();
return String(arr.begin(), arr.size());
}
case capnp::DynamicValue::DATA:
{
auto arr = value.as<capnp::Data>().asChars();
return String(arr.begin(), arr.size());
}
case capnp::DynamicValue::LIST:
{
auto listValue = value.as<capnp::DynamicList>();
Array res(listValue.size());
for (auto i : kj::indices(listValue))
res[i] = convertNodeToField(listValue[i]);
return res;
}
case capnp::DynamicValue::ENUM:
return UInt64(value.as<capnp::DynamicEnum>().getRaw());
case capnp::DynamicValue::STRUCT:
throw Exception("STRUCT type not supported, read individual fields instead");
case capnp::DynamicValue::CAPABILITY:
throw Exception("CAPABILITY type not supported");
case capnp::DynamicValue::ANY_POINTER:
throw Exception("ANY_POINTER type not supported");
case capnp::DynamicValue::UNKNOWN:
throw Exception("Unknown field type");
case capnp::DynamicValue::VOID:
return Field();
case capnp::DynamicValue::BOOL:
return UInt64(value.as<bool>() ? 1 : 0);
case capnp::DynamicValue::INT:
return Int64((value.as<int64_t>()));
case capnp::DynamicValue::UINT:
return UInt64(value.as<uint64_t>());
case capnp::DynamicValue::FLOAT:
return Float64(value.as<double>());
case capnp::DynamicValue::TEXT:
{
auto arr = value.as<capnp::Text>();
return String(arr.begin(), arr.size());
}
case capnp::DynamicValue::DATA:
{
auto arr = value.as<capnp::Data>().asChars();
return String(arr.begin(), arr.size());
}
case capnp::DynamicValue::LIST:
{
auto listValue = value.as<capnp::DynamicList>();
Array res(listValue.size());
for (auto i : kj::indices(listValue))
res[i] = convertNodeToField(listValue[i]);
return res;
}
case capnp::DynamicValue::ENUM:
return UInt64(value.as<capnp::DynamicEnum>().getRaw());
case capnp::DynamicValue::STRUCT:
throw Exception("STRUCT type not supported, read individual fields instead");
case capnp::DynamicValue::CAPABILITY:
throw Exception("CAPABILITY type not supported");
case capnp::DynamicValue::ANY_POINTER:
throw Exception("ANY_POINTER type not supported");
}
}
@ -172,19 +173,21 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns)
for (auto action : actions)
{
switch (action.type) {
case Action::READ: {
auto & col = columns[action.column];
Field value = convertNodeToField(stack.back().get(action.field));
col->insert(value);
break;
}
case Action::POP:
stack.pop_back();
break;
case Action::PUSH:
stack.push_back(stack.back().get(action.field).as<capnp::DynamicStruct>());
break;
switch (action.type)
{
case Action::READ:
{
auto & col = columns[action.column];
Field value = convertNodeToField(stack.back().get(action.field));
col->insert(value);
break;
}
case Action::POP:
stack.pop_back();
break;
case Action::PUSH:
stack.push_back(stack.back().get(action.field).as<capnp::DynamicStruct>());
break;
}
}

View File

@ -21,6 +21,8 @@ public:
String getID() const override;
Block getHeader() override { return ref_definition; }
protected:
Block readImpl() override;

View File

@ -48,6 +48,8 @@ public:
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -51,6 +51,8 @@ public:
return res.str();
}
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
/// Can return 1 more records than max_block_size.
Block readImpl() override;

View File

@ -67,6 +67,8 @@ public:
void readSuffixImpl() override;
Block getHeader() override { return children.at(0)->getHeader(); }
/// for use in implementations of IColumn::gather()
template <typename Column>
void gather(Column & column_res);

View File

@ -41,6 +41,8 @@ public:
return res.str();
}
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override
{

View File

@ -54,6 +54,8 @@ public:
return res.str();
}
Block getHeader() override { return children.back()->getHeader(); }
/// Takes `totals` only from the main source, not from subquery sources.
const Block & getTotals() override;

View File

@ -24,6 +24,8 @@ public:
String getID() const override;
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -27,6 +27,8 @@ public:
String getID() const override;
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -1,27 +0,0 @@
#pragma once
#include <DataStreams/IBlockOutputStream.h>
#include <Common/Exception.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM;
}
/** When trying to write blocks to this stream of blocks, throws an exception.
* Used where, in general, you need to pass a stream of blocks, but in some cases, it should not be used.
*/
class EmptyBlockOutputStream : public IBlockOutputStream
{
public:
void write(const Block & block) override
{
throw Exception("Cannot write to EmptyBlockOutputStream", ErrorCodes::CANNOT_WRITE_TO_EMPTY_BLOCK_OUTPUT_STREAM);
}
};
}

View File

@ -31,6 +31,13 @@ const Block & ExpressionBlockInputStream::getTotals()
return totals;
}
Block ExpressionBlockInputStream::getHeader()
{
auto res = children.back()->getHeader();
expression->execute(res);
return res;
}
Block ExpressionBlockInputStream::readImpl()
{
Block res = children.back()->read();

View File

@ -24,6 +24,7 @@ public:
String getName() const override;
String getID() const override;
const Block & getTotals() override;
Block getHeader() override;
protected:
Block readImpl() override;

View File

@ -23,10 +23,14 @@ FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input
children.push_back(input);
}
FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name_)
: expression(expression_), filter_column(-1), filter_column_name(filter_column_name_)
FilterBlockInputStream::FilterBlockInputStream(const BlockInputStreamPtr & input, const ExpressionActionsPtr & expression_, const String & filter_column_name)
: expression(expression_)
{
children.push_back(input);
/// Determine position of filter column.
Block src_header = input->getHeader();
filter_column = src_header.getPositionByName(filter_column_name);
}
@ -36,7 +40,7 @@ String FilterBlockInputStream::getName() const { return "Filter"; }
String FilterBlockInputStream::getID() const
{
std::stringstream res;
res << "Filter(" << children.back()->getID() << ", " << expression->getID() << ", " << filter_column << ", " << filter_column_name << ")";
res << "Filter(" << children.back()->getID() << ", " << expression->getID() << ", " << filter_column << ")";
return res.str();
}
@ -53,34 +57,39 @@ const Block & FilterBlockInputStream::getTotals()
}
Block FilterBlockInputStream::getHeader()
{
auto res = children.back()->getHeader();
expression->execute(res);
/// Isn't the filter already constant?
ColumnPtr column = res.safeGetByPosition(filter_column).column;
if (!have_constant_filter_description)
{
have_constant_filter_description = true;
if (column)
constant_filter_description = ConstantFilterDescription(*column);
}
if (constant_filter_description.always_false
|| constant_filter_description.always_true)
return res;
/// Replace the filter column to a constant with value 1.
auto res_filter_elem = res.getByPosition(filter_column);
res_filter_elem.column = res_filter_elem.type->createColumnConst(res.rows(), UInt64(1));
return res;
}
Block FilterBlockInputStream::readImpl()
{
Block res;
if (is_first)
if (!have_constant_filter_description)
{
is_first = false;
const Block & sample_block = expression->getSampleBlock();
/// Find the current position of the filter column in the block.
/** sample_block has the result structure of evaluating the expression.
* But this structure does not necessarily match expression->execute(res) below,
* because the expression can be applied to a block that also contains additional,
* columns unnecessary for this expression, but needed later, in the next stages of the query execution pipeline.
* There will be no such columns in sample_block.
* Therefore, the position of the filter column in it can be different.
*/
ssize_t filter_column_in_sample_block = filter_column;
if (filter_column_in_sample_block == -1)
filter_column_in_sample_block = sample_block.getPositionByName(filter_column_name);
/// Let's check if the filter column is a constant containing 0 or 1.
ColumnPtr column = sample_block.safeGetByPosition(filter_column_in_sample_block).column;
if (column)
constant_filter_description = ConstantFilterDescription(*column);
getHeader();
if (constant_filter_description.always_false)
return res;
}
@ -97,10 +106,6 @@ Block FilterBlockInputStream::readImpl()
if (constant_filter_description.always_true)
return res;
/// Find the current position of the filter column in the block.
if (filter_column == -1)
filter_column = res.getPositionByName(filter_column_name);
size_t columns = res.columns();
ColumnPtr column = res.safeGetByPosition(filter_column).column;

View File

@ -27,6 +27,7 @@ public:
String getName() const override;
String getID() const override;
const Block & getTotals() override;
Block getHeader() override;
protected:
Block readImpl() override;
@ -34,11 +35,9 @@ protected:
private:
ExpressionActionsPtr expression;
ssize_t filter_column;
String filter_column_name;
bool is_first = true;
ConstantFilterDescription constant_filter_description;
bool have_constant_filter_description = false;
};
}

View File

@ -15,6 +15,18 @@ String FilterColumnsBlockInputStream::getID() const
return res.str();
}
Block FilterColumnsBlockInputStream::getHeader()
{
Block block = children.back()->getHeader();
Block filtered;
for (const auto & it : columns_to_save)
if (throw_if_column_not_found || block.has(it))
filtered.insert(std::move(block.getByName(it)));
return filtered;
}
Block FilterColumnsBlockInputStream::readImpl()
{
Block block = children.back()->read();

View File

@ -26,6 +26,8 @@ public:
String getID() const override;
Block getHeader() override;
protected:
Block readImpl() override;

View File

@ -1,74 +0,0 @@
#pragma once
#include <DataStreams/QueueBlockIOStream.h>
namespace DB
{
/** Allows you to make several sources from one.
* Used for single-pass execution of several queries at once.
*
* Multiple received sources should be read from different threads!
* Uses O(1) RAM (does not buffer all data).
* For this, readings from different sources are synchronized:
* reading of next block is blocked until all sources have read the current block.
*/
class ForkBlockInputStreams : private boost::noncopyable
{
public:
ForkBlockInputStreams(const BlockInputStreamPtr & source_) : source(source_) {}
/// Create a source. Call the function as many times as many forked sources you need.
BlockInputStreamPtr createInput()
{
destinations.emplace_back(std::make_shared<QueueBlockIOStream>(1));
return destinations.back();
}
/// Before you can read from the sources you have to "run" this construct.
void run()
{
while (1)
{
if (destinations.empty())
return;
Block block = source->read();
for (Destinations::iterator it = destinations.begin(); it != destinations.end();)
{
if ((*it)->isCancelled())
{
destinations.erase(it++);
}
else
{
(*it)->write(block);
++it;
}
}
if (!block)
return;
}
}
private:
/// From where to read.
BlockInputStreamPtr source;
/** Forked sources.
* Made on the basis of a queue of small length.
* A block from `source` is put in each queue.
*/
using Destination = std::shared_ptr<QueueBlockIOStream>;
using Destinations = std::list<Destination>;
Destinations destinations;
};
using ForkPtr = std::shared_ptr<ForkBlockInputStreams>;
using Forks = std::vector<ForkPtr>;
}

View File

@ -158,6 +158,8 @@ public:
current_pattern->function->destroy(place_for_aggregate_state.data());
}
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -48,6 +48,13 @@ class IBlockInputStream : private boost::noncopyable
public:
IBlockInputStream() {}
/** Get data structure of the stream in a form of "header" block (it is also called "sample block").
* Header block contains column names, data types, and constant columns.
* If the column is not constant (and not dummy), then the header block will contain nullptr instead of column.
* It is guaranteed that method "read" returns blocks of exactly that structure.
*/
virtual Block getHeader() = 0;
/** Read next block.
* If there are no more blocks, return an empty block (for which operator `bool` returns false).
* NOTE: Only one thread can read from one instance of IBlockInputStream simultaneously.

View File

@ -1,10 +1,20 @@
#include <Parsers/ASTInsertQuery.h>
#include <Interpreters/Context.h>
#include <IO/ConcatReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
#include <DataStreams/BlockIO.h>
#include <DataStreams/InputStreamFromASTInsertQuery.h>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
const ASTPtr & ast, ReadBuffer & input_buffer_tail_part, const BlockIO & streams, Context & context)
{
@ -19,19 +29,18 @@ InputStreamFromASTInsertQuery::InputStreamFromASTInsertQuery(
/// Data could be in parsed (ast_insert_query.data) and in not parsed yet (input_buffer_tail_part) part of query.
input_buffer_ast_part = std::make_unique<ReadBufferFromMemory>(
ast_insert_query->data, ast_insert_query->data ? ast_insert_query->end - ast_insert_query->data : 0);
input_buffer_ast_part = ReadBufferFromMemory(ast_insert_query->data, ast_insert_query->data ? ast_insert_query->end - ast_insert_query->data : 0);
ConcatReadBuffer::ReadBuffers buffers;
if (ast_insert_query->data)
buffers.push_back(input_buffer_ast_part.get());
buffers.push_back(&input_buffer_ast_part.value());
buffers.push_back(&input_buffer_tail_part);
/** NOTE Must not read from 'input_buffer_tail_part' before read all between 'ast_insert_query.data' and 'ast_insert_query.end'.
* - because 'query.data' could refer to memory piece, used as buffer for 'input_buffer_tail_part'.
*/
input_buffer_contacenated = std::make_unique<ConcatReadBuffer>(buffers);
input_buffer_contacenated = ConcatReadBuffer(buffers);
res_stream = context.getInputFormat(format, *input_buffer_contacenated, streams.out_sample, context.getSettings().max_insert_block_size);
}

View File

@ -1,20 +1,16 @@
#pragma once
#include <Parsers/ASTInsertQuery.h>
#include <Interpreters/Context.h>
#include <IO/ConcatReadBuffer.h>
#include <Parsers/IAST.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataStreams/BlockIO.h>
#include <cstddef>
#include <optional>
namespace DB
{
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
struct BlockIO;
class Context;
/** Prepares an input stream which produce data containing in INSERT query
* Head of inserting data could be stored in INSERT ast directly
@ -23,7 +19,6 @@ namespace ErrorCodes
class InputStreamFromASTInsertQuery : public IProfilingBlockInputStream
{
public:
InputStreamFromASTInsertQuery(const ASTPtr & ast, ReadBuffer & input_buffer_tail_part, const BlockIO & streams, Context & context);
Block readImpl() override { return res_stream->read(); }
@ -33,10 +28,11 @@ public:
String getName() const override { return "InputStreamFromASTInsertQuery"; }
String getID() const override { return "InputStreamFromASTInsertQuery(" + toString(std::intptr_t(this)) + ")"; }
private:
Block getHeader() override { return res_stream->getHeader(); }
std::unique_ptr<ReadBuffer> input_buffer_ast_part;
std::unique_ptr<ReadBuffer> input_buffer_contacenated;
private:
std::optional<ReadBuffer> input_buffer_ast_part;
std::optional<ReadBuffer> input_buffer_contacenated;
BlockInputStreamPtr res_stream;
};

View File

@ -34,55 +34,21 @@ public:
IProfilingBlockInputStream::cancel();
}
Block getHeader() override
{
init();
if (!input)
return {};
return input->getHeader();
}
protected:
Block readImpl() override
{
init();
if (!input)
{
input = generator();
if (!input)
return Block();
auto * p_input = dynamic_cast<IProfilingBlockInputStream *>(input.get());
if (p_input)
{
/// They could have been set before, but were not passed into the `input`.
if (progress_callback)
p_input->setProgressCallback(progress_callback);
if (process_list_elem)
p_input->setProcessListElement(process_list_elem);
}
input->readPrefix();
{
std::lock_guard<std::mutex> lock(cancel_mutex);
/** TODO Data race here. See IProfilingBlockInputStream::collectAndSendTotalRowsApprox.
Assume following pipeline:
RemoteBlockInputStream
AsynchronousBlockInputStream
LazyBlockInputStream
RemoteBlockInputStream calls AsynchronousBlockInputStream::readPrefix
and AsynchronousBlockInputStream spawns a thread and returns.
The separate thread will call LazyBlockInputStream::read
LazyBlockInputStream::read will add more children to itself
In the same moment, in main thread, RemoteBlockInputStream::read is called,
then IProfilingBlockInputStream::collectAndSendTotalRowsApprox is called
and iterates over set of children.
*/
children.push_back(input);
if (isCancelled() && p_input)
p_input->cancel();
}
}
return {};
return input->read();
}
@ -91,9 +57,61 @@ private:
const char * name = "Lazy";
Generator generator;
bool initialized = false;
BlockInputStreamPtr input;
std::mutex cancel_mutex;
void init()
{
if (initialized)
return;
input = generator();
initialized = true;
if (!input)
return;
auto * p_input = dynamic_cast<IProfilingBlockInputStream *>(input.get());
if (p_input)
{
/// They could have been set before, but were not passed into the `input`.
if (progress_callback)
p_input->setProgressCallback(progress_callback);
if (process_list_elem)
p_input->setProcessListElement(process_list_elem);
}
input->readPrefix();
{
std::lock_guard<std::mutex> lock(cancel_mutex);
/** TODO Data race here. See IProfilingBlockInputStream::collectAndSendTotalRowsApprox.
Assume following pipeline:
RemoteBlockInputStream
AsynchronousBlockInputStream
LazyBlockInputStream
RemoteBlockInputStream calls AsynchronousBlockInputStream::readPrefix
and AsynchronousBlockInputStream spawns a thread and returns.
The separate thread will call LazyBlockInputStream::read
LazyBlockInputStream::read will add more children to itself
In the same moment, in main thread, RemoteBlockInputStream::read is called,
then IProfilingBlockInputStream::collectAndSendTotalRowsApprox is called
and iterates over set of children.
*/
children.push_back(input);
if (isCancelled() && p_input)
p_input->cancel();
}
}
};
}

View File

@ -28,6 +28,8 @@ public:
return res.str();
}
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -1,4 +1,6 @@
#include <DataStreams/LimitByBlockInputStream.h>
#include <Common/SipHash.h>
namespace DB
{

View File

@ -3,9 +3,9 @@
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Common/HashTable/HashMap.h>
#include <Common/SipHash.h>
#include <Common/UInt128.h>
namespace DB
{
@ -22,6 +22,8 @@ public:
String getName() const override { return "LimitBy"; }
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -22,6 +22,18 @@ String MaterializingBlockInputStream::getID() const
return res.str();
}
Block MaterializingBlockInputStream::getHeader()
{
Block res = children.back()->getHeader();
/// Constant columns become non constant.
for (auto & elem : res)
if (!elem.column->isColumnConst())
elem.column = nullptr;
return res;
}
Block MaterializingBlockInputStream::readImpl()
{
return materializeBlock(children.back()->read());

View File

@ -13,6 +13,7 @@ public:
MaterializingBlockInputStream(const BlockInputStreamPtr & input);
String getName() const override;
String getID() const override;
Block getHeader() override;
protected:
Block readImpl() override;

View File

@ -39,6 +39,8 @@ public:
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;
@ -96,6 +98,8 @@ public:
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -6,6 +6,11 @@
namespace DB
{
Block MergingAggregatedBlockInputStream::getHeader()
{
return aggregator.getHeader(final);
}
Block MergingAggregatedBlockInputStream::readImpl()
{

View File

@ -29,6 +29,8 @@ public:
return res.str();
}
Block getHeader() override;
protected:
Block readImpl() override;

View File

@ -90,6 +90,12 @@ MergingAggregatedMemoryEfficientBlockInputStream::MergingAggregatedMemoryEfficie
}
Block MergingAggregatedMemoryEfficientBlockInputStream::getHeader()
{
return aggregator.getHeader(final);
}
String MergingAggregatedMemoryEfficientBlockInputStream::getID() const
{
std::stringstream res;

View File

@ -80,6 +80,8 @@ public:
*/
void cancel() override;
Block getHeader() override;
protected:
Block readImpl() override;

View File

@ -74,6 +74,8 @@ public:
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
struct RowRef
{

View File

@ -19,6 +19,7 @@ namespace ErrorCodes
extern const int INCORRECT_INDEX;
extern const int LOGICAL_ERROR;
extern const int CANNOT_READ_ALL_DATA;
extern const int NOT_IMPLEMENTED;
}
NativeBlockInputStream::NativeBlockInputStream(
@ -50,6 +51,19 @@ void NativeBlockInputStream::readData(const IDataType & type, IColumn & column,
}
Block NativeBlockInputStream::getHeader()
{
/// Note: we may read first block and stash it for further use just to get header.
if (!use_index)
throw Exception("Method getHeader for NativeBlockInputStream requires index", ErrorCodes::NOT_IMPLEMENTED);
Block res;
for (const auto & column : index_block_it->columns)
res.insert({ nullptr, DataTypeFactory::instance().get(column.type), column.name });
return res;
}
Block NativeBlockInputStream::readImpl()
{
Block res;

View File

@ -81,6 +81,8 @@ public:
static void readData(const IDataType & type, IColumn & column, ReadBuffer & istr, size_t rows, double avg_value_size_hint);
Block getHeader() override;
protected:
Block readImpl() override;

View File

@ -35,6 +35,8 @@ public:
return res.str();
}
Block getHeader() override { return {}; }
protected:
Block readImpl() override
{

View File

@ -6,14 +6,19 @@
namespace DB
{
/** Empty stream of blocks.
/** Empty stream of blocks of specified structure.
*/
class NullBlockInputStream : public IBlockInputStream
{
public:
Block read() override { return Block(); }
NullBlockInputStream(const Block & header) : header(header) {}
Block read() override { return {}; }
Block getHeader() override { return header; }
String getName() const override { return "Null"; }
private:
Block header;
};
}

View File

@ -17,6 +17,7 @@ namespace ErrorCodes
NullableAdapterBlockInputStream::NullableAdapterBlockInputStream(
const BlockInputStreamPtr & input,
const Block & in_sample_, const Block & out_sample_)
: header(out_sample_)
{
buildActions(in_sample_, out_sample_);
children.push_back(input);

View File

@ -25,6 +25,8 @@ public:
String getID() const override;
Block getHeader() override { return header; }
protected:
Block readImpl() override;
@ -52,6 +54,7 @@ private:
void buildActions(const Block & in_sample, const Block & out_sample);
private:
Block header;
Actions actions;
std::vector<std::optional<String>> rename;
bool must_transform = false;

View File

@ -16,6 +16,14 @@ public:
String getName() const override { return "One"; }
Block getHeader() override
{
Block res;
for (const auto & elem : block)
res.insert({ elem.column->isColumnConst() ? elem.column->cloneEmpty() : nullptr, elem.type, elem.name });
return res;
}
protected:
Block readImpl() override
{

View File

@ -20,6 +20,8 @@ public:
children.push_back(stream);
}
Block getHeader() override { return children.at(0)->getHeader(); }
private:
Block readImpl() override { return stream->read(); }

View File

@ -29,6 +29,12 @@ ParallelAggregatingBlockInputStream::ParallelAggregatingBlockInputStream(
}
Block ParallelAggregatingBlockInputStream::getHeader()
{
return aggregator.getHeader(final);
}
String ParallelAggregatingBlockInputStream::getID() const
{
std::stringstream res;

View File

@ -31,6 +31,8 @@ public:
void cancel() override;
Block getHeader() override;
protected:
/// Do nothing that preparation to execution of the query be done in parallel, in ParallelInputsProcessor.
void readPrefix() override

View File

@ -39,6 +39,8 @@ public:
bool isSortedOutput() const override { return true; }
const SortDescription & getSortDescription() const override { return description; }
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -1,69 +0,0 @@
#pragma once
#include <limits>
#include <Common/ConcurrentBoundedQueue.h>
#include <DataStreams/IProfilingBlockInputStream.h>
#include <DataStreams/IBlockOutputStream.h>
namespace DB
{
/** Is both an InputStream and an OutputStream.
* When writing, puts the blocks in the queue.
* When reading, it takes them out of the queue.
* A thread-safe queue is used.
* If the queue is empty, the read is blocked.
* If the queue is full, the write is blocked.
*
* Used to temporarily store the result somewhere, and later pass it further.
* Also used for synchronization, when you need to make several sources from one
* - for single-pass execution of several queries at once.
* It can also be used for parallelization: several threads put blocks in the queue, and one - takes out.
*/
class QueueBlockIOStream : public IProfilingBlockInputStream, public IBlockOutputStream
{
public:
QueueBlockIOStream(size_t queue_size_ = std::numeric_limits<int>::max())
: queue_size(queue_size_), queue(queue_size) {}
String getName() const override { return "QueueBlockIOStream"; }
String getID() const override
{
std::stringstream res;
res << this;
return res.str();
}
void write(const Block & block) override
{
queue.push(block);
}
void cancel() override
{
IProfilingBlockInputStream::cancel();
queue.clear();
}
protected:
Block readImpl() override
{
Block res;
queue.pop(res);
return res;
}
private:
size_t queue_size;
using Queue = ConcurrentBoundedQueue<Block>;
Queue queue;
};
}

View File

@ -148,7 +148,8 @@ void RemoteBlockInputStream::sendExternalTables()
multiplexed_connections->sendExternalTablesData(external_tables_data);
}
Block RemoteBlockInputStream::readImpl()
Block RemoteBlockInputStream::receiveBlock()
{
if (!sent_query)
{
@ -161,17 +162,14 @@ Block RemoteBlockInputStream::readImpl()
while (true)
{
if (isCancelled())
return Block();
return {};
Connection::Packet packet = multiplexed_connections->receivePacket();
switch (packet.type)
{
case Protocol::Server::Data:
/// If the block is not empty and is not a header block
if (packet.block && (packet.block.rows() > 0))
return packet.block;
break; /// If the block is empty - we will receive other packets before EndOfStream.
return packet.block;
case Protocol::Server::Exception:
got_exception_from_replica = true;
@ -182,7 +180,7 @@ Block RemoteBlockInputStream::readImpl()
if (!multiplexed_connections->hasActiveConnections())
{
finished = true;
return Block();
return {};
}
break;
@ -216,6 +214,42 @@ Block RemoteBlockInputStream::readImpl()
}
}
Block RemoteBlockInputStream::getHeader()
{
if (header)
return header;
Block res = receiveBlock();
if (res.rows() > 0)
throw Exception("Logical error: the header block must be sent before data", ErrorCodes::LOGICAL_ERROR);
header = res;
return header;
}
Block RemoteBlockInputStream::readImpl()
{
while (true)
{
Block res = receiveBlock();
if (finished)
return {};
/// If the block is empty - we will receive other packets before EndOfStream.
if (!res)
continue;
if (res.rows() > 0)
return res;
/// Block with zero rows is a header block. The data will be sent in a following packet.
header = res;
}
}
void RemoteBlockInputStream::readSuffixImpl()
{
/** If one of:

View File

@ -78,6 +78,8 @@ public:
return multiplexed_connections->getBlockExtraInfo();
}
Block getHeader() override;
protected:
/// Send all temporary tables to remote servers
void sendExternalTables();
@ -95,10 +97,14 @@ protected:
private:
void sendQuery();
Block receiveBlock();
/// If wasn't sent yet, send request to cancell all connections to replicas
void tryCancel(const char * reason);
private:
Block header;
std::function<std::unique_ptr<MultiplexedConnections>()> create_multiplexed_connections;
std::unique_ptr<MultiplexedConnections> multiplexed_connections;

View File

@ -34,6 +34,17 @@ public:
return res.str();
}
Block getHeader() override
{
Block res = children.back()->getHeader();
for (const auto & it : columns_to_remove)
if (res.has(it))
res.erase(it);
return res;
}
protected:
Block readImpl() override
{

View File

@ -41,6 +41,8 @@ public:
return res.str();
}
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
/// Can return 1 more records than max_block_size.
Block readImpl() override;

View File

@ -23,6 +23,8 @@ public:
return res.str();
}
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
Block readImpl() override;

View File

@ -37,6 +37,8 @@ public:
String getID() const override;
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
/// Can return 1 more records than max_block_size.
Block readImpl() override;

View File

@ -70,7 +70,7 @@ const Block & TotalsHavingBlockInputStream::getTotals()
addToTotals(current_totals, overflow_aggregates, nullptr);
}
totals = header.cloneWithColumns(std::move(current_totals));
totals = unfinalized_header.cloneWithColumns(std::move(current_totals));
finalize(totals);
}
@ -81,8 +81,20 @@ const Block & TotalsHavingBlockInputStream::getTotals()
}
Block TotalsHavingBlockInputStream::getHeader()
{
unfinalized_header = children.at(0)->getHeader();
auto res = unfinalized_header;
finalize(res);
return res;
}
Block TotalsHavingBlockInputStream::readImpl()
{
if (!unfinalized_header)
getHeader();
Block finalized;
Block block;
@ -90,9 +102,6 @@ Block TotalsHavingBlockInputStream::readImpl()
{
block = children[0]->read();
if (!header)
header = block.cloneEmpty();
/// Block with values not included in `max_rows_to_group_by`. We'll postpone it.
if (overflow_row && block && block.info.is_overflows)
{

View File

@ -30,6 +30,8 @@ public:
const Block & getTotals() override;
Block getHeader() override;
protected:
Block readImpl() override;
@ -42,7 +44,7 @@ private:
size_t passed_keys = 0;
size_t total_keys = 0;
Block header;
Block unfinalized_header;
/** Here are the values that did not pass max_rows_to_group_by.
* They are added or not added to the current_totals, depending on the totals_mode.

View File

@ -139,6 +139,8 @@ public:
return doGetBlockExtraInfo();
}
Block getHeader() override { return children.at(0)->getHeader(); }
protected:
void finalize()
{

View File

@ -1,63 +0,0 @@
#include <set>
#include <DataStreams/glueBlockInputStreams.h>
namespace DB
{
using IDsMap = std::map<String, BlockInputStreams>;
using ForksMap = std::map<String, ForkPtr>;
static void createIDsMap(BlockInputStreamPtr & node, IDsMap & ids_map)
{
ids_map[node->getID()].push_back(node);
BlockInputStreams & children = node->getChildren();
for (size_t i = 0, size = children.size(); i < size; ++i)
createIDsMap(children[i], ids_map);
}
static void glue(BlockInputStreamPtr & node, IDsMap & ids_map, ForksMap & forks_map)
{
String id = node->getID();
if (ids_map.end() != ids_map.find(id) && ids_map[id].size() > 1)
{
/// Insert a "fork" or use an existing one.
if (forks_map.end() == forks_map.find(id))
{
forks_map[id] = std::make_shared<ForkBlockInputStreams>(node);
std::cerr << "Forking at " << id << std::endl;
}
std::cerr << "Replacing node with fork end" << std::endl;
node = forks_map[id]->createInput();
}
else
{
BlockInputStreams & children = node->getChildren();
for (size_t i = 0, size = children.size(); i < size; ++i)
glue(children[i], ids_map, forks_map);
}
}
void glueBlockInputStreams(BlockInputStreams & inputs, Forks & forks)
{
IDsMap ids_map;
for (size_t i = 0, size = inputs.size(); i < size; ++i)
createIDsMap(inputs[i], ids_map);
ForksMap forks_map;
for (size_t i = 0, size = inputs.size(); i < size; ++i)
glue(inputs[i], ids_map, forks_map);
for (ForksMap::iterator it = forks_map.begin(); it != forks_map.end(); ++it)
forks.push_back(it->second);
}
}

View File

@ -1,17 +0,0 @@
#pragma once
#include <DataStreams/ForkBlockInputStreams.h>
namespace DB
{
/** If passed sources (query execution pipelines) have the same parts,
* then glues these parts, replacing them with one source and inserting "forks" (multipliers).
* This is used for single-pass execution of multiple queries.
*
* To execute a glued pipeline, all `inputs` and `forks` must be used in different threads.
*/
void glueBlockInputStreams(BlockInputStreams & inputs, Forks & forks);
}

View File

@ -28,9 +28,3 @@ target_link_libraries (union_stream2 dbms)
add_executable (collapsing_sorted_stream collapsing_sorted_stream.cpp ${SRCS})
target_link_libraries (collapsing_sorted_stream dbms)
add_executable (fork_streams fork_streams.cpp ${SRCS})
target_link_libraries (fork_streams dbms clickhouse_storages_system)
add_executable (glue_streams glue_streams.cpp ${SRCS})
target_link_libraries (glue_streams dbms)

View File

@ -1,110 +0,0 @@
#include <iostream>
#include <iomanip>
#include <thread>
#include <IO/WriteBufferFromOStream.h>
#include <Storages/System/StorageSystemNumbers.h>
#include <DataStreams/LimitBlockInputStream.h>
#include <DataStreams/ExpressionBlockInputStream.h>
#include <DataStreams/FilterBlockInputStream.h>
#include <DataStreams/TabSeparatedRowOutputStream.h>
#include <DataStreams/ForkBlockInputStreams.h>
#include <DataStreams/copyData.h>
#include <DataTypes/DataTypesNumber.h>
#include <Parsers/ParserSelectQuery.h>
#include <Parsers/parseQuery.h>
#include <Parsers/formatAST.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/Context.h>
void thread1(DB::BlockInputStreamPtr in, DB::BlockOutputStreamPtr out, DB::WriteBuffer & out_buf)
{
while (DB::Block block = in->read())
{
out->write(block);
out_buf.next();
}
}
void thread2(DB::BlockInputStreamPtr in, DB::BlockOutputStreamPtr out, DB::WriteBuffer & out_buf)
{
while (DB::Block block = in->read())
{
out->write(block);
out_buf.next();
}
}
int main(int, char **)
try
{
using namespace DB;
std::string input = "SELECT number, number % 10000000 == 1";
ParserSelectQuery parser;
ASTPtr ast = parseQuery(parser, input.data(), input.data() + input.size(), "");
formatAST(*ast, std::cerr);
std::cerr << std::endl;
Context context = Context::createGlobal();
ExpressionAnalyzer analyzer(ast, context, {}, {NameAndTypePair("number", std::make_shared<DataTypeUInt64>())});
ExpressionActionsChain chain;
analyzer.appendSelect(chain, false);
analyzer.appendProjectResult(chain);
chain.finalize();
ExpressionActionsPtr expression = chain.getLastActions();
StoragePtr table = StorageSystemNumbers::create("numbers", false);
Names column_names;
column_names.push_back("number");
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0];
ForkBlockInputStreams fork(in);
BlockInputStreamPtr in1 = fork.createInput();
BlockInputStreamPtr in2 = fork.createInput();
in1 = std::make_shared<FilterBlockInputStream>(in1, expression, 1);
in1 = std::make_shared<LimitBlockInputStream>(in1, 10, 0);
in2 = std::make_shared<FilterBlockInputStream>(in2, expression, 1);
in2 = std::make_shared<LimitBlockInputStream>(in2, 20, 5);
Block out_sample = expression->getSampleBlock();
WriteBufferFromOStream ob1(std::cout);
WriteBufferFromOStream ob2(std::cerr);
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", ob1, out_sample);
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", ob2, out_sample);
std::thread thr1(std::bind(thread1, in1, out1, std::ref(ob1)));
std::thread thr2(std::bind(thread2, in2, out2, std::ref(ob2)));
fork.run();
thr1.join();
thr2.join();
return 0;
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -1,98 +0,0 @@
#include <iostream>
#include <iomanip>
#include <common/ThreadPool.h>
#include <IO/WriteBufferFromFileDescriptor.h>
#include <Interpreters/Context.h>
#include <Interpreters/loadMetadata.h>
#include <Interpreters/executeQuery.h>
#include <DataStreams/glueBlockInputStreams.h>
using namespace DB;
void inputThread(const BlockInputStreamPtr & in, BlockOutputStreamPtr out, WriteBuffer & wb, std::mutex & mutex)
{
while (Block block = in->read())
{
std::lock_guard<std::mutex> lock(mutex);
out->write(block);
wb.next();
}
}
void forkThread(ForkPtr fork)
{
fork->run();
}
int main(int, char **)
try
{
Context context = Context::createGlobal();
context.setGlobalContext(context);
context.setPath("./");
loadMetadata(context);
context.setCurrentDatabase("default");
context.setSetting("max_threads", UInt64(1));
BlockIO io1 = executeQuery(
"SELECT SearchPhrase, count()"
" FROM hits"
" WHERE SearchPhrase != ''"
" GROUP BY SearchPhrase"
" ORDER BY count() DESC"
" LIMIT 10",
context, false, QueryProcessingStage::Complete);
BlockIO io2 = executeQuery(
"SELECT count()"
" FROM hits"
" WHERE SearchPhrase != ''",
context, false, QueryProcessingStage::Complete);
WriteBufferFromFileDescriptor wb(STDOUT_FILENO);
BlockOutputStreamPtr out1 = context.getOutputFormat("TabSeparated", wb, io1.in_sample);
BlockOutputStreamPtr out2 = context.getOutputFormat("TabSeparated", wb, io2.in_sample);
BlockInputStreams inputs;
inputs.push_back(io1.in);
inputs.push_back(io2.in);
for (size_t i = 0; i < inputs.size(); ++i)
std::cerr << inputs[i]->getID() << std::endl;
Forks forks;
glueBlockInputStreams(inputs, forks);
std::cerr << forks.size() << std::endl;
std::mutex mutex;
ThreadPool pool(inputs.size() + forks.size());
pool.schedule(std::bind(inputThread, inputs[0], out1, std::ref(wb), std::ref(mutex)));
pool.schedule(std::bind(inputThread, inputs[1], out2, std::ref(wb), std::ref(mutex)));
for (size_t i = 0; i < forks.size(); ++i)
pool.schedule(std::bind(forkThread, forks[i]));
pool.wait();
return 0;
}
catch (const DB::Exception & e)
{
std::cerr << e.what() << ", " << e.displayText() << std::endl;
throw;
}

View File

@ -138,7 +138,8 @@ void DatabaseMemory::shutdown()
tables_snapshot = tables;
}
for (const auto & kv: tables_snapshot) {
for (const auto & kv: tables_snapshot)
{
kv.second->shutdown();
}

View File

@ -427,7 +427,8 @@ void DatabaseOrdinary::shutdown()
tables_snapshot = tables;
}
for (const auto & kv: tables_snapshot) {
for (const auto & kv: tables_snapshot)
{
kv.second->shutdown();
}

View File

@ -474,14 +474,16 @@ private:
if (!outdated_keys.empty())
{
std::vector<size_t> required_rows(outdated_keys.size());
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair) {
std::transform(std::begin(outdated_keys), std::end(outdated_keys), std::begin(required_rows), [](auto & pair)
{
return pair.second.front();
});
update(key_columns,
keys_array,
required_rows,
[&](const StringRef key, const size_t cell_idx) {
[&](const StringRef key, const size_t cell_idx)
{
const StringRef attribute_value = attribute_array[cell_idx];
/// We must copy key and value to own memory, because it may be replaced with another
@ -492,7 +494,8 @@ private:
map[copied_key] = copied_value;
total_length += (attribute_value.size + 1) * outdated_keys[key].size();
},
[&](const StringRef key, const size_t) {
[&](const StringRef key, const size_t)
{
for (const auto row : outdated_keys[key])
total_length += get_default(row).size + 1;
});

View File

@ -244,12 +244,14 @@ void ComplexKeyHashedDictionary::loadData()
element_count += rows;
const auto key_column_ptrs = ext::map<Columns>(ext::range(0, keys_size),
[&] (const size_t attribute_idx) {
[&] (const size_t attribute_idx)
{
return block.safeGetByPosition(attribute_idx).column;
});
const auto attribute_column_ptrs = ext::map<Columns>(ext::range(0, attributes_size),
[&] (const size_t attribute_idx) {
[&] (const size_t attribute_idx)
{
return block.safeGetByPosition(keys_size + attribute_idx).column;
});

View File

@ -31,7 +31,7 @@ public:
const std::vector<StringRef> & keys, const Names & column_names);
using GetColumnsFunction =
std::function<ColumnsWithTypeAndName(const Columns &, const std::vector<DictionaryAttribute>& attributes)>;
std::function<ColumnsWithTypeAndName(const Columns &, const std::vector<DictionaryAttribute> & attributes)>;
// Used to separate key columns format for storage and view.
// Calls get_key_columns_function to get key column for dictionary get fuction call
// and get_view_columns_function to get key representation.
@ -41,7 +41,8 @@ public:
GetColumnsFunction && get_key_columns_function,
GetColumnsFunction && get_view_columns_function);
String getName() const override {
String getName() const override
{
return "DictionaryBlockInputStream";
}
@ -96,7 +97,7 @@ private:
ColumnPtr getColumnFromStringAttribute(Getter getter, const PaddedPODArray<Key> & ids,
const Columns & keys, const DataTypes & data_types,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const;
ColumnPtr getColumnFromIds(const PaddedPODArray<Key>& ids) const;
ColumnPtr getColumnFromIds(const PaddedPODArray<Key> & ids) const;
void fillKeyColumns(const std::vector<StringRef> & keys, size_t start, size_t size,
const DictionaryStructure & dictionary_structure, ColumnsWithTypeAndName & columns) const;
@ -107,7 +108,7 @@ private:
ColumnsWithTypeAndName key_columns;
Poco::Logger * logger;
Block (DictionaryBlockInputStream<DictionaryType, Key>::*fillBlockFunction)(
const PaddedPODArray<Key>& ids, const Columns& keys,
const PaddedPODArray<Key> & ids, const Columns& keys,
const DataTypes & types, ColumnsWithTypeAndName && view) const;
Columns data_columns;
@ -240,7 +241,7 @@ void DictionaryBlockInputStream<DictionaryType, Key>::callGetter(
template <typename DictionaryType, typename Key>
template <template <typename> class Getter, typename StringGetter>
Block DictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key>& ids, const Columns& keys, const DataTypes & types, ColumnsWithTypeAndName && view) const
const PaddedPODArray<Key> & ids, const Columns & keys, const DataTypes & types, ColumnsWithTypeAndName && view) const
{
std::unordered_set<std::string> names(column_names.begin(), column_names.end());
@ -349,7 +350,7 @@ ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromStringAt
}
template <typename DictionaryType, typename Key>
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(const PaddedPODArray<Key>& ids) const
ColumnPtr DictionaryBlockInputStream<DictionaryType, Key>::getColumnFromIds(const PaddedPODArray<Key> & ids) const
{
auto column_vector = ColumnVector<UInt64>::create();
column_vector->getData().reserve(ids.size());

View File

@ -26,4 +26,15 @@ Block DictionaryBlockInputStreamBase::readImpl()
return block;
}
Block DictionaryBlockInputStreamBase::getHeader()
{
Block block = getBlock(0, 0);
/// Columns are non constant. Reset them for header block.
for (auto & elem : block)
elem.column = nullptr;
return block;
}
}

View File

@ -15,13 +15,14 @@ protected:
virtual Block getBlock(size_t start, size_t length) const = 0;
Block getHeader() override;
private:
const size_t rows_count;
const size_t max_block_size;
size_t next_row;
size_t next_row = 0;
Block readImpl() override;
void readPrefixImpl() override { next_row = 0; }
};
}

View File

@ -47,7 +47,8 @@ private:
struct language_alias { const char * const name; const Language lang; };
static const language_alias * getLanguageAliases()
{
static constexpr const language_alias language_aliases[] {
static constexpr const language_alias language_aliases[]
{
{ "ru", Language::RU },
{ "en", Language::EN },
{ "ua", Language::UA },

View File

@ -101,6 +101,8 @@ public:
}
}
Block getHeader() override { return stream->getHeader(); };
private:
Block readImpl() override { return stream->read(); }

View File

@ -539,11 +539,8 @@ PaddedPODArray<FlatDictionary::Key> FlatDictionary::getIds() const
PaddedPODArray<Key> ids;
for (auto idx : ext::range(0, ids_count))
{
if (loaded_ids[idx]) {
if (loaded_ids[idx])
ids.push_back(idx);
}
}
return ids;
}

View File

@ -494,9 +494,9 @@ PaddedPODArray<HashedDictionary::Key> HashedDictionary::getIds(const Attribute &
PaddedPODArray<Key> ids;
ids.reserve(attr.size());
for (const auto & value : attr) {
for (const auto & value : attr)
ids.push_back(value.first);
}
return ids;
}

View File

@ -28,7 +28,7 @@ MongoDBBlockInputStream::MongoDBBlockInputStream(
std::unique_ptr<Poco::MongoDB::Cursor> cursor_,
const Block & sample_block,
const size_t max_block_size)
: connection(connection_), cursor{std::move(cursor_)}, max_block_size{max_block_size}
: connection(connection_), cursor{std::move(cursor_)}, max_block_size{max_block_size}
{
description.init(sample_block);
}

View File

@ -34,6 +34,8 @@ public:
String getID() const override;
Block getHeader() override { return description.sample_block; };
private:
Block readImpl() override;

View File

@ -23,6 +23,8 @@ public:
String getID() const override;
Block getHeader() override { return description.sample_block; };
private:
Block readImpl() override;

View File

@ -29,6 +29,8 @@ public:
String getID() const override;
Block getHeader() override { return description.sample_block; };
private:
Block readImpl() override;

View File

@ -28,7 +28,8 @@ public:
DictionatyPtr dictionary, size_t max_block_size, const Names & column_names, PaddedPODArray<Key> && ids,
PaddedPODArray<UInt16> && start_dates, PaddedPODArray<UInt16> && end_dates);
String getName() const override {
String getName() const override
{
return "RangeDictionaryBlockInputStream";
}
@ -42,16 +43,16 @@ private:
template <typename AttributeType>
ColumnPtr getColumnFromAttribute(DictionaryGetter<AttributeType> getter,
const PaddedPODArray<Key>& ids, const PaddedPODArray<UInt16> & dates,
const PaddedPODArray<Key> & ids, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const;
ColumnPtr getColumnFromAttributeString(const PaddedPODArray<Key>& ids, const PaddedPODArray<UInt16> & dates,
ColumnPtr getColumnFromAttributeString(const PaddedPODArray<Key> & ids, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const;
template <typename T>
ColumnPtr getColumnFromPODArray(const PaddedPODArray<T>& array) const;
ColumnPtr getColumnFromPODArray(const PaddedPODArray<T> & array) const;
template <typename T>
void addSpecialColumn(
const std::optional<DictionarySpecialAttribute>& attribute, DataTypePtr type,
const std::optional<DictionarySpecialAttribute> & attribute, DataTypePtr type,
const std::string & default_name, const std::unordered_set<std::string> & column_names,
const PaddedPODArray<T> & values, ColumnsWithTypeAndName& columns) const;
@ -102,7 +103,7 @@ Block RangeDictionaryBlockInputStream<DictionaryType, Key>::getBlock(size_t star
template <typename DictionaryType, typename Key>
template <typename AttributeType>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttribute(
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key>& ids,
DictionaryGetter<AttributeType> getter, const PaddedPODArray<Key> & ids,
const PaddedPODArray<UInt16> & dates, const DictionaryAttribute& attribute, const DictionaryType& dictionary) const
{
auto column_vector = ColumnVector<AttributeType>::create(ids.size());
@ -112,7 +113,7 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
template <typename DictionaryType, typename Key>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAttributeString(
const PaddedPODArray<Key>& ids, const PaddedPODArray<UInt16> & dates,
const PaddedPODArray<Key> & ids, const PaddedPODArray<UInt16> & dates,
const DictionaryAttribute& attribute, const DictionaryType& dictionary) const
{
auto column_string = ColumnString::create();
@ -122,7 +123,7 @@ ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromAtt
template <typename DictionaryType, typename Key>
template <typename T>
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPODArray(const PaddedPODArray<T>& array) const
ColumnPtr RangeDictionaryBlockInputStream<DictionaryType, Key>::getColumnFromPODArray(const PaddedPODArray<T> & array) const
{
auto column_vector = ColumnVector<T>::create();
column_vector->getData().reserve(array.size());
@ -140,12 +141,11 @@ void RangeDictionaryBlockInputStream<DictionaryType, Key>::addSpecialColumn(
const PaddedPODArray<T> & values, ColumnsWithTypeAndName & columns) const
{
std::string name = default_name;
if (attribute) {
if (attribute)
name = attribute->name;
}
if (column_names.find(name) != column_names.end()) {
if (column_names.find(name) != column_names.end())
columns.emplace_back(getColumnFromPODArray(values), type, name);
}
}
template <typename DictionaryType, typename Key>
@ -167,7 +167,7 @@ PaddedPODArray<UInt16> RangeDictionaryBlockInputStream<DictionaryType, Key>::mak
template <typename DictionaryType, typename Key>
Block RangeDictionaryBlockInputStream<DictionaryType, Key>::fillBlock(
const PaddedPODArray<Key>& ids,
const PaddedPODArray<Key> & ids,
const PaddedPODArray<UInt16> & start_dates, const PaddedPODArray<UInt16> & end_dates) const
{
ColumnsWithTypeAndName columns;

View File

@ -323,7 +323,8 @@ void RangeHashedDictionary::setAttributeValue(Attribute & attribute, const Key i
auto & values = it->second;
const auto insert_it = std::lower_bound(std::begin(values), std::end(values), range,
[] (const Value<StringRef> & lhs, const Range & range) {
[] (const Value<StringRef> & lhs, const Range & range)
{
return lhs.range < range;
});

View File

@ -608,7 +608,8 @@ Columns TrieDictionary::getKeyColumns() const
auto mask_column = ColumnVector<UInt8>::create();
#if defined(__SIZEOF_INT128__)
auto getter = [& ip_column, & mask_column](__uint128_t ip, size_t mask) {
auto getter = [& ip_column, & mask_column](__uint128_t ip, size_t mask)
{
UInt64 * ip_array = reinterpret_cast<UInt64 *>(&ip);
ip_array[0] = Poco::ByteOrder::fromNetwork(ip_array[0]);
ip_array[1] = Poco::ByteOrder::fromNetwork(ip_array[1]);

View File

@ -47,7 +47,7 @@ enum ClusterOperation
/// functions (eg. Hamming distance) using Clickhouse lambdas.
// Centroids array has the same size as number of clusters.
size_t find_centroid(Float64 x, std::vector<Float64>& centroids)
size_t find_centroid(Float64 x, std::vector<Float64> & centroids)
{
// Centroids array has to have at least one element, and if it has only one element,
// it is also the result of this Function.

View File

@ -181,7 +181,8 @@ public:
private:
struct Ellipse {
struct Ellipse
{
Float64 x;
Float64 y;
Float64 a;

View File

@ -835,7 +835,8 @@ struct ImplMetroHash64
static auto Hash128to64(const uint128_t & x) { return CityHash_v1_0_2::Hash128to64(x); }
static auto Hash64(const char * s, const size_t len)
{
union {
union
{
UInt64 u64;
UInt8 u8[sizeof(u64)];
};

View File

@ -1396,7 +1396,8 @@ public:
AggregateFunctionPtr aggregate_function_ptr = column_with_states->getAggregateFunction();
const IAggregateFunction & agg_func = *aggregate_function_ptr;
auto deleter = [&agg_func](char * ptr) {
auto deleter = [&agg_func](char * ptr)
{
agg_func.destroy(ptr);
free(ptr);
};
@ -1529,7 +1530,8 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
DataTypePtr res;
dispatchForSourceType(*arguments[0], [&](auto field_type_tag) {
dispatchForSourceType(*arguments[0], [&](auto field_type_tag)
{
res = std::make_shared<DataTypeNumber<DstFieldType<decltype(field_type_tag)>>>();
});

View File

@ -11,8 +11,8 @@ namespace ProfileEvents
}
namespace DB {
namespace DB
{
namespace Regexps
{

View File

@ -35,9 +35,8 @@ ReadBufferFromFile::ReadBufferFromFile(
#ifdef __APPLE__
bool o_direct = (flags != -1) && (flags & O_DIRECT);
if (o_direct) {
if (o_direct)
flags = flags & ~O_DIRECT;
}
#endif
fd = open(file_name.c_str(), flags == -1 ? O_RDONLY : flags);

View File

@ -37,9 +37,8 @@ WriteBufferFromFile::WriteBufferFromFile(
#ifdef __APPLE__
bool o_direct = (flags != -1) && (flags & O_DIRECT);
if (o_direct) {
if (o_direct)
flags = flags & ~O_DIRECT;
}
#endif
fd = open(file_name.c_str(), flags == -1 ? O_WRONLY | O_TRUNC | O_CREAT : flags, mode);

Some files were not shown because too many files have changed in this diff Show More