mirror of https://github.com/ByConity/ByConity
commit
e9f3d4d6b8
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
|
@ -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;
|
||||
};
|
||||
|
|
|
@ -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 });
|
||||
|
||||
|
|
|
@ -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.")
|
||||
|
||||
|
|
|
@ -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];
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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});
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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).
|
||||
|
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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.
|
||||
*/
|
||||
|
|
|
@ -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() {}
|
||||
};
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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())
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
|
|
@ -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 (...)
|
||||
{
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
||||
}
|
||||
|
|
|
@ -0,0 +1,4 @@
|
|||
if (ENABLE_TESTS)
|
||||
add_subdirectory (tests)
|
||||
endif ()
|
||||
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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();
|
||||
// });
|
||||
// }
|
||||
//}
|
||||
|
||||
}
|
||||
|
|
@ -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;
|
||||
//};
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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>;
|
||||
|
||||
}
|
|
@ -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);
|
||||
//}
|
||||
|
||||
}
|
||||
|
|
@ -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;
|
||||
//};
|
||||
|
||||
}
|
|
@ -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));
|
||||
//}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -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(); }
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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)
|
||||
{
|
||||
}
|
||||
};
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
@ -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); }
|
||||
};
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
|
@ -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;
|
||||
}
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -0,0 +1,3 @@
|
|||
if (ENABLE_TESTS)
|
||||
add_subdirectory (tests)
|
||||
endif ()
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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
|
|
@ -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
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
||||
};
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
};
|
||||
|
||||
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
};
|
||||
|
||||
}
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
|
@ -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;
|
||||
// });
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
|
@ -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
Loading…
Reference in New Issue