Merge pull request #4914 from yandex/processors

Processors
This commit is contained in:
Nikolai Kochetov 2019-07-11 19:22:02 +03:00 committed by GitHub
commit e9f3d4d6b8
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
195 changed files with 19273 additions and 172 deletions

View File

@ -114,6 +114,13 @@ add_headers_and_sources(dbms src/Storages/Distributed)
add_headers_and_sources(dbms src/Storages/MergeTree)
add_headers_and_sources(dbms src/Client)
add_headers_and_sources(dbms src/Formats)
add_headers_and_sources(dbms src/Processors)
add_headers_and_sources(dbms src/Processors/Executors)
add_headers_and_sources(dbms src/Processors/Formats)
add_headers_and_sources(dbms src/Processors/Formats/Impl)
add_headers_and_sources(dbms src/Processors/Transforms)
add_headers_and_sources(dbms src/Processors/Sources)
add_headers_only(dbms src/Server)
if(USE_RDKAFKA)
add_headers_and_sources(dbms src/Storages/Kafka)

View File

@ -29,6 +29,8 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <Compression/CompressionFactory.h>
#include <Processors/Formats/LazyOutputFormat.h>
#include "TCPHandler.h"
@ -207,6 +209,8 @@ void TCPHandler::runImpl()
/// Does the request require receive data from client?
if (state.need_receive_data_for_insert)
processInsertQuery(global_settings);
else if (state.io.pipeline.initialized())
processOrdinaryQueryWithProcessors(query_context->getSettingsRef().max_threads);
else
processOrdinaryQuery();
@ -447,9 +451,9 @@ void TCPHandler::processOrdinaryQuery()
*/
if (!block && !isQueryCancelled())
{
sendTotals();
sendExtremes();
sendProfileInfo();
sendTotals(state.io.in->getTotals());
sendExtremes(state.io.in->getExtremes());
sendProfileInfo(state.io.in->getProfileInfo());
sendProgress();
sendLogs();
}
@ -465,6 +469,129 @@ void TCPHandler::processOrdinaryQuery()
state.io.onFinish();
}
void TCPHandler::processOrdinaryQueryWithProcessors(size_t num_threads)
{
auto & pipeline = state.io.pipeline;
/// Send header-block, to allow client to prepare output format for data to send.
{
auto & header = pipeline.getHeader();
if (header)
sendData(header);
}
auto lazy_format = std::make_shared<LazyOutputFormat>(pipeline.getHeader());
pipeline.setOutput(lazy_format);
{
auto thread_group = CurrentThread::getGroup();
ThreadPool pool(1);
auto executor = pipeline.execute();
std::atomic_bool exception = false;
pool.schedule([&]()
{
/// ThreadStatus thread_status;
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
CurrentMetrics::Increment query_thread_metric_increment{CurrentMetrics::QueryThread};
setThreadName("QueryPipelineEx");
try
{
executor->execute(num_threads);
}
catch (...)
{
exception = true;
throw;
}
});
/// Wait in case of exception. Delete pipeline to release memory.
SCOPE_EXIT(
/// Clear queue in case if somebody is waiting lazy_format to push.
lazy_format->finish();
lazy_format->clearQueue();
pool.wait();
pipeline = QueryPipeline()
);
while (true)
{
Block block;
while (true)
{
if (isQueryCancelled())
{
/// A packet was received requesting to stop execution of the request.
executor->cancel();
break;
}
else
{
if (after_send_progress.elapsed() / 1000 >= query_context->getSettingsRef().interactive_delay)
{
/// Some time passed and there is a progress.
after_send_progress.restart();
sendProgress();
}
sendLogs();
if ((block = lazy_format->getBlock(query_context->getSettingsRef().interactive_delay / 1000)))
break;
if (lazy_format->isFinished())
break;
if (exception)
{
pool.wait();
break;
}
}
}
/** If data has run out, we will send the profiling data and total values to
* the last zero block to be able to use
* this information in the suffix output of stream.
* If the request was interrupted, then `sendTotals` and other methods could not be called,
* because we have not read all the data yet,
* and there could be ongoing calculations in other threads at the same time.
*/
if (!block && !isQueryCancelled())
{
pool.wait();
pipeline.finalize();
sendTotals(lazy_format->getTotals());
sendExtremes(lazy_format->getExtremes());
sendProfileInfo(lazy_format->getProfileInfo());
sendProgress();
sendLogs();
}
sendData(block);
if (!block)
break;
}
}
state.io.onFinish();
}
void TCPHandler::processTablesStatusRequest()
{
@ -495,18 +622,16 @@ void TCPHandler::processTablesStatusRequest()
}
void TCPHandler::sendProfileInfo()
void TCPHandler::sendProfileInfo(const BlockStreamProfileInfo & info)
{
writeVarUInt(Protocol::Server::ProfileInfo, *out);
state.io.in->getProfileInfo().write(*out);
info.write(*out);
out->next();
}
void TCPHandler::sendTotals()
void TCPHandler::sendTotals(const Block & totals)
{
const Block & totals = state.io.in->getTotals();
if (totals)
{
initBlockOutput(totals);
@ -521,10 +646,8 @@ void TCPHandler::sendTotals()
}
void TCPHandler::sendExtremes()
void TCPHandler::sendExtremes(const Block & extremes)
{
Block extremes = state.io.in->getExtremes();
if (extremes)
{
initBlockOutput(extremes);

View File

@ -145,6 +145,8 @@ private:
/// Process a request that does not require the receiving of data blocks from the client
void processOrdinaryQuery();
void processOrdinaryQueryWithProcessors(size_t num_threads);
void processTablesStatusRequest();
void sendHello();
@ -155,9 +157,9 @@ private:
void sendProgress();
void sendLogs();
void sendEndOfStream();
void sendProfileInfo();
void sendTotals();
void sendExtremes();
void sendProfileInfo(const BlockStreamProfileInfo & info);
void sendTotals(const Block & totals);
void sendExtremes(const Block & extremes);
/// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled.
void initBlockInput();

View File

@ -12,5 +12,6 @@ add_subdirectory (Interpreters)
add_subdirectory (AggregateFunctions)
add_subdirectory (Client)
add_subdirectory (TableFunctions)
add_subdirectory (Processors)
add_subdirectory (Formats)
add_subdirectory (Compression)

View File

@ -0,0 +1,47 @@
#pragma once
#include <cstdint>
#include <mutex>
#include <condition_variable>
/** Allow to subscribe for multiple events and wait for them one by one in arbitrary order.
*/
class EventCounter
{
private:
size_t events_happened = 0;
size_t events_waited = 0;
mutable std::mutex mutex;
std::condition_variable condvar;
public:
void notify()
{
{
std::lock_guard lock(mutex);
++events_happened;
}
condvar.notify_all();
}
void wait()
{
std::unique_lock lock(mutex);
condvar.wait(lock, [&]{ return events_happened > events_waited; });
++events_waited;
}
template <typename Duration>
bool waitFor(Duration && duration)
{
std::unique_lock lock(mutex);
if (condvar.wait(lock, std::forward<Duration>(duration), [&]{ return events_happened > events_waited; }))
{
++events_waited;
return true;
}
return false;
}
};

View File

@ -189,3 +189,16 @@ private:
Timestamp stop_ts;
bool is_running = false;
};
template <typename TStopwatch>
class StopwatchGuard : public TStopwatch
{
public:
explicit StopwatchGuard(UInt64 & elapsed_ns_) : elapsed_ns(elapsed_ns_) {}
~StopwatchGuard() { elapsed_ns += TStopwatch::elapsedNanoseconds(); }
private:
UInt64 & elapsed_ns;
};

View File

@ -370,6 +370,11 @@ Block Block::cloneWithColumns(const Columns & columns) const
Block res;
size_t num_columns = data.size();
if (num_columns != columns.size())
throw Exception("Cannot clone block with columns because block has " + toString(num_columns) + " columns, "
"but " + toString(columns.size()) + " columns given.", ErrorCodes::LOGICAL_ERROR);
for (size_t i = 0; i < num_columns; ++i)
res.insert({ columns[i], data[i].type, data[i].name });

View File

@ -326,8 +326,10 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, external_table_functions_use_nulls, true, "If it is set to true, external table functions will implicitly use Nullable type if needed. Otherwise NULLs will be substituted with default values. Currently supported only for 'mysql' table function.") \
M(SettingBool, allow_experimental_data_skipping_indices, false, "If it is set to true, data skipping indices can be used in CREATE TABLE/ALTER TABLE queries.") \
\
M(SettingBool, allow_hyperscan, 1, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \
M(SettingBool, allow_simdjson, 1, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \
M(SettingBool, experimental_use_processors, true, "Use processors pipeline.") \
\
M(SettingBool, allow_hyperscan, true, "Allow functions that use Hyperscan library. Disable to avoid potentially long compilation times and excessive resource usage.") \
M(SettingBool, allow_simdjson, true, "Allow using simdjson library in 'JSON*' functions if AVX2 instructions are available. If disabled rapidjson will be used.") \
\
M(SettingUInt64, max_partitions_per_insert_block, 100, "Limit maximum number of partitions in single INSERTed block. Zero means unlimited. Throw exception if the block contains too many partitions. This setting is a safety threshold, because using large number of partitions is a common misconception.")

View File

@ -3,6 +3,7 @@
#include <Common/typeid_cast.h>
#include <Core/SortDescription.h>
#include <Core/Block.h>
#include <Core/ColumnNumbers.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnString.h>
@ -47,26 +48,44 @@ struct SortCursorImpl
reset(block);
}
SortCursorImpl(const Columns & columns, const SortDescription & desc_, size_t order_ = 0)
: desc(desc_), sort_columns_size(desc.size()), order(order_), need_collation(desc.size())
{
for (auto & column_desc : desc)
{
if (!column_desc.column_name.empty())
throw Exception("SortDesctiption should contain column position if SortCursor was used without header.",
ErrorCodes::LOGICAL_ERROR);
}
reset(columns, {});
}
bool empty() const { return rows == 0; }
/// Set the cursor to the beginning of the new block.
void reset(const Block & block)
{
reset(block.getColumns(), block);
}
/// Set the cursor to the beginning of the new block.
void reset(const Columns & columns, const Block & block)
{
all_columns.clear();
sort_columns.clear();
size_t num_columns = block.columns();
size_t num_columns = columns.size();
for (size_t j = 0; j < num_columns; ++j)
all_columns.push_back(block.safeGetByPosition(j).column.get());
all_columns.push_back(columns[j].get());
for (size_t j = 0, size = desc.size(); j < size; ++j)
{
size_t column_number = !desc[j].column_name.empty()
? block.getPositionByName(desc[j].column_name)
: desc[j].column_number;
sort_columns.push_back(block.safeGetByPosition(column_number).column.get());
auto & column_desc = desc[j];
size_t column_number = !column_desc.column_name.empty()
? block.getPositionByName(column_desc.column_name)
: column_desc.column_number;
sort_columns.push_back(columns[column_number].get());
need_collation[j] = desc[j].collator != nullptr && typeid_cast<const ColumnString *>(sort_columns.back()); /// TODO Nullable(String)
has_collation |= need_collation[j];

View File

@ -4,6 +4,8 @@
#include <functional>
#include <Processors/QueryPipeline.h>
namespace DB
{
@ -25,6 +27,8 @@ struct BlockIO
BlockOutputStreamPtr out;
BlockInputStreamPtr in;
QueryPipeline pipeline;
/// Callbacks for query logging could be set here.
std::function<void(IBlockInputStream *, IBlockOutputStream *)> finish_callback;
std::function<void()> exception_callback;
@ -54,6 +58,7 @@ struct BlockIO
process_list_entry = rhs.process_list_entry;
in = rhs.in;
out = rhs.out;
pipeline = rhs.pipeline;
finish_callback = rhs.finish_callback;
exception_callback = rhs.exception_callback;

View File

@ -71,6 +71,9 @@ void BlockStreamProfileInfo::update(Block & block)
void BlockStreamProfileInfo::collectInfosForStreamsWithName(const char * name, BlockStreamProfileInfos & res) const
{
if (!parent)
return;
if (parent->getName() == name)
{
res.push_back(this);

View File

@ -50,6 +50,13 @@ struct BlockStreamProfileInfo
/// If skip_block_size_info if true, then rows, bytes and block fields are ignored.
void setFrom(const BlockStreamProfileInfo & rhs, bool skip_block_size_info);
/// Only for Processors.
void setRowsBeforeLimit(size_t rows_before_limit_)
{
applied_limit = true;
rows_before_limit = rows_before_limit_;
}
private:
void calculateRowsBeforeLimit() const;

View File

@ -23,7 +23,7 @@ Block LimitBlockInputStream::readImpl()
Block res;
UInt64 rows = 0;
/// pos - how many lines were read, including the last read block
/// pos - how many rows were read, including the last read block
if (pos >= offset + limit)
{
@ -46,7 +46,7 @@ Block LimitBlockInputStream::readImpl()
pos += rows;
} while (pos <= offset);
/// give away the whole block
/// return the whole block
if (pos >= offset + rows && pos <= offset + limit)
return res;
@ -61,7 +61,7 @@ Block LimitBlockInputStream::readImpl()
static_cast<Int64>(limit) + static_cast<Int64>(offset) - static_cast<Int64>(pos) + static_cast<Int64>(rows)));
for (size_t i = 0; i < res.columns(); ++i)
res.safeGetByPosition(i).column = res.safeGetByPosition(i).column->cut(start, length);
res.getByPosition(i).column = res.getByPosition(i).column->cut(start, length);
// TODO: we should provide feedback to child-block, so it will know how many rows are actually consumed.
// It's crucial for streaming engines like Kafka.

View File

@ -9,7 +9,7 @@ void removeConstantsFromBlock(Block & block)
size_t i = 0;
while (i < columns)
{
if (isColumnConst(*block.getByPosition(i).column))
if (block.getByPosition(i).column && isColumnConst(*block.getByPosition(i).column))
{
block.erase(i);
--columns;
@ -22,13 +22,14 @@ void removeConstantsFromBlock(Block & block)
void removeConstantsFromSortDescription(const Block & header, SortDescription & description)
{
/// Note: This code is not correct if column description contains column numbers instead of column names.
/// Hopefully, everywhere where it is used, column description contains names.
description.erase(std::remove_if(description.begin(), description.end(),
[&](const SortColumnDescription & elem)
{
if (!elem.column_name.empty())
return isColumnConst(*header.getByName(elem.column_name).column);
else
return isColumnConst(*header.safeGetByPosition(elem.column_number).column);
auto & column = !elem.column_name.empty() ? header.getByName(elem.column_name)
: header.safeGetByPosition(elem.column_number);
return column.column && isColumnConst(*column.column);
}), description.end());
}
@ -41,7 +42,7 @@ void enrichBlockWithConstants(Block & block, const Block & header)
for (size_t i = 0; i < columns; ++i)
{
const auto & col_type_name = header.getByPosition(i);
if (isColumnConst(*col_type_name.column))
if (col_type_name.column && isColumnConst(*col_type_name.column))
block.insert(i, {col_type_name.column->cloneResized(rows), col_type_name.type, col_type_name.name});
}
}

View File

@ -74,6 +74,8 @@ ClickHouseDictionarySource::ClickHouseDictionarySource(
{
/// We should set user info even for the case when the dictionary is loaded in-process (without TCP communication).
context.setUser(user, password, Poco::Net::SocketAddress("127.0.0.1", 0), {});
/// Processors are not supported here yet.
context.getSettingsRef().experimental_use_processors = false;
}

View File

@ -4,6 +4,7 @@
#include <DataStreams/MaterializingBlockOutputStream.h>
#include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IRowInputFormat.h>
namespace DB
@ -26,6 +27,50 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name)
throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT);
}
const FormatFactory::ProcessorCreators & FormatFactory::getProcessorCreators(const String & name) const
{
auto it = processors_dict.find(name);
if (processors_dict.end() != it)
return it->second;
throw Exception("Unknown format " + name, ErrorCodes::UNKNOWN_FORMAT);
}
static FormatSettings getInputFormatSetting(const Settings & settings)
{
FormatSettings format_settings;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields;
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
format_settings.with_names_use_header = settings.input_format_with_names_use_header;
format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields;
format_settings.import_nested_json = settings.input_format_import_nested_json;
format_settings.date_time_input_format = settings.date_time_input_format;
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
return format_settings;
}
static FormatSettings getOutputFormatSetting(const Settings & settings)
{
FormatSettings format_settings;
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width;
format_settings.pretty.color = settings.output_format_pretty_color;
format_settings.write_statistics = settings.output_format_write_statistics;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
return format_settings;
}
BlockInputStreamPtr FormatFactory::getInput(
const String & name,
@ -41,19 +86,7 @@ BlockInputStreamPtr FormatFactory::getInput(
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
const Settings & settings = context.getSettingsRef();
FormatSettings format_settings;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
format_settings.csv.empty_as_default = settings.input_format_defaults_for_omitted_fields;
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
format_settings.with_names_use_header = settings.input_format_with_names_use_header;
format_settings.skip_unknown_fields = settings.input_format_skip_unknown_fields;
format_settings.import_nested_json = settings.input_format_import_nested_json;
format_settings.date_time_input_format = settings.date_time_input_format;
format_settings.input_allow_errors_num = settings.input_format_allow_errors_num;
format_settings.input_allow_errors_ratio = settings.input_format_allow_errors_ratio;
FormatSettings format_settings = getInputFormatSetting(settings);
return input_getter(
buf, sample, context, max_block_size, rows_portion_size, callback ? callback : ReadCallback(), format_settings);
@ -67,19 +100,7 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer &
throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT);
const Settings & settings = context.getSettingsRef();
FormatSettings format_settings;
format_settings.json.quote_64bit_integers = settings.output_format_json_quote_64bit_integers;
format_settings.json.quote_denormals = settings.output_format_json_quote_denormals;
format_settings.json.escape_forward_slashes = settings.output_format_json_escape_forward_slashes;
format_settings.csv.delimiter = settings.format_csv_delimiter;
format_settings.csv.allow_single_quotes = settings.format_csv_allow_single_quotes;
format_settings.csv.allow_double_quotes = settings.format_csv_allow_double_quotes;
format_settings.pretty.max_rows = settings.output_format_pretty_max_rows;
format_settings.pretty.max_column_pad_width = settings.output_format_pretty_max_column_pad_width;
format_settings.pretty.color = settings.output_format_pretty_color;
format_settings.write_statistics = settings.output_format_write_statistics;
format_settings.parquet.row_group_size = settings.output_format_parquet_row_group_size;
FormatSettings format_settings = getOutputFormatSetting(settings);
/** Materialization is needed, because formats can use the functions `IDataType`,
* which only work with full columns.
@ -89,12 +110,46 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name, WriteBuffer &
}
InputFormatPtr FormatFactory::getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, const Context & context, UInt64 max_block_size) const
{
const auto & input_getter = getProcessorCreators(name).first;
if (!input_getter)
throw Exception("Format " + name + " is not suitable for input", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_INPUT);
const Settings & settings = context.getSettingsRef();
FormatSettings format_settings = getInputFormatSetting(settings);
RowInputFormatParams params;
params.max_block_size = max_block_size;
params.allow_errors_num = format_settings.input_allow_errors_num;
params.allow_errors_ratio = format_settings.input_allow_errors_ratio;
return input_getter(buf, sample, context, params, format_settings);
}
OutputFormatPtr FormatFactory::getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample, const Context & context) const
{
const auto & output_getter = getProcessorCreators(name).second;
if (!output_getter)
throw Exception("Format " + name + " is not suitable for output", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT);
const Settings & settings = context.getSettingsRef();
FormatSettings format_settings = getOutputFormatSetting(settings);
/** TODO: Materialization is needed, because formats can use the functions `IDataType`,
* which only work with full columns.
*/
return output_getter(buf, sample, context, format_settings);
}
void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator)
{
auto & target = dict[name].first;
if (target)
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = input_creator;
target = std::move(input_creator);
}
void FormatFactory::registerOutputFormat(const String & name, OutputCreator output_creator)
@ -102,7 +157,23 @@ void FormatFactory::registerOutputFormat(const String & name, OutputCreator outp
auto & target = dict[name].second;
if (target)
throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = output_creator;
target = std::move(output_creator);
}
void FormatFactory::registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator)
{
auto & target = processors_dict[name].first;
if (target)
throw Exception("FormatFactory: Input format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(input_creator);
}
void FormatFactory::registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator)
{
auto & target = processors_dict[name].second;
if (target)
throw Exception("FormatFactory: Output format " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(output_creator);
}
@ -127,6 +198,25 @@ void registerOutputFormatParquet(FormatFactory & factory);
void registerInputFormatProtobuf(FormatFactory & factory);
void registerOutputFormatProtobuf(FormatFactory & factory);
void registerInputFormatProcessorNative(FormatFactory & factory);
void registerOutputFormatProcessorNative(FormatFactory & factory);
void registerInputFormatProcessorRowBinary(FormatFactory & factory);
void registerOutputFormatProcessorRowBinary(FormatFactory & factory);
void registerInputFormatProcessorTabSeparated(FormatFactory & factory);
void registerOutputFormatProcessorTabSeparated(FormatFactory & factory);
void registerInputFormatProcessorValues(FormatFactory & factory);
void registerOutputFormatProcessorValues(FormatFactory & factory);
void registerInputFormatProcessorCSV(FormatFactory & factory);
void registerOutputFormatProcessorCSV(FormatFactory & factory);
void registerInputFormatProcessorTSKV(FormatFactory & factory);
void registerOutputFormatProcessorTSKV(FormatFactory & factory);
void registerInputFormatProcessorJSONEachRow(FormatFactory & factory);
void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory);
void registerInputFormatProcessorParquet(FormatFactory & factory);
void registerOutputFormatProcessorParquet(FormatFactory & factory);
void registerInputFormatProcessorProtobuf(FormatFactory & factory);
void registerOutputFormatProcessorProtobuf(FormatFactory & factory);
/// Output only (presentational) formats.
void registerOutputFormatPretty(FormatFactory & factory);
@ -141,9 +231,22 @@ void registerOutputFormatODBCDriver2(FormatFactory & factory);
void registerOutputFormatNull(FormatFactory & factory);
void registerOutputFormatMySQLWire(FormatFactory & factory);
void registerOutputFormatProcessorPretty(FormatFactory & factory);
void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory);
void registerOutputFormatProcessorPrettySpace(FormatFactory & factory);
void registerOutputFormatProcessorVertical(FormatFactory & factory);
void registerOutputFormatProcessorJSON(FormatFactory & factory);
void registerOutputFormatProcessorJSONCompact(FormatFactory & factory);
void registerOutputFormatProcessorXML(FormatFactory & factory);
void registerOutputFormatProcessorODBCDriver(FormatFactory & factory);
void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory);
void registerOutputFormatProcessorNull(FormatFactory & factory);
void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory);
/// Input only formats.
void registerInputFormatCapnProto(FormatFactory & factory);
void registerInputFormatProcessorCapnProto(FormatFactory & factory);
FormatFactory::FormatFactory()
@ -168,6 +271,28 @@ FormatFactory::FormatFactory()
registerInputFormatParquet(*this);
registerOutputFormatParquet(*this);
registerOutputFormatMySQLWire(*this);
registerInputFormatProcessorNative(*this);
registerOutputFormatProcessorNative(*this);
registerInputFormatProcessorRowBinary(*this);
registerOutputFormatProcessorRowBinary(*this);
registerInputFormatProcessorTabSeparated(*this);
registerOutputFormatProcessorTabSeparated(*this);
registerInputFormatProcessorValues(*this);
registerOutputFormatProcessorValues(*this);
registerInputFormatProcessorCSV(*this);
registerOutputFormatProcessorCSV(*this);
registerInputFormatProcessorTSKV(*this);
registerOutputFormatProcessorTSKV(*this);
registerInputFormatProcessorJSONEachRow(*this);
registerOutputFormatProcessorJSONEachRow(*this);
registerInputFormatProcessorProtobuf(*this);
registerOutputFormatProcessorProtobuf(*this);
registerInputFormatProcessorCapnProto(*this);
registerInputFormatProcessorParquet(*this);
registerOutputFormatProcessorParquet(*this);
registerOutputFormatPretty(*this);
registerOutputFormatPrettyCompact(*this);
registerOutputFormatPrettySpace(*this);
@ -178,7 +303,18 @@ FormatFactory::FormatFactory()
registerOutputFormatODBCDriver(*this);
registerOutputFormatODBCDriver2(*this);
registerOutputFormatNull(*this);
registerOutputFormatMySQLWire(*this);
registerOutputFormatProcessorPretty(*this);
registerOutputFormatProcessorPrettyCompact(*this);
registerOutputFormatProcessorPrettySpace(*this);
registerOutputFormatProcessorVertical(*this);
registerOutputFormatProcessorJSON(*this);
registerOutputFormatProcessorJSONCompact(*this);
registerOutputFormatProcessorXML(*this);
registerOutputFormatProcessorODBCDriver(*this);
registerOutputFormatProcessorODBCDriver2(*this);
registerOutputFormatProcessorNull(*this);
registerOutputFormatProcessorMySQLWrite(*this);
}
}

View File

@ -19,6 +19,18 @@ struct FormatSettings;
class ReadBuffer;
class WriteBuffer;
class IProcessor;
using ProcessorPtr = std::shared_ptr<IProcessor>;
class IInputFormat;
class IOutputFormat;
struct RowInputFormatParams;
using InputFormatPtr = std::shared_ptr<IInputFormat>;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
/** Allows to create an IBlockInputStream or IBlockOutputStream by the name of the format.
* Note: format and compression are independent things.
*/
@ -45,9 +57,24 @@ private:
const Context & context,
const FormatSettings & settings)>;
using InputProcessorCreator = std::function<InputFormatPtr(
ReadBuffer & buf,
const Block & header,
const Context & context,
const RowInputFormatParams & params,
const FormatSettings & settings)>;
using OutputProcessorCreator = std::function<OutputFormatPtr(
WriteBuffer & buf,
const Block & sample,
const Context & context,
const FormatSettings & settings)>;
using Creators = std::pair<InputCreator, OutputCreator>;
using ProcessorCreators = std::pair<InputProcessorCreator, OutputProcessorCreator>;
using FormatsDictionary = std::unordered_map<String, Creators>;
using FormatProcessorsDictionary = std::unordered_map<String, ProcessorCreators>;
public:
BlockInputStreamPtr getInput(
@ -62,10 +89,19 @@ public:
BlockOutputStreamPtr getOutput(const String & name, WriteBuffer & buf,
const Block & sample, const Context & context) const;
InputFormatPtr getInputFormat(const String & name, ReadBuffer & buf,
const Block & sample, const Context & context, UInt64 max_block_size) const;
OutputFormatPtr getOutputFormat(const String & name, WriteBuffer & buf,
const Block & sample, const Context & context) const;
/// Register format by its name.
void registerInputFormat(const String & name, InputCreator input_creator);
void registerOutputFormat(const String & name, OutputCreator output_creator);
void registerInputFormatProcessor(const String & name, InputProcessorCreator input_creator);
void registerOutputFormatProcessor(const String & name, OutputProcessorCreator output_creator);
const FormatsDictionary & getAllFormats() const
{
return dict;
@ -73,11 +109,13 @@ public:
private:
FormatsDictionary dict;
FormatProcessorsDictionary processors_dict;
FormatFactory();
friend class ext::singleton<FormatFactory>;
const Creators & getCreators(const String & name) const;
const ProcessorCreators & getProcessorCreators(const String & name) const;
};
}

View File

@ -34,9 +34,11 @@ class DoubleConverter
DoubleConverter() = default;
public:
/** @todo Add commentary on how this constant is deduced.
* e.g. it's minus sign, integral zero, decimal point, up to 5 leading zeros and kBase10MaximalLength digits. */
static constexpr auto MAX_REPRESENTATION_LENGTH = 26;
/// Sign (1 byte) + DigitsBeforePoint + point (1 byte) + DigitsAfterPoint + zero byte.
/// See comment to DoubleToStringConverter::ToFixed method for explanation.
static constexpr auto MAX_REPRESENTATION_LENGTH =
1 + double_conversion::DoubleToStringConverter::kMaxFixedDigitsBeforePoint +
1 + double_conversion::DoubleToStringConverter::kMaxFixedDigitsAfterPoint + 1;
using BufferType = char[MAX_REPRESENTATION_LENGTH];
static const auto & instance()

View File

@ -2053,7 +2053,6 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
* Then the calculations can be parallelized by buckets.
* We decompose the blocks to the bucket numbers indicated in them.
*/
using BucketToBlocks = std::map<Int32, BlocksList>;
BucketToBlocks bucket_to_blocks;
/// Read all the data.
@ -2071,11 +2070,22 @@ void Aggregator::mergeStream(const BlockInputStreamPtr & stream, AggregatedDataV
bucket_to_blocks[block.info.bucket_num].emplace_back(std::move(block));
}
LOG_TRACE(log, "Read " << total_input_blocks << " blocks of partially aggregated data, total " << total_input_rows << " rows.");
LOG_TRACE(log, "Read " << total_input_blocks << " blocks of partially aggregated data, total " << total_input_rows
<< " rows.");
mergeBlocks(bucket_to_blocks, result, max_threads);
}
void Aggregator::mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads)
{
if (bucket_to_blocks.empty())
return;
UInt64 total_input_rows = 0;
for (auto & bucket : bucket_to_blocks)
for (auto & block : bucket.second)
total_input_rows += block.rows();
/** `minus one` means the absence of information about the bucket
* - in the case of single-level aggregation, as well as for blocks with "overflowing" values.
* If there is at least one block with a bucket number greater or equal than zero, then there was a two-level aggregation.

View File

@ -857,6 +857,10 @@ public:
*/
void mergeStream(const BlockInputStreamPtr & stream, AggregatedDataVariants & result, size_t max_threads);
using BucketToBlocks = std::map<Int32, BlocksList>;
/// Merge partially aggregated blocks separated to buckets into one data structure.
void mergeBlocks(BucketToBlocks bucket_to_blocks, AggregatedDataVariants & result, size_t max_threads);
/// Merge several partially aggregated blocks into one.
/// Precondition: for all blocks block.info.is_overflows flag must be the same.
/// (either all blocks are from overflow data or none blocks are).

View File

@ -1779,6 +1779,11 @@ BlockOutputStreamPtr Context::getOutputFormat(const String & name, WriteBuffer &
return FormatFactory::instance().getOutput(name, buf, sample, *this);
}
OutputFormatPtr Context::getOutputFormatProcessor(const String & name, WriteBuffer & buf, const Block & sample) const
{
return FormatFactory::instance().getOutputFormat(name, buf, sample, *this);
}
time_t Context::getUptimeSeconds() const
{

View File

@ -74,6 +74,8 @@ class ShellCommand;
class ICompressionCodec;
class SettingsConstraints;
class IOutputFormat;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
#if USE_EMBEDDED_COMPILER
@ -289,6 +291,8 @@ public:
BlockInputStreamPtr getInputFormat(const String & name, ReadBuffer & buf, const Block & sample, UInt64 max_block_size) const;
BlockOutputStreamPtr getOutputFormat(const String & name, WriteBuffer & buf, const Block & sample) const;
OutputFormatPtr getOutputFormatProcessor(const String & name, WriteBuffer & buf, const Block & sample) const;
InterserverIOHandler & getInterserverIOHandler();
/// How other servers can access this for downloading replicated data.

View File

@ -740,22 +740,27 @@ void ExpressionActions::execute(Block & block, bool dry_run) const
}
}
bool ExpressionActions::hasTotalsInJoin() const
{
bool has_totals_in_join = false;
for (const auto & action : actions)
{
if (action.join && action.join->hasTotals())
{
has_totals_in_join = true;
break;
}
}
return has_totals_in_join;
}
void ExpressionActions::executeOnTotals(Block & block) const
{
/// If there is `totals` in the subquery for JOIN, but we do not have totals, then take the block with the default values instead of `totals`.
if (!block)
{
bool has_totals_in_join = false;
for (const auto & action : actions)
{
if (action.join && action.join->hasTotals())
{
has_totals_in_join = true;
break;
}
}
if (has_totals_in_join)
if (hasTotalsInJoin())
{
for (const auto & name_and_type : input_columns)
{

View File

@ -223,6 +223,9 @@ public:
/// Execute the expression on the block. The block must contain all the columns returned by getRequiredColumns.
void execute(Block & block, bool dry_run = false) const;
/// Check if joined subquery has totals.
bool hasTotalsInJoin() const;
/** Execute the expression on the block of total values.
* Almost the same as `execute`. The difference is only when JOIN is executed.
*/

View File

@ -2,6 +2,7 @@
#include <DataStreams/BlockIO.h>
#include <Processors/QueryPipeline.h>
namespace DB
{
@ -17,6 +18,10 @@ public:
*/
virtual BlockIO execute() = 0;
virtual QueryPipeline executeWithProcessors() { throw Exception("executeWithProcessors not implemented", ErrorCodes::NOT_IMPLEMENTED); }
virtual bool canExecuteWithProcessors() const { return false; }
virtual ~IInterpreter() {}
};

View File

@ -265,6 +265,11 @@ Block InterpreterKillQueryQuery::getSelectResult(const String & columns, const S
if (where_expression)
select_query += " WHERE " + queryToString(where_expression);
auto use_processors = context.getSettingsRef().experimental_use_processors;
context.getSettingsRef().experimental_use_processors = false;
SCOPE_EXIT(context.getSettingsRef().experimental_use_processors = use_processors);
BlockIO block_io = executeQuery(select_query, context, true);
Block res = block_io.in->read();

View File

@ -58,6 +58,26 @@
#include <ext/map.h>
#include <memory>
#include <Processors/Sources/NullSource.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include <Processors/Transforms/FilterTransform.h>
#include <Processors/Transforms/ExpressionTransform.h>
#include <Processors/Transforms/AggregatingTransform.h>
#include <Processors/Transforms/MergingAggregatedTransform.h>
#include <Processors/Transforms/MergingAggregatedMemoryEfficientTransform.h>
#include <Processors/Transforms/TotalsHavingTransform.h>
#include <Processors/Transforms/PartialSortingTransform.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MergeSortingTransform.h>
#include <Processors/Transforms/MergingSortedTransform.h>
#include <Processors/Transforms/DistinctTransform.h>
#include <Processors/Transforms/LimitByTransform.h>
#include <Processors/Transforms/ExtremesTransform.h>
#include <Processors/Transforms/CreatingSetsTransform.h>
#include <Processors/Transforms/RollupTransform.h>
#include <Processors/Transforms/CubeTransform.h>
#include <Processors/LimitTransform.h>
namespace DB
{
@ -356,6 +376,13 @@ BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams()
return pipeline.streams;
}
QueryPipeline InterpreterSelectQuery::executeWithProcessors()
{
QueryPipeline query_pipeline;
executeImpl(query_pipeline, input, options.only_analyze);
return query_pipeline;
}
InterpreterSelectQuery::AnalysisResult
InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run, const FilterInfoPtr & filter_info)
{
@ -555,7 +582,8 @@ InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage
}
void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run)
template <typename TPipeline>
void InterpreterSelectQuery::executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run)
{
/** Streams of data. When the query is executed in parallel, we have several data streams.
* If there is no GROUP BY, then perform all operations before ORDER BY and LIMIT in parallel, then
@ -567,6 +595,8 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
* then perform the remaining operations with one resulting stream.
*/
constexpr bool pipeline_with_processors = std::is_same<TPipeline, QueryPipeline>::value;
/// Now we will compose block streams that perform the necessary actions.
auto & query = getSelectQuery();
const Settings & settings = context.getSettingsRef();
@ -611,21 +641,42 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
if (dry_run)
{
pipeline.streams.emplace_back(std::make_shared<NullBlockInputStream>(source_header));
if constexpr (pipeline_with_processors)
pipeline.init({std::make_shared<NullSource>(source_header)});
else
pipeline.streams.emplace_back(std::make_shared<NullBlockInputStream>(source_header));
expressions = analyzeExpressions(QueryProcessingStage::FetchColumns, true, filter_info);
if (storage && expressions.filter_info && expressions.prewhere_info)
throw Exception("PREWHERE is not supported if the table is filtered by row-level security expression", ErrorCodes::ILLEGAL_PREWHERE);
if (expressions.prewhere_info)
pipeline.streams.back() = std::make_shared<FilterBlockInputStream>(
{
if constexpr (pipeline_with_processors)
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<FilterTransform>(
header,
expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_column_name,
expressions.prewhere_info->remove_prewhere_column);
});
else
pipeline.streams.back() = std::make_shared<FilterBlockInputStream>(
pipeline.streams.back(), expressions.prewhere_info->prewhere_actions,
expressions.prewhere_info->prewhere_column_name, expressions.prewhere_info->remove_prewhere_column);
}
}
else
{
if (prepared_input)
pipeline.streams.push_back(prepared_input);
{
if constexpr (pipeline_with_processors)
pipeline.init({std::make_shared<SourceFromInputStream>(prepared_input)});
else
pipeline.streams.push_back(prepared_input);
}
expressions = analyzeExpressions(from_stage, false, filter_info);
@ -662,25 +713,78 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
{
if (expressions.filter_info)
{
pipeline.transform([&](auto & stream)
if constexpr (pipeline_with_processors)
{
stream = std::make_shared<FilterBlockInputStream>(
stream,
expressions.filter_info->actions,
expressions.filter_info->column_name,
expressions.filter_info->do_remove_column);
});
pipeline.addSimpleTransform([&](const Block & block, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipeline::StreamType::Totals)
return nullptr;
return std::make_shared<FilterTransform>(
block,
expressions.filter_info->actions,
expressions.filter_info->column_name,
expressions.filter_info->do_remove_column);
});
}
else
{
pipeline.transform([&](auto & stream)
{
stream = std::make_shared<FilterBlockInputStream>(
stream,
expressions.filter_info->actions,
expressions.filter_info->column_name,
expressions.filter_info->do_remove_column);
});
}
}
if (expressions.hasJoin())
{
Block header_before_join;
if constexpr (pipeline_with_processors)
{
header_before_join = pipeline.getHeader();
/// In case joined subquery has totals, and we don't, add default chunk to totals.
bool default_totals = false;
if (!pipeline.hasTotals())
{
pipeline.addDefaultTotals();
default_totals = true;
}
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType type)
{
bool on_totals = type == QueryPipeline::StreamType::Totals;
return std::make_shared<ExpressionTransform>(header, expressions.before_join, on_totals, default_totals);
});
}
else
{
header_before_join = pipeline.firstStream()->getHeader();
/// Applies to all sources except stream_with_non_joined_data.
for (auto & stream : pipeline.streams)
stream = std::make_shared<ExpressionBlockInputStream>(stream, expressions.before_join);
}
const auto & join = query.join()->table_join->as<ASTTableJoin &>();
if (isRightOrFull(join.kind))
pipeline.stream_with_non_joined_data = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(
pipeline.firstStream()->getHeader(), settings.max_block_size);
{
auto stream = expressions.before_join->createStreamWithNonJoinedDataIfFullOrRightJoin(
header_before_join, settings.max_block_size);
for (auto & stream : pipeline.streams) /// Applies to all sources except stream_with_non_joined_data.
stream = std::make_shared<ExpressionBlockInputStream>(stream, expressions.before_join);
if constexpr (pipeline_with_processors)
{
auto source = std::make_shared<SourceFromInputStream>(std::move(stream));
pipeline.addDelayedStream(source);
}
else
pipeline.stream_with_non_joined_data = std::move(stream);
}
}
if (expressions.has_where)
@ -810,13 +914,18 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
if (need_second_distinct_pass
|| query.limitLength()
|| query.limitBy()
|| pipeline.stream_with_non_joined_data)
|| pipeline.hasDelayedStream())
{
need_merge_streams = true;
}
if (need_merge_streams)
executeUnion(pipeline);
{
if constexpr (pipeline_with_processors)
pipeline.resize(1);
else
executeUnion(pipeline);
}
/** If there was more than one stream,
* then DISTINCT needs to be performed once again after merging all streams.
@ -888,10 +997,13 @@ static UInt64 getLimitForSorting(const ASTSelectQuery & query, const Context & c
}
template <typename TPipeline>
void InterpreterSelectQuery::executeFetchColumns(
QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
QueryProcessingStage::Enum processing_stage, TPipeline & pipeline,
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere)
{
constexpr bool pipeline_with_processors = std::is_same<TPipeline, QueryPipeline>::value;
auto & query = getSelectQuery();
const Settings & settings = context.getSettingsRef();
@ -1105,7 +1217,7 @@ void InterpreterSelectQuery::executeFetchColumns(
throw Exception("Setting 'max_block_size' cannot be zero", ErrorCodes::PARAMETER_OUT_OF_BOUND);
/// Initialize the initial data streams to which the query transforms are superimposed. Table or subquery or prepared input?
if (!pipeline.streams.empty())
if (pipeline.initialized())
{
/// Prepared input.
}
@ -1127,7 +1239,11 @@ void InterpreterSelectQuery::executeFetchColumns(
interpreter_subquery->ignoreWithTotals();
}
pipeline.streams = interpreter_subquery->executeWithMultipleStreams();
if constexpr (pipeline_with_processors)
/// Just use pipeline from subquery.
pipeline = interpreter_subquery->executeWithProcessors();
else
pipeline.streams = interpreter_subquery->executeWithMultipleStreams();
}
else if (storage)
{
@ -1146,27 +1262,23 @@ void InterpreterSelectQuery::executeFetchColumns(
query_info.sets = query_analyzer->getPreparedSets();
query_info.prewhere_info = prewhere_info;
pipeline.streams = storage->read(required_columns, query_info, context, processing_stage, max_block_size, max_streams);
auto streams = storage->read(required_columns, query_info, context, processing_stage, max_block_size, max_streams);
if (pipeline.streams.empty())
if (streams.empty())
{
pipeline.streams = {std::make_shared<NullBlockInputStream>(storage->getSampleBlockForColumns(required_columns))};
streams = {std::make_shared<NullBlockInputStream>(storage->getSampleBlockForColumns(required_columns))};
if (query_info.prewhere_info)
pipeline.transform([&](auto & stream)
{
stream = std::make_shared<FilterBlockInputStream>(
stream,
prewhere_info->prewhere_actions,
prewhere_info->prewhere_column_name,
prewhere_info->remove_prewhere_column);
});
streams.back() = std::make_shared<FilterBlockInputStream>(
streams.back(),
prewhere_info->prewhere_actions,
prewhere_info->prewhere_column_name,
prewhere_info->remove_prewhere_column);
}
pipeline.transform([&](auto & stream)
{
for (auto & stream : streams)
stream->addTableLock(table_lock);
});
/// Set the limits and quota for reading data, the speed and time of the query.
{
@ -1194,15 +1306,52 @@ void InterpreterSelectQuery::executeFetchColumns(
QuotaForIntervals & quota = context.getQuota();
pipeline.transform([&](auto & stream)
for (auto & stream : streams)
{
if (!options.ignore_limits)
stream->setLimits(limits);
if (options.to_stage == QueryProcessingStage::Complete)
stream->setQuota(quota);
});
}
}
if constexpr (pipeline_with_processors)
{
/// Unify streams. They must have same headers.
if (streams.size() > 1)
{
/// Unify streams in case they have different headers.
auto first_header = streams.at(0)->getHeader();
for (size_t i = 1; i < streams.size(); ++i)
{
auto & stream = streams[i];
auto header = stream->getHeader();
auto mode = ConvertingBlockInputStream::MatchColumnsMode::Name;
if (!blocksHaveEqualStructure(first_header, header))
stream = std::make_shared<ConvertingBlockInputStream>(context, stream, first_header, mode);
}
}
Processors sources;
sources.reserve(streams.size());
for (auto & stream : streams)
{
bool force_add_agg_info = processing_stage == QueryProcessingStage::WithMergeableState;
auto source = std::make_shared<SourceFromInputStream>(stream, force_add_agg_info);
if (processing_stage == QueryProcessingStage::Complete)
source->addTotalsPort();
sources.emplace_back(std::move(source));
}
pipeline.init(std::move(sources));
}
else
pipeline.streams = std::move(streams);
}
else
throw Exception("Logical error in InterpreterSelectQuery: nowhere to read", ErrorCodes::LOGICAL_ERROR);
@ -1210,10 +1359,20 @@ void InterpreterSelectQuery::executeFetchColumns(
/// Aliases in table declaration.
if (processing_stage == QueryProcessingStage::FetchColumns && alias_actions)
{
pipeline.transform([&](auto & stream)
if constexpr (pipeline_with_processors)
{
stream = std::make_shared<ExpressionBlockInputStream>(stream, alias_actions);
});
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, alias_actions);
});
}
else
{
pipeline.transform([&](auto & stream)
{
stream = std::make_shared<ExpressionBlockInputStream>(stream, alias_actions);
});
}
}
}
@ -1226,6 +1385,13 @@ void InterpreterSelectQuery::executeWhere(Pipeline & pipeline, const ExpressionA
});
}
void InterpreterSelectQuery::executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_fiter)
{
pipeline.addSimpleTransform([&](const Block & block)
{
return std::make_shared<FilterTransform>(block, expression, getSelectQuery().where()->getColumnName(), remove_fiter);
});
}
void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final)
{
@ -1294,6 +1460,76 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre
}
void InterpreterSelectQuery::executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final)
{
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<ExpressionTransform>(header, expression);
});
Names key_names;
AggregateDescriptions aggregates;
query_analyzer->getAggregateInfo(key_names, aggregates);
Block header_before_aggregation = pipeline.getHeader();
ColumnNumbers keys;
for (const auto & name : key_names)
keys.push_back(header_before_aggregation.getPositionByName(name));
for (auto & descr : aggregates)
if (descr.arguments.empty())
for (const auto & name : descr.argument_names)
descr.arguments.push_back(header_before_aggregation.getPositionByName(name));
const Settings & settings = context.getSettingsRef();
/** Two-level aggregation is useful in two cases:
* 1. Parallel aggregation is done, and the results should be merged in parallel.
* 2. An aggregation is done with store of temporary data on the disk, and they need to be merged in a memory efficient way.
*/
bool allow_to_use_two_level_group_by = pipeline.getNumMainStreams() > 1 || settings.max_bytes_before_external_group_by != 0;
Aggregator::Params params(header_before_aggregation, keys, aggregates,
overflow_row, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile,
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold : SettingUInt64(0),
allow_to_use_two_level_group_by ? settings.group_by_two_level_threshold_bytes : SettingUInt64(0),
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath(), settings.max_threads);
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
pipeline.dropTotalsIfHas();
/// If there are several sources, then we perform parallel aggregation
if (pipeline.getNumMainStreams() > 1)
{
pipeline.resize(max_streams);
auto many_data = std::make_shared<ManyAggregatedData>(max_streams);
auto merge_threads = settings.aggregation_memory_efficient_merge_threads
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
size_t counter = 0;
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AggregatingTransform>(header, transform_params, many_data, counter++, max_streams, merge_threads);
});
pipeline.resize(1);
}
else
{
pipeline.resize(1);
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AggregatingTransform>(header, transform_params);
});
}
}
void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool overflow_row, bool final)
{
Names key_names;
@ -1345,6 +1581,67 @@ void InterpreterSelectQuery::executeMergeAggregated(Pipeline & pipeline, bool ov
}
}
void InterpreterSelectQuery::executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final)
{
Names key_names;
AggregateDescriptions aggregates;
query_analyzer->getAggregateInfo(key_names, aggregates);
Block header_before_merge = pipeline.getHeader();
ColumnNumbers keys;
for (const auto & name : key_names)
keys.push_back(header_before_merge.getPositionByName(name));
/** There are two modes of distributed aggregation.
*
* 1. In different threads read from the remote servers blocks.
* Save all the blocks in the RAM. Merge blocks.
* If the aggregation is two-level - parallelize to the number of buckets.
*
* 2. In one thread, read blocks from different servers in order.
* RAM stores only one block from each server.
* If the aggregation is a two-level aggregation, we consistently merge the blocks of each next level.
*
* The second option consumes less memory (up to 256 times less)
* in the case of two-level aggregation, which is used for large results after GROUP BY,
* but it can work more slowly.
*/
const Settings & settings = context.getSettingsRef();
Aggregator::Params params(header_before_merge, keys, aggregates, overflow_row, settings.max_threads);
auto transform_params = std::make_shared<AggregatingTransformParams>(params, final);
if (!settings.distributed_aggregation_memory_efficient)
{
/// We union several sources into one, parallelizing the work.
pipeline.resize(1);
/// Now merge the aggregated blocks
pipeline.addSimpleTransform([&](const Block & header)
{
return std::make_shared<MergingAggregatedTransform>(header, transform_params, settings.max_threads);
});
}
else
{
/// pipeline.resize(max_streams); - Seem we don't need it.
auto num_merge_threads = settings.aggregation_memory_efficient_merge_threads
? static_cast<size_t>(settings.aggregation_memory_efficient_merge_threads)
: static_cast<size_t>(settings.max_threads);
auto pipe = createMergingAggregatedMemoryEfficientPipe(
pipeline.getHeader(),
transform_params,
pipeline.getNumStreams(),
num_merge_threads);
pipeline.addPipe(std::move(pipe));
}
}
void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const ExpressionActionsPtr & expression)
{
@ -1354,6 +1651,18 @@ void InterpreterSelectQuery::executeHaving(Pipeline & pipeline, const Expression
});
}
void InterpreterSelectQuery::executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression)
{
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipeline::StreamType::Totals)
return nullptr;
/// TODO: do we need to save filter there?
return std::make_shared<FilterTransform>(header, expression, getSelectQuery().having()->getColumnName(), false);
});
}
void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final)
{
@ -1371,6 +1680,19 @@ void InterpreterSelectQuery::executeTotalsAndHaving(Pipeline & pipeline, bool ha
final);
}
void InterpreterSelectQuery::executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final)
{
const Settings & settings = context.getSettingsRef();
auto totals_having = std::make_shared<TotalsHavingTransform>(
pipeline.getHeader(), overflow_row, expression,
has_having ? getSelectQuery().having()->getColumnName() : "",
settings.totals_mode, settings.totals_auto_threshold, final);
pipeline.addTotalsHavingTransform(std::move(totals_having));
}
void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificator modificator)
{
executeUnion(pipeline);
@ -1401,6 +1723,44 @@ void InterpreterSelectQuery::executeRollupOrCube(Pipeline & pipeline, Modificato
pipeline.firstStream() = std::make_shared<CubeBlockInputStream>(pipeline.firstStream(), params);
}
void InterpreterSelectQuery::executeRollupOrCube(QueryPipeline & pipeline, Modificator modificator)
{
pipeline.resize(1);
Names key_names;
AggregateDescriptions aggregates;
query_analyzer->getAggregateInfo(key_names, aggregates);
Block header_before_transform = pipeline.getHeader();
ColumnNumbers keys;
for (const auto & name : key_names)
keys.push_back(header_before_transform.getPositionByName(name));
const Settings & settings = context.getSettingsRef();
Aggregator::Params params(header_before_transform, keys, aggregates,
false, settings.max_rows_to_group_by, settings.group_by_overflow_mode,
settings.compile ? &context.getCompiler() : nullptr, settings.min_count_to_compile,
SettingUInt64(0), SettingUInt64(0),
settings.max_bytes_before_external_group_by, settings.empty_result_for_aggregation_by_empty_set,
context.getTemporaryPath(), settings.max_threads);
auto transform_params = std::make_shared<AggregatingTransformParams>(params, true);
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipeline::StreamType::Totals)
return nullptr;
if (modificator == Modificator::ROLLUP)
return std::make_shared<RollupTransform>(header, std::move(transform_params));
else
return std::make_shared<CubeTransform>(header, std::move(transform_params));
});
}
void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const ExpressionActionsPtr & expression)
{
@ -1410,6 +1770,13 @@ void InterpreterSelectQuery::executeExpression(Pipeline & pipeline, const Expres
});
}
void InterpreterSelectQuery::executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression)
{
pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr
{
return std::make_shared<ExpressionTransform>(header, expression);
});
}
static SortDescription getSortDescription(const ASTSelectQuery & query)
{
@ -1462,6 +1829,42 @@ void InterpreterSelectQuery::executeOrder(Pipeline & pipeline)
settings.max_bytes_before_external_sort, context.getTemporaryPath());
}
void InterpreterSelectQuery::executeOrder(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query);
UInt64 limit = getLimitForSorting(query, context);
const Settings & settings = context.getSettingsRef();
/// TODO: Limits on sorting
// IBlockInputStream::LocalLimits limits;
// limits.mode = IBlockInputStream::LIMITS_TOTAL;
// limits.size_limits = SizeLimits(settings.max_rows_to_sort, settings.max_bytes_to_sort, settings.sort_overflow_mode);
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type)
{
bool do_count_rows = stream_type == QueryPipeline::StreamType::Main;
return std::make_shared<PartialSortingTransform>(header, order_descr, limit, do_count_rows);
});
/// If there are several streams, we merge them into one
pipeline.resize(1);
/// Merge the sorted blocks.
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipeline::StreamType::Totals)
return nullptr;
return std::make_shared<MergeSortingTransform>(
header, order_descr, settings.max_block_size, limit,
settings.max_bytes_before_remerge_sort,
settings.max_bytes_before_external_sort, context.getTemporaryPath());
});
}
void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
{
@ -1490,6 +1893,27 @@ void InterpreterSelectQuery::executeMergeSorted(Pipeline & pipeline)
}
}
void InterpreterSelectQuery::executeMergeSorted(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
SortDescription order_descr = getSortDescription(query);
UInt64 limit = getLimitForSorting(query, context);
const Settings & settings = context.getSettingsRef();
/// If there are several streams, then we merge them into one
if (pipeline.getNumStreams() > 1)
{
auto transform = std::make_shared<MergingSortedTransform>(
pipeline.getHeader(),
pipeline.getNumStreams(),
order_descr,
settings.max_block_size, limit);
pipeline.addPipe({ std::move(transform) });
}
}
void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const ExpressionActionsPtr & expression)
{
@ -1499,6 +1923,14 @@ void InterpreterSelectQuery::executeProjection(Pipeline & pipeline, const Expres
});
}
void InterpreterSelectQuery::executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression)
{
pipeline.addSimpleTransform([&](const Block & header) -> ProcessorPtr
{
return std::make_shared<ExpressionTransform>(header, expression);
});
}
void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_order, Names columns)
{
@ -1522,6 +1954,32 @@ void InterpreterSelectQuery::executeDistinct(Pipeline & pipeline, bool before_or
}
}
void InterpreterSelectQuery::executeDistinct(QueryPipeline & pipeline, bool before_order, Names columns)
{
auto & query = getSelectQuery();
if (query.distinct)
{
const Settings & settings = context.getSettingsRef();
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
UInt64 limit_for_distinct = 0;
/// If after this stage of DISTINCT ORDER BY is not executed, then you can get no more than limit_length + limit_offset of different rows.
if (!query.orderBy() || !before_order)
limit_for_distinct = limit_length + limit_offset;
SizeLimits limits(settings.max_rows_in_distinct, settings.max_bytes_in_distinct, settings.distinct_overflow_mode);
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipeline::StreamType::Totals)
return nullptr;
return std::make_shared<DistinctTransform>(header, limits, limit_for_distinct, columns);
});
}
}
void InterpreterSelectQuery::executeUnion(Pipeline & pipeline)
{
@ -1558,6 +2016,24 @@ void InterpreterSelectQuery::executePreLimit(Pipeline & pipeline)
}
}
/// Preliminary LIMIT - is used in every source, if there are several sources, before they are combined.
void InterpreterSelectQuery::executePreLimit(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
/// If there is LIMIT
if (query.limitLength())
{
auto [limit_length, limit_offset] = getLimitLengthAndOffset(query, context);
pipeline.addSimpleTransform([&, limit = limit_length + limit_offset](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipeline::StreamType::Totals)
return nullptr;
return std::make_shared<LimitTransform>(header, limit, 0);
});
}
}
void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline)
{
@ -1577,6 +2053,28 @@ void InterpreterSelectQuery::executeLimitBy(Pipeline & pipeline)
});
}
void InterpreterSelectQuery::executeLimitBy(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
if (!query.limitByLength() || !query.limitBy())
return;
Names columns;
for (const auto & elem : query.limitBy()->children)
columns.emplace_back(elem->getColumnName());
UInt64 length = getLimitUIntValue(query.limitByLength(), context);
UInt64 offset = (query.limitByOffset() ? getLimitUIntValue(query.limitByOffset(), context) : 0);
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type == QueryPipeline::StreamType::Totals)
return nullptr;
return std::make_shared<LimitByTransform>(header, length, offset, columns);
});
}
// TODO: move to anonymous namespace
bool hasWithTotalsInAnySubqueryInFromClause(const ASTSelectQuery & query)
@ -1636,6 +2134,44 @@ void InterpreterSelectQuery::executeLimit(Pipeline & pipeline)
}
}
void InterpreterSelectQuery::executeLimit(QueryPipeline & pipeline)
{
auto & query = getSelectQuery();
/// If there is LIMIT
if (query.limitLength())
{
/** Rare case:
* if there is no WITH TOTALS and there is a subquery in FROM, and there is WITH TOTALS on one of the levels,
* then when using LIMIT, you should read the data to the end, rather than cancel the query earlier,
* because if you cancel the query, we will not get `totals` data from the remote server.
*
* Another case:
* if there is WITH TOTALS and there is no ORDER BY, then read the data to the end,
* otherwise TOTALS is counted according to incomplete data.
*/
bool always_read_till_end = false;
if (query.group_by_with_totals && !query.orderBy())
always_read_till_end = true;
if (!query.group_by_with_totals && hasWithTotalsInAnySubqueryInFromClause(query))
always_read_till_end = true;
UInt64 limit_length;
UInt64 limit_offset;
std::tie(limit_length, limit_offset) = getLimitLengthAndOffset(query, context);
pipeline.addSimpleTransform([&](const Block & header, QueryPipeline::StreamType stream_type) -> ProcessorPtr
{
if (stream_type != QueryPipeline::StreamType::Main)
return nullptr;
return std::make_shared<LimitTransform>(
header, limit_length, limit_offset, always_read_till_end);
});
}
}
void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline)
{
@ -1648,6 +2184,15 @@ void InterpreterSelectQuery::executeExtremes(Pipeline & pipeline)
});
}
void InterpreterSelectQuery::executeExtremes(QueryPipeline & pipeline)
{
if (!context.getSettingsRef().extremes)
return;
auto transform = std::make_shared<ExtremesTransform>(pipeline.getHeader());
pipeline.addExtremesTransform(std::move(transform));
}
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline, SubqueriesForSets & subqueries_for_sets)
{
@ -1656,6 +2201,19 @@ void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(Pipeline & pipeline
pipeline.firstStream(), subqueries_for_sets, context);
}
void InterpreterSelectQuery::executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, SubqueriesForSets & subqueries_for_sets)
{
const Settings & settings = context.getSettingsRef();
auto creating_sets = std::make_shared<CreatingSetsTransform>(
pipeline.getHeader(), subqueries_for_sets,
SizeLimits(settings.max_rows_to_transfer, settings.max_bytes_to_transfer, settings.transfer_overflow_mode),
context);
pipeline.addCreatingSetsTransform(std::move(creating_sets));
}
void InterpreterSelectQuery::unifyStreams(Pipeline & pipeline)
{
if (pipeline.hasMoreThanOneStream())

View File

@ -13,6 +13,7 @@
#include <Storages/SelectQueryInfo.h>
#include <Storages/TableStructureLockHolder.h>
#include <Processors/QueryPipeline.h>
namespace Poco { class Logger; }
@ -69,6 +70,9 @@ public:
/// Execute the query and return multuple streams for parallel processing.
BlockInputStreams executeWithMultipleStreams();
QueryPipeline executeWithProcessors() override;
bool canExecuteWithProcessors() const override { return true; }
Block getSampleBlock();
void ignoreWithTotals();
@ -125,10 +129,13 @@ private:
{
return hasMoreThanOneStream() || union_stream;
}
bool hasDelayedStream() const { return stream_with_non_joined_data != nullptr; }
bool initialized() const { return !streams.empty(); }
};
void executeImpl(Pipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run);
template <typename TPipeline>
void executeImpl(TPipeline & pipeline, const BlockInputStreamPtr & prepared_input, bool dry_run);
struct AnalysisResult
{
@ -177,7 +184,8 @@ private:
/// dry_run - don't read from table, use empty header block instead.
void executeWithMultipleStreamsImpl(Pipeline & pipeline, const BlockInputStreamPtr & input, bool dry_run);
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, Pipeline & pipeline,
template <typename TPipeline>
void executeFetchColumns(QueryProcessingStage::Enum processing_stage, TPipeline & pipeline,
const PrewhereInfoPtr & prewhere_info, const Names & columns_to_remove_after_prewhere);
void executeWhere(Pipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_filter);
@ -197,6 +205,22 @@ private:
void executeExtremes(Pipeline & pipeline);
void executeSubqueriesInSetsAndJoins(Pipeline & pipeline, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
void executeWhere(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool remove_fiter);
void executeAggregation(QueryPipeline & pipeline, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeMergeAggregated(QueryPipeline & pipeline, bool overflow_row, bool final);
void executeTotalsAndHaving(QueryPipeline & pipeline, bool has_having, const ExpressionActionsPtr & expression, bool overflow_row, bool final);
void executeHaving(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeExpression(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeOrder(QueryPipeline & pipeline);
void executeMergeSorted(QueryPipeline & pipeline);
void executePreLimit(QueryPipeline & pipeline);
void executeLimitBy(QueryPipeline & pipeline);
void executeLimit(QueryPipeline & pipeline);
void executeProjection(QueryPipeline & pipeline, const ExpressionActionsPtr & expression);
void executeDistinct(QueryPipeline & pipeline, bool before_order, Names columns);
void executeExtremes(QueryPipeline & pipeline);
void executeSubqueriesInSetsAndJoins(QueryPipeline & pipeline, std::unordered_map<String, SubqueryForSet> & subqueries_for_sets);
/// If pipeline has several streams with different headers, add ConvertingBlockInputStream to first header.
void unifyStreams(Pipeline & pipeline);
@ -208,6 +232,8 @@ private:
void executeRollupOrCube(Pipeline & pipeline, Modificator modificator);
void executeRollupOrCube(QueryPipeline & pipeline, Modificator modificator);
/** If there is a SETTINGS section in the SELECT query, then apply settings from it.
*
* Section SETTINGS - settings for a specific query.

View File

@ -12,6 +12,9 @@
#include <Parsers/queryToString.h>
#include <Parsers/ASTExpressionList.h>
#include <Processors/Sources/NullSource.h>
#include <Processors/QueryPipeline.h>
namespace DB
{
@ -101,16 +104,25 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
for (size_t query_num = 0; query_num < num_selects; ++query_num)
headers[query_num] = nested_interpreters[query_num]->getSampleBlock();
result_header = headers.front();
size_t num_columns = result_header.columns();
result_header = getCommonHeaderForUnion(headers);
}
}
for (size_t query_num = 1; query_num < num_selects; ++query_num)
if (headers[query_num].columns() != num_columns)
throw Exception("Different number of columns in UNION ALL elements:\n"
+ result_header.dumpNames()
+ "\nand\n"
+ headers[query_num].dumpNames() + "\n",
ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
Block InterpreterSelectWithUnionQuery::getCommonHeaderForUnion(const Blocks & headers)
{
size_t num_selects = headers.size();
Block common_header = headers.front();
size_t num_columns = common_header.columns();
for (size_t query_num = 1; query_num < num_selects; ++query_num)
{
if (headers[query_num].columns() != num_columns)
throw Exception("Different number of columns in UNION ALL elements:\n"
+ common_header.dumpNames()
+ "\nand\n"
+ headers[query_num].dumpNames() + "\n",
ErrorCodes::UNION_ALL_RESULT_STRUCTURES_MISMATCH);
for (size_t column_num = 0; column_num < num_columns; ++column_num)
{
@ -119,10 +131,12 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
for (size_t i = 0; i < num_selects; ++i)
columns.push_back(&headers[i].getByPosition(column_num));
ColumnWithTypeAndName & result_elem = result_header.getByPosition(column_num);
ColumnWithTypeAndName & result_elem = common_header.getByPosition(column_num);
result_elem = getLeastSuperColumn(columns);
}
}
return common_header;
}
@ -197,6 +211,43 @@ BlockIO InterpreterSelectWithUnionQuery::execute()
}
QueryPipeline InterpreterSelectWithUnionQuery::executeWithProcessors()
{
QueryPipeline main_pipeline;
std::vector<QueryPipeline> pipelines;
bool has_main_pipeline = false;
Blocks headers;
headers.reserve(nested_interpreters.size());
for (auto & interpreter : nested_interpreters)
{
if (!has_main_pipeline)
{
has_main_pipeline = true;
main_pipeline = interpreter->executeWithProcessors();
headers.emplace_back(main_pipeline.getHeader());
}
else
{
pipelines.emplace_back(interpreter->executeWithProcessors());
headers.emplace_back(pipelines.back().getHeader());
}
}
if (!has_main_pipeline)
main_pipeline.init({ std::make_shared<NullSource>(getSampleBlock()) });
if (!pipelines.empty())
{
auto common_header = getCommonHeaderForUnion(headers);
main_pipeline.unitePipelines(std::move(pipelines), common_header, context);
}
return main_pipeline;
}
void InterpreterSelectWithUnionQuery::ignoreWithTotals()
{
for (auto & interpreter : nested_interpreters)

View File

@ -5,6 +5,7 @@
#include <Interpreters/IInterpreter.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Processors/QueryPipeline.h>
namespace DB
{
@ -30,6 +31,9 @@ public:
/// Execute the query without union of streams.
BlockInputStreams executeWithMultipleStreams();
QueryPipeline executeWithProcessors() override;
bool canExecuteWithProcessors() const override { return true; }
Block getSampleBlock();
static Block getSampleBlock(
@ -48,6 +52,8 @@ private:
std::vector<std::unique_ptr<InterpreterSelectQuery>> nested_interpreters;
Block result_header;
static Block getCommonHeaderForUnion(const Blocks & headers);
};
}

View File

@ -28,8 +28,11 @@
#include <Interpreters/InterpreterSetQuery.h>
#include <Interpreters/ReplaceQueryParameterVisitor.h>
#include <Interpreters/executeQuery.h>
#include "DNSCacheUpdater.h"
#include <Interpreters/DNSCacheUpdater.h>
#include <Processors/Transforms/LimitsCheckingTransform.h>
#include <Processors/Transforms/MaterializingTransform.h>
#include <Processors/Formats/IOutputFormat.h>
namespace DB
{
@ -200,6 +203,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
/// Copy query into string. It will be written to log and presented in processlist. If an INSERT query, string will not include data to insertion.
String query(begin, query_end);
BlockIO res;
QueryPipeline & pipeline = res.pipeline;
try
{
@ -235,7 +239,13 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
context.initializeExternalTablesIfSet();
auto interpreter = InterpreterFactory::get(ast, context, stage);
res = interpreter->execute();
bool use_processors = settings.experimental_use_processors && interpreter->canExecuteWithProcessors();
if (use_processors)
pipeline = interpreter->executeWithProcessors();
else
res = interpreter->execute();
if (auto * insert_interpreter = typeid_cast<const InterpreterInsertQuery *>(&*interpreter))
context.setInsertionTable(insert_interpreter->getDatabaseTable());
@ -245,36 +255,57 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
if ((*process_list_entry)->isKilled())
throw Exception("Query '" + (*process_list_entry)->getInfo().client_info.current_query_id + "' is killed in pending state",
ErrorCodes::QUERY_WAS_CANCELLED);
else
else if (!use_processors)
(*process_list_entry)->setQueryStreams(res);
}
/// Hold element of process list till end of query execution.
res.process_list_entry = process_list_entry;
if (res.in)
IBlockInputStream::LocalLimits limits;
limits.mode = IBlockInputStream::LIMITS_CURRENT;
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
if (use_processors)
{
res.in->setProgressCallback(context.getProgressCallback());
res.in->setProcessListElement(context.getProcessListElement());
pipeline.setProgressCallback(context.getProgressCallback());
pipeline.setProcessListElement(context.getProcessListElement());
/// Limits on the result, the quota on the result, and also callback for progress.
/// Limits apply only to the final result.
if (stage == QueryProcessingStage::Complete)
{
IBlockInputStream::LocalLimits limits;
limits.mode = IBlockInputStream::LIMITS_CURRENT;
limits.size_limits = SizeLimits(settings.max_result_rows, settings.max_result_bytes, settings.result_overflow_mode);
res.in->setLimits(limits);
res.in->setQuota(quota);
pipeline.resize(1);
pipeline.addSimpleTransform([&](const Block & header)
{
auto transform = std::make_shared<LimitsCheckingTransform>(header, limits);
transform->setQuota(quota);
return transform;
});
}
}
if (res.out)
else
{
if (auto stream = dynamic_cast<CountingBlockOutputStream *>(res.out.get()))
if (res.in)
{
stream->setProcessListElement(context.getProcessListElement());
res.in->setProgressCallback(context.getProgressCallback());
res.in->setProcessListElement(context.getProcessListElement());
/// Limits on the result, the quota on the result, and also callback for progress.
/// Limits apply only to the final result.
if (stage == QueryProcessingStage::Complete)
{
res.in->setLimits(limits);
res.in->setQuota(quota);
}
}
if (res.out)
{
if (auto stream = dynamic_cast<CountingBlockOutputStream *>(res.out.get()))
{
stream->setProcessListElement(context.getProcessListElement());
}
}
}
@ -304,7 +335,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
/// Also make possible for caller to log successful query finish and exception during execution.
res.finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable
auto finish_callback = [elem, &context, log_queries] (IBlockInputStream * stream_in, IBlockOutputStream * stream_out) mutable
{
QueryStatus * process_list_elem = context.getProcessListElement();
@ -373,7 +404,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
};
res.exception_callback = [elem, &context, log_queries] () mutable
auto exception_callback = [elem, &context, log_queries] () mutable
{
context.getQuota().addError();
@ -416,6 +447,9 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
}
};
res.finish_callback = std::move(finish_callback);
res.exception_callback = std::move(exception_callback);
if (!internal && res.in)
{
std::stringstream log_str;
@ -498,6 +532,8 @@ void executeQuery(
std::tie(ast, streams) = executeQueryImpl(begin, end, context, false, QueryProcessingStage::Complete, may_have_tail);
auto & pipeline = streams.pipeline;
try
{
if (streams.out)
@ -551,6 +587,63 @@ void executeQuery(
copyData(*streams.in, *out);
}
if (pipeline.initialized())
{
const ASTQueryWithOutput * ast_query_with_output = dynamic_cast<const ASTQueryWithOutput *>(ast.get());
WriteBuffer * out_buf = &ostr;
std::optional<WriteBufferFromFile> out_file_buf;
if (ast_query_with_output && ast_query_with_output->out_file)
{
if (!allow_into_outfile)
throw Exception("INTO OUTFILE is not allowed", ErrorCodes::INTO_OUTFILE_NOT_ALLOWED);
const auto & out_file = typeid_cast<const ASTLiteral &>(*ast_query_with_output->out_file).value.safeGet<std::string>();
out_file_buf.emplace(out_file, DBMS_DEFAULT_BUFFER_SIZE, O_WRONLY | O_EXCL | O_CREAT);
out_buf = &*out_file_buf;
}
String format_name = ast_query_with_output && (ast_query_with_output->format != nullptr)
? *getIdentifierName(ast_query_with_output->format)
: context.getDefaultFormat();
if (ast_query_with_output && ast_query_with_output->settings_ast)
InterpreterSetQuery(ast_query_with_output->settings_ast, context).executeForCurrentContext();
pipeline.addSimpleTransform([](const Block & header)
{
return std::make_shared<MaterializingTransform>(header);
});
auto out = context.getOutputFormatProcessor(format_name, *out_buf, pipeline.getHeader());
/// Save previous progress callback if any. TODO Do it more conveniently.
auto previous_progress_callback = context.getProgressCallback();
/// NOTE Progress callback takes shared ownership of 'out'.
pipeline.setProgressCallback([out, previous_progress_callback] (const Progress & progress)
{
if (previous_progress_callback)
previous_progress_callback(progress);
out->onProgress(progress);
});
if (set_content_type)
set_content_type(out->getContentType());
if (set_query_id)
set_query_id(context.getClientInfo().current_query_id);
pipeline.setOutput(std::move(out));
{
auto executor = pipeline.execute();
executor->execute(context.getSettingsRef().max_threads);
}
pipeline.finalize();
}
}
catch (...)
{

View File

@ -3,6 +3,7 @@
#include <Core/QueryProcessingStage.h>
#include <DataStreams/BlockIO.h>
#include <Processors/QueryPipeline.h>
namespace DB
{
@ -45,4 +46,13 @@ BlockIO executeQuery(
bool may_have_embedded_data = false /// If insert query may have embedded data
);
QueryPipeline executeQueryWithProcessors(
const String & query, /// Query text without INSERT data. The latter must be written to BlockIO::out.
Context & context, /// DB, tables, data types, storage engines, functions, aggregate functions...
bool internal = false, /// If true, this query is caused by another query and thus needn't be registered in the ProcessList.
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete, /// To which stage the query must be executed.
bool may_have_embedded_data = false /// If insert query may have embedded data
);
}

View File

@ -0,0 +1,4 @@
if (ENABLE_TESTS)
add_subdirectory (tests)
endif ()

View File

@ -0,0 +1,155 @@
#include <Processors/Chunk.h>
#include <IO/WriteHelpers.h>
#include <IO/Operators.h>
namespace DB
{
namespace ErrorCodes
{
extern const int POSITION_OUT_OF_BOUND;
}
Chunk::Chunk(DB::Columns columns_, UInt64 num_rows_) : columns(std::move(columns_)), num_rows(num_rows_)
{
checkNumRowsIsConsistent();
}
Chunk::Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_)
: columns(std::move(columns_)), num_rows(num_rows_), chunk_info(std::move(chunk_info_))
{
checkNumRowsIsConsistent();
}
static Columns unmuteColumns(MutableColumns && mut_columns)
{
Columns columns;
columns.reserve(mut_columns.size());
for (auto & col : mut_columns)
columns.emplace_back(std::move(col));
return columns;
}
Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_)
: columns(unmuteColumns(std::move(columns_))), num_rows(num_rows_)
{
}
Chunk::Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_)
: columns(unmuteColumns(std::move(columns_))), num_rows(num_rows_), chunk_info(std::move(chunk_info_))
{
}
Chunk Chunk::clone() const
{
return Chunk(getColumns(), getNumRows());
}
void Chunk::setColumns(Columns columns_, UInt64 num_rows_)
{
columns = std::move(columns_);
num_rows = num_rows_;
checkNumRowsIsConsistent();
}
void Chunk::setColumns(MutableColumns columns_, UInt64 num_rows_)
{
columns = unmuteColumns(std::move(columns_));
num_rows = num_rows_;
checkNumRowsIsConsistent();
}
void Chunk::checkNumRowsIsConsistent()
{
for (auto & column : columns)
if (column->size() != num_rows)
throw Exception("Invalid number of rows in Chunk column " + column->getName()+ ": expected " +
toString(num_rows) + ", got " + toString(column->size()), ErrorCodes::LOGICAL_ERROR);
}
MutableColumns Chunk::mutateColumns()
{
size_t num_columns = columns.size();
MutableColumns mut_columns(num_columns);
for (size_t i = 0; i < num_columns; ++i)
mut_columns[i] = (*std::move(columns[i])).mutate();
columns.clear();
num_rows = 0;
return mut_columns;
}
MutableColumns Chunk::cloneEmptyColumns() const
{
size_t num_columns = columns.size();
MutableColumns mut_columns(num_columns);
for (size_t i = 0; i < num_columns; ++i)
mut_columns[i] = columns[i]->cloneEmpty();
return mut_columns;
}
Columns Chunk::detachColumns()
{
num_rows = 0;
return std::move(columns);
}
void Chunk::erase(size_t position)
{
if (columns.empty())
throw Exception("Chunk is empty", ErrorCodes::POSITION_OUT_OF_BOUND);
if (position >= columns.size())
throw Exception("Position " + toString(position) + " out of bound in Chunk::erase(), max position = "
+ toString(columns.size() - 1), ErrorCodes::POSITION_OUT_OF_BOUND);
columns.erase(columns.begin() + position);
}
UInt64 Chunk::bytes() const
{
UInt64 res = 0;
for (const auto & column : columns)
res += column->byteSize();
return res;
}
UInt64 Chunk::allocatedBytes() const
{
UInt64 res = 0;
for (const auto & column : columns)
res += column->allocatedBytes();
return res;
}
std::string Chunk::dumpStructure() const
{
WriteBufferFromOwnString out;
for (auto & column : columns)
out << ' ' << column->dumpStructure();
return out.str();
}
void ChunkMissingValues::setBit(size_t column_idx, size_t row_idx)
{
RowsBitMask & mask = rows_mask_by_column_id[column_idx];
mask.resize(row_idx + 1);
mask[row_idx] = true;
}
const ChunkMissingValues::RowsBitMask & ChunkMissingValues::getDefaultsBitmask(size_t column_idx) const
{
static RowsBitMask none;
auto it = rows_mask_by_column_id.find(column_idx);
if (it != rows_mask_by_column_id.end())
return it->second;
return none;
}
}

117
dbms/src/Processors/Chunk.h Normal file
View File

@ -0,0 +1,117 @@
#pragma once
#include <Columns/IColumn.h>
#include <unordered_map>
namespace DB
{
class ChunkInfo
{
public:
virtual ~ChunkInfo() = default;
ChunkInfo() = default;
};
using ChunkInfoPtr = std::shared_ptr<const ChunkInfo>;
class Chunk
{
public:
Chunk() = default;
Chunk(const Chunk & other) = delete;
Chunk(Chunk && other) noexcept
: columns(std::move(other.columns))
, num_rows(other.num_rows)
, chunk_info(std::move(other.chunk_info))
{
other.num_rows = 0;
}
Chunk(Columns columns_, UInt64 num_rows_);
Chunk(Columns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_);
Chunk(MutableColumns columns_, UInt64 num_rows_);
Chunk(MutableColumns columns_, UInt64 num_rows_, ChunkInfoPtr chunk_info_);
Chunk & operator=(const Chunk & other) = delete;
Chunk & operator=(Chunk && other) noexcept
{
columns = std::move(other.columns);
chunk_info = std::move(other.chunk_info);
num_rows = other.num_rows;
other.num_rows = 0;
return *this;
}
Chunk clone() const;
void swap(Chunk & other)
{
columns.swap(other.columns);
chunk_info.swap(other.chunk_info);
std::swap(num_rows, other.num_rows);
}
void clear()
{
num_rows = 0;
columns.clear();
chunk_info.reset();
}
const Columns & getColumns() const { return columns; }
void setColumns(Columns columns_, UInt64 num_rows_);
void setColumns(MutableColumns columns_, UInt64 num_rows_);
Columns detachColumns();
MutableColumns mutateColumns();
/** Get empty columns with the same types as in block. */
MutableColumns cloneEmptyColumns() const;
const ChunkInfoPtr & getChunkInfo() const { return chunk_info; }
void setChunkInfo(ChunkInfoPtr chunk_info_) { chunk_info = std::move(chunk_info_); }
UInt64 getNumRows() const { return num_rows; }
UInt64 getNumColumns() const { return columns.size(); }
bool hasNoRows() const { return num_rows == 0; }
bool hasNoColumns() const { return columns.empty(); }
bool empty() const { return hasNoRows() && hasNoColumns(); }
operator bool() const { return !empty(); }
void erase(size_t position);
UInt64 bytes() const;
UInt64 allocatedBytes() const;
std::string dumpStructure() const;
private:
Columns columns;
UInt64 num_rows = 0;
ChunkInfoPtr chunk_info;
void checkNumRowsIsConsistent();
};
using Chunks = std::vector<Chunk>;
/// Block extension to support delayed defaults. AddingDefaultsProcessor uses it to replace missing values with column defaults.
class ChunkMissingValues : public ChunkInfo
{
public:
using RowsBitMask = std::vector<bool>; /// a bit per row for a column
const RowsBitMask & getDefaultsBitmask(size_t column_idx) const;
void setBit(size_t column_idx, size_t row_idx);
bool empty() const { return rows_mask_by_column_id.empty(); }
size_t size() const { return rows_mask_by_column_id.size(); }
void clear() { rows_mask_by_column_id.clear(); }
private:
using RowsMaskByColumnId = std::unordered_map<size_t, RowsBitMask>;
/// If rows_mask_by_column_id[column_id][row_id] is true related value in Block should be replaced with column default.
/// It could contain less columns and rows then related block.
RowsMaskByColumnId rows_mask_by_column_id;
};
}

View File

@ -0,0 +1,63 @@
#include <Processors/ConcatProcessor.h>
namespace DB
{
ConcatProcessor::Status ConcatProcessor::prepare()
{
auto & output = outputs.front();
/// Check can output.
if (output.isFinished())
{
for (; current_input != inputs.end(); ++current_input)
current_input->close();
return Status::Finished;
}
if (!output.isNeeded())
{
if (current_input != inputs.end())
current_input->setNotNeeded();
return Status::PortFull;
}
if (!output.canPush())
return Status::PortFull;
/// Check can input.
if (current_input == inputs.end())
return Status::Finished;
if (current_input->isFinished())
{
++current_input;
if (current_input == inputs.end())
{
output.finish();
return Status::Finished;
}
}
auto & input = *current_input;
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
/// Move data.
output.push(input.pull());
/// Now, we pushed to output, and it must be full.
return Status::PortFull;
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <Processors/IProcessor.h>
namespace DB
{
/** Has arbitary non zero number of inputs and one output.
* All of them have the same structure.
*
* Pulls all data from first input, then all data from second input, etc...
* Doesn't do any heavy calculations.
* Preserves an order of data.
*/
class ConcatProcessor : public IProcessor
{
public:
ConcatProcessor(const Block & header, size_t num_inputs)
: IProcessor(InputPorts(num_inputs, header), OutputPorts{header}), current_input(inputs.begin())
{
}
String getName() const override { return "Concat"; }
Status prepare() override;
OutputPort & getOutputPort() { return outputs.front(); }
private:
InputPorts::iterator current_input;
};
}

View File

@ -0,0 +1,101 @@
#include <Common/EventCounter.h>
#include <Common/ThreadPool.h>
#include <Processors/Executors/ParallelPipelineExecutor.h>
#include <Processors/Executors/traverse.h>
namespace DB
{
//
//ParallelPipelineExecutor::ParallelPipelineExecutor(const std::vector<ProcessorPtr> & processors, ThreadPool & pool)
// : processors(processors), pool(pool)
//{
//}
//
//
//ParallelPipelineExecutor::Status ParallelPipelineExecutor::prepare()
//{
// current_processor = nullptr;
//
// bool has_someone_to_wait = false;
//
// for (auto & element : processors)
// {
// traverse(*element,
// [&] (IProcessor & processor)
// {
// {
// std::lock_guard lock(mutex);
// if (active_processors.count(&processor))
// {
// has_someone_to_wait = true;
// return Status::Wait;
// }
// }
//
// Status status = processor.prepare();
//
// if (status == Status::Wait)
// has_someone_to_wait = true;
//
// if (status == Status::Ready || status == Status::Async)
// {
// current_processor = &processor;
// current_status = status;
// }
//
// return status;
// });
//
// if (current_processor)
// break;
// }
//
// if (current_processor)
// return Status::Async;
//
// if (has_someone_to_wait)
// return Status::Wait;
//
// for (auto & element : processors)
// {
// if (element->prepare() == Status::NeedData)
// throw Exception("Pipeline stuck: " + element->getName() + " processor needs input data but no one is going to generate it", ErrorCodes::LOGICAL_ERROR);
// if (element->prepare() == Status::PortFull)
// throw Exception("Pipeline stuck: " + element->getName() + " processor has data in output port but no one is going to consume it", ErrorCodes::LOGICAL_ERROR);
// }
//
// return Status::Finished;
//}
//
//
//void ParallelPipelineExecutor::schedule(EventCounter & watch)
//{
// if (!current_processor)
// throw Exception("Bad pipeline", ErrorCodes::LOGICAL_ERROR);
//
// if (current_status == Status::Async)
// {
// current_processor->schedule(watch);
// }
// else
// {
// {
// std::lock_guard lock(mutex);
// active_processors.insert(current_processor);
// }
//
// pool.schedule([processor = current_processor, &watch, this]
// {
// processor->work();
// {
// std::lock_guard lock(mutex);
// active_processors.erase(processor);
// }
// watch.notify();
// });
// }
//}
}

View File

@ -0,0 +1,41 @@
#pragma once
#include <vector>
#include <set>
#include <mutex>
#include <Processors/IProcessor.h>
template <typename>
class ThreadPoolImpl;
class ThreadFromGlobalPool;
using ThreadPool = ThreadPoolImpl<ThreadFromGlobalPool>;
namespace DB
{
/** Wraps pipeline in a single processor.
* This processor has no inputs and outputs and just executes the pipeline,
* performing all synchronous work within a threadpool.
*/
//class ParallelPipelineExecutor : public IProcessor
//{
//private:
// Processors processors;
// ThreadPool & pool;
//
// std::set<IProcessor *> active_processors;
// std::mutex mutex;
//
// IProcessor * current_processor = nullptr;
// Status current_status;
//
//public:
// ParallelPipelineExecutor(const Processors & processors, ThreadPool & pool);
//
// String getName() const override { return "ParallelPipelineExecutor"; }
//
// Status prepare() override;
// void schedule(EventCounter & watch) override;
//};
}

View File

@ -0,0 +1,653 @@
#include <Processors/Executors/PipelineExecutor.h>
#include <unordered_map>
#include <queue>
#include <IO/WriteBufferFromString.h>
#include <Processors/printPipeline.h>
#include <Common/EventCounter.h>
#include <ext/scope_guard.h>
#include <Common/CurrentThread.h>
#include <boost/lockfree/queue.hpp>
#include <Common/Stopwatch.h>
namespace DB
{
namespace ErrorCodes
{
extern const int TOO_MANY_ROWS_OR_BYTES;
extern const int QUOTA_EXPIRED;
extern const int QUERY_WAS_CANCELLED;
}
static bool checkCanAddAdditionalInfoToException(const DB::Exception & exception)
{
/// Don't add additional info to limits and quota exceptions, and in case of kill query (to pass tests).
return exception.code() != ErrorCodes::TOO_MANY_ROWS_OR_BYTES
&& exception.code() != ErrorCodes::QUOTA_EXPIRED
&& exception.code() != ErrorCodes::QUERY_WAS_CANCELLED;
}
PipelineExecutor::PipelineExecutor(Processors & processors)
: processors(processors)
, cancelled(false)
, finished(false)
, num_processing_executors(0)
, expand_pipeline_task(nullptr)
{
buildGraph();
}
bool PipelineExecutor::addEdges(UInt64 node)
{
auto throwUnknownProcessor = [](const IProcessor * proc, const IProcessor * parent, bool from_input_port)
{
String msg = "Processor " + proc->getName() + " was found as " + (from_input_port ? "input" : "output")
+ " for processor " + parent->getName() + ", but not found in list of processors.";
throw Exception(msg, ErrorCodes::LOGICAL_ERROR);
};
const IProcessor * cur = graph[node].processor;
auto add_edge = [&](auto & from_port, const IProcessor * to_proc, Edges & edges)
{
auto it = processors_map.find(to_proc);
if (it == processors_map.end())
throwUnknownProcessor(to_proc, cur, true);
UInt64 proc_num = it->second;
Edge * edge_ptr = nullptr;
for (auto & edge : edges)
if (edge.to == proc_num)
edge_ptr = &edge;
if (!edge_ptr)
{
edge_ptr = &edges.emplace_back();
edge_ptr->to = proc_num;
}
from_port.setVersion(&edge_ptr->version);
};
bool was_edge_added = false;
auto & inputs = processors[node]->getInputs();
auto from_input = graph[node].backEdges.size();
if (from_input < inputs.size())
{
was_edge_added = true;
for (auto it = std::next(inputs.begin(), from_input); it != inputs.end(); ++it)
{
const IProcessor * proc = &it->getOutputPort().getProcessor();
add_edge(*it, proc, graph[node].backEdges);
}
}
auto & outputs = processors[node]->getOutputs();
auto from_output = graph[node].directEdges.size();
if (from_output < outputs.size())
{
was_edge_added = true;
for (auto it = std::next(outputs.begin(), from_output); it != outputs.end(); ++it)
{
const IProcessor * proc = &it->getInputPort().getProcessor();
add_edge(*it, proc, graph[node].directEdges);
}
}
return was_edge_added;
}
void PipelineExecutor::buildGraph()
{
UInt64 num_processors = processors.size();
graph.reserve(num_processors);
for (UInt64 node = 0; node < num_processors; ++node)
{
IProcessor * proc = processors[node].get();
processors_map[proc] = node;
graph.emplace_back(proc, node);
}
for (UInt64 node = 0; node < num_processors; ++node)
addEdges(node);
}
void PipelineExecutor::addChildlessProcessorsToStack(Stack & stack)
{
UInt64 num_processors = processors.size();
for (UInt64 proc = 0; proc < num_processors; ++proc)
{
if (graph[proc].directEdges.empty())
{
stack.push(proc);
graph[proc].status = ExecStatus::Preparing;
}
}
}
static void executeJob(IProcessor * processor)
{
try
{
processor->work();
}
catch (Exception & exception)
{
if (checkCanAddAdditionalInfoToException(exception))
exception.addMessage("While executing " + processor->getName() + " ("
+ toString(reinterpret_cast<std::uintptr_t>(processor)) + ") ");
throw;
}
}
void PipelineExecutor::addJob(ExecutionState * execution_state)
{
auto job = [execution_state]()
{
try
{
Stopwatch watch;
executeJob(execution_state->processor);
execution_state->execution_time_ns += watch.elapsed();
++execution_state->num_executed_jobs;
}
catch (...)
{
execution_state->exception = std::current_exception();
}
};
execution_state->job = std::move(job);
}
void PipelineExecutor::expandPipeline(Stack & stack, UInt64 pid)
{
auto & cur_node = graph[pid];
auto new_processors = cur_node.processor->expandPipeline();
for (const auto & processor : new_processors)
{
if (processors_map.count(processor.get()))
throw Exception("Processor " + processor->getName() + " was already added to pipeline.",
ErrorCodes::LOGICAL_ERROR);
processors_map[processor.get()] = graph.size();
graph.emplace_back(processor.get(), graph.size());
}
processors.insert(processors.end(), new_processors.begin(), new_processors.end());
UInt64 num_processors = processors.size();
for (UInt64 node = 0; node < num_processors; ++node)
{
if (addEdges(node))
{
if (graph[node].status == ExecStatus::Idle || graph[node].status == ExecStatus::New)
{
graph[node].status = ExecStatus::Preparing;
stack.push(node);
}
}
}
}
bool PipelineExecutor::tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stack)
{
/// In this method we have ownership on edge, but node can be concurrently accessed.
auto & node = graph[edge.to];
ExecStatus status = node.status.load();
/// Don't add processor if nothing was read from port.
if (status != ExecStatus::New && edge.version == edge.prev_version)
return false;
if (status == ExecStatus::Finished)
return false;
/// Signal that node need to be prepared.
node.need_to_be_prepared = true;
edge.prev_version = edge.version;
/// Try to get ownership for node.
/// Assume that current status is New or Idle. Otherwise, can't prepare node.
if (status != ExecStatus::New)
status = ExecStatus::Idle;
/// Statuses but New and Idle are not interesting because they own node.
/// Prepare will be called in owning thread before changing status.
while (!node.status.compare_exchange_weak(status, ExecStatus::Preparing))
if (!(status == ExecStatus::New || status == ExecStatus::Idle) || !node.need_to_be_prepared)
return false;
stack.push(edge.to);
return true;
}
bool PipelineExecutor::prepareProcessor(UInt64 pid, Stack & children, Stack & parents, size_t thread_number, bool async)
{
/// In this method we have ownership on node.
auto & node = graph[pid];
{
/// Stopwatch watch;
/// Disable flag before prepare call. Otherwise, we can skip prepare request.
/// Prepare can be called more times than needed, but it's ok.
node.need_to_be_prepared = false;
auto status = node.processor->prepare();
/// node.execution_state->preparation_time_ns += watch.elapsed();
node.last_processor_status = status;
}
auto add_neighbours_to_prepare_queue = [&] ()
{
for (auto & edge : node.backEdges)
tryAddProcessorToStackIfUpdated(edge, parents);
for (auto & edge : node.directEdges)
tryAddProcessorToStackIfUpdated(edge, children);
};
auto try_release_ownership = [&] ()
{
/// This function can be called after expand pipeline, where node from outer scope is not longer valid.
auto & node_ = graph[pid];
ExecStatus expected = ExecStatus::Idle;
node_.status = ExecStatus::Idle;
if (node_.need_to_be_prepared)
{
while (!node_.status.compare_exchange_weak(expected, ExecStatus::Preparing))
if (!(expected == ExecStatus::Idle) || !node_.need_to_be_prepared)
return;
children.push(pid);
}
};
switch (node.last_processor_status)
{
case IProcessor::Status::NeedData:
{
add_neighbours_to_prepare_queue();
try_release_ownership();
break;
}
case IProcessor::Status::PortFull:
{
add_neighbours_to_prepare_queue();
try_release_ownership();
break;
}
case IProcessor::Status::Finished:
{
add_neighbours_to_prepare_queue();
node.status = ExecStatus::Finished;
break;
}
case IProcessor::Status::Ready:
{
node.status = ExecStatus::Executing;
return true;
}
case IProcessor::Status::Async:
{
throw Exception("Async is temporary not supported.", ErrorCodes::LOGICAL_ERROR);
// node.status = ExecStatus::Executing;
// addAsyncJob(pid);
// break;
}
case IProcessor::Status::Wait:
{
if (!async)
throw Exception("Processor returned status Wait before Async.", ErrorCodes::LOGICAL_ERROR);
break;
}
case IProcessor::Status::ExpandPipeline:
{
executor_contexts[thread_number]->task_list.emplace_back(
node.execution_state.get(),
&parents
);
ExpandPipelineTask * desired = &executor_contexts[thread_number]->task_list.back();
ExpandPipelineTask * expected = nullptr;
while (!expand_pipeline_task.compare_exchange_strong(expected, desired))
{
doExpandPipeline(expected, true);
expected = nullptr;
}
doExpandPipeline(desired, true);
/// node is not longer valid after pipeline was expanded
graph[pid].need_to_be_prepared = true;
try_release_ownership();
break;
}
}
return false;
}
void PipelineExecutor::doExpandPipeline(ExpandPipelineTask * task, bool processing)
{
std::unique_lock lock(task->mutex);
if (processing)
++task->num_waiting_processing_threads;
task->condvar.wait(lock, [&]()
{
return task->num_waiting_processing_threads >= num_processing_executors || expand_pipeline_task != task;
});
/// After condvar.wait() task may point to trash. Can change it only if it is still in expand_pipeline_task.
if (expand_pipeline_task == task)
{
expandPipeline(*task->stack, task->node_to_expand->processors_id);
expand_pipeline_task = nullptr;
lock.unlock();
task->condvar.notify_all();
}
}
void PipelineExecutor::finish()
{
{
std::lock_guard lock(task_queue_mutex);
finished = true;
}
task_queue_condvar.notify_all();
}
void PipelineExecutor::execute(size_t num_threads)
{
try
{
executeImpl(num_threads);
/// Execution can be stopped because of exception. Check and rethrow if any.
for (auto & node : graph)
if (node.execution_state->exception)
std::rethrow_exception(node.execution_state->exception);
}
catch (Exception & exception)
{
if (checkCanAddAdditionalInfoToException(exception))
exception.addMessage("\nCurrent state:\n" + dumpPipeline());
throw;
}
if (cancelled)
return;
bool all_processors_finished = true;
for (auto & node : graph)
if (node.status != ExecStatus::Finished)
all_processors_finished = false;
if (!all_processors_finished)
throw Exception("Pipeline stuck. Current state:\n" + dumpPipeline(), ErrorCodes::LOGICAL_ERROR);
}
void PipelineExecutor::executeSingleThread(size_t thread_num, size_t num_threads)
{
UInt64 total_time_ns = 0;
UInt64 execution_time_ns = 0;
UInt64 processing_time_ns = 0;
UInt64 wait_time_ns = 0;
Stopwatch total_time_watch;
ExecutionState * state = nullptr;
auto prepare_processor = [&](UInt64 pid, Stack & children, Stack & parents)
{
try
{
return prepareProcessor(pid, children, parents, thread_num, false);
}
catch (...)
{
graph[pid].execution_state->exception = std::current_exception();
finish();
}
return false;
};
using Queue = std::queue<ExecutionState *>;
auto prepare_all_processors = [&](Queue & queue, Stack & stack, Stack & children, Stack & parents)
{
while (!stack.empty() && !finished)
{
auto current_processor = stack.top();
stack.pop();
if (prepare_processor(current_processor, children, parents))
queue.push(graph[current_processor].execution_state.get());
}
};
while (!finished)
{
/// First, find any processor to execute.
/// Just travers graph and prepare any processor.
while (!finished)
{
std::unique_lock lock(task_queue_mutex);
if (!task_queue.empty())
{
state = task_queue.front();
task_queue.pop();
break;
}
++num_waiting_threads;
if (num_waiting_threads == num_threads)
{
finished = true;
lock.unlock();
task_queue_condvar.notify_all();
break;
}
task_queue_condvar.wait(lock, [&]()
{
return finished || !task_queue.empty();
});
--num_waiting_threads;
}
if (finished)
break;
while (state)
{
if (finished)
break;
addJob(state);
{
Stopwatch execution_time_watch;
state->job();
execution_time_ns += execution_time_watch.elapsed();
}
if (state->exception)
finish();
if (finished)
break;
Stopwatch processing_time_watch;
/// Try to execute neighbour processor.
{
Stack children;
Stack parents;
Queue queue;
++num_processing_executors;
while (auto task = expand_pipeline_task.load())
doExpandPipeline(task, true);
/// Execute again if can.
if (!prepare_processor(state->processors_id, children, parents))
state = nullptr;
/// Process all neighbours. Children will be on the top of stack, then parents.
prepare_all_processors(queue, children, children, parents);
if (!state && !queue.empty())
{
state = queue.front();
queue.pop();
}
prepare_all_processors(queue, parents, parents, parents);
if (!queue.empty())
{
std::lock_guard lock(task_queue_mutex);
while (!queue.empty() && !finished)
{
task_queue.push(queue.front());
queue.pop();
}
task_queue_condvar.notify_all();
}
--num_processing_executors;
while (auto task = expand_pipeline_task.load())
doExpandPipeline(task, false);
}
processing_time_ns += processing_time_watch.elapsed();
}
}
total_time_ns = total_time_watch.elapsed();
wait_time_ns = total_time_ns - execution_time_ns - processing_time_ns;
LOG_TRACE(log, "Thread finished."
<< " Total time: " << (total_time_ns / 1e9) << " sec."
<< " Execution time: " << (execution_time_ns / 1e9) << " sec."
<< " Processing time: " << (processing_time_ns / 1e9) << " sec."
<< " Wait time: " << (wait_time_ns / 1e9) << "sec.");
}
void PipelineExecutor::executeImpl(size_t num_threads)
{
Stack stack;
executor_contexts.reserve(num_threads);
for (size_t i = 0; i < num_threads; ++i)
executor_contexts.emplace_back(std::make_unique<ExecutorContext>());
addChildlessProcessorsToStack(stack);
while (!stack.empty())
{
UInt64 proc = stack.top();
stack.pop();
if (prepareProcessor(proc, stack, stack, 0, false))
{
auto cur_state = graph[proc].execution_state.get();
task_queue.push(cur_state);
}
}
ThreadPool pool(num_threads);
SCOPE_EXIT(
finish();
pool.wait()
);
auto thread_group = CurrentThread::getGroup();
for (size_t i = 0; i < num_threads; ++i)
{
pool.schedule([this, thread_group, thread_num = i, num_threads]
{
/// ThreadStatus thread_status;
if (thread_group)
CurrentThread::attachTo(thread_group);
SCOPE_EXIT(
if (thread_group)
CurrentThread::detachQueryIfNotDetached();
);
executeSingleThread(thread_num, num_threads);
});
}
pool.wait();
}
String PipelineExecutor::dumpPipeline() const
{
for (auto & node : graph)
{
if (node.execution_state)
node.processor->setDescription(
"(" + std::to_string(node.execution_state->num_executed_jobs) + " jobs, execution time: "
+ std::to_string(node.execution_state->execution_time_ns / 1e9) + " sec., preparation time: "
+ std::to_string(node.execution_state->preparation_time_ns / 1e9) + " sec.)");
}
std::vector<IProcessor::Status> statuses;
std::vector<IProcessor *> proc_list;
statuses.reserve(graph.size());
proc_list.reserve(graph.size());
for (auto & proc : graph)
{
proc_list.emplace_back(proc.processor);
statuses.emplace_back(proc.last_processor_status);
}
WriteBufferFromOwnString out;
printPipeline(processors, statuses, out);
out.finish();
return out.str();
}
}

View File

@ -0,0 +1,195 @@
#pragma once
#include <queue>
#include <stack>
#include <Processors/IProcessor.h>
#include <mutex>
#include <Common/ThreadPool.h>
#include <Common/EventCounter.h>
#include <common/logger_useful.h>
#include <boost/lockfree/stack.hpp>
namespace DB
{
/// Executes query pipeline.
class PipelineExecutor
{
public:
/// Get pipeline as a set of processors.
/// Processors should represent full graph. All ports must be connected, all connected nodes are mentioned in set.
/// Executor doesn't own processors, just stores reference.
/// During pipeline execution new processors can appear. They will be added to existing set.
///
/// Explicit graph representation is built in constructor. Throws if graph is not correct.
explicit PipelineExecutor(Processors & processors);
/// Execute pipeline in multiple threads. Must be called once.
/// In case of exception during execution throws any occurred.
void execute(size_t num_threads);
String getName() const { return "PipelineExecutor"; }
const Processors & getProcessors() const { return processors; }
/// Cancel execution. May be called from another thread.
void cancel()
{
cancelled = true;
finish();
}
private:
Processors & processors;
struct Edge
{
UInt64 to = std::numeric_limits<UInt64>::max();
/// Edge version is increased when port's state is changed (e.g. when data is pushed). See Port.h for details.
/// To compare version with prev_version we can decide if neighbour processor need to be prepared.
UInt64 version = 0;
UInt64 prev_version = 0;
};
/// Use std::list because new ports can be added to processor during execution.
using Edges = std::list<Edge>;
/// Status for processor.
/// Can be owning or not. Owning means that executor who set this status can change node's data and nobody else can.
enum class ExecStatus
{
New, /// prepare wasn't called yet. Initial state. Non-owning.
Idle, /// prepare returned NeedData or PortFull. Non-owning.
Preparing, /// some executor is preparing processor, or processor is in task_queue. Owning.
Executing, /// prepare returned Ready and task is executing. Owning.
Finished, /// prepare returned Finished. Non-owning.
Async /// prepare returned Async. Owning.
};
/// Small structure with context of executing job.
struct ExecutionState
{
std::exception_ptr exception;
std::function<void()> job;
IProcessor * processor;
UInt64 processors_id;
/// Counters for profiling.
size_t num_executed_jobs = 0;
UInt64 execution_time_ns = 0;
UInt64 preparation_time_ns = 0;
};
struct Node
{
IProcessor * processor = nullptr;
Edges directEdges;
Edges backEdges;
std::atomic<ExecStatus> status;
/// This flag can be set by any executor.
/// When enabled, any executor can try to atomically set Preparing state to status.
std::atomic_bool need_to_be_prepared;
/// Last state for profiling.
IProcessor::Status last_processor_status = IProcessor::Status::NeedData;
std::unique_ptr<ExecutionState> execution_state;
Node(IProcessor * processor_, UInt64 processor_id)
: processor(processor_), status(ExecStatus::New), need_to_be_prepared(false)
{
execution_state = std::make_unique<ExecutionState>();
execution_state->processor = processor;
execution_state->processors_id = processor_id;
}
Node(Node && other) noexcept
: processor(other.processor), status(other.status.load())
, need_to_be_prepared(other.need_to_be_prepared.load()), execution_state(std::move(other.execution_state))
{
}
};
using Nodes = std::vector<Node>;
Nodes graph;
using Stack = std::stack<UInt64>;
using TaskQueue = std::queue<ExecutionState *>;
/// Queue with pointers to tasks. Each thread will concurrently read from it until finished flag is set.
/// Stores processors need to be prepared. Preparing status is already set for them.
TaskQueue task_queue;
std::mutex task_queue_mutex;
std::condition_variable task_queue_condvar;
std::atomic_bool cancelled;
std::atomic_bool finished;
Poco::Logger * log = &Poco::Logger::get("PipelineExecutor");
/// Num threads waiting condvar. Last thread finish execution if task_queue is empty.
size_t num_waiting_threads = 0;
/// Things to stop execution to expand pipeline.
struct ExpandPipelineTask
{
ExecutionState * node_to_expand;
Stack * stack;
size_t num_waiting_processing_threads = 0;
std::mutex mutex;
std::condition_variable condvar;
ExpandPipelineTask(ExecutionState * node_to_expand_, Stack * stack_)
: node_to_expand(node_to_expand_), stack(stack_) {}
};
std::atomic<size_t> num_processing_executors;
std::atomic<ExpandPipelineTask *> expand_pipeline_task;
/// Context for each thread.
struct ExecutorContext
{
/// Will store context for all expand pipeline tasks (it's easy and we don't expect many).
/// This can be solved by using atomic shard ptr.
std::list<ExpandPipelineTask> task_list;
};
std::vector<std::unique_ptr<ExecutorContext>> executor_contexts;
/// Processor ptr -> node number
using ProcessorsMap = std::unordered_map<const IProcessor *, UInt64>;
ProcessorsMap processors_map;
/// Graph related methods.
bool addEdges(UInt64 node);
void buildGraph();
void expandPipeline(Stack & stack, UInt64 pid);
/// Pipeline execution related methods.
void addChildlessProcessorsToStack(Stack & stack);
bool tryAddProcessorToStackIfUpdated(Edge & edge, Stack & stack);
static void addJob(ExecutionState * execution_state);
// TODO: void addAsyncJob(UInt64 pid);
/// Prepare processor with pid number.
/// Check parents and children of current processor and push them to stacks if they also need to be prepared.
/// If processor wants to be expanded, ExpandPipelineTask from thread_number's execution context will be used.
bool prepareProcessor(size_t pid, Stack & children, Stack & parents, size_t thread_number, bool async);
void doExpandPipeline(ExpandPipelineTask * task, bool processing);
void executeImpl(size_t num_threads);
void executeSingleThread(size_t thread_num, size_t num_threads);
void finish();
String dumpPipeline() const;
};
using PipelineExecutorPtr = std::shared_ptr<PipelineExecutor>;
}

View File

@ -0,0 +1,79 @@
#include <Processors/Executors/SequentialPipelineExecutor.h>
#include <Processors/Executors/traverse.h>
namespace DB
{
//SequentialPipelineExecutor::SequentialPipelineExecutor(const Processors & processors)
// : processors(processors)
//{
//}
//
//
//SequentialPipelineExecutor::Status SequentialPipelineExecutor::prepare()
//{
// current_processor = nullptr;
//
// bool has_someone_to_wait = false;
// Status found_status = Status::Finished;
//
// for (auto & element : processors)
// {
// traverse(*element,
// [&] (IProcessor & processor)
// {
// Status status = processor.prepare();
//
// if (status == Status::Wait)
// has_someone_to_wait = true;
//
// if (status == Status::Ready || status == Status::Async)
// {
// current_processor = &processor;
// found_status = status;
// }
//
// return status;
// });
//
// if (current_processor)
// break;
// }
//
// if (current_processor)
// return found_status;
// if (has_someone_to_wait)
// return Status::Wait;
//
// for (auto & element : processors)
// {
// if (element->prepare() == Status::NeedData)
// throw Exception("Pipeline stuck: " + element->getName() + " processor needs input data but no one is going to generate it", ErrorCodes::LOGICAL_ERROR);
// if (element->prepare() == Status::PortFull)
// throw Exception("Pipeline stuck: " + element->getName() + " processor has data in output port but no one is going to consume it", ErrorCodes::LOGICAL_ERROR);
// }
//
// return Status::Finished;
//}
//
//
//void SequentialPipelineExecutor::work()
//{
// if (!current_processor)
// throw Exception("Bad pipeline", ErrorCodes::LOGICAL_ERROR);
//
// current_processor->work();
//}
//
//
//void SequentialPipelineExecutor::schedule(EventCounter & watch)
//{
// if (!current_processor)
// throw Exception("Bad pipeline", ErrorCodes::LOGICAL_ERROR);
//
// current_processor->schedule(watch);
//}
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <vector>
#include <Processors/IProcessor.h>
namespace DB
{
/** Wraps pipeline in a single processor.
* This processor has no inputs and outputs and just executes the pipeline,
* performing all synchronous work from the current thread.
*/
//class SequentialPipelineExecutor : public IProcessor
//{
//private:
// Processors processors;
// IProcessor * current_processor = nullptr;
//
//public:
// SequentialPipelineExecutor(const Processors & processors);
//
// String getName() const override { return "SequentialPipelineExecutor"; }
//
// Status prepare() override;
// void work() override;
// void schedule(EventCounter & watch) override;
//};
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Processors/IProcessor.h>
namespace DB
{
/// Look for first Ready or Async processor by depth-first search in needed input ports and full output ports.
/// NOTE: Pipeline must not have cycles.
//template <typename Visit>
//void traverse(IProcessor & processor, Visit && visit)
//{
// IProcessor::Status status = visit(processor);
//
// if (status == IProcessor::Status::Ready || status == IProcessor::Status::Async)
// return;
//
// if (status == IProcessor::Status::NeedData)
// for (auto & input : processor.getInputs())
// if (input.isNeeded() && !input.hasData())
// traverse(input.getOutputPort().getProcessor(), std::forward<Visit>(visit));
//
// if (status == IProcessor::Status::PortFull)
// for (auto & output : processor.getOutputs())
// if (output.hasData())
// traverse(output.getInputPort().getProcessor(), std::forward<Visit>(visit));
//}
}

View File

@ -0,0 +1,81 @@
#include <Processors/ForkProcessor.h>
namespace DB
{
ForkProcessor::Status ForkProcessor::prepare()
{
auto & input = inputs.front();
/// Check can output.
bool all_finished = true;
bool all_can_push = true;
size_t num_active_outputs = 0;
for (const auto & output : outputs)
{
if (!output.isFinished())
{
all_finished = false;
++num_active_outputs;
/// The order is important.
if (!output.canPush())
all_can_push = false;
}
}
if (all_finished)
{
input.close();
return Status::Finished;
}
if (!all_can_push)
{
input.setNotNeeded();
return Status::PortFull;
}
/// Check can input.
if (input.isFinished())
{
for (auto & output : outputs)
output.finish();
return Status::Finished;
}
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
/// Move data.
auto data = input.pull();
size_t num_processed_outputs = 0;
for (auto & output : outputs)
{
if (!output.isFinished()) /// Skip finished outputs.
{
++num_processed_outputs;
if (num_processed_outputs == num_active_outputs)
output.push(std::move(data)); /// Can push because no full or unneeded outputs.
else
output.push(data.clone());
}
}
/// Now, we pulled from input. It must be empty.
return Status::NeedData;
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <Processors/IProcessor.h>
namespace DB
{
/** Has one input and arbitrary non zero number of outputs.
* All of them have the same structure.
*
* Pulls data input and copies it to every output.
* You may have heard about it under the name 'tee'.
*
* Doesn't do any heavy calculations.
* Preserves an order of data.
*/
class ForkProcessor : public IProcessor
{
public:
ForkProcessor(const Block & header, size_t num_outputs)
: IProcessor(InputPorts{header}, OutputPorts(num_outputs, header))
{
}
String getName() const override { return "Fork"; }
Status prepare() override;
InputPort & getInputPort() { return inputs.front(); }
};
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <Processors/ISource.h>
namespace DB
{
class ReadBuffer;
/** Input format is a source, that reads data from ReadBuffer.
*/
class IInputFormat : public ISource
{
protected:
/// Skip GCC warning: maybe_unused attribute ignored
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wattributes"
ReadBuffer & in [[maybe_unused]];
#pragma GCC diagnostic pop
public:
IInputFormat(Block header, ReadBuffer & in)
: ISource(std::move(header)), in(in)
{
}
};
}

View File

@ -0,0 +1,78 @@
#include <Processors/Formats/IOutputFormat.h>
#include <IO/WriteBuffer.h>
namespace DB
{
IOutputFormat::IOutputFormat(const Block & header, WriteBuffer & out)
: IProcessor({header, header, header}, {}), out(out)
{
}
IOutputFormat::Status IOutputFormat::prepare()
{
if (has_input)
return Status::Ready;
for (auto kind : {Main, Totals, Extremes})
{
auto & input = getPort(kind);
if (kind != Main && !input.isConnected())
continue;
if (input.isFinished())
continue;
input.setNeeded();
if (!input.hasData())
return Status::NeedData;
current_chunk = input.pull();
current_block_kind = kind;
has_input = true;
return Status::Ready;
}
finished = true;
if (!finalized)
return Status::Ready;
return Status::Finished;
}
void IOutputFormat::work()
{
if (finished && !finalized)
{
finalize();
finalized = true;
return;
}
switch (current_block_kind)
{
case Main:
consume(std::move(current_chunk));
break;
case Totals:
consumeTotals(std::move(current_chunk));
break;
case Extremes:
consumeExtremes(std::move(current_chunk));
break;
}
has_input = false;
}
void IOutputFormat::flush()
{
out.next();
}
}

View File

@ -0,0 +1,63 @@
#pragma once
#include <string>
#include <Processors/IProcessor.h>
#include <IO/Progress.h>
namespace DB
{
class WriteBuffer;
/** Output format have three inputs and no outputs. It writes data from WriteBuffer.
*
* First input is for main resultset, second is for "totals" and third is for "extremes".
* It's not necessarily to connect "totals" or "extremes" ports (they may remain dangling).
*
* Data from input ports are pulled in order: first, from main input, then totals, then extremes.
*
* By default, data for "totals" and "extremes" is ignored.
*/
class IOutputFormat : public IProcessor
{
public:
enum PortKind { Main = 0, Totals = 1, Extremes = 2 };
protected:
WriteBuffer & out;
Chunk current_chunk;
PortKind current_block_kind = PortKind::Main;
bool has_input = false;
bool finished = false;
bool finalized = false;
virtual void consume(Chunk) = 0;
virtual void consumeTotals(Chunk) {}
virtual void consumeExtremes(Chunk) {}
virtual void finalize() {}
public:
IOutputFormat(const Block & header, WriteBuffer & out);
Status prepare() override;
void work() override;
/// Flush output buffers if any.
virtual void flush();
/// Value for rows_before_limit_at_least field.
virtual void setRowsBeforeLimit(size_t /*rows_before_limit*/) {}
/// Notify about progress. Method could be called from different threads.
/// Passed value are delta, that must be summarized.
virtual void onProgress(const Progress & /*progress*/) {}
/// Content-Type to set when sending HTTP response.
virtual std::string getContentType() const { return "text/plain; charset=UTF-8"; }
InputPort & getPort(PortKind kind) { return *std::next(inputs.begin(), kind); }
};
}

View File

@ -0,0 +1,151 @@
#include <Processors/Formats/IRowInputFormat.h>
#include <IO/WriteHelpers.h> // toString
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int CANNOT_PARSE_DATE;
extern const int CANNOT_PARSE_DATETIME;
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
extern const int CANNOT_PARSE_NUMBER;
extern const int CANNOT_PARSE_UUID;
extern const int TOO_LARGE_STRING_SIZE;
extern const int INCORRECT_NUMBER_OF_COLUMNS;
}
static bool isParseError(int code)
{
return code == ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED
|| code == ErrorCodes::CANNOT_PARSE_QUOTED_STRING
|| code == ErrorCodes::CANNOT_PARSE_DATE
|| code == ErrorCodes::CANNOT_PARSE_DATETIME
|| code == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT
|| code == ErrorCodes::CANNOT_PARSE_NUMBER
|| code == ErrorCodes::CANNOT_PARSE_UUID
|| code == ErrorCodes::TOO_LARGE_STRING_SIZE;
}
Chunk IRowInputFormat::generate()
{
if (total_rows == 0)
readPrefix();
const Block & header = getPort().getHeader();
size_t num_columns = header.columns();
MutableColumns columns = header.cloneEmptyColumns();
size_t prev_rows = total_rows;
auto chunk_missing_values = std::make_unique<ChunkMissingValues>();
try
{
for (size_t rows = 0; rows < params.max_block_size; ++rows)
{
try
{
++total_rows;
RowReadExtension info;
if (!readRow(columns, info))
break;
for (size_t column_idx = 0; column_idx < info.read_columns.size(); ++column_idx)
{
if (!info.read_columns[column_idx])
{
size_t column_size = columns[column_idx]->size();
if (column_size == 0)
throw Exception("Unexpected empty column", ErrorCodes::INCORRECT_NUMBER_OF_COLUMNS);
chunk_missing_values->setBit(column_idx, column_size - 1);
}
}
}
catch (Exception & e)
{
/// Logic for possible skipping of errors.
if (!isParseError(e.code()))
throw;
if (params.allow_errors_num == 0 && params.allow_errors_ratio == 0)
throw;
++num_errors;
Float64 current_error_ratio = static_cast<Float64>(num_errors) / total_rows;
if (num_errors > params.allow_errors_num
&& current_error_ratio > params.allow_errors_ratio)
{
e.addMessage("(Already have " + toString(num_errors) + " errors"
" out of " + toString(total_rows) + " rows"
", which is " + toString(current_error_ratio) + " of all rows)");
throw;
}
if (!allowSyncAfterError())
{
e.addMessage("(Input format doesn't allow to skip errors)");
throw;
}
syncAfterError();
/// Truncate all columns in block to minimal size (remove values, that was appended to only part of columns).
size_t min_size = std::numeric_limits<size_t>::max();
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
min_size = std::min(min_size, columns[column_idx]->size());
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
{
auto & column = columns[column_idx];
if (column->size() > min_size)
column->popBack(column->size() - min_size);
}
}
}
}
catch (Exception & e)
{
if (!isParseError(e.code()))
throw;
String verbose_diagnostic;
try
{
verbose_diagnostic = getDiagnosticInfo();
}
catch (...)
{
/// Error while trying to obtain verbose diagnostic. Ok to ignore.
}
e.addMessage("(at row " + toString(total_rows) + ")\n" + verbose_diagnostic);
throw;
}
if (columns.empty() || columns[0]->empty())
{
readSuffix();
return {};
}
Chunk chunk(std::move(columns), total_rows - prev_rows);
chunk.setChunkInfo(std::move(chunk_missing_values));
return chunk;
}
void IRowInputFormat::syncAfterError()
{
throw Exception("Method syncAfterError is not implemented for input format", ErrorCodes::NOT_IMPLEMENTED);
}
}

View File

@ -0,0 +1,72 @@
#pragma once
#include <string>
#include <Columns/IColumn.h>
#include <Processors/Formats/IInputFormat.h>
namespace DB
{
/// Contains extra information about read data.
struct RowReadExtension
{
/// IRowInputStream.read() output. It contains non zero for columns that actually read from the source and zero otherwise.
/// It's used to attach defaults for partially filled rows.
std::vector<UInt8> read_columns;
};
/// Common parameters for generating blocks.
struct RowInputFormatParams
{
size_t max_block_size;
UInt64 allow_errors_num;
Float64 allow_errors_ratio;
};
///Row oriented input format: reads data row by row.
class IRowInputFormat : public IInputFormat
{
public:
using Params = RowInputFormatParams;
IRowInputFormat(
Block header,
ReadBuffer & in_,
Params params)
: IInputFormat(std::move(header), in_), params(params)
{
}
Chunk generate() override;
protected:
/** Read next row and append it to the columns.
* If no more rows - return false.
*/
virtual bool readRow(MutableColumns & columns, RowReadExtension & extra) = 0;
virtual void readPrefix() {} /// delimiter before begin of result
virtual void readSuffix() {} /// delimiter after end of result
/// Skip data until next row.
/// This is intended for text streams, that allow skipping of errors.
/// By default - throws not implemented exception.
virtual bool allowSyncAfterError() const { return false; }
virtual void syncAfterError();
/// In case of parse error, try to roll back and parse last one or two rows very carefully
/// and collect as much as possible diagnostic information about error.
/// If not implemented, returns empty string.
virtual std::string getDiagnosticInfo() { return {}; }
private:
Params params;
size_t total_rows = 0;
size_t num_errors = 0;
};
}

View File

@ -0,0 +1,101 @@
#include <string>
#include <Processors/Formats/IRowOutputFormat.h>
#include <IO/WriteHelpers.h>
namespace DB
{
void IRowOutputFormat::consume(DB::Chunk chunk)
{
writePrefixIfNot();
auto num_rows = chunk.getNumRows();
auto & columns = chunk.getColumns();
for (UInt64 row = 0; row < num_rows; ++row)
{
if (!first_row)
writeRowBetweenDelimiter();
first_row = false;
write(columns, row);
}
}
void IRowOutputFormat::consumeTotals(DB::Chunk chunk)
{
writePrefixIfNot();
writeSuffixIfNot();
auto num_rows = chunk.getNumRows();
if (num_rows != 1)
throw Exception("Got " + toString(num_rows) + " in totals chunk, expected 1", ErrorCodes::LOGICAL_ERROR);
auto & columns = chunk.getColumns();
writeBeforeTotals();
writeTotals(columns, 0);
writeAfterTotals();
}
void IRowOutputFormat::consumeExtremes(DB::Chunk chunk)
{
writePrefixIfNot();
writeSuffixIfNot();
auto num_rows = chunk.getNumRows();
auto & columns = chunk.getColumns();
if (num_rows != 2)
throw Exception("Got " + toString(num_rows) + " in extremes chunk, expected 2", ErrorCodes::LOGICAL_ERROR);
writeBeforeExtremes();
writeMinExtreme(columns, 0);
writeRowBetweenDelimiter();
writeMaxExtreme(columns, 1);
writeAfterExtremes();
}
void IRowOutputFormat::finalize()
{
writePrefixIfNot();
writeSuffixIfNot();
writeLastSuffix();
}
void IRowOutputFormat::write(const Columns & columns, size_t row_num)
{
size_t num_columns = columns.size();
writeRowStartDelimiter();
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
writeFieldDelimiter();
writeField(*columns[i], *types[i], row_num);
}
writeRowEndDelimiter();
}
void IRowOutputFormat::writeMinExtreme(const DB::Columns & columns, size_t row_num)
{
write(columns, row_num);
}
void IRowOutputFormat::writeMaxExtreme(const DB::Columns & columns, size_t row_num)
{
write(columns, row_num);
}
void IRowOutputFormat::writeTotals(const DB::Columns & columns, size_t row_num)
{
write(columns, row_num);
}
}

View File

@ -0,0 +1,80 @@
#pragma once
#include <string>
#include <Processors/Formats/IOutputFormat.h>
namespace DB
{
class WriteBuffer;
/** Output format that writes data row by row.
*/
class IRowOutputFormat : public IOutputFormat
{
protected:
DataTypes types;
void consume(Chunk chunk) override;
void consumeTotals(Chunk chunk) override;
void consumeExtremes(Chunk chunk) override;
void finalize() override;
public:
IRowOutputFormat(const Block & header, WriteBuffer & out_)
: IOutputFormat(header, out_), types(header.getDataTypes())
{
}
/** Write a row.
* Default implementation calls methods to write single values and delimiters
* (except delimiter between rows (writeRowBetweenDelimiter())).
*/
virtual void write(const Columns & columns, size_t row_num);
virtual void writeMinExtreme(const Columns & columns, size_t row_num);
virtual void writeMaxExtreme(const Columns & columns, size_t row_num);
virtual void writeTotals(const Columns & columns, size_t row_num);
/** Write single value. */
virtual void writeField(const IColumn & column, const IDataType & type, size_t row_num) = 0;
/** Write delimiter. */
virtual void writeFieldDelimiter() {} /// delimiter between values
virtual void writeRowStartDelimiter() {} /// delimiter before each row
virtual void writeRowEndDelimiter() {} /// delimiter after each row
virtual void writeRowBetweenDelimiter() {} /// delimiter between rows
virtual void writePrefix() {} /// delimiter before resultset
virtual void writeSuffix() {} /// delimiter after resultset
virtual void writeBeforeTotals() {}
virtual void writeAfterTotals() {}
virtual void writeBeforeExtremes() {}
virtual void writeAfterExtremes() {}
virtual void writeLastSuffix() {} /// Write something after resultset, totals end extremes.
private:
bool first_row = true;
bool prefix_written = false;
bool suffix_written = false;
void writePrefixIfNot()
{
if (!prefix_written)
writePrefix();
prefix_written = true;
}
void writeSuffixIfNot()
{
if (!suffix_written)
writeSuffix();
suffix_written = true;
}
};
}

View File

@ -0,0 +1,82 @@
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <Processors/Formats/Impl/BinaryRowInputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params, bool with_names_, bool with_types_)
: IRowInputFormat(std::move(header), in_, params), with_names(with_names_), with_types(with_types_)
{
}
bool BinaryRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
if (in.eof())
return false;
size_t num_columns = columns.size();
for (size_t i = 0; i < num_columns; ++i)
getPort().getHeader().getByPosition(i).type->deserializeBinary(*columns[i], in);
return true;
}
void BinaryRowInputFormat::readPrefix()
{
/// NOTE The header is completely ignored. This can be easily improved.
UInt64 columns = 0;
String tmp;
if (with_names || with_types)
{
readVarUInt(columns, in);
}
if (with_names)
{
for (size_t i = 0; i < columns; ++i)
{
readStringBinary(tmp, in);
}
}
if (with_types)
{
for (size_t i = 0; i < columns; ++i)
{
readStringBinary(tmp, in);
}
}
}
void registerInputFormatProcessorRowBinary(FormatFactory & factory)
{
factory.registerInputFormatProcessor("RowBinary", [](
ReadBuffer & buf,
const Block & sample,
const Context &,
const IRowInputFormat::Params & params,
const FormatSettings &)
{
return std::make_shared<BinaryRowInputFormat>(buf, sample, params, false, false);
});
factory.registerInputFormatProcessor("RowBinaryWithNamesAndTypes", [](
ReadBuffer & buf,
const Block & sample,
const Context &,
const IRowInputFormat::Params & params,
const FormatSettings &)
{
return std::make_shared<BinaryRowInputFormat>(buf, sample, params, true, true);
});
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
namespace DB
{
class ReadBuffer;
/** A stream for inputting data in a binary line-by-line format.
*/
class BinaryRowInputFormat : public IRowInputFormat
{
public:
BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params, bool with_names_, bool with_types_);
bool readRow(MutableColumns & columns, RowReadExtension &) override;
void readPrefix() override;
String getName() const override { return "BinaryRowInputFormat"; }
private:
bool with_names;
bool with_types;
};
}

View File

@ -0,0 +1,71 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Columns/IColumn.h>
#include <DataTypes/IDataType.h>
#include <Processors/Formats/Impl/BinaryRowOutputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
BinaryRowOutputFormat::BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_)
: IRowOutputFormat(header, out_), with_names(with_names_), with_types(with_types_)
{
}
void BinaryRowOutputFormat::writePrefix()
{
auto & header = getPort(PortKind::Main).getHeader();
size_t columns = header.columns();
if (with_names || with_types)
{
writeVarUInt(columns, out);
}
if (with_names)
{
for (size_t i = 0; i < columns; ++i)
{
writeStringBinary(header.safeGetByPosition(i).name, out);
}
}
if (with_types)
{
for (size_t i = 0; i < columns; ++i)
{
writeStringBinary(header.safeGetByPosition(i).type->getName(), out);
}
}
}
void BinaryRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{
type.serializeBinary(column, row_num, out);
}
void registerOutputFormatProcessorRowBinary(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("RowBinary", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings &)
{
return std::make_shared<BinaryRowOutputFormat>(buf, sample, false, false);
});
factory.registerOutputFormatProcessor("RowBinaryWithNamesAndTypes", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings &)
{
return std::make_shared<BinaryRowOutputFormat>(buf, sample, true, true);
});
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <Processors/Formats/IRowOutputFormat.h>
#include <Core/Block.h>
namespace DB
{
class IColumn;
class IDataType;
class WriteBuffer;
/** A stream for outputting data in a binary line-by-line format.
*/
class BinaryRowOutputFormat: public IRowOutputFormat
{
public:
BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_);
String getName() const override { return "BinaryRowOutputFormat"; }
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writePrefix() override;
String getContentType() const override { return "application/octet-stream"; }
protected:
bool with_names;
bool with_types;
};
}

View File

@ -0,0 +1,3 @@
if (ENABLE_TESTS)
add_subdirectory (tests)
endif ()

View File

@ -0,0 +1,367 @@
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <Formats/verbosePrintString.h>
#include <Processors/Formats/Impl/CSVRowInputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
}
CSVRowInputFormat::CSVRowInputFormat(
ReadBuffer & in_, Block header, Params params, bool with_names_, const FormatSettings & format_settings)
: IRowInputFormat(std::move(header), in_, params), with_names(with_names_), format_settings(format_settings)
{
auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
data_types.resize(num_columns);
for (size_t i = 0; i < num_columns; ++i)
data_types[i] = sample.safeGetByPosition(i).type;
}
static void skipEndOfLine(ReadBuffer & istr)
{
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
if (*istr.position() == '\n')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\r')
++istr.position();
}
else if (*istr.position() == '\r')
{
++istr.position();
if (!istr.eof() && *istr.position() == '\n')
++istr.position();
else
throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)."
" Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA);
}
else if (!istr.eof())
throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA);
}
static void skipDelimiter(ReadBuffer & istr, const char delimiter, bool is_last_column)
{
if (is_last_column)
{
if (istr.eof())
return;
/// we support the extra delimiter at the end of the line
if (*istr.position() == delimiter)
{
++istr.position();
if (istr.eof())
return;
}
skipEndOfLine(istr);
}
else
assertChar(delimiter, istr);
}
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & buf)
{
while (!buf.eof()
&& (*buf.position() == ' '
|| *buf.position() == '\t'))
++buf.position();
}
static void skipRow(ReadBuffer & istr, const FormatSettings::CSV & settings, size_t num_columns)
{
String tmp;
for (size_t i = 0; i < num_columns; ++i)
{
skipWhitespacesAndTabs(istr);
readCSVString(tmp, istr, settings);
skipWhitespacesAndTabs(istr);
skipDelimiter(istr, settings.delimiter, i + 1 == num_columns);
}
}
void CSVRowInputFormat::readPrefix()
{
/// In this format, we assume, that if first string field contain BOM as value, it will be written in quotes,
/// so BOM at beginning of stream cannot be confused with BOM in first string value, and it is safe to skip it.
skipBOMIfExists(in);
size_t num_columns = data_types.size();
String tmp;
if (with_names)
skipRow(in, format_settings.csv, num_columns);
}
bool CSVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
if (in.eof())
return false;
updateDiagnosticInfo();
size_t size = data_types.size();
for (size_t i = 0; i < size; ++i)
{
skipWhitespacesAndTabs(in);
data_types[i]->deserializeAsTextCSV(*columns[i], in, format_settings);
skipWhitespacesAndTabs(in);
skipDelimiter(in, format_settings.csv.delimiter, i + 1 == size);
}
return true;
}
String CSVRowInputFormat::getDiagnosticInfo()
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
WriteBufferFromOwnString out;
auto & header = getPort().getHeader();
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last rows are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current row and parse all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
in.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
bool CSVRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
const char delimiter = format_settings.csv.delimiter;
auto & header = getPort().getHeader();
size_t size = data_types.size();
for (size_t i = 0; i < size; ++i)
{
if (i == 0 && in.eof())
{
out << "<End of stream>\n";
return false;
}
out << "Column " << i << ", " << std::string((i < 10 ? 2 : i < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(i).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(i).name.size(), ' ')
<< "type: " << data_types[i]->getName() << ", " << std::string(max_length_of_data_type_name - data_types[i]->getName().size(), ' ');
BufferBase::Position prev_position = in.position();
BufferBase::Position curr_position = in.position();
std::exception_ptr exception;
try
{
skipWhitespacesAndTabs(in);
prev_position = in.position();
data_types[i]->deserializeAsTextCSV(*columns[i], in, format_settings);
curr_position = in.position();
skipWhitespacesAndTabs(in);
}
catch (...)
{
exception = std::current_exception();
}
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i]))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << data_types[i]->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (data_types[i]->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (data_types[i]->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (data_types[i]->haveMaximumSizeOfValue())
{
if (*curr_position != '\n' && *curr_position != '\r' && *curr_position != delimiter)
{
out << "ERROR: garbage after " << data_types[i]->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (data_types[i]->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (data_types[i]->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
/// Delimiters
if (i + 1 == size)
{
if (in.eof())
return false;
/// we support the extra delimiter at the end of the line
if (*in.position() == delimiter)
{
++in.position();
if (in.eof())
break;
}
if (!in.eof() && *in.position() != '\n' && *in.position() != '\r')
{
out << "ERROR: There is no line feed. ";
verbosePrintString(in.position(), in.position() + 1, out);
out << " found instead.\n"
" It's like your file has more columns than expected.\n"
"And if your file have right number of columns, maybe it have unquoted string value with comma.\n";
return false;
}
skipEndOfLine(in);
}
else
{
try
{
assertChar(delimiter, in);
}
catch (const DB::Exception &)
{
if (*in.position() == '\n' || *in.position() == '\r')
{
out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped quotes in values.\n";
}
else
{
out << "ERROR: There is no delimiter (" << delimiter << "). ";
verbosePrintString(in.position(), in.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
return true;
}
void CSVRowInputFormat::syncAfterError()
{
skipToNextLineOrEOF(in);
}
void CSVRowInputFormat::updateDiagnosticInfo()
{
++row_num;
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = in.position();
}
void registerInputFormatProcessorCSV(FormatFactory & factory)
{
for (bool with_names : {false, true})
{
factory.registerInputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<CSVRowInputFormat>(buf, sample, params, with_names, settings);
});
}
}
}

View File

@ -0,0 +1,56 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class ReadBuffer;
/** A stream for inputting data in csv format.
* Does not conform with https://tools.ietf.org/html/rfc4180 because it skips spaces and tabs between values.
*/
class CSVRowInputFormat : public IRowInputFormat
{
public:
/** with_names - in the first line the header with column names
* with_types - on the next line header with type names
*/
CSVRowInputFormat(ReadBuffer & in_, Block header, Params params, bool with_names, const FormatSettings & format_settings);
String getName() const override { return "CSVRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension &) override;
void readPrefix() override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
std::string getDiagnosticInfo() override;
private:
bool with_names;
DataTypes data_types;
const FormatSettings format_settings;
/// For convenient diagnostics in case of an error.
size_t row_num = 0;
/// How many bytes were read, not counting those that are still in the buffer.
size_t bytes_read_at_start_of_buffer_on_current_row = 0;
size_t bytes_read_at_start_of_buffer_on_prev_row = 0;
char * pos_of_current_row = nullptr;
char * pos_of_prev_row = nullptr;
void updateDiagnosticInfo();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name);
};
}

View File

@ -0,0 +1,87 @@
#include <Processors/Formats/Impl/CSVRowOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
namespace DB
{
CSVRowOutputFormat::CSVRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, const FormatSettings & format_settings)
: IRowOutputFormat(header, out_), with_names(with_names_), format_settings(format_settings)
{
auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
data_types.resize(columns);
for (size_t i = 0; i < columns; ++i)
data_types[i] = sample.safeGetByPosition(i).type;
}
void CSVRowOutputFormat::writePrefix()
{
auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
if (with_names)
{
for (size_t i = 0; i < columns; ++i)
{
writeCSVString(sample.safeGetByPosition(i).name, out);
char delimiter = format_settings.csv.delimiter;
if (i + 1 == columns)
delimiter = '\n';
writeChar(delimiter, out);
}
}
}
void CSVRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{
type.serializeAsTextCSV(column, row_num, out, format_settings);
}
void CSVRowOutputFormat::writeFieldDelimiter()
{
writeChar(format_settings.csv.delimiter, out);
}
void CSVRowOutputFormat::writeRowEndDelimiter()
{
writeChar('\n', out);
}
void CSVRowOutputFormat::writeBeforeTotals()
{
writeChar('\n', out);
}
void CSVRowOutputFormat::writeBeforeExtremes()
{
writeChar('\n', out);
}
void registerOutputFormatProcessorCSV(FormatFactory & factory)
{
for (bool with_names : {false, true})
{
factory.registerOutputFormatProcessor(with_names ? "CSVWithNames" : "CSV", [=](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<CSVRowOutputFormat>(buf, sample, with_names, format_settings);
});
}
}
}

View File

@ -0,0 +1,48 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowOutputFormat.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class WriteBuffer;
/** The stream for outputting data in csv format.
* Does not conform with https://tools.ietf.org/html/rfc4180 because it uses LF, not CR LF.
*/
class CSVRowOutputFormat : public IRowOutputFormat
{
public:
/** with_names - output in the first line a header with column names
* with_types - output in the next line header with the names of the types
*/
CSVRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, const FormatSettings & format_settings);
String getName() const override { return "CSVRowOutputFormat"; }
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeFieldDelimiter() override;
void writeRowEndDelimiter() override;
void writePrefix() override;
void writeBeforeTotals() override;
void writeBeforeExtremes() override;
/// https://www.iana.org/assignments/media-types/text/csv
String getContentType() const override
{
return String("text/csv; charset=UTF-8; header=") + (with_names ? "present" : "absent");
}
protected:
bool with_names;
const FormatSettings format_settings;
DataTypes data_types;
};
}

View File

@ -0,0 +1,314 @@
#include "config_formats.h"
#if USE_CAPNP
#include <IO/ReadBuffer.h>
#include <Interpreters/Context.h>
#include <Processors/Formats/Impl/CapnProtoRowInputFormat.h> // Y_IGNORE
#include <Formats/FormatFactory.h>
#include <Formats/FormatSchemaInfo.h>
#include <capnp/serialize.h> // Y_IGNORE
#include <capnp/dynamic.h> // Y_IGNORE
#include <capnp/common.h> // Y_IGNORE
#include <boost/algorithm/string.hpp>
#include <boost/range/join.hpp>
#include <common/logger_useful.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_TYPE_OF_FIELD;
extern const int BAD_ARGUMENTS;
extern const int THERE_IS_NO_COLUMN;
extern const int LOGICAL_ERROR;
}
static CapnProtoRowInputFormat::NestedField split(const Block & header, size_t i)
{
CapnProtoRowInputFormat::NestedField field = {{}, i};
// Remove leading dot in field definition, e.g. ".msg" -> "msg"
String name(header.safeGetByPosition(i).name);
if (!name.empty() && name[0] == '.')
name.erase(0, 1);
boost::split(field.tokens, name, boost::is_any_of("._"));
return field;
}
static Field convertNodeToField(const capnp::DynamicValue::Reader & value)
{
switch (value.getType())
{
case capnp::DynamicValue::UNKNOWN:
throw Exception("Unknown field type", ErrorCodes::BAD_TYPE_OF_FIELD);
case capnp::DynamicValue::VOID:
return Field();
case capnp::DynamicValue::BOOL:
return value.as<bool>() ? 1u : 0u;
case capnp::DynamicValue::INT:
return value.as<int64_t>();
case capnp::DynamicValue::UINT:
return value.as<uint64_t>();
case capnp::DynamicValue::FLOAT:
return 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 value.as<capnp::DynamicEnum>().getRaw();
case capnp::DynamicValue::STRUCT:
{
auto structValue = value.as<capnp::DynamicStruct>();
const auto & fields = structValue.getSchema().getFields();
Field field = Tuple(TupleBackend(fields.size()));
TupleBackend & tuple = get<Tuple &>(field).toUnderType();
for (auto i : kj::indices(fields))
tuple[i] = convertNodeToField(structValue.get(fields[i]));
return field;
}
case capnp::DynamicValue::CAPABILITY:
throw Exception("CAPABILITY type not supported", ErrorCodes::BAD_TYPE_OF_FIELD);
case capnp::DynamicValue::ANY_POINTER:
throw Exception("ANY_POINTER type not supported", ErrorCodes::BAD_TYPE_OF_FIELD);
}
return Field();
}
static capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std::string & field)
{
KJ_IF_MAYBE(child, node.findFieldByName(field))
return *child;
else
throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr(), ErrorCodes::THERE_IS_NO_COLUMN);
}
void CapnProtoRowInputFormat::createActions(const NestedFieldList & sorted_fields, capnp::StructSchema reader)
{
/// Columns in a table can map to fields in Cap'n'Proto or to structs.
/// Store common parents and their tokens in order to backtrack.
std::vector<capnp::StructSchema::Field> parents;
std::vector<std::string> parent_tokens;
capnp::StructSchema cur_reader = reader;
for (const auto & field : sorted_fields)
{
if (field.tokens.empty())
throw Exception("Logical error in CapnProtoRowInputFormat", ErrorCodes::LOGICAL_ERROR);
// Backtrack to common parent
while (field.tokens.size() < parent_tokens.size() + 1
|| !std::equal(parent_tokens.begin(), parent_tokens.end(), field.tokens.begin()))
{
actions.push_back({Action::POP});
parents.pop_back();
parent_tokens.pop_back();
if (parents.empty())
{
cur_reader = reader;
break;
}
else
cur_reader = parents.back().getType().asStruct();
}
// Go forward
while (parent_tokens.size() + 1 < field.tokens.size())
{
const auto & token = field.tokens[parents.size()];
auto node = getFieldOrThrow(cur_reader, token);
if (node.getType().isStruct())
{
// Descend to field structure
parents.emplace_back(node);
parent_tokens.emplace_back(token);
cur_reader = node.getType().asStruct();
actions.push_back({Action::PUSH, node});
}
else if (node.getType().isList())
{
break; // Collect list
}
else
throw Exception("Field " + token + " is neither Struct nor List", ErrorCodes::BAD_TYPE_OF_FIELD);
}
// Read field from the structure
auto node = getFieldOrThrow(cur_reader, field.tokens[parents.size()]);
if (node.getType().isList() && !actions.empty() && actions.back().field == node)
{
// The field list here flattens Nested elements into multiple arrays
// In order to map Nested types in Cap'nProto back, they need to be collected
// Since the field names are sorted, the order of field positions must be preserved
// For example, if the fields are { b @0 :Text, a @1 :Text }, the `a` would come first
// even though it's position is second.
auto & columns = actions.back().columns;
auto it = std::upper_bound(columns.cbegin(), columns.cend(), field.pos);
columns.insert(it, field.pos);
}
else
{
actions.push_back({Action::READ, node, {field.pos}});
}
}
}
CapnProtoRowInputFormat::CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params, const FormatSchemaInfo & info)
: IRowInputFormat(std::move(header), in_, params), parser(std::make_shared<SchemaParser>())
{
// Parse the schema and fetch the root object
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wdeprecated-declarations"
auto schema = parser->impl.parseDiskFile(info.schemaPath(), info.absoluteSchemaPath(), {});
#pragma GCC diagnostic pop
root = schema.getNested(info.messageName()).asStruct();
/**
* The schema typically consists of fields in various nested structures.
* Here we gather the list of fields and sort them in a way so that fields in the same structure are adjacent,
* and the nesting level doesn't decrease to make traversal easier.
*/
auto & sample = getPort().getHeader();
NestedFieldList list;
size_t num_columns = sample.columns();
for (size_t i = 0; i < num_columns; ++i)
list.push_back(split(sample, i));
// Order list first by value of strings then by length of string vector.
std::sort(list.begin(), list.end(), [](const NestedField & a, const NestedField & b) { return a.tokens < b.tokens; });
createActions(list, root);
}
bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
if (in.eof())
return false;
// Read from underlying buffer directly
auto buf = in.buffer();
auto base = reinterpret_cast<const capnp::word *>(in.position());
// Check if there's enough bytes in the buffer to read the full message
kj::Array<capnp::word> heap_array;
auto array = kj::arrayPtr(base, buf.size() - in.offset());
auto expected_words = capnp::expectedSizeInWordsFromPrefix(array);
if (expected_words * sizeof(capnp::word) > array.size())
{
// We'll need to reassemble the message in a contiguous buffer
heap_array = kj::heapArray<capnp::word>(expected_words);
in.readStrict(heap_array.asChars().begin(), heap_array.asChars().size());
array = heap_array.asPtr();
}
#if CAPNP_VERSION >= 8000
capnp::UnalignedFlatArrayMessageReader msg(array);
#else
capnp::FlatArrayMessageReader msg(array);
#endif
std::vector<capnp::DynamicStruct::Reader> stack;
stack.push_back(msg.getRoot<capnp::DynamicStruct>(root));
for (auto action : actions)
{
switch (action.type)
{
case Action::READ:
{
Field value = convertNodeToField(stack.back().get(action.field));
if (action.columns.size() > 1)
{
// Nested columns must be flattened into several arrays
// e.g. Array(Tuple(x ..., y ...)) -> Array(x ...), Array(y ...)
const auto & collected = DB::get<const Array &>(value);
size_t size = collected.size();
// The flattened array contains an array of a part of the nested tuple
Array flattened(size);
for (size_t column_index = 0; column_index < action.columns.size(); ++column_index)
{
// Populate array with a single tuple elements
for (size_t off = 0; off < size; ++off)
{
const TupleBackend & tuple = DB::get<const Tuple &>(collected[off]).toUnderType();
flattened[off] = tuple[column_index];
}
auto & col = columns[action.columns[column_index]];
col->insert(flattened);
}
}
else
{
auto & col = columns[action.columns[0]];
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;
}
}
// Advance buffer position if used directly
if (heap_array.size() == 0)
{
auto parsed = (msg.getEnd() - base) * sizeof(capnp::word);
in.position() += parsed;
}
return true;
}
void registerInputFormatProcessorCapnProto(FormatFactory & factory)
{
factory.registerInputFormatProcessor(
"CapnProto",
[](ReadBuffer & buf, const Block & sample, const Context & context, IRowInputFormat::Params params, const FormatSettings &)
{
return std::make_shared<CapnProtoRowInputFormat>(buf, sample, params, FormatSchemaInfo(context, "capnp"));
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerInputFormatProcessorCapnProto(FormatFactory &) {}
}
#endif // USE_CAPNP

View File

@ -0,0 +1,75 @@
#pragma once
#include <Common/config.h>
#if USE_CAPNP
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <capnp/schema-parser.h>
namespace DB
{
class FormatSchemaInfo;
class ReadBuffer;
/** A stream for reading messages in Cap'n Proto format in given schema.
* Like Protocol Buffers and Thrift (but unlike JSON or MessagePack),
* Cap'n Proto messages are strongly-typed and not self-describing.
* The schema in this case cannot be compiled in, so it uses a runtime schema parser.
* See https://capnproto.org/cxx.html
*/
class CapnProtoRowInputFormat : public IRowInputFormat
{
public:
struct NestedField
{
std::vector<std::string> tokens;
size_t pos;
};
using NestedFieldList = std::vector<NestedField>;
/** schema_dir - base path for schema files
* schema_file - location of the capnproto schema, e.g. "schema.capnp"
* root_object - name to the root object, e.g. "Message"
*/
CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params, const FormatSchemaInfo & info);
String getName() const override { return "CapnProtoRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension &) override;
private:
// Build a traversal plan from a sorted list of fields
void createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader);
/* Action for state machine for traversing nested structures. */
using BlockPositionList = std::vector<size_t>;
struct Action
{
enum Type { POP, PUSH, READ };
Type type;
capnp::StructSchema::Field field = {};
BlockPositionList columns = {};
};
// Wrapper for classes that could throw in destructor
// https://github.com/capnproto/capnproto/issues/553
template <typename T>
struct DestructorCatcher
{
T impl;
template <typename ... Arg>
DestructorCatcher(Arg && ... args) : impl(kj::fwd<Arg>(args)...) {}
~DestructorCatcher() noexcept try { } catch (...) { return; }
};
using SchemaParser = DestructorCatcher<capnp::SchemaParser>;
std::shared_ptr<SchemaParser> parser;
capnp::StructSchema root;
std::vector<Action> actions;
};
}
#endif // USE_CAPNP

View File

@ -0,0 +1,92 @@
#include <Processors/Formats/Impl/JSONCompactRowOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteHelpers.h>
namespace DB
{
JSONCompactRowOutputFormat::JSONCompactRowOutputFormat(
WriteBuffer & out_, const Block & header, const FormatSettings & settings_)
: JSONRowOutputFormat(out_, header, settings_)
{
}
void JSONCompactRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{
type.serializeAsTextJSON(column, row_num, *ostr, settings);
++field_number;
}
void JSONCompactRowOutputFormat::writeFieldDelimiter()
{
writeCString(", ", *ostr);
}
void JSONCompactRowOutputFormat::writeTotalsFieldDelimiter()
{
writeCString(",", *ostr);
}
void JSONCompactRowOutputFormat::writeRowStartDelimiter()
{
if (row_count > 0)
writeCString(",\n", *ostr);
writeCString("\t\t[", *ostr);
}
void JSONCompactRowOutputFormat::writeRowEndDelimiter()
{
writeChar(']', *ostr);
field_number = 0;
++row_count;
}
void JSONCompactRowOutputFormat::writeBeforeTotals()
{
writeCString(",\n", *ostr);
writeChar('\n', *ostr);
writeCString("\t\"totals\": [", *ostr);
}
void JSONCompactRowOutputFormat::writeAfterTotals()
{
writeChar(']', *ostr);
}
void JSONCompactRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num)
{
writeCString("\t\t\"", *ostr);
writeCString(title, *ostr);
writeCString("\": [", *ostr);
size_t extremes_columns = columns.size();
for (size_t i = 0; i < extremes_columns; ++i)
{
if (i != 0)
writeTotalsFieldDelimiter();
writeField(*columns[i], *types[i], row_num);
}
writeChar(']', *ostr);
}
void registerOutputFormatProcessorJSONCompact(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("JSONCompact", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<JSONCompactRowOutputFormat>(buf, sample, format_settings);
});
}
}

View File

@ -0,0 +1,43 @@
#pragma once
#include <Core/Block.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteBufferValidUTF8.h>
#include <Processors/Formats/Impl/JSONRowOutputFormat.h>
namespace DB
{
struct FormatSettings;
/** The stream for outputting data in the JSONCompact format.
*/
class JSONCompactRowOutputFormat : public JSONRowOutputFormat
{
public:
JSONCompactRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings_);
String getName() const override { return "JSONCompactRowOutputFormat"; }
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeFieldDelimiter() override;
void writeRowStartDelimiter() override;
void writeRowEndDelimiter() override;
void writeBeforeTotals() override;
void writeAfterTotals() override;
protected:
void writeExtremesElement(const char * title, const Columns & columns, size_t row_num) override;
void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num) override
{
return writeField(column, type, row_num);
}
void writeTotalsFieldDelimiter() override;
};
}

View File

@ -0,0 +1,270 @@
#include <IO/ReadHelpers.h>
#include <Processors/Formats/Impl/JSONEachRowRowInputFormat.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/NestedUtils.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int CANNOT_READ_ALL_DATA;
extern const int LOGICAL_ERROR;
}
namespace
{
enum
{
UNKNOWN_FIELD = size_t(-1),
NESTED_FIELD = size_t(-2)
};
}
JSONEachRowRowInputFormat::JSONEachRowRowInputFormat(
ReadBuffer & in_, const Block & header, Params params, const FormatSettings & format_settings)
: IRowInputFormat(header, in_, params), format_settings(format_settings), name_map(header.columns())
{
/// In this format, BOM at beginning of stream cannot be confused with value, so it is safe to skip it.
skipBOMIfExists(in);
size_t num_columns = getPort().getHeader().columns();
for (size_t i = 0; i < num_columns; ++i)
{
const String & column_name = columnName(i);
name_map[column_name] = i; /// NOTE You could place names more cache-locally.
if (format_settings.import_nested_json)
{
const auto splitted = Nested::splitName(column_name);
if (!splitted.second.empty())
{
const StringRef table_name(column_name.data(), splitted.first.size());
name_map[table_name] = NESTED_FIELD;
}
}
}
prev_positions.assign(num_columns, name_map.end());
}
const String & JSONEachRowRowInputFormat::columnName(size_t i) const
{
return getPort().getHeader().getByPosition(i).name;
}
inline size_t JSONEachRowRowInputFormat::columnIndex(const StringRef & name, size_t key_index)
{
/// Optimization by caching the order of fields (which is almost always the same)
/// and a quick check to match the next expected field, instead of searching the hash table.
if (prev_positions.size() > key_index
&& prev_positions[key_index] != name_map.end()
&& name == prev_positions[key_index]->getFirst())
{
return prev_positions[key_index]->getSecond();
}
else
{
const auto it = name_map.find(name);
if (name_map.end() != it)
{
if (key_index < prev_positions.size())
prev_positions[key_index] = it;
return it->getSecond();
}
else
return UNKNOWN_FIELD;
}
}
/** Read the field name and convert it to column name
* (taking into account the current nested name prefix)
* Resulting StringRef is valid only before next read from buf.
*/
StringRef JSONEachRowRowInputFormat::readColumnName(ReadBuffer & buf)
{
// This is just an optimization: try to avoid copying the name into current_column_name
if (nested_prefix_length == 0 && buf.position() + 1 < buf.buffer().end())
{
char * next_pos = find_first_symbols<'\\', '"'>(buf.position() + 1, buf.buffer().end());
if (next_pos != buf.buffer().end() && *next_pos != '\\')
{
/// The most likely option is that there is no escape sequence in the key name, and the entire name is placed in the buffer.
assertChar('"', buf);
StringRef res(buf.position(), next_pos - buf.position());
buf.position() = next_pos + 1;
return res;
}
}
current_column_name.resize(nested_prefix_length);
readJSONStringInto(current_column_name, buf);
return current_column_name;
}
static inline void skipColonDelimeter(ReadBuffer & istr)
{
skipWhitespaceIfAny(istr);
assertChar(':', istr);
skipWhitespaceIfAny(istr);
}
void JSONEachRowRowInputFormat::skipUnknownField(const StringRef & name_ref)
{
if (!format_settings.skip_unknown_fields)
throw Exception("Unknown field found while parsing JSONEachRow format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
skipJSONField(in, name_ref);
}
void JSONEachRowRowInputFormat::readField(size_t index, MutableColumns & columns)
{
if (read_columns[index])
throw Exception("Duplicate field found while parsing JSONEachRow format: " + columnName(index), ErrorCodes::INCORRECT_DATA);
try
{
auto & header = getPort().getHeader();
header.getByPosition(index).type->deserializeAsTextJSON(*columns[index], in, format_settings);
}
catch (Exception & e)
{
e.addMessage("(while read the value of key " + columnName(index) + ")");
throw;
}
read_columns[index] = true;
}
inline bool JSONEachRowRowInputFormat::advanceToNextKey(size_t key_index)
{
skipWhitespaceIfAny(in);
if (in.eof())
throw Exception("Unexpected end of stream while parsing JSONEachRow format", ErrorCodes::CANNOT_READ_ALL_DATA);
else if (*in.position() == '}')
{
++in.position();
return false;
}
if (key_index > 0)
{
assertChar(',', in);
skipWhitespaceIfAny(in);
}
return true;
}
void JSONEachRowRowInputFormat::readJSONObject(MutableColumns & columns)
{
assertChar('{', in);
for (size_t key_index = 0; advanceToNextKey(key_index); ++key_index)
{
StringRef name_ref = readColumnName(in);
const size_t column_index = columnIndex(name_ref, key_index);
if (unlikely(ssize_t(column_index) < 0))
{
/// name_ref may point directly to the input buffer
/// and input buffer may be filled with new data on next read
/// If we want to use name_ref after another reads from buffer, we must copy it to temporary string.
current_column_name.assign(name_ref.data, name_ref.size);
name_ref = StringRef(current_column_name);
skipColonDelimeter(in);
if (column_index == UNKNOWN_FIELD)
skipUnknownField(name_ref);
else if (column_index == NESTED_FIELD)
readNestedData(name_ref.toString(), columns);
else
throw Exception("Logical error: illegal value of column_index", ErrorCodes::LOGICAL_ERROR);
}
else
{
skipColonDelimeter(in);
readField(column_index, columns);
}
}
}
void JSONEachRowRowInputFormat::readNestedData(const String & name, MutableColumns & columns)
{
current_column_name = name;
current_column_name.push_back('.');
nested_prefix_length = current_column_name.size();
readJSONObject(columns);
nested_prefix_length = 0;
}
bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
{
skipWhitespaceIfAny(in);
/// We consume ;, or \n before scanning a new row, instead scanning to next row at the end.
/// The reason is that if we want an exact number of rows read with LIMIT x
/// from a streaming table engine with text data format, like File or Kafka
/// then seeking to next ;, or \n would trigger reading of an extra row at the end.
/// Semicolon is added for convenience as it could be used at end of INSERT query.
if (!in.eof() && (*in.position() == ',' || *in.position() == ';'))
++in.position();
skipWhitespaceIfAny(in);
if (in.eof())
return false;
size_t num_columns = columns.size();
/// Set of columns for which the values were read. The rest will be filled with default values.
read_columns.assign(num_columns, false);
nested_prefix_length = 0;
readJSONObject(columns);
auto & header = getPort().getHeader();
/// Fill non-visited columns with the default values.
for (size_t i = 0; i < num_columns; ++i)
if (!read_columns[i])
header.getByPosition(i).type->insertDefaultInto(*columns[i]);
/// return info about defaults set
ext.read_columns = read_columns;
return true;
}
void JSONEachRowRowInputFormat::syncAfterError()
{
skipToUnescapedNextLineOrEOF(in);
}
void registerInputFormatProcessorJSONEachRow(FormatFactory & factory)
{
factory.registerInputFormatProcessor("JSONEachRow", [](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<JSONEachRowRowInputFormat>(buf, sample, params, settings);
});
}
}

View File

@ -0,0 +1,68 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Formats/FormatSettings.h>
#include <Common/HashTable/HashMap.h>
namespace DB
{
class ReadBuffer;
/** A stream for reading data in JSON format, where each row is represented by a separate JSON object.
* Objects can be separated by feed return, other whitespace characters in any number and possibly a comma.
* Fields can be listed in any order (including, in different lines there may be different order),
* and some fields may be missing.
*/
class JSONEachRowRowInputFormat : public IRowInputFormat
{
public:
JSONEachRowRowInputFormat(ReadBuffer & in_, const Block & header, Params params, const FormatSettings & format_settings);
String getName() const override { return "JSONEachRowRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
private:
const String & columnName(size_t i) const;
size_t columnIndex(const StringRef & name, size_t key_index);
bool advanceToNextKey(size_t key_index);
void skipUnknownField(const StringRef & name_ref);
StringRef readColumnName(ReadBuffer & buf);
void readField(size_t index, MutableColumns & columns);
void readJSONObject(MutableColumns & columns);
void readNestedData(const String & name, MutableColumns & columns);
private:
const FormatSettings format_settings;
/// Buffer for the read from the stream field name. Used when you have to copy it.
/// Also, if processing of Nested data is in progress, it holds the common prefix
/// of the nested column names (so that appending the field name to it produces
/// the full column name)
String current_column_name;
/// If processing Nested data, holds the length of the common prefix
/// of the names of related nested columns. For example, for a table
/// created as follows
/// CREATE TABLE t (n Nested (i Int32, s String))
/// the nested column names are 'n.i' and 'n.s' and the nested prefix is 'n.'
size_t nested_prefix_length = 0;
std::vector<UInt8> read_columns;
/// Hash table match `field name -> position in the block`. NOTE You can use perfect hash map.
using NameMap = HashMap<StringRef, size_t, StringRefHash>;
NameMap name_map;
/// Cached search results for previous row (keyed as index in JSON object) - used as a hint.
std::vector<NameMap::iterator> prev_positions;
};
}

View File

@ -0,0 +1,66 @@
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferValidUTF8.h>
#include <Processors/Formats/Impl/JSONEachRowRowOutputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings)
: IRowOutputFormat(header, out_), settings(settings)
{
auto & sample = getPort(PortKind::Main).getHeader();
size_t columns = sample.columns();
fields.resize(columns);
for (size_t i = 0; i < columns; ++i)
{
WriteBufferFromString buf(fields[i]);
writeJSONString(sample.getByPosition(i).name, buf, settings);
}
}
void JSONEachRowRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{
writeString(fields[field_number], out);
writeChar(':', out);
type.serializeAsTextJSON(column, row_num, out, settings);
++field_number;
}
void JSONEachRowRowOutputFormat::writeFieldDelimiter()
{
writeChar(',', out);
}
void JSONEachRowRowOutputFormat::writeRowStartDelimiter()
{
writeChar('{', out);
}
void JSONEachRowRowOutputFormat::writeRowEndDelimiter()
{
writeCString("}\n", out);
field_number = 0;
}
void registerOutputFormatProcessorJSONEachRow(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("JSONEachRow", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<JSONEachRowRowOutputFormat>(buf, sample, format_settings);
});
}
}

View File

@ -0,0 +1,35 @@
#pragma once
#include <Core/Block.h>
#include <IO/WriteBuffer.h>
#include <Processors/Formats/IRowOutputFormat.h>
#include <Formats/FormatSettings.h>
namespace DB
{
/** The stream for outputting data in JSON format, by object per line.
* Does not validate UTF-8.
*/
class JSONEachRowRowOutputFormat : public IRowOutputFormat
{
public:
JSONEachRowRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings);
String getName() const override { return "JSONEachRowRowOutputFormat"; }
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeFieldDelimiter() override;
void writeRowStartDelimiter() override;
void writeRowEndDelimiter() override;
private:
size_t field_number = 0;
Names fields;
FormatSettings settings;
};
}

View File

@ -0,0 +1,257 @@
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferValidUTF8.h>
#include <Processors/Formats/Impl/JSONRowOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <Formats/BlockOutputStreamFromRowOutputStream.h>
namespace DB
{
JSONRowOutputFormat::JSONRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings_)
: IRowOutputFormat(header, out_), settings(settings_)
{
auto & sample = getPort(PortKind::Main).getHeader();
NamesAndTypesList columns(sample.getNamesAndTypesList());
fields.assign(columns.begin(), columns.end());
bool need_validate_utf8 = false;
for (size_t i = 0; i < sample.columns(); ++i)
{
if (!sample.getByPosition(i).type->textCanContainOnlyValidUTF8())
need_validate_utf8 = true;
WriteBufferFromOwnString buf;
writeJSONString(fields[i].name, buf, settings);
fields[i].name = buf.str();
}
if (need_validate_utf8)
{
validating_ostr = std::make_unique<WriteBufferValidUTF8>(out);
ostr = validating_ostr.get();
}
else
ostr = &out;
}
void JSONRowOutputFormat::writePrefix()
{
writeCString("{\n", *ostr);
writeCString("\t\"meta\":\n", *ostr);
writeCString("\t[\n", *ostr);
for (size_t i = 0; i < fields.size(); ++i)
{
writeCString("\t\t{\n", *ostr);
writeCString("\t\t\t\"name\": ", *ostr);
writeString(fields[i].name, *ostr);
writeCString(",\n", *ostr);
writeCString("\t\t\t\"type\": ", *ostr);
writeJSONString(fields[i].type->getName(), *ostr, settings);
writeChar('\n', *ostr);
writeCString("\t\t}", *ostr);
if (i + 1 < fields.size())
writeChar(',', *ostr);
writeChar('\n', *ostr);
}
writeCString("\t],\n", *ostr);
writeChar('\n', *ostr);
writeCString("\t\"data\":\n", *ostr);
writeCString("\t[\n", *ostr);
}
void JSONRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{
writeCString("\t\t\t", *ostr);
writeString(fields[field_number].name, *ostr);
writeCString(": ", *ostr);
type.serializeAsTextJSON(column, row_num, *ostr, settings);
++field_number;
}
void JSONRowOutputFormat::writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num)
{
writeCString("\t\t", *ostr);
writeString(fields[field_number].name, *ostr);
writeCString(": ", *ostr);
type.serializeAsTextJSON(column, row_num, *ostr, settings);
++field_number;
}
void JSONRowOutputFormat::writeFieldDelimiter()
{
writeCString(",\n", *ostr);
}
void JSONRowOutputFormat::writeRowStartDelimiter()
{
writeCString("\t\t{\n", *ostr);
}
void JSONRowOutputFormat::writeRowEndDelimiter()
{
writeChar('\n', *ostr);
writeCString("\t\t}", *ostr);
field_number = 0;
++row_count;
}
void JSONRowOutputFormat::writeRowBetweenDelimiter()
{
writeCString(",\n", *ostr);
}
void JSONRowOutputFormat::writeSuffix()
{
writeChar('\n', *ostr);
writeCString("\t]", *ostr);
}
void JSONRowOutputFormat::writeBeforeTotals()
{
writeCString(",\n", *ostr);
writeChar('\n', *ostr);
writeCString("\t\"totals\":\n", *ostr);
writeCString("\t{\n", *ostr);
}
void JSONRowOutputFormat::writeTotals(const Columns & columns, size_t row_num)
{
size_t num_columns = columns.size();
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
writeTotalsFieldDelimiter();
writeTotalsField(*columns[i], *types[i], row_num);
}
}
void JSONRowOutputFormat::writeAfterTotals()
{
writeChar('\n', *ostr);
writeCString("\t}", *ostr);
field_number = 0;
}
void JSONRowOutputFormat::writeBeforeExtremes()
{
writeCString(",\n", *ostr);
writeChar('\n', *ostr);
writeCString("\t\"extremes\":\n", *ostr);
writeCString("\t{\n", *ostr);
}
void JSONRowOutputFormat::writeExtremesElement(const char * title, const Columns & columns, size_t row_num)
{
writeCString("\t\t\"", *ostr);
writeCString(title, *ostr);
writeCString("\":\n", *ostr);
writeCString("\t\t{\n", *ostr);
size_t extremes_columns = columns.size();
for (size_t i = 0; i < extremes_columns; ++i)
{
if (i != 0)
writeFieldDelimiter();
writeField(*columns[i], *types[i], row_num);
}
writeChar('\n', *ostr);
writeCString("\t\t}", *ostr);
field_number = 0;
}
void JSONRowOutputFormat::writeMinExtreme(const Columns & columns, size_t row_num)
{
writeExtremesElement("min", columns, row_num);
}
void JSONRowOutputFormat::writeMaxExtreme(const Columns & columns, size_t row_num)
{
writeExtremesElement("max", columns, row_num);
}
void JSONRowOutputFormat::writeAfterExtremes()
{
writeChar('\n', *ostr);
writeCString("\t}", *ostr);
}
void JSONRowOutputFormat::writeLastSuffix()
{
writeCString(",\n\n", *ostr);
writeCString("\t\"rows\": ", *ostr);
writeIntText(row_count, *ostr);
writeRowsBeforeLimitAtLeast();
if (settings.write_statistics)
writeStatistics();
writeChar('\n', *ostr);
writeCString("}\n", *ostr);
ostr->next();
}
void JSONRowOutputFormat::writeRowsBeforeLimitAtLeast()
{
if (applied_limit)
{
writeCString(",\n\n", *ostr);
writeCString("\t\"rows_before_limit_at_least\": ", *ostr);
writeIntText(rows_before_limit, *ostr);
}
}
void JSONRowOutputFormat::writeStatistics()
{
writeCString(",\n\n", *ostr);
writeCString("\t\"statistics\":\n", *ostr);
writeCString("\t{\n", *ostr);
writeCString("\t\t\"elapsed\": ", *ostr);
writeText(watch.elapsedSeconds(), *ostr);
writeCString(",\n", *ostr);
writeCString("\t\t\"rows_read\": ", *ostr);
writeText(progress.read_rows.load(), *ostr);
writeCString(",\n", *ostr);
writeCString("\t\t\"bytes_read\": ", *ostr);
writeText(progress.read_bytes.load(), *ostr);
writeChar('\n', *ostr);
writeCString("\t}", *ostr);
}
void JSONRowOutputFormat::onProgress(const Progress & value)
{
progress.incrementPiecewiseAtomically(value);
}
void registerOutputFormatProcessorJSON(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("JSON", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<JSONRowOutputFormat>(buf, sample, format_settings);
});
}
}

View File

@ -0,0 +1,84 @@
#pragma once
#include <Core/Block.h>
#include <IO/Progress.h>
#include <IO/WriteBuffer.h>
#include <Common/Stopwatch.h>
#include <Processors/Formats/IRowOutputFormat.h>
#include <Formats/FormatSettings.h>
namespace DB
{
/** Stream for output data in JSON format.
*/
class JSONRowOutputFormat : public IRowOutputFormat
{
public:
JSONRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & settings_);
String getName() const override { return "JSONRowOutputFormat"; }
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeFieldDelimiter() override;
void writeRowStartDelimiter() override;
void writeRowEndDelimiter() override;
void writeRowBetweenDelimiter() override;
void writePrefix() override;
void writeSuffix() override;
void writeMinExtreme(const Columns & columns, size_t row_num) override;
void writeMaxExtreme(const Columns & columns, size_t row_num) override;
void writeTotals(const Columns & columns, size_t row_num) override;
void writeBeforeTotals() override;
void writeAfterTotals() override;
void writeBeforeExtremes() override;
void writeAfterExtremes() override;
void writeLastSuffix() override;
void flush() override
{
ostr->next();
if (validating_ostr)
out.next();
}
void setRowsBeforeLimit(size_t rows_before_limit_) override
{
applied_limit = true;
rows_before_limit = rows_before_limit_;
}
void onProgress(const Progress & value) override;
String getContentType() const override { return "application/json; charset=UTF-8"; }
protected:
virtual void writeTotalsField(const IColumn & column, const IDataType & type, size_t row_num);
virtual void writeExtremesElement(const char * title, const Columns & columns, size_t row_num);
virtual void writeTotalsFieldDelimiter() { writeFieldDelimiter(); }
void writeRowsBeforeLimitAtLeast();
void writeStatistics();
std::unique_ptr<WriteBuffer> validating_ostr; /// Validates UTF-8 sequences, replaces bad sequences with replacement character.
WriteBuffer * ostr;
size_t field_number = 0;
size_t row_count = 0;
bool applied_limit = false;
size_t rows_before_limit = 0;
NamesAndTypes fields;
Progress progress;
Stopwatch watch;
FormatSettings settings;
};
}

View File

@ -0,0 +1,105 @@
#include <Processors/Formats/Impl/MySQLOutputFormat.h>
#include <Core/MySQLProtocol.h>
#include <Interpreters/ProcessList.h>
#include <Formats/FormatFactory.h>
#include <Interpreters/Context.h>
#include <iomanip>
#include <sstream>
namespace DB
{
using namespace MySQLProtocol;
MySQLOutputFormat::MySQLOutputFormat(WriteBuffer & out_, const Block & header, const Context & context, const FormatSettings & settings)
: IOutputFormat(header, out_)
, context(context)
, packet_sender(std::make_shared<PacketSender>(out, const_cast<size_t &>(context.sequence_id))) /// TODO: fix it
, format_settings(settings)
{
}
void MySQLOutputFormat::consume(Chunk chunk)
{
auto & header = getPort(PortKind::Main).getHeader();
if (!initialized)
{
initialized = true;
if (header.columns())
{
packet_sender->sendPacket(LengthEncodedNumber(header.columns()));
for (const ColumnWithTypeAndName & column : header.getColumnsWithTypeAndName())
{
ColumnDefinition column_definition(column.name, CharacterSet::binary, 0, ColumnType::MYSQL_TYPE_STRING,
0, 0);
packet_sender->sendPacket(column_definition);
}
if (!(context.client_capabilities & Capability::CLIENT_DEPRECATE_EOF))
{
packet_sender->sendPacket(EOF_Packet(0, 0));
}
}
}
size_t rows = chunk.getNumRows();
auto & columns = chunk.getColumns();
for (size_t i = 0; i < rows; i++)
{
ResultsetRow row_packet;
for (size_t col = 0; col < columns.size(); ++col)
{
String column_value;
WriteBufferFromString ostr(column_value);
header.getByPosition(col).type->serializeAsText(*columns[col], i, ostr, format_settings);
ostr.finish();
row_packet.appendColumn(std::move(column_value));
}
packet_sender->sendPacket(row_packet);
}
}
void MySQLOutputFormat::finalize()
{
QueryStatus * process_list_elem = context.getProcessListElement();
CurrentThread::finalizePerformanceCounters();
QueryStatusInfo info = process_list_elem->getInfo();
size_t affected_rows = info.written_rows;
std::stringstream human_readable_info;
human_readable_info << std::fixed << std::setprecision(3)
<< "Read " << info.read_rows << " rows, " << formatReadableSizeWithBinarySuffix(info.read_bytes) << " in " << info.elapsed_seconds << " sec., "
<< static_cast<size_t>(info.read_rows / info.elapsed_seconds) << " rows/sec., "
<< formatReadableSizeWithBinarySuffix(info.read_bytes / info.elapsed_seconds) << "/sec.";
auto & header = getPort(PortKind::Main).getHeader();
if (header.columns() == 0)
packet_sender->sendPacket(OK_Packet(0x0, context.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true);
else
if (context.client_capabilities & CLIENT_DEPRECATE_EOF)
packet_sender->sendPacket(OK_Packet(0xfe, context.client_capabilities, affected_rows, 0, 0, "", human_readable_info.str()), true);
else
packet_sender->sendPacket(EOF_Packet(0, 0), true);
}
void registerOutputFormatProcessorMySQLWrite(FormatFactory & factory)
{
factory.registerOutputFormatProcessor(
"MySQLWire", [](WriteBuffer & buf, const Block & sample, const Context & context, const FormatSettings & settings)
{
return std::make_shared<MySQLOutputFormat>(buf, sample, context, settings);
});
}
}

View File

@ -0,0 +1,39 @@
#pragma once
#include <Processors/Formats/IRowOutputFormat.h>
#include <Core/Block.h>
#include <Core/MySQLProtocol.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class IColumn;
class IDataType;
class WriteBuffer;
class Context;
/** A stream for outputting data in a binary line-by-line format.
*/
class MySQLOutputFormat: public IOutputFormat
{
public:
MySQLOutputFormat(WriteBuffer & out_, const Block & header, const Context & context, const FormatSettings & settings);
String getName() const override { return "MySQLOutputFormat"; }
void consume(Chunk) override;
void finalize() override;
private:
bool initialized = false;
const Context & context;
std::shared_ptr<MySQLProtocol::PacketSender> packet_sender;
FormatSettings format_settings;
};
}

View File

@ -0,0 +1,170 @@
#include <DataStreams/NativeBlockInputStream.h>
#include <DataStreams/NativeBlockOutputStream.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/IOutputFormat.h>
namespace DB
{
class NativeInputFormatFromNativeBlockInputStream : public IInputFormat
{
public:
NativeInputFormatFromNativeBlockInputStream(const Block & header, ReadBuffer & in_)
: IInputFormat(header, in_)
, stream(std::make_shared<NativeBlockInputStream>(in, header, 0))
{
}
String getName() const override { return "NativeInputFormatFromNativeBlockInputStream"; }
protected:
Chunk generate() override
{
/// TODO: do something with totals and extremes.
if (!read_prefix)
{
stream->readPrefix();
read_prefix = true;
}
auto block = stream->read();
if (!block)
{
if (!read_suffix)
{
stream->readSuffix();
read_suffix = true;
}
return Chunk();
}
assertBlocksHaveEqualStructure(getPort().getHeader(), block, getName());
block.checkNumberOfRows();
UInt64 num_rows = block.rows();
return Chunk(block.getColumns(), num_rows);
}
private:
std::shared_ptr<NativeBlockInputStream> stream;
bool read_prefix = false;
bool read_suffix = false;
};
class NativeOutputFormatFromNativeBlockOutputStream : public IOutputFormat
{
public:
NativeOutputFormatFromNativeBlockOutputStream(const Block & header, WriteBuffer & out_)
: IOutputFormat(header, out_)
, stream(std::make_shared<NativeBlockOutputStream>(out, 0, header))
{
}
String getName() const override { return "NativeOutputFormatFromNativeBlockOutputStream"; }
void setRowsBeforeLimit(size_t rows_before_limit) override
{
stream->setRowsBeforeLimit(rows_before_limit);
}
void onProgress(const Progress & progress) override
{
stream->onProgress(progress);
}
std::string getContentType() const override
{
return stream->getContentType();
}
protected:
void consume(Chunk chunk) override
{
writePrefixIfNot();
if (chunk)
{
auto block = getPort(PortKind::Main).getHeader();
block.setColumns(chunk.detachColumns());
stream->write(block);
}
}
void consumeTotals(Chunk chunk) override
{
writePrefixIfNot();
auto block = getPort(PortKind::Totals).getHeader();
block.setColumns(chunk.detachColumns());
stream->setTotals(block);
}
void consumeExtremes(Chunk chunk) override
{
writePrefixIfNot();
auto block = getPort(PortKind::Extremes).getHeader();
block.setColumns(chunk.detachColumns());
stream->setExtremes(block);
}
void finalize() override
{
writePrefixIfNot();
writeSuffixIfNot();
}
private:
std::shared_ptr<NativeBlockOutputStream> stream;
bool prefix_written = false;
bool suffix_written = false;
void writePrefixIfNot()
{
if (!prefix_written)
stream->writePrefix();
prefix_written = true;
}
void writeSuffixIfNot()
{
if (!suffix_written)
stream->writeSuffix();
suffix_written = true;
}
};
void registerInputFormatProcessorNative(FormatFactory & factory)
{
factory.registerInputFormatProcessor("Native", [](
ReadBuffer & buf,
const Block & sample,
const Context &,
const RowInputFormatParams &,
const FormatSettings &)
{
return std::make_shared<NativeInputFormatFromNativeBlockInputStream>(sample, buf);
});
}
void registerOutputFormatProcessorNative(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("Native", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings &)
{
return std::make_shared<NativeOutputFormatFromNativeBlockOutputStream>(sample, buf);
});
}
}

View File

@ -0,0 +1,31 @@
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
class NullOutputFormat : public IOutputFormat
{
public:
NullOutputFormat(const Block & header, WriteBuffer & out_) : IOutputFormat(header, out_) {}
String getName() const override { return "NullOutputFormat"; }
protected:
void consume(Chunk) override {}
};
void registerOutputFormatProcessorNull(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("Null", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings &)
{
return std::make_shared<NullOutputFormat>(sample, buf);
});
}
}

View File

@ -0,0 +1,113 @@
#include <Core/Block.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/Impl/ODBCDriver2BlockOutputFormat.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Core/iostream_debug_helpers.h>
namespace DB
{
ODBCDriver2BlockOutputFormat::ODBCDriver2BlockOutputFormat(
WriteBuffer & out_, const Block & header, const FormatSettings & format_settings)
: IOutputFormat(header, out_), format_settings(format_settings)
{
}
static void writeODBCString(WriteBuffer & out, const std::string & str)
{
writeIntBinary(Int32(str.size()), out);
out.write(str.data(), str.size());
}
void ODBCDriver2BlockOutputFormat::writeRow(const Block & header, const Columns & columns, size_t row_idx, std::string & buffer)
{
size_t num_columns = columns.size();
for (size_t column_idx = 0; column_idx < num_columns; ++column_idx)
{
buffer.clear();
auto & column = columns[column_idx];
if (column->isNullAt(row_idx))
{
writeIntBinary(Int32(-1), out);
}
else
{
{
WriteBufferFromString text_out(buffer);
header.getByPosition(row_idx).type->serializeAsText(*column, row_idx, text_out, format_settings);
}
writeODBCString(out, buffer);
}
}
}
void ODBCDriver2BlockOutputFormat::write(Chunk chunk, PortKind port_kind)
{
String text_value;
auto & header = getPort(port_kind).getHeader();
auto & columns = chunk.getColumns();
const size_t rows = chunk.getNumRows();
for (size_t i = 0; i < rows; ++i)
writeRow(header, columns, i, text_value);
}
void ODBCDriver2BlockOutputFormat::consume(Chunk chunk)
{
writePrefixIfNot();
write(std::move(chunk), PortKind::Main);
}
void ODBCDriver2BlockOutputFormat::consumeTotals(Chunk chunk)
{
writePrefixIfNot();
write(std::move(chunk), PortKind::Totals);
}
void ODBCDriver2BlockOutputFormat::finalize()
{
writePrefixIfNot();
}
void ODBCDriver2BlockOutputFormat::writePrefix()
{
auto & header = getPort(PortKind::Main).getHeader();
const size_t columns = header.columns();
/// Number of header rows.
writeIntBinary(Int32(2), out);
/// Names of columns.
/// Number of columns + 1 for first name column.
writeIntBinary(Int32(columns + 1), out);
writeODBCString(out, "name");
for (size_t i = 0; i < columns; ++i)
{
const ColumnWithTypeAndName & col = header.getByPosition(i);
writeODBCString(out, col.name);
}
/// Types of columns.
writeIntBinary(Int32(columns + 1), out);
writeODBCString(out, "type");
for (size_t i = 0; i < columns; ++i)
{
const ColumnWithTypeAndName & col = header.getByPosition(i);
writeODBCString(out, col.type->getName());
}
}
void registerOutputFormatProcessorODBCDriver2(FormatFactory & factory)
{
factory.registerOutputFormatProcessor(
"ODBCDriver2", [](WriteBuffer & buf, const Block & sample, const Context &, const FormatSettings & format_settings)
{
return std::make_shared<ODBCDriver2BlockOutputFormat>(buf, sample, format_settings);
});
}
}

View File

@ -0,0 +1,55 @@
#pragma once
#include <string>
#include <Core/Block.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class WriteBuffer;
/** A data format designed to simplify the implementation of the ODBC driver.
* ODBC driver is designed to be build for different platforms without dependencies from the main code,
* so the format is made that way so that it can be as easy as possible to parse it.
* A header is displayed with the required information.
* The data is then output in the order of the rows. Each value is displayed as follows: length in Int32 format (-1 for NULL), then data in text form.
*/
class ODBCDriver2BlockOutputFormat final : public IOutputFormat
{
public:
ODBCDriver2BlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings);
String getName() const override { return "ODBCDriver2BlockOutputFormat"; }
void consume(Chunk) override;
void consumeTotals(Chunk) override;
void finalize() override;
std::string getContentType() const override
{
return "application/octet-stream";
}
private:
const FormatSettings format_settings;
bool prefix_written = false;
void writePrefixIfNot()
{
if (!prefix_written)
writePrefix();
prefix_written = true;
}
void writeRow(const Block & header, const Columns & columns, size_t row_idx, std::string & buffer);
void write(Chunk chunk, PortKind port_kind);
void writePrefix();
};
}

View File

@ -0,0 +1,79 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Core/Block.h>
#include <Processors/Formats/Impl/ODBCDriverBlockOutputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
ODBCDriverBlockOutputFormat::ODBCDriverBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings)
: IOutputFormat(header, out_), format_settings(format_settings)
{
}
void ODBCDriverBlockOutputFormat::consume(Chunk chunk)
{
writePrefixIfNot();
const size_t num_rows = chunk.getNumRows();
const size_t num_columns = chunk.getNumColumns();
auto & columns = chunk.getColumns();
auto & header = getPort(PortKind::Main).getHeader();
String text_value;
for (size_t i = 0; i < num_rows; ++i)
{
for (size_t j = 0; j < num_columns; ++j)
{
text_value.resize(0);
auto & column = columns[j];
auto & type = header.getByPosition(j).type;
{
WriteBufferFromString text_out(text_value);
type->serializeAsText(*column, i, text_out, format_settings);
}
writeStringBinary(text_value, out);
}
}
}
void ODBCDriverBlockOutputFormat::writePrefix()
{
auto & header = getPort(PortKind::Main).getHeader();
const size_t columns = header.columns();
/// Number of columns.
writeVarUInt(columns, out);
/// Names and types of columns.
for (size_t i = 0; i < columns; ++i)
{
const ColumnWithTypeAndName & col = header.getByPosition(i);
writeStringBinary(col.name, out);
writeStringBinary(col.type->getName(), out);
}
}
void ODBCDriverBlockOutputFormat::finalize()
{
writePrefixIfNot();
}
void registerOutputFormatProcessorODBCDriver(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("ODBCDriver", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<ODBCDriverBlockOutputFormat>(buf, sample, format_settings);
});
}
}

View File

@ -0,0 +1,48 @@
#pragma once
#include <string>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatSettings.h>
#include <Core/Block.h>
namespace DB
{
class WriteBuffer;
/** A data format designed to simplify the implementation of the ODBC driver.
* ODBC driver is designed to be build for different platforms without dependencies from the main code,
* so the format is made that way so that it can be as easy as possible to parse it.
* A header is displayed with the required information.
* The data is then output in the order of the rows. Each value is displayed as follows: length in VarUInt format, then data in text form.
*/
class ODBCDriverBlockOutputFormat : public IOutputFormat
{
public:
ODBCDriverBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings);
String getName() const override { return "ODBCDriverBlockOutputFormat"; }
void consume(Chunk) override;
void finalize() override;
std::string getContentType() const override { return "application/octet-stream"; }
private:
const FormatSettings format_settings;
bool prefix_written = false;
void writePrefixIfNot()
{
if (!prefix_written)
writePrefix();
prefix_written = true;
}
void writePrefix();
};
}

View File

@ -0,0 +1,497 @@
#include "config_formats.h"
#if USE_PARQUET
#include "ParquetBlockInputFormat.h"
#include <algorithm>
#include <iterator>
#include <vector>
// TODO: clear includes
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/IColumn.h>
#include <Core/ColumnWithTypeAndName.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/DataTypesNumber.h>
#include <Formats/FormatFactory.h>
#include <IO/BufferBase.h>
#include <IO/ReadBufferFromMemory.h>
#include <IO/WriteBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include <Interpreters/castColumn.h>
#include <common/DateLUTImpl.h>
#include <ext/range.h>
#include <arrow/api.h>
//#include <arrow/buffer.h>
//#include <arrow/io/api.h>
#include <parquet/arrow/reader.h>
//#include <parquet/arrow/writer.h>
//#include <parquet/exception.h>
#include <parquet/file_reader.h>
#include <Core/iostream_debug_helpers.h> // REMOVE ME
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_TYPE;
extern const int VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE;
extern const int CANNOT_READ_ALL_DATA;
extern const int EMPTY_DATA_PASSED;
extern const int SIZES_OF_COLUMNS_DOESNT_MATCH;
extern const int CANNOT_CONVERT_TYPE;
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
extern const int THERE_IS_NO_COLUMN;
}
ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header, const Context & context)
: IInputFormat(std::move(header), in_), context{context}
{
}
/// Inserts numeric data right into internal column data to reduce an overhead
template <typename NumericType, typename VectorType = ColumnVector<NumericType>>
static void fillColumnWithNumericData(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & internal_column)
{
auto & column_data = static_cast<VectorType &>(*internal_column).getData();
column_data.reserve(arrow_column->length());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
std::shared_ptr<arrow::Array> chunk = arrow_column->data()->chunk(chunk_i);
/// buffers[0] is a null bitmap and buffers[1] are actual values
std::shared_ptr<arrow::Buffer> buffer = chunk->data()->buffers[1];
const auto * raw_data = reinterpret_cast<const NumericType *>(buffer->data());
column_data.insert_assume_reserved(raw_data, raw_data + chunk->length());
}
}
/// Inserts chars and offsets right into internal column data to reduce an overhead.
/// Internal offsets are shifted by one to the right in comparison with Arrow ones. So the last offset should map to the end of all chars.
/// Also internal strings are null terminated.
static void fillColumnWithStringData(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & internal_column)
{
PaddedPODArray<UInt8> & column_chars_t = static_cast<ColumnString &>(*internal_column).getChars();
PaddedPODArray<UInt64> & column_offsets = static_cast<ColumnString &>(*internal_column).getOffsets();
size_t chars_t_size = 0;
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
arrow::BinaryArray & chunk = static_cast<arrow::BinaryArray &>(*(arrow_column->data()->chunk(chunk_i)));
const size_t chunk_length = chunk.length();
chars_t_size += chunk.value_offset(chunk_length - 1) + chunk.value_length(chunk_length - 1);
chars_t_size += chunk_length; /// additional space for null bytes
}
column_chars_t.reserve(chars_t_size);
column_offsets.reserve(arrow_column->length());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
arrow::BinaryArray & chunk = static_cast<arrow::BinaryArray &>(*(arrow_column->data()->chunk(chunk_i)));
std::shared_ptr<arrow::Buffer> buffer = chunk.value_data();
const size_t chunk_length = chunk.length();
for (size_t offset_i = 0; offset_i != chunk_length; ++offset_i)
{
if (!chunk.IsNull(offset_i) && buffer)
{
const UInt8 * raw_data = buffer->data() + chunk.value_offset(offset_i);
column_chars_t.insert_assume_reserved(raw_data, raw_data + chunk.value_length(offset_i));
}
column_chars_t.emplace_back('\0');
column_offsets.emplace_back(column_chars_t.size());
}
}
}
static void fillColumnWithBooleanData(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & internal_column)
{
auto & column_data = static_cast<ColumnVector<UInt8> &>(*internal_column).getData();
column_data.resize(arrow_column->length());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
arrow::BooleanArray & chunk = static_cast<arrow::BooleanArray &>(*(arrow_column->data()->chunk(chunk_i)));
/// buffers[0] is a null bitmap and buffers[1] are actual values
std::shared_ptr<arrow::Buffer> buffer = chunk.data()->buffers[1];
for (size_t bool_i = 0; bool_i != static_cast<size_t>(chunk.length()); ++bool_i)
column_data[bool_i] = chunk.Value(bool_i);
}
}
/// Arrow stores Parquet::DATE in Int32, while ClickHouse stores Date in UInt16. Therefore, it should be checked before saving
static void fillColumnWithDate32Data(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & internal_column)
{
PaddedPODArray<UInt16> & column_data = static_cast<ColumnVector<UInt16> &>(*internal_column).getData();
column_data.reserve(arrow_column->length());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
arrow::Date32Array & chunk = static_cast<arrow::Date32Array &>(*(arrow_column->data()->chunk(chunk_i)));
for (size_t value_i = 0, length = static_cast<size_t>(chunk.length()); value_i < length; ++value_i)
{
UInt32 days_num = static_cast<UInt32>(chunk.Value(value_i));
if (days_num > DATE_LUT_MAX_DAY_NUM)
{
// TODO: will it rollback correctly?
throw Exception{"Input value " + std::to_string(days_num) + " of a column \"" + arrow_column->name()
+ "\" is greater than "
"max allowed Date value, which is "
+ std::to_string(DATE_LUT_MAX_DAY_NUM),
ErrorCodes::VALUE_IS_OUT_OF_RANGE_OF_DATA_TYPE};
}
column_data.emplace_back(days_num);
}
}
}
/// Arrow stores Parquet::DATETIME in Int64, while ClickHouse stores DateTime in UInt32. Therefore, it should be checked before saving
static void fillColumnWithDate64Data(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & internal_column)
{
auto & column_data = static_cast<ColumnVector<UInt32> &>(*internal_column).getData();
column_data.reserve(arrow_column->length());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
auto & chunk = static_cast<arrow::Date64Array &>(*(arrow_column->data()->chunk(chunk_i)));
for (size_t value_i = 0, length = static_cast<size_t>(chunk.length()); value_i < length; ++value_i)
{
auto timestamp = static_cast<UInt32>(chunk.Value(value_i) / 1000); // Always? in ms
column_data.emplace_back(timestamp);
}
}
}
static void fillColumnWithTimestampData(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & internal_column)
{
auto & column_data = static_cast<ColumnVector<UInt32> &>(*internal_column).getData();
column_data.reserve(arrow_column->length());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
auto & chunk = static_cast<arrow::TimestampArray &>(*(arrow_column->data()->chunk(chunk_i)));
const auto & type = static_cast<const ::arrow::TimestampType &>(*chunk.type());
UInt32 divide = 1;
const auto unit = type.unit();
switch (unit)
{
case arrow::TimeUnit::SECOND:
divide = 1;
break;
case arrow::TimeUnit::MILLI:
divide = 1000;
break;
case arrow::TimeUnit::MICRO:
divide = 1000000;
break;
case arrow::TimeUnit::NANO:
divide = 1000000000;
break;
}
for (size_t value_i = 0, length = static_cast<size_t>(chunk.length()); value_i < length; ++value_i)
{
auto timestamp = static_cast<UInt32>(chunk.Value(value_i) / divide); // ms! TODO: check other 's' 'ns' ...
column_data.emplace_back(timestamp);
}
}
}
static void fillColumnWithDecimalData(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & internal_column)
{
auto & column = static_cast<ColumnDecimal<Decimal128> &>(*internal_column);
auto & column_data = column.getData();
column_data.reserve(arrow_column->length());
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->data()->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
auto & chunk = static_cast<arrow::DecimalArray &>(*(arrow_column->data()->chunk(chunk_i)));
for (size_t value_i = 0, length = static_cast<size_t>(chunk.length()); value_i < length; ++value_i)
{
column_data.emplace_back(chunk.IsNull(value_i) ? Decimal128(0) : *reinterpret_cast<const Decimal128 *>(chunk.Value(value_i))); // TODO: copy column
}
}
}
/// Creates a null bytemap from arrow's null bitmap
static void fillByteMapFromArrowColumn(std::shared_ptr<arrow::Column> & arrow_column, MutableColumnPtr & bytemap)
{
PaddedPODArray<UInt8> & bytemap_data = static_cast<ColumnVector<UInt8> &>(*bytemap).getData();
bytemap_data.reserve(arrow_column->length());
for (size_t chunk_i = 0; chunk_i != static_cast<size_t>(arrow_column->data()->num_chunks()); ++chunk_i)
{
std::shared_ptr<arrow::Array> chunk = arrow_column->data()->chunk(chunk_i);
for (size_t value_i = 0; value_i != static_cast<size_t>(chunk->length()); ++value_i)
bytemap_data.emplace_back(chunk->IsNull(value_i));
}
}
# define FOR_ARROW_NUMERIC_TYPES(M) \
M(arrow::Type::UINT8, UInt8) \
M(arrow::Type::INT8, Int8) \
M(arrow::Type::UINT16, UInt16) \
M(arrow::Type::INT16, Int16) \
M(arrow::Type::UINT32, UInt32) \
M(arrow::Type::INT32, Int32) \
M(arrow::Type::UINT64, UInt64) \
M(arrow::Type::INT64, Int64) \
M(arrow::Type::FLOAT, Float32) \
M(arrow::Type::DOUBLE, Float64)
//M(arrow::Type::HALF_FLOAT, Float32) // TODO
using NameToColumnPtr = std::unordered_map<std::string, std::shared_ptr<arrow::Column>>;
const std::unordered_map<arrow::Type::type, std::shared_ptr<IDataType>> arrow_type_to_internal_type = {
//{arrow::Type::DECIMAL, std::make_shared<DataTypeDecimal>()},
{arrow::Type::UINT8, std::make_shared<DataTypeUInt8>()},
{arrow::Type::INT8, std::make_shared<DataTypeInt8>()},
{arrow::Type::UINT16, std::make_shared<DataTypeUInt16>()},
{arrow::Type::INT16, std::make_shared<DataTypeInt16>()},
{arrow::Type::UINT32, std::make_shared<DataTypeUInt32>()},
{arrow::Type::INT32, std::make_shared<DataTypeInt32>()},
{arrow::Type::UINT64, std::make_shared<DataTypeUInt64>()},
{arrow::Type::INT64, std::make_shared<DataTypeInt64>()},
{arrow::Type::HALF_FLOAT, std::make_shared<DataTypeFloat32>()},
{arrow::Type::FLOAT, std::make_shared<DataTypeFloat32>()},
{arrow::Type::DOUBLE, std::make_shared<DataTypeFloat64>()},
{arrow::Type::BOOL, std::make_shared<DataTypeUInt8>()},
//{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
//{arrow::Type::DATE32, std::make_shared<DataTypeDateTime>()},
{arrow::Type::DATE64, std::make_shared<DataTypeDateTime>()},
{arrow::Type::TIMESTAMP, std::make_shared<DataTypeDateTime>()},
//{arrow::Type::TIME32, std::make_shared<DataTypeDateTime>()},
{arrow::Type::STRING, std::make_shared<DataTypeString>()},
{arrow::Type::BINARY, std::make_shared<DataTypeString>()},
//{arrow::Type::FIXED_SIZE_BINARY, std::make_shared<DataTypeString>()},
//{arrow::Type::UUID, std::make_shared<DataTypeString>()},
// TODO: add other types that are convertable to internal ones:
// 0. ENUM?
// 1. UUID -> String
// 2. JSON -> String
// Full list of types: contrib/arrow/cpp/src/arrow/type.h
};
Chunk ParquetBlockInputFormat::generate()
{
Chunk res;
Columns columns_list;
UInt64 num_rows = 0;
auto & header = getPort().getHeader();
columns_list.reserve(header.rows());
if (!in.eof())
{
/*
First we load whole stream into string (its very bad and limiting .parquet file size to half? of RAM)
Then producing blocks for every row_group (dont load big .parquet files with one row_group - it can eat x10+ RAM from .parquet file size)
*/
if (row_group_current < row_group_total)
throw Exception{"Got new data, but data from previous chunks not readed " + std::to_string(row_group_current) + "/" + std::to_string(row_group_total), ErrorCodes::CANNOT_READ_ALL_DATA};
file_data.clear();
{
WriteBufferFromString file_buffer(file_data);
copyData(in, file_buffer);
}
buffer = std::make_unique<arrow::Buffer>(file_data);
// TODO: maybe use parquet::RandomAccessSource?
auto reader = parquet::ParquetFileReader::Open(std::make_shared<::arrow::io::BufferReader>(*buffer));
file_reader = std::make_unique<parquet::arrow::FileReader>(::arrow::default_memory_pool(), std::move(reader));
row_group_total = file_reader->num_row_groups();
row_group_current = 0;
}
//DUMP(row_group_current, row_group_total);
if (row_group_current >= row_group_total)
return res;
// TODO: also catch a ParquetException thrown by filereader?
//arrow::Status read_status = filereader.ReadTable(&table);
std::shared_ptr<arrow::Table> table;
arrow::Status read_status = file_reader->ReadRowGroup(row_group_current, &table);
if (!read_status.ok())
throw Exception{"Error while reading parquet data: " + read_status.ToString(), ErrorCodes::CANNOT_READ_ALL_DATA};
if (0 == table->num_rows())
throw Exception{"Empty table in input data", ErrorCodes::EMPTY_DATA_PASSED};
if (header.columns() > static_cast<size_t>(table->num_columns()))
// TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable?
throw Exception{"Number of columns is less than the table has", ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH};
++row_group_current;
NameToColumnPtr name_to_column_ptr;
for (size_t i = 0, num_columns = static_cast<size_t>(table->num_columns()); i < num_columns; ++i)
{
std::shared_ptr<arrow::Column> arrow_column = table->column(i);
name_to_column_ptr[arrow_column->name()] = arrow_column;
}
for (size_t column_i = 0, columns = header.columns(); column_i < columns; ++column_i)
{
ColumnWithTypeAndName header_column = header.getByPosition(column_i);
if (name_to_column_ptr.find(header_column.name) == name_to_column_ptr.end())
// TODO: What if some columns were not presented? Insert NULLs? What if a column is not nullable?
throw Exception{"Column \"" + header_column.name + "\" is not presented in input data", ErrorCodes::THERE_IS_NO_COLUMN};
std::shared_ptr<arrow::Column> arrow_column = name_to_column_ptr[header_column.name];
arrow::Type::type arrow_type = arrow_column->type()->id();
// TODO: check if a column is const?
if (!header_column.type->isNullable() && arrow_column->null_count())
{
throw Exception{"Can not insert NULL data into non-nullable column \"" + header_column.name + "\"",
ErrorCodes::CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN};
}
const bool target_column_is_nullable = header_column.type->isNullable() || arrow_column->null_count();
DataTypePtr internal_nested_type;
if (arrow_type == arrow::Type::DECIMAL)
{
const auto decimal_type = static_cast<arrow::DecimalType *>(arrow_column->type().get());
internal_nested_type = std::make_shared<DataTypeDecimal<Decimal128>>(decimal_type->precision(), decimal_type->scale());
}
else if (arrow_type_to_internal_type.find(arrow_type) != arrow_type_to_internal_type.end())
{
internal_nested_type = arrow_type_to_internal_type.at(arrow_type);
}
else
{
throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name()
+ "\" is not supported for conversion from a Parquet data format",
ErrorCodes::CANNOT_CONVERT_TYPE};
}
const DataTypePtr internal_type = target_column_is_nullable ? makeNullable(internal_nested_type) : internal_nested_type;
const std::string internal_nested_type_name = internal_nested_type->getName();
const DataTypePtr column_nested_type = header_column.type->isNullable()
? static_cast<const DataTypeNullable *>(header_column.type.get())->getNestedType()
: header_column.type;
const DataTypePtr column_type = header_column.type;
const std::string column_nested_type_name = column_nested_type->getName();
ColumnWithTypeAndName column;
column.name = header_column.name;
column.type = internal_type;
/// Data
MutableColumnPtr read_column = internal_nested_type->createColumn();
switch (arrow_type)
{
case arrow::Type::STRING:
case arrow::Type::BINARY:
//case arrow::Type::FIXED_SIZE_BINARY:
fillColumnWithStringData(arrow_column, read_column);
break;
case arrow::Type::BOOL:
fillColumnWithBooleanData(arrow_column, read_column);
break;
case arrow::Type::DATE32:
fillColumnWithDate32Data(arrow_column, read_column);
break;
case arrow::Type::DATE64:
fillColumnWithDate64Data(arrow_column, read_column);
break;
case arrow::Type::TIMESTAMP:
fillColumnWithTimestampData(arrow_column, read_column);
break;
case arrow::Type::DECIMAL:
//fillColumnWithNumericData<Decimal128, ColumnDecimal<Decimal128>>(arrow_column, read_column); // Have problems with trash values under NULL, but faster
fillColumnWithDecimalData(arrow_column, read_column /*, internal_nested_type*/);
break;
# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \
case ARROW_NUMERIC_TYPE: \
fillColumnWithNumericData<CPP_NUMERIC_TYPE>(arrow_column, read_column); \
break;
FOR_ARROW_NUMERIC_TYPES(DISPATCH)
# undef DISPATCH
// TODO: support TIMESTAMP_MICROS and TIMESTAMP_MILLIS with truncated micro- and milliseconds?
// TODO: read JSON as a string?
// TODO: read UUID as a string?
default:
throw Exception{"Unsupported parquet type \"" + arrow_column->type()->name() + "\" of an input column \""
+ arrow_column->name() + "\"",
ErrorCodes::UNKNOWN_TYPE};
}
if (column.type->isNullable())
{
MutableColumnPtr null_bytemap = DataTypeUInt8().createColumn();
fillByteMapFromArrowColumn(arrow_column, null_bytemap);
column.column = ColumnNullable::create(std::move(read_column), std::move(null_bytemap));
}
else
{
column.column = std::move(read_column);
}
column.column = castColumn(column, column_type, context);
column.type = column_type;
num_rows = column.column->size();
columns_list.push_back(std::move(column.column));
}
res.setColumns(columns_list, num_rows);
return res;
}
void registerInputFormatProcessorParquet(FormatFactory & factory)
{
factory.registerInputFormatProcessor(
"Parquet",
[](ReadBuffer & buf,
const Block & sample,
const Context & context,
const RowInputFormatParams &,
const FormatSettings & /* settings */){ return std::make_shared<ParquetBlockInputFormat>(buf, sample, context); });
}
}
#else
namespace DB
{
class FormatFactory;
void registerInputFormatProcessorParquet(FormatFactory &)
{
}
}
#endif

View File

@ -0,0 +1,43 @@
#pragma once
#include <Common/config.h>
#if USE_PARQUET
# include <Processors/Formats/IInputFormat.h>
//# include <parquet/file_reader.h>
//# include <parquet/arrow/reader.h>
//# include <arrow/buffer.h>
namespace parquet { namespace arrow { class FileReader; } }
namespace arrow { class Buffer; }
namespace DB
{
class Context;
class ParquetBlockInputFormat: public IInputFormat
{
public:
ParquetBlockInputFormat(ReadBuffer & in_, Block header, const Context & context);
String getName() const override { return "ParquetBlockInputFormat"; }
protected:
Chunk generate() override;
private:
// TODO: check that this class implements every part of its parent
const Context & context;
std::unique_ptr<parquet::arrow::FileReader> file_reader;
std::string file_data;
std::unique_ptr<arrow::Buffer> buffer;
int row_group_total = 0;
int row_group_current = 0;
};
}
#endif

View File

@ -0,0 +1,451 @@
#include "config_formats.h"
#if USE_PARQUET
# include "ParquetBlockOutputFormat.h"
// TODO: clean includes
# include <Columns/ColumnDecimal.h>
# include <Columns/ColumnFixedString.h>
# include <Columns/ColumnNullable.h>
# include <Columns/ColumnString.h>
# include <Columns/ColumnVector.h>
# include <Columns/ColumnsNumber.h>
# include <Core/ColumnWithTypeAndName.h>
# include <Core/callOnTypeIndex.h>
# include <DataTypes/DataTypeDateTime.h>
# include <DataTypes/DataTypeNullable.h>
# include <DataTypes/DataTypesDecimal.h>
# include <DataStreams/SquashingBlockOutputStream.h>
# include <Formats/FormatFactory.h>
# include <IO/WriteHelpers.h>
# include <arrow/api.h>
# include <arrow/io/api.h>
# include <arrow/util/decimal.h>
# include <parquet/arrow/writer.h>
# include <parquet/exception.h>
# include <parquet/util/memory.h>
# include <Core/iostream_debug_helpers.h> // REMOVE ME
namespace DB
{
namespace ErrorCodes
{
extern const int UNKNOWN_EXCEPTION;
extern const int UNKNOWN_TYPE;
}
ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings)
: IOutputFormat(header, out_), format_settings{format_settings}
{
}
static void checkStatus(arrow::Status & status, const std::string & column_name)
{
if (!status.ok())
throw Exception{"Error with a parquet column \"" + column_name + "\": " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION};
}
template <typename NumericType, typename ArrowBuilderType>
static void fillArrowArrayWithNumericColumnData(
ColumnPtr write_column, std::shared_ptr<arrow::Array> & arrow_array, const PaddedPODArray<UInt8> * null_bytemap)
{
const PaddedPODArray<NumericType> & internal_data = static_cast<const ColumnVector<NumericType> &>(*write_column).getData();
ArrowBuilderType builder;
arrow::Status status;
const UInt8 * arrow_null_bytemap_raw_ptr = nullptr;
PaddedPODArray<UInt8> arrow_null_bytemap;
if (null_bytemap)
{
/// Invert values since Arrow interprets 1 as a non-null value, while CH as a null
arrow_null_bytemap.reserve(null_bytemap->size());
for (size_t i = 0, size = null_bytemap->size(); i < size; ++i)
arrow_null_bytemap.emplace_back(1 ^ (*null_bytemap)[i]);
arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data();
}
status = builder.AppendValues(internal_data.data(), internal_data.size(), arrow_null_bytemap_raw_ptr);
checkStatus(status, write_column->getName());
status = builder.Finish(&arrow_array);
checkStatus(status, write_column->getName());
}
template <typename ColumnType>
static void fillArrowArrayWithStringColumnData(
ColumnPtr write_column, std::shared_ptr<arrow::Array> & arrow_array, const PaddedPODArray<UInt8> * null_bytemap)
{
const auto & internal_column = static_cast<const ColumnType &>(*write_column);
arrow::StringBuilder builder;
arrow::Status status;
for (size_t string_i = 0, size = internal_column.size(); string_i < size; ++string_i)
{
if (null_bytemap && (*null_bytemap)[string_i])
{
status = builder.AppendNull();
}
else
{
StringRef string_ref = internal_column.getDataAt(string_i);
status = builder.Append(string_ref.data, string_ref.size);
}
checkStatus(status, write_column->getName());
}
status = builder.Finish(&arrow_array);
checkStatus(status, write_column->getName());
}
static void fillArrowArrayWithDateColumnData(
ColumnPtr write_column, std::shared_ptr<arrow::Array> & arrow_array, const PaddedPODArray<UInt8> * null_bytemap)
{
const PaddedPODArray<UInt16> & internal_data = static_cast<const ColumnVector<UInt16> &>(*write_column).getData();
//arrow::Date32Builder date_builder;
arrow::UInt16Builder builder;
arrow::Status status;
for (size_t value_i = 0, size = internal_data.size(); value_i < size; ++value_i)
{
if (null_bytemap && (*null_bytemap)[value_i])
status = builder.AppendNull();
else
/// Implicitly converts UInt16 to Int32
status = builder.Append(internal_data[value_i]);
checkStatus(status, write_column->getName());
}
status = builder.Finish(&arrow_array);
checkStatus(status, write_column->getName());
}
static void fillArrowArrayWithDateTimeColumnData(
ColumnPtr write_column, std::shared_ptr<arrow::Array> & arrow_array, const PaddedPODArray<UInt8> * null_bytemap)
{
auto & internal_data = static_cast<const ColumnVector<UInt32> &>(*write_column).getData();
//arrow::Date64Builder builder;
arrow::UInt32Builder builder;
arrow::Status status;
for (size_t value_i = 0, size = internal_data.size(); value_i < size; ++value_i)
{
if (null_bytemap && (*null_bytemap)[value_i])
status = builder.AppendNull();
else
/// Implicitly converts UInt16 to Int32
//status = date_builder.Append(static_cast<int64_t>(internal_data[value_i]) * 1000); // now ms. TODO check other units
status = builder.Append(internal_data[value_i]);
checkStatus(status, write_column->getName());
}
status = builder.Finish(&arrow_array);
checkStatus(status, write_column->getName());
}
template <typename DataType>
static void fillArrowArrayWithDecimalColumnData(
ColumnPtr write_column,
std::shared_ptr<arrow::Array> & arrow_array,
const PaddedPODArray<UInt8> * null_bytemap,
const DataType * decimal_type)
{
const auto & column = static_cast<const typename DataType::ColumnType &>(*write_column);
arrow::DecimalBuilder builder(arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale()));
arrow::Status status;
for (size_t value_i = 0, size = column.size(); value_i < size; ++value_i)
{
if (null_bytemap && (*null_bytemap)[value_i])
status = builder.AppendNull();
else
status = builder.Append(
arrow::Decimal128(reinterpret_cast<const uint8_t *>(&column.getElement(value_i).value))); // TODO: try copy column
checkStatus(status, write_column->getName());
}
status = builder.Finish(&arrow_array);
checkStatus(status, write_column->getName());
/* TODO column copy
const auto & internal_data = static_cast<const typename DataType::ColumnType &>(*write_column).getData();
//ArrowBuilderType numeric_builder;
arrow::DecimalBuilder builder(arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale()));
arrow::Status status;
const uint8_t * arrow_null_bytemap_raw_ptr = nullptr;
PaddedPODArray<UInt8> arrow_null_bytemap;
if (null_bytemap)
{
/// Invert values since Arrow interprets 1 as a non-null value, while CH as a null
arrow_null_bytemap.reserve(null_bytemap->size());
for (size_t i = 0, size = null_bytemap->size(); i < size; ++i)
arrow_null_bytemap.emplace_back(1 ^ (*null_bytemap)[i]);
arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data();
}
status = builder.AppendValues(reinterpret_cast<const uint8_t*>(internal_data.data()), internal_data.size(), arrow_null_bytemap_raw_ptr);
checkStatus(status, write_column->getName());
status = builder.Finish(&arrow_array);
checkStatus(status, write_column->getName());
*/
}
# define FOR_INTERNAL_NUMERIC_TYPES(M) \
M(UInt8, arrow::UInt8Builder) \
M(Int8, arrow::Int8Builder) \
M(UInt16, arrow::UInt16Builder) \
M(Int16, arrow::Int16Builder) \
M(UInt32, arrow::UInt32Builder) \
M(Int32, arrow::Int32Builder) \
M(UInt64, arrow::UInt64Builder) \
M(Int64, arrow::Int64Builder) \
M(Float32, arrow::FloatBuilder) \
M(Float64, arrow::DoubleBuilder)
const std::unordered_map<String, std::shared_ptr<arrow::DataType>> internal_type_to_arrow_type = {
{"UInt8", arrow::uint8()},
{"Int8", arrow::int8()},
{"UInt16", arrow::uint16()},
{"Int16", arrow::int16()},
{"UInt32", arrow::uint32()},
{"Int32", arrow::int32()},
{"UInt64", arrow::uint64()},
{"Int64", arrow::int64()},
{"Float32", arrow::float32()},
{"Float64", arrow::float64()},
//{"Date", arrow::date64()},
//{"Date", arrow::date32()},
{"Date", arrow::uint16()}, // CHECK
//{"DateTime", arrow::date64()}, // BUG! saves as date32
{"DateTime", arrow::uint32()},
// TODO: ClickHouse can actually store non-utf8 strings!
{"String", arrow::utf8()},
{"FixedString", arrow::utf8()},
};
static const PaddedPODArray<UInt8> * extractNullBytemapPtr(ColumnPtr column)
{
ColumnPtr null_column = static_cast<const ColumnNullable &>(*column).getNullMapColumnPtr();
const PaddedPODArray<UInt8> & null_bytemap = static_cast<const ColumnVector<UInt8> &>(*null_column).getData();
return &null_bytemap;
}
class OstreamOutputStream : public parquet::OutputStream
{
public:
explicit OstreamOutputStream(WriteBuffer & ostr_) : ostr(ostr_) {}
virtual ~OstreamOutputStream() {}
virtual void Close() {}
virtual int64_t Tell() { return total_length; }
virtual void Write(const uint8_t * data, int64_t length)
{
ostr.write(reinterpret_cast<const char *>(data), length);
total_length += length;
}
private:
WriteBuffer & ostr;
int64_t total_length = 0;
PARQUET_DISALLOW_COPY_AND_ASSIGN(OstreamOutputStream);
};
void ParquetBlockOutputFormat::consume(Chunk chunk)
{
auto & header = getPort(PortKind::Main).getHeader();
const size_t columns_num = chunk.getNumColumns();
/// For arrow::Schema and arrow::Table creation
std::vector<std::shared_ptr<arrow::Field>> arrow_fields;
std::vector<std::shared_ptr<arrow::Array>> arrow_arrays;
arrow_fields.reserve(columns_num);
arrow_arrays.reserve(columns_num);
for (size_t column_i = 0; column_i < columns_num; ++column_i)
{
// TODO: constructed every iteration
ColumnWithTypeAndName column = header.safeGetByPosition(column_i);
column.column = chunk.getColumns()[column_i];
const bool is_column_nullable = column.type->isNullable();
const auto & column_nested_type
= is_column_nullable ? static_cast<const DataTypeNullable *>(column.type.get())->getNestedType() : column.type;
const std::string column_nested_type_name = column_nested_type->getFamilyName();
if (isDecimal(column_nested_type))
{
const auto add_decimal_field = [&](const auto & types) -> bool {
using Types = std::decay_t<decltype(types)>;
using ToDataType = typename Types::LeftType;
if constexpr (
std::is_same_v<
ToDataType,
DataTypeDecimal<
Decimal32>> || std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> || std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
{
const auto & decimal_type = static_cast<const ToDataType *>(column_nested_type.get());
arrow_fields.emplace_back(std::make_shared<arrow::Field>(
column.name, arrow::decimal(decimal_type->getPrecision(), decimal_type->getScale()), is_column_nullable));
}
return false;
};
callOnIndexAndDataType<void>(column_nested_type->getTypeId(), add_decimal_field);
}
else
{
if (internal_type_to_arrow_type.find(column_nested_type_name) == internal_type_to_arrow_type.end())
{
throw Exception{"The type \"" + column_nested_type_name + "\" of a column \"" + column.name
+ "\""
" is not supported for conversion into a Parquet data format",
ErrorCodes::UNKNOWN_TYPE};
}
arrow_fields.emplace_back(std::make_shared<arrow::Field>(column.name, internal_type_to_arrow_type.at(column_nested_type_name), is_column_nullable));
}
std::shared_ptr<arrow::Array> arrow_array;
ColumnPtr nested_column
= is_column_nullable ? static_cast<const ColumnNullable &>(*column.column).getNestedColumnPtr() : column.column;
const PaddedPODArray<UInt8> * null_bytemap = is_column_nullable ? extractNullBytemapPtr(column.column) : nullptr;
if ("String" == column_nested_type_name)
{
fillArrowArrayWithStringColumnData<ColumnString>(nested_column, arrow_array, null_bytemap);
}
else if ("FixedString" == column_nested_type_name)
{
fillArrowArrayWithStringColumnData<ColumnFixedString>(nested_column, arrow_array, null_bytemap);
}
else if ("Date" == column_nested_type_name)
{
fillArrowArrayWithDateColumnData(nested_column, arrow_array, null_bytemap);
}
else if ("DateTime" == column_nested_type_name)
{
fillArrowArrayWithDateTimeColumnData(nested_column, arrow_array, null_bytemap);
}
else if (isDecimal(column_nested_type))
{
auto fill_decimal = [&](const auto & types) -> bool
{
using Types = std::decay_t<decltype(types)>;
using ToDataType = typename Types::LeftType;
if constexpr (
std::is_same_v<
ToDataType,
DataTypeDecimal<
Decimal32>> || std::is_same_v<ToDataType, DataTypeDecimal<Decimal64>> || std::is_same_v<ToDataType, DataTypeDecimal<Decimal128>>)
{
const auto & decimal_type = static_cast<const ToDataType *>(column_nested_type.get());
fillArrowArrayWithDecimalColumnData(nested_column, arrow_array, null_bytemap, decimal_type);
}
return false;
};
callOnIndexAndDataType<void>(column_nested_type->getTypeId(), fill_decimal);
}
# define DISPATCH(CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE) \
else if (#CPP_NUMERIC_TYPE == column_nested_type_name) \
{ \
fillArrowArrayWithNumericColumnData<CPP_NUMERIC_TYPE, ARROW_BUILDER_TYPE>(nested_column, arrow_array, null_bytemap); \
}
FOR_INTERNAL_NUMERIC_TYPES(DISPATCH)
# undef DISPATCH
else
{
throw Exception{"Internal type \"" + column_nested_type_name + "\" of a column \"" + column.name
+ "\""
" is not supported for conversion into a Parquet data format",
ErrorCodes::UNKNOWN_TYPE};
}
arrow_arrays.emplace_back(std::move(arrow_array));
}
std::shared_ptr<arrow::Schema> arrow_schema = std::make_shared<arrow::Schema>(std::move(arrow_fields));
std::shared_ptr<arrow::Table> arrow_table = arrow::Table::Make(arrow_schema, arrow_arrays);
auto sink = std::make_shared<OstreamOutputStream>(out);
if (!file_writer)
{
parquet::WriterProperties::Builder builder;
#if USE_SNAPPY
builder.compression(parquet::Compression::SNAPPY);
#endif
auto props = builder.build();
auto status = parquet::arrow::FileWriter::Open(
*arrow_table->schema(),
arrow::default_memory_pool(),
sink,
props, /*parquet::default_writer_properties(),*/
parquet::arrow::default_arrow_writer_properties(),
&file_writer);
if (!status.ok())
throw Exception{"Error while opening a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION};
}
// TODO: calculate row_group_size depending on a number of rows and table size
auto status = file_writer->WriteTable(*arrow_table, format_settings.parquet.row_group_size);
if (!status.ok())
throw Exception{"Error while writing a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION};
}
void ParquetBlockOutputFormat::finalize()
{
if (file_writer)
{
auto status = file_writer->Close();
if (!status.ok())
throw Exception{"Error while closing a table: " + status.ToString(), ErrorCodes::UNKNOWN_EXCEPTION};
}
}
void registerOutputFormatProcessorParquet(FormatFactory & factory)
{
factory.registerOutputFormatProcessor(
"Parquet", [](WriteBuffer & buf, const Block & sample, const Context & /*context*/, const FormatSettings & format_settings)
{
auto impl = std::make_shared<ParquetBlockOutputFormat>(buf, sample, format_settings);
/// TODO
// auto res = std::make_shared<SquashingBlockOutputStream>(impl, impl->getHeader(), format_settings.parquet.row_group_size, 0);
// res->disableFlush();
return impl;
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerOutputFormatProcessorParquet(FormatFactory &)
{
}
}
#endif

View File

@ -0,0 +1,43 @@
#pragma once
#include <Common/config.h>
#if USE_PARQUET
# include <Processors/Formats/IOutputFormat.h>
# include <Formats/FormatSettings.h>
namespace arrow
{
class Array;
class DataType;
}
namespace parquet
{
namespace arrow
{
class FileWriter;
}
}
namespace DB
{
class ParquetBlockOutputFormat : public IOutputFormat
{
public:
ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings);
String getName() const override { return "ParquetBlockOutputFormat"; }
void consume(Chunk) override;
void finalize() override;
String getContentType() const override { return "application/octet-stream"; }
private:
const FormatSettings format_settings;
std::unique_ptr<parquet::arrow::FileWriter> file_writer;
};
}
#endif

View File

@ -0,0 +1,281 @@
#include <sys/ioctl.h>
#include <port/unistd.h>
#include <Processors/Formats/Impl/PrettyBlockOutputFormat.h>
#include <Formats/FormatFactory.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <Common/UTF8Helpers.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
PrettyBlockOutputFormat::PrettyBlockOutputFormat(
WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_)
: IOutputFormat(header, out_), format_settings(format_settings_)
{
struct winsize w;
if (0 == ioctl(STDOUT_FILENO, TIOCGWINSZ, &w))
terminal_width = w.ws_col;
}
/// Evaluate the visible width of the values and column names.
/// Note that number of code points is just a rough approximation of visible string width.
void PrettyBlockOutputFormat::calculateWidths(
const Block & header, const Chunk & chunk,
WidthsPerColumn & widths, Widths & max_widths, Widths & name_widths)
{
size_t num_rows = chunk.getNumRows();
size_t num_columns = chunk.getNumColumns();
auto & columns = chunk.getColumns();
widths.resize(num_columns);
max_widths.resize_fill(num_columns);
name_widths.resize(num_columns);
/// Calculate widths of all values.
String serialized_value;
size_t prefix = 2; // Tab character adjustment
for (size_t i = 0; i < num_columns; ++i)
{
auto & elem = header.getByPosition(i);
auto & column = columns[i];
widths[i].resize(num_rows);
for (size_t j = 0; j < num_rows; ++j)
{
{
WriteBufferFromString out(serialized_value);
elem.type->serializeAsText(*column, j, out, format_settings);
}
widths[i][j] = std::min<UInt64>(format_settings.pretty.max_column_pad_width,
UTF8::computeWidth(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size(), prefix));
max_widths[i] = std::max(max_widths[i], widths[i][j]);
}
/// And also calculate widths for names of columns.
{
// name string doesn't contain Tab, no need to pass `prefix`
name_widths[i] = std::min<UInt64>(format_settings.pretty.max_column_pad_width,
UTF8::computeWidth(reinterpret_cast<const UInt8 *>(elem.name.data()), elem.name.size()));
max_widths[i] = std::max(max_widths[i], name_widths[i]);
}
prefix += max_widths[i] + 3;
}
}
void PrettyBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind)
{
UInt64 max_rows = format_settings.pretty.max_rows;
if (total_rows >= max_rows)
{
total_rows += chunk.getNumRows();
return;
}
auto num_rows = chunk.getNumRows();
auto num_columns = chunk.getNumColumns();
auto & columns = chunk.getColumns();
auto & header = getPort(port_kind).getHeader();
WidthsPerColumn widths;
Widths max_widths;
Widths name_widths;
calculateWidths(header, chunk, widths, max_widths, name_widths);
/// Create separators
std::stringstream top_separator;
std::stringstream middle_names_separator;
std::stringstream middle_values_separator;
std::stringstream bottom_separator;
top_separator << "";
middle_names_separator << "";
middle_values_separator << "";
bottom_separator << "";
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
{
top_separator << "";
middle_names_separator << "";
middle_values_separator << "";
bottom_separator << "";
}
for (size_t j = 0; j < max_widths[i] + 2; ++j)
{
top_separator << "";
middle_names_separator << "";
middle_values_separator << "";
bottom_separator << "";
}
}
top_separator << "\n";
middle_names_separator << "\n";
middle_values_separator << "\n";
bottom_separator << "\n";
std::string top_separator_s = top_separator.str();
std::string middle_names_separator_s = middle_names_separator.str();
std::string middle_values_separator_s = middle_values_separator.str();
std::string bottom_separator_s = bottom_separator.str();
/// Output the block
writeString(top_separator_s, out);
/// Names
writeCString("", out);
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
writeCString("", out);
auto & col = header.getByPosition(i);
if (format_settings.pretty.color)
writeCString("\033[1m", out);
if (col.type->shouldAlignRightInPrettyFormats())
{
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeChar(' ', out);
writeString(col.name, out);
}
else
{
writeString(col.name, out);
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeChar(' ', out);
}
if (format_settings.pretty.color)
writeCString("\033[0m", out);
}
writeCString("\n", out);
writeString(middle_names_separator_s, out);
for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i)
{
if (i != 0)
writeString(middle_values_separator_s, out);
writeCString("", out);
for (size_t j = 0; j < num_columns; ++j)
{
if (j != 0)
writeCString("", out);
auto & type = *header.getByPosition(j).type;
writeValueWithPadding(*columns[j], type, i, widths[j].empty() ? max_widths[j] : widths[j][i], max_widths[j]);
}
writeCString("\n", out);
}
writeString(bottom_separator_s, out);
total_rows += num_rows;
}
void PrettyBlockOutputFormat::writeValueWithPadding(
const IColumn & column, const IDataType & type, size_t row_num, size_t value_width, size_t pad_to_width)
{
auto writePadding = [&]()
{
for (size_t k = 0; k < pad_to_width - value_width; ++k)
writeChar(' ', out);
};
if (type.shouldAlignRightInPrettyFormats())
{
writePadding();
type.serializeAsText(column, row_num, out, format_settings);
}
else
{
type.serializeAsText(column, row_num, out, format_settings);
writePadding();
}
}
void PrettyBlockOutputFormat::consume(Chunk chunk)
{
write(chunk, PortKind::Main);
}
void PrettyBlockOutputFormat::consumeTotals(Chunk chunk)
{
total_rows = 0;
writeSuffixIfNot();
writeCString("\nExtremes:\n", out);
write(chunk, PortKind::Totals);
}
void PrettyBlockOutputFormat::consumeExtremes(Chunk chunk)
{
total_rows = 0;
writeSuffixIfNot();
writeCString("\nTotals:\n", out);
write(chunk, PortKind::Extremes);
}
void PrettyBlockOutputFormat::writeSuffix()
{
if (total_rows >= format_settings.pretty.max_rows)
{
writeCString(" Showed first ", out);
writeIntText(format_settings.pretty.max_rows, out);
writeCString(".\n", out);
}
}
void PrettyBlockOutputFormat::finalize()
{
writeSuffixIfNot();
}
void registerOutputFormatProcessorPretty(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("Pretty", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<PrettyBlockOutputFormat>(buf, sample, format_settings);
});
factory.registerOutputFormatProcessor("PrettyNoEscapes", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
FormatSettings changed_settings = format_settings;
changed_settings.pretty.color = false;
return std::make_shared<PrettyBlockOutputFormat>(buf, sample, changed_settings);
});
}
}

View File

@ -0,0 +1,61 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/FormatSettings.h>
namespace DB
{
class WriteBuffer;
class Context;
/** Prints the result in the form of beautiful tables.
*/
class PrettyBlockOutputFormat : public IOutputFormat
{
public:
/// no_escapes - do not use ANSI escape sequences - to display in the browser, not in the console.
PrettyBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_);
String getName() const override { return "PrettyBlockOutputFormat"; }
void consume(Chunk) override;
void consumeTotals(Chunk) override;
void consumeExtremes(Chunk) override;
void finalize() override;
protected:
size_t total_rows = 0;
size_t terminal_width = 0;
bool suffix_written = false;
const FormatSettings format_settings;
using Widths = PODArray<size_t>;
using WidthsPerColumn = std::vector<Widths>;
virtual void write(const Chunk & chunk, PortKind port_kind);
virtual void writeSuffix();
void writeSuffixIfNot()
{
if (!suffix_written)
writeSuffix();
suffix_written = true;
}
void calculateWidths(
const Block & header, const Chunk & chunk,
WidthsPerColumn & widths, Widths & max_widths, Widths & name_widths);
void writeValueWithPadding(
const IColumn & column, const IDataType & type, size_t row_num, size_t value_width, size_t pad_to_width);
};
}

View File

@ -0,0 +1,167 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
///#include <DataStreams/SquashingBlockOutputStream.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/Impl/PrettyCompactBlockOutputFormat.h>
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
}
void PrettyCompactBlockOutputFormat::writeHeader(
const Block & block,
const Widths & max_widths,
const Widths & name_widths)
{
/// Names
writeCString("┌─", out);
for (size_t i = 0; i < max_widths.size(); ++i)
{
if (i != 0)
writeCString("─┬─", out);
const ColumnWithTypeAndName & col = block.getByPosition(i);
if (col.type->shouldAlignRightInPrettyFormats())
{
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeCString("", out);
if (format_settings.pretty.color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (format_settings.pretty.color)
writeCString("\033[0m", out);
}
else
{
if (format_settings.pretty.color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (format_settings.pretty.color)
writeCString("\033[0m", out);
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
writeCString("", out);
}
}
writeCString("─┐\n", out);
}
void PrettyCompactBlockOutputFormat::writeBottom(const Widths & max_widths)
{
/// Create delimiters
std::stringstream bottom_separator;
bottom_separator << "";
for (size_t i = 0; i < max_widths.size(); ++i)
{
if (i != 0)
bottom_separator << "";
for (size_t j = 0; j < max_widths[i] + 2; ++j)
bottom_separator << "";
}
bottom_separator << "\n";
writeString(bottom_separator.str(), out);
}
void PrettyCompactBlockOutputFormat::writeRow(
size_t row_num,
const Block & header,
const Columns & columns,
const WidthsPerColumn & widths,
const Widths & max_widths)
{
size_t num_columns = max_widths.size();
writeCString("", out);
for (size_t j = 0; j < num_columns; ++j)
{
if (j != 0)
writeCString("", out);
auto & type = *header.getByPosition(j).type;
auto & cur_widths = widths[j].empty() ? max_widths[j] : widths[j][row_num];
writeValueWithPadding(*columns[j], type, row_num, cur_widths, max_widths[j]);
}
writeCString("\n", out);
}
void PrettyCompactBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind)
{
UInt64 max_rows = format_settings.pretty.max_rows;
if (total_rows >= max_rows)
{
total_rows += chunk.getNumRows();
return;
}
size_t num_rows = chunk.getNumRows();
auto & header = getPort(port_kind).getHeader();
auto & columns = chunk.getColumns();
WidthsPerColumn widths;
Widths max_widths;
Widths name_widths;
calculateWidths(header, chunk, widths, max_widths, name_widths);
writeHeader(header, max_widths, name_widths);
for (size_t i = 0; i < num_rows && total_rows + i < max_rows; ++i)
writeRow(i, header, columns, widths, max_widths);
writeBottom(max_widths);
total_rows += num_rows;
}
void registerOutputFormatProcessorPrettyCompact(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("PrettyCompact", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<PrettyCompactBlockOutputFormat>(buf, sample, format_settings);
});
factory.registerOutputFormatProcessor("PrettyCompactNoEscapes", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
FormatSettings changed_settings = format_settings;
changed_settings.pretty.color = false;
return std::make_shared<PrettyCompactBlockOutputFormat>(buf, sample, changed_settings);
});
/// TODO
// factory.registerOutputFormat("PrettyCompactMonoBlock", [](
// WriteBuffer & buf,
// const Block & sample,
// const Context &,
// const FormatSettings & format_settings)
// {
// BlockOutputStreamPtr impl = std::make_shared<PrettyCompactBlockOutputFormat>(buf, sample, format_settings);
// auto res = std::make_shared<SquashingBlockOutputStream>(impl, impl->getHeader(), format_settings.pretty.max_rows, 0);
// res->disableFlush();
// return res;
// });
}
}

View File

@ -0,0 +1,31 @@
#pragma once
#include <Processors/Formats/Impl/PrettyBlockOutputFormat.h>
namespace DB
{
/** Prints the result in the form of beautiful tables, but with fewer delimiter lines.
*/
class PrettyCompactBlockOutputFormat : public PrettyBlockOutputFormat
{
public:
PrettyCompactBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_)
: PrettyBlockOutputFormat(out_, header, format_settings_) {}
String getName() const override { return "PrettyCompactBlockOutputFormat"; }
protected:
void write(const Chunk & chunk, PortKind port_kind) override;
void writeHeader(const Block & block, const Widths & max_widths, const Widths & name_widths);
void writeBottom(const Widths & max_widths);
void writeRow(
size_t row_num,
const Block & header,
const Columns & columns,
const WidthsPerColumn & widths,
const Widths & max_widths);
};
}

View File

@ -0,0 +1,123 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/Impl/PrettySpaceBlockOutputFormat.h>
namespace DB
{
void PrettySpaceBlockOutputFormat::write(const Chunk & chunk, PortKind port_kind)
{
UInt64 max_rows = format_settings.pretty.max_rows;
if (total_rows >= max_rows)
{
total_rows += chunk.getNumRows();
return;
}
size_t num_rows = chunk.getNumRows();
size_t num_columns = chunk.getNumColumns();
auto & header = getPort(port_kind).getHeader();
auto & columns = chunk.getColumns();
WidthsPerColumn widths;
Widths max_widths;
Widths name_widths;
calculateWidths(header, chunk, widths, max_widths, name_widths);
/// Do not align on too long values.
if (terminal_width > 80)
for (size_t i = 0; i < num_columns; ++i)
if (max_widths[i] > terminal_width / 2)
max_widths[i] = terminal_width / 2;
/// Names
for (size_t i = 0; i < num_columns; ++i)
{
if (i != 0)
writeCString(" ", out);
const ColumnWithTypeAndName & col = header.getByPosition(i);
if (col.type->shouldAlignRightInPrettyFormats())
{
for (ssize_t k = 0; k < std::max(static_cast<ssize_t>(0), static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
writeChar(' ', out);
if (format_settings.pretty.color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (format_settings.pretty.color)
writeCString("\033[0m", out);
}
else
{
if (format_settings.pretty.color)
writeCString("\033[1m", out);
writeString(col.name, out);
if (format_settings.pretty.color)
writeCString("\033[0m", out);
for (ssize_t k = 0; k < std::max(static_cast<ssize_t>(0), static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
writeChar(' ', out);
}
}
writeCString("\n\n", out);
for (size_t row = 0; row < num_rows && total_rows + row < max_rows; ++row)
{
for (size_t column = 0; column < num_columns; ++column)
{
if (column != 0)
writeCString(" ", out);
auto & type = *header.getByPosition(column).type;
auto & cur_width = widths[column].empty() ? max_widths[column] : widths[column][row];
writeValueWithPadding(*columns[column], type, row, cur_width, max_widths[column]);
}
writeChar('\n', out);
}
total_rows += num_rows;
}
void PrettySpaceBlockOutputFormat::writeSuffix()
{
if (total_rows >= format_settings.pretty.max_rows)
{
writeCString("\nShowed first ", out);
writeIntText(format_settings.pretty.max_rows, out);
writeCString(".\n", out);
}
}
void registerOutputFormatProcessorPrettySpace(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("PrettySpace", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
return std::make_shared<PrettySpaceBlockOutputFormat>(buf, sample, format_settings);
});
factory.registerOutputFormatProcessor("PrettySpaceNoEscapes", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & format_settings)
{
FormatSettings changed_settings = format_settings;
changed_settings.pretty.color = false;
return std::make_shared<PrettySpaceBlockOutputFormat>(buf, sample, changed_settings);
});
}
}

View File

@ -0,0 +1,24 @@
#pragma once
#include <Processors/Formats/Impl/PrettyBlockOutputFormat.h>
namespace DB
{
/** Prints the result, aligned with spaces.
*/
class PrettySpaceBlockOutputFormat : public PrettyBlockOutputFormat
{
public:
PrettySpaceBlockOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_)
: PrettyBlockOutputFormat(out_, header, format_settings_) {}
String getName() const override { return "PrettySpaceBlockOutputFormat"; }
protected:
void write(const Chunk & chunk, PortKind port_kind) override;
void writeSuffix() override;
};
}

View File

@ -0,0 +1,73 @@
#include <Formats/FormatFactory.h>
#include "config_formats.h"
#if USE_PROTOBUF
#include <Processors/Formats/Impl/ProtobufBlockOutputFormat.h>
#include <Core/Block.h>
#include <Formats/FormatSchemaInfo.h>
#include <Formats/ProtobufSchemas.h>
#include <Interpreters/Context.h>
#include <google/protobuf/descriptor.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
extern const int NO_DATA_FOR_REQUIRED_PROTOBUF_FIELD;
}
ProtobufBlockOutputFormat::ProtobufBlockOutputFormat(
WriteBuffer & out_,
const Block & header,
const FormatSchemaInfo & format_schema)
: IOutputFormat(header, out_)
, data_types(header.getDataTypes())
, writer(out, ProtobufSchemas::instance().getMessageTypeForFormatSchema(format_schema), header.getNames())
{
value_indices.resize(header.columns());
}
void ProtobufBlockOutputFormat::consume(Chunk chunk)
{
auto & columns = chunk.getColumns();
auto num_rows = chunk.getNumRows();
for (UInt64 row_num = 0; row_num < num_rows; ++row_num)
{
writer.startMessage();
std::fill(value_indices.begin(), value_indices.end(), 0);
size_t column_index;
while (writer.writeField(column_index))
data_types[column_index]->serializeProtobuf(
*columns[column_index], row_num, writer, value_indices[column_index]);
writer.endMessage();
}
}
void registerOutputFormatProcessorProtobuf(FormatFactory & factory)
{
factory.registerOutputFormatProcessor(
"Protobuf", [](WriteBuffer & buf, const Block & header, const Context & context, const FormatSettings &)
{
return std::make_shared<ProtobufBlockOutputFormat>(buf, header, FormatSchemaInfo(context, "Protobuf"));
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerOutputFormatProcessorProtobuf(FormatFactory &) {}
}
#endif

View File

@ -0,0 +1,53 @@
#pragma once
#include <Common/config.h>
#if USE_PROTOBUF
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Formats/ProtobufWriter.h>
#include <Formats/FormatSchemaInfo.h>
#include <Processors/Formats/IOutputFormat.h>
namespace google
{
namespace protobuf
{
class Message;
}
}
namespace DB
{
/** Stream designed to serialize data in the google protobuf format.
* Each row is written as a separated message.
* These messages are delimited according to documentation
* https://github.com/protocolbuffers/protobuf/blob/master/src/google/protobuf/util/delimited_message_util.h
* Serializing in the protobuf format requires the 'format_schema' setting to be set, e.g.
* SELECT * from table FORMAT Protobuf SETTINGS format_schema = 'schema:Message'
* where schema is the name of "schema.proto" file specifying protobuf schema.
*/
class ProtobufBlockOutputFormat : public IOutputFormat
{
public:
ProtobufBlockOutputFormat(
WriteBuffer & out_,
const Block & header,
const FormatSchemaInfo & format_schema);
String getName() const override { return "ProtobufBlockOutputFormat"; }
void consume(Chunk) override;
std::string getContentType() const override { return "application/octet-stream"; }
private:
DataTypes data_types;
ProtobufWriter writer;
std::vector<size_t> value_indices;
};
}
#endif

View File

@ -0,0 +1,92 @@
#include "config_formats.h"
#if USE_PROTOBUF
#include <Core/Block.h>
#include <Formats/FormatFactory.h>
#include <Formats/FormatSchemaInfo.h>
#include <Formats/ProtobufSchemas.h>
#include <Processors/Formats/Impl/ProtobufRowInputFormat.h>
namespace DB
{
ProtobufRowInputFormat::ProtobufRowInputFormat(ReadBuffer & in_, const Block & header, Params params, const FormatSchemaInfo & info)
: IRowInputFormat(header, in_, params)
, data_types(header.getDataTypes())
, reader(in, ProtobufSchemas::instance().getMessageTypeForFormatSchema(info), header.getNames())
{
}
ProtobufRowInputFormat::~ProtobufRowInputFormat() = default;
bool ProtobufRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & extra)
{
if (!reader.startMessage())
return false; // EOF reached, no more messages.
// Set of columns for which the values were read. The rest will be filled with default values.
auto & read_columns = extra.read_columns;
read_columns.assign(columns.size(), false);
// Read values from this message and put them to the columns while it's possible.
size_t column_index;
while (reader.readColumnIndex(column_index))
{
bool allow_add_row = !static_cast<bool>(read_columns[column_index]);
do
{
bool row_added;
data_types[column_index]->deserializeProtobuf(*columns[column_index], reader, allow_add_row, row_added);
if (row_added)
{
read_columns[column_index] = true;
allow_add_row = false;
}
} while (reader.maybeCanReadValue());
}
// Fill non-visited columns with the default values.
for (column_index = 0; column_index < read_columns.size(); ++column_index)
if (!read_columns[column_index])
data_types[column_index]->insertDefaultInto(*columns[column_index]);
reader.endMessage();
return true;
}
bool ProtobufRowInputFormat::allowSyncAfterError() const
{
return true;
}
void ProtobufRowInputFormat::syncAfterError()
{
reader.endMessage();
}
void registerInputFormatProcessorProtobuf(FormatFactory & factory)
{
factory.registerInputFormatProcessor("Protobuf", [](
ReadBuffer & buf,
const Block & sample,
const Context & context,
IRowInputFormat::Params params,
const FormatSettings &)
{
return std::make_shared<ProtobufRowInputFormat>(buf, sample, params, FormatSchemaInfo(context, "proto"));
});
}
}
#else
namespace DB
{
class FormatFactory;
void registerInputFormatProcessorProtobuf(FormatFactory &) {}
}
#endif

View File

@ -0,0 +1,36 @@
#pragma once
#include <Common/config.h>
#if USE_PROTOBUF
#include <DataTypes/IDataType.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Formats/ProtobufReader.h>
namespace DB
{
class Block;
class FormatSchemaInfo;
/** Interface of stream, that allows to read data by rows.
*/
class ProtobufRowInputFormat : public IRowInputFormat
{
public:
ProtobufRowInputFormat(ReadBuffer & in_, const Block & header, Params params, const FormatSchemaInfo & info);
~ProtobufRowInputFormat() override;
String getName() const override { return "ProtobufRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension & extra) override;
bool allowSyncAfterError() const override;
void syncAfterError() override;
private:
DataTypes data_types;
ProtobufReader reader;
};
}
#endif

View File

@ -0,0 +1,207 @@
#include <IO/ReadHelpers.h>
#include <Processors/Formats/Impl/TSKVRowInputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
extern const int CANNOT_READ_ALL_DATA;
extern const int CANNOT_PARSE_INPUT_ASSERTION_FAILED;
}
TSKVRowInputFormat::TSKVRowInputFormat(ReadBuffer & in_, Block header, Params params, const FormatSettings & format_settings)
: IRowInputFormat(std::move(header), in_, params), format_settings(format_settings), name_map(header.columns())
{
/// In this format, we assume that column name cannot contain BOM,
/// so BOM at beginning of stream cannot be confused with name of field, and it is safe to skip it.
skipBOMIfExists(in);
size_t num_columns = header.columns();
for (size_t i = 0; i < num_columns; ++i)
name_map[header.safeGetByPosition(i).name] = i; /// NOTE You could place names more cache-locally.
}
/** Read the field name in the `tskv` format.
* Return true if the field is followed by an equal sign,
* otherwise (field with no value) return false.
* The reference to the field name will be written to `ref`.
* A temporary `tmp` buffer can also be used to copy the field name to it.
* When reading, skips the name and the equal sign after it.
*/
static bool readName(ReadBuffer & buf, StringRef & ref, String & tmp)
{
tmp.clear();
while (!buf.eof())
{
const char * next_pos = find_first_symbols<'\t', '\n', '\\', '='>(buf.position(), buf.buffer().end());
if (next_pos == buf.buffer().end())
{
tmp.append(buf.position(), next_pos - buf.position());
buf.next();
continue;
}
/// Came to the end of the name.
if (*next_pos != '\\')
{
bool have_value = *next_pos == '=';
if (tmp.empty())
{
/// No need to copy data, you can refer directly to the `buf`.
ref = StringRef(buf.position(), next_pos - buf.position());
buf.position() += next_pos + have_value - buf.position();
}
else
{
/// Copy the data to a temporary string and return a reference to it.
tmp.append(buf.position(), next_pos - buf.position());
buf.position() += next_pos + have_value - buf.position();
ref = StringRef(tmp);
}
return have_value;
}
/// The name has an escape sequence.
else
{
tmp.append(buf.position(), next_pos - buf.position());
buf.position() += next_pos + 1 - buf.position();
if (buf.eof())
throw Exception("Cannot parse escape sequence", ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE);
tmp.push_back(parseEscapeSequence(*buf.position()));
++buf.position();
continue;
}
}
throw Exception("Unexpected end of stream while reading key name from TSKV format", ErrorCodes::CANNOT_READ_ALL_DATA);
}
bool TSKVRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & ext)
{
if (in.eof())
return false;
auto & header = getPort().getHeader();
size_t num_columns = columns.size();
/// Set of columns for which the values were read. The rest will be filled with default values.
read_columns.assign(num_columns, false);
if (unlikely(*in.position() == '\n'))
{
/// An empty string. It is permissible, but it is unclear why.
++in.position();
}
else
{
while (true)
{
StringRef name_ref;
bool has_value = readName(in, name_ref, name_buf);
ssize_t index = -1;
if (has_value)
{
/// NOTE Optimization is possible by caching the order of fields (which is almost always the same)
/// and quickly checking for the next expected field, instead of searching the hash table.
auto it = name_map.find(name_ref);
if (name_map.end() == it)
{
if (!format_settings.skip_unknown_fields)
throw Exception("Unknown field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
/// If the key is not found, skip the value.
NullSink sink;
readEscapedStringInto(sink, in);
}
else
{
index = it->getSecond();
if (read_columns[index])
throw Exception("Duplicate field found while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
read_columns[index] = true;
header.getByPosition(index).type->deserializeAsTextEscaped(*columns[index], in, format_settings);
}
}
else
{
/// The only thing that can go without value is `tskv` fragment that is ignored.
if (!(name_ref.size == 4 && 0 == memcmp(name_ref.data, "tskv", 4)))
throw Exception("Found field without value while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
}
if (in.eof())
{
throw Exception("Unexpected end of stream after field in TSKV format: " + name_ref.toString(), ErrorCodes::CANNOT_READ_ALL_DATA);
}
else if (*in.position() == '\t')
{
++in.position();
continue;
}
else if (*in.position() == '\n')
{
++in.position();
break;
}
else
{
/// Possibly a garbage was written into column, remove it
if (index >= 0)
{
columns[index]->popBack(1);
read_columns[index] = false;
}
throw Exception("Found garbage after field in TSKV format: " + name_ref.toString(), ErrorCodes::CANNOT_PARSE_INPUT_ASSERTION_FAILED);
}
}
}
/// Fill in the not met columns with default values.
for (size_t i = 0; i < num_columns; ++i)
if (!read_columns[i])
header.getByPosition(i).type->insertDefaultInto(*columns[i]);
/// return info about defaults set
ext.read_columns = read_columns;
return true;
}
void TSKVRowInputFormat::syncAfterError()
{
skipToUnescapedNextLineOrEOF(in);
}
void registerInputFormatProcessorTSKV(FormatFactory & factory)
{
factory.registerInputFormatProcessor("TSKV", [](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TSKVRowInputFormat>(buf, sample, params, settings);
});
}
}

View File

@ -0,0 +1,47 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Formats/FormatSettings.h>
#include <Common/HashTable/HashMap.h>
namespace DB
{
class ReadBuffer;
/** Stream for reading data in TSKV format.
* TSKV is a very inefficient data format.
* Similar to TSV, but each field is written as key=value.
* Fields can be listed in any order (including, in different lines there may be different order),
* and some fields may be missing.
* An equal sign can be escaped in the field name.
* Also, as an additional element there may be a useless tskv fragment - it needs to be ignored.
*/
class TSKVRowInputFormat : public IRowInputFormat
{
public:
TSKVRowInputFormat(ReadBuffer & in_, Block header, Params params, const FormatSettings & format_settings);
String getName() const override { return "TSKVRowInputFormat"; }
bool readRow(MutableColumns & columns, RowReadExtension &) override;
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
private:
const FormatSettings format_settings;
/// Buffer for the read from the stream the field name. Used when you have to copy it.
String name_buf;
/// Hash table matching `field name -> position in the block`. NOTE You can use perfect hash map.
using NameMap = HashMap<StringRef, size_t, StringRefHash>;
NameMap name_map;
std::vector<UInt8> read_columns;
};
}

View File

@ -0,0 +1,55 @@
#include <IO/WriteHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <Processors/Formats/Impl/TSKVRowOutputFormat.h>
#include <Formats/FormatFactory.h>
namespace DB
{
TSKVRowOutputFormat::TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings_)
: TabSeparatedRowOutputFormat(out_, header, false, false, format_settings_)
{
auto & sample = getPort(PortKind::Main).getHeader();
NamesAndTypesList columns(sample.getNamesAndTypesList());
fields.assign(columns.begin(), columns.end());
for (auto & field : fields)
{
WriteBufferFromOwnString wb;
writeAnyEscapedString<'='>(field.name.data(), field.name.data() + field.name.size(), wb);
writeCString("=", wb);
field.name = wb.str();
}
}
void TSKVRowOutputFormat::writeField(const IColumn & column, const IDataType & type, size_t row_num)
{
writeString(fields[field_number].name, out);
type.serializeAsTextEscaped(column, row_num, out, format_settings);
++field_number;
}
void TSKVRowOutputFormat::writeRowEndDelimiter()
{
writeChar('\n', out);
field_number = 0;
}
void registerOutputFormatProcessorTSKV(FormatFactory & factory)
{
factory.registerOutputFormatProcessor("TSKV", [](
WriteBuffer & buf,
const Block & sample,
const Context &,
const FormatSettings & settings)
{
return std::make_shared<TSKVRowOutputFormat>(buf, sample, settings);
});
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Formats/FormatSettings.h>
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
namespace DB
{
/** The stream for outputting data in the TSKV format.
* TSKV is similar to TabSeparated, but before every value, its name and equal sign are specified: name=value.
* This format is very inefficient.
*/
class TSKVRowOutputFormat: public TabSeparatedRowOutputFormat
{
public:
TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const FormatSettings & format_settings);
String getName() const override { return "TSKVRowOutputFormat"; }
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override;
void writeRowEndDelimiter() override;
protected:
NamesAndTypes fields;
size_t field_number = 0;
};
}

View File

@ -0,0 +1,28 @@
#pragma once
#include <Formats/FormatSettings.h>
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
namespace DB
{
/** A stream for outputting data in tsv format, but without escaping individual values.
* (That is, the output is irreversible.)
*/
class TabSeparatedRawRowOutputFormat : public TabSeparatedRowOutputFormat
{
public:
TabSeparatedRawRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: TabSeparatedRowOutputFormat(out_, header, with_names_, with_types_, format_settings_) {}
String getName() const override { return "TabSeparatedRawRowOutputFormat"; }
void writeField(const IColumn & column, const IDataType & type, size_t row_num) override
{
type.serializeAsText(column, row_num, out, format_settings);
}
};
}

View File

@ -0,0 +1,370 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int LOGICAL_ERROR;
}
TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(
ReadBuffer & in_, Block header, bool with_names, bool with_types, Params params, const FormatSettings & format_settings)
: IRowInputFormat(std::move(header), in_, params), with_names(with_names), with_types(with_types), format_settings(format_settings)
{
auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
data_types.resize(num_columns);
for (size_t i = 0; i < num_columns; ++i)
data_types[i] = sample.safeGetByPosition(i).type;
}
void TabSeparatedRowInputFormat::readPrefix()
{
auto & header = getPort().getHeader();
size_t num_columns = header.columns();
String tmp;
if (with_names || with_types)
{
/// In this format, we assume that column name or type cannot contain BOM,
/// so, if format has header,
/// then BOM at beginning of stream cannot be confused with name or type of field, and it is safe to skip it.
skipBOMIfExists(in);
}
if (with_names)
{
for (size_t i = 0; i < num_columns; ++i)
{
readEscapedString(tmp, in);
assertChar(i == num_columns - 1 ? '\n' : '\t', in);
}
}
if (with_types)
{
for (size_t i = 0; i < num_columns; ++i)
{
readEscapedString(tmp, in);
assertChar(i == num_columns - 1 ? '\n' : '\t', in);
}
}
}
/** Check for a common error case - usage of Windows line feed.
*/
static void checkForCarriageReturn(ReadBuffer & in)
{
if (in.position()[0] == '\r' || (in.position() != in.buffer().begin() && in.position()[-1] == '\r'))
throw Exception("\nYou have carriage return (\\r, 0x0D, ASCII 13) at end of first row."
"\nIt's like your input data has DOS/Windows style line separators, that are illegal in TabSeparated format."
" You must transform your file to Unix format."
"\nBut if you really need carriage return at end of string value of last column, you need to escape it as \\r.",
ErrorCodes::INCORRECT_DATA);
}
bool TabSeparatedRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &)
{
if (in.eof())
return false;
updateDiagnosticInfo();
size_t size = data_types.size();
for (size_t i = 0; i < size; ++i)
{
data_types[i]->deserializeAsTextEscaped(*columns[i], in, format_settings);
/// skip separators
if (i + 1 == size)
{
if (!in.eof())
{
if (unlikely(row_num == 1))
checkForCarriageReturn(in);
assertChar('\n', in);
}
}
else
assertChar('\t', in);
}
return true;
}
String TabSeparatedRowInputFormat::getDiagnosticInfo()
{
if (in.eof()) /// Buffer has gone, cannot extract information about what has been parsed.
return {};
auto & header = getPort().getHeader();
WriteBufferFromOwnString out;
MutableColumns columns = header.cloneEmptyColumns();
/// It is possible to display detailed diagnostics only if the last and next to last lines are still in the read buffer.
size_t bytes_read_at_start_of_buffer = in.count() - in.offset();
if (bytes_read_at_start_of_buffer != bytes_read_at_start_of_buffer_on_prev_row)
{
out << "Could not print diagnostic info because two last rows aren't in buffer (rare case)\n";
return out.str();
}
size_t max_length_of_column_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).name.size() > max_length_of_column_name)
max_length_of_column_name = header.safeGetByPosition(i).name.size();
size_t max_length_of_data_type_name = 0;
for (size_t i = 0; i < header.columns(); ++i)
if (header.safeGetByPosition(i).type->getName().size() > max_length_of_data_type_name)
max_length_of_data_type_name = header.safeGetByPosition(i).type->getName().size();
/// Roll back the cursor to the beginning of the previous or current line and pars all over again. But now we derive detailed information.
if (pos_of_prev_row)
{
in.position() = pos_of_prev_row;
out << "\nRow " << (row_num - 1) << ":\n";
if (!parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name))
return out.str();
}
else
{
if (!pos_of_current_row)
{
out << "Could not print diagnostic info because parsing of data hasn't started.\n";
return out.str();
}
in.position() = pos_of_current_row;
}
out << "\nRow " << row_num << ":\n";
parseRowAndPrintDiagnosticInfo(columns, out, max_length_of_column_name, max_length_of_data_type_name);
out << "\n";
return out.str();
}
bool TabSeparatedRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
WriteBuffer & out, size_t max_length_of_column_name, size_t max_length_of_data_type_name)
{
auto & header = getPort().getHeader();
size_t size = data_types.size();
for (size_t i = 0; i < size; ++i)
{
if (i == 0 && in.eof())
{
out << "<End of stream>\n";
return false;
}
out << "Column " << i << ", " << std::string((i < 10 ? 2 : i < 100 ? 1 : 0), ' ')
<< "name: " << header.safeGetByPosition(i).name << ", " << std::string(max_length_of_column_name - header.safeGetByPosition(i).name.size(), ' ')
<< "type: " << data_types[i]->getName() << ", " << std::string(max_length_of_data_type_name - data_types[i]->getName().size(), ' ');
auto prev_position = in.position();
std::exception_ptr exception;
try
{
data_types[i]->deserializeAsTextEscaped(*columns[i], in, format_settings);
}
catch (...)
{
exception = std::current_exception();
}
auto curr_position = in.position();
if (curr_position < prev_position)
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i]))
{
/// An empty string instead of a value.
if (curr_position == prev_position)
{
out << "ERROR: text ";
verbosePrintString(prev_position, std::min(prev_position + 10, in.buffer().end()), out);
out << " is not like " << data_types[i]->getName() << "\n";
return false;
}
}
out << "parsed text: ";
verbosePrintString(prev_position, curr_position, out);
if (exception)
{
if (data_types[i]->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (data_types[i]->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
else
out << "ERROR\n";
return false;
}
out << "\n";
if (data_types[i]->haveMaximumSizeOfValue())
{
if (*curr_position != '\n' && *curr_position != '\t')
{
out << "ERROR: garbage after " << data_types[i]->getName() << ": ";
verbosePrintString(curr_position, std::min(curr_position + 10, in.buffer().end()), out);
out << "\n";
if (data_types[i]->getName() == "DateTime")
out << "ERROR: DateTime must be in YYYY-MM-DD hh:mm:ss or NNNNNNNNNN (unix timestamp, exactly 10 digits) format.\n";
else if (data_types[i]->getName() == "Date")
out << "ERROR: Date must be in YYYY-MM-DD format.\n";
return false;
}
}
/// Delimiters
if (i + 1 == size)
{
if (!in.eof())
{
try
{
assertChar('\n', in);
}
catch (const DB::Exception &)
{
if (*in.position() == '\t')
{
out << "ERROR: Tab found where line feed is expected."
" It's like your file has more columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped tab in value.\n";
}
else if (*in.position() == '\r')
{
out << "ERROR: Carriage return found where line feed is expected."
" It's like your file has DOS/Windows style line separators, that is illegal in TabSeparated format.\n";
}
else
{
out << "ERROR: There is no line feed. ";
verbosePrintString(in.position(), in.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
else
{
try
{
assertChar('\t', in);
}
catch (const DB::Exception &)
{
if (*in.position() == '\n')
{
out << "ERROR: Line feed found where tab is expected."
" It's like your file has less columns than expected.\n"
"And if your file have right number of columns, maybe it have unescaped backslash in value before tab, which cause tab has escaped.\n";
}
else if (*in.position() == '\r')
{
out << "ERROR: Carriage return found where tab is expected.\n";
}
else
{
out << "ERROR: There is no tab. ";
verbosePrintString(in.position(), in.position() + 1, out);
out << " found instead.\n";
}
return false;
}
}
}
return true;
}
void TabSeparatedRowInputFormat::syncAfterError()
{
skipToUnescapedNextLineOrEOF(in);
}
void TabSeparatedRowInputFormat::updateDiagnosticInfo()
{
++row_num;
bytes_read_at_start_of_buffer_on_prev_row = bytes_read_at_start_of_buffer_on_current_row;
bytes_read_at_start_of_buffer_on_current_row = in.count() - in.offset();
pos_of_prev_row = pos_of_current_row;
pos_of_current_row = in.position();
}
void registerInputFormatProcessorTabSeparated(FormatFactory & factory)
{
for (auto name : {"TabSeparated", "TSV"})
{
factory.registerInputFormatProcessor(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, false, false, params, settings);
});
}
for (auto name : {"TabSeparatedWithNames", "TSVWithNames"})
{
factory.registerInputFormatProcessor(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, true, false, params, settings);
});
}
for (auto name : {"TabSeparatedWithNamesAndTypes", "TSVWithNamesAndTypes"})
{
factory.registerInputFormatProcessor(name, [](
ReadBuffer & buf,
const Block & sample,
const Context &,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
return std::make_shared<TabSeparatedRowInputFormat>(buf, sample, true, true, params, settings);
});
}
}
}

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