mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-24 16:42:05 +00:00
better
This commit is contained in:
parent
8667842390
commit
2dde73f700
@ -1364,9 +1364,6 @@ private:
|
||||
if (with_output && with_output->settings_ast)
|
||||
apply_query_settings(*with_output->settings_ast);
|
||||
|
||||
// if (context.getSettingsRef().output_format_parallel_formatting)
|
||||
// need_render_progress = false;
|
||||
|
||||
connection->forceConnected(connection_parameters.timeouts);
|
||||
|
||||
ASTPtr input_function;
|
||||
@ -1939,7 +1936,7 @@ private:
|
||||
|
||||
if (!is_interactive && !need_render_progress)
|
||||
block_out_stream = context.getOutputFormatParallelIfPossible(current_format, *out_buf, block);
|
||||
|
||||
|
||||
if (!block_out_stream)
|
||||
block_out_stream = context.getOutputFormat(current_format, *out_buf, block);
|
||||
|
||||
|
@ -224,7 +224,7 @@ BlockOutputStreamPtr FormatFactory::getOutputParallelIfPossible(const String & n
|
||||
/** TODO: Materialization is needed, because formats can use the functions `IDataType`,
|
||||
* which only work with full columns.
|
||||
*/
|
||||
|
||||
|
||||
auto formatter_creator = [output_getter, sample, callback, format_settings]
|
||||
(WriteBuffer & output) -> OutputFormatPtr
|
||||
{ return output_getter(output, sample, {std::move(callback)}, format_settings);};
|
||||
@ -244,7 +244,6 @@ BlockOutputStreamPtr FormatFactory::getOutputParallelIfPossible(const String & n
|
||||
}
|
||||
|
||||
|
||||
|
||||
BlockOutputStreamPtr FormatFactory::getOutput(const String & name,
|
||||
WriteBuffer & buf, const Block & sample, const Context & context,
|
||||
WriteCallback callback, const std::optional<FormatSettings> & _format_settings) const
|
||||
@ -256,7 +255,7 @@ BlockOutputStreamPtr FormatFactory::getOutput(const String & name,
|
||||
{
|
||||
throw Exception("Format " + name + " is not suitable for output (with processors)", ErrorCodes::FORMAT_IS_NOT_SUITABLE_FOR_OUTPUT);
|
||||
}
|
||||
|
||||
|
||||
auto format = getOutputFormat(name, buf, sample, context, std::move(callback), format_settings);
|
||||
return std::make_shared<MaterializingBlockOutputStream>(std::make_shared<OutputStreamToOutputFormat>(format), sample);
|
||||
}
|
||||
|
@ -246,7 +246,7 @@ bool JSONEachRowRowInputFormat::readRow(MutableColumns & columns, RowReadExtensi
|
||||
|
||||
/// Semicolon is added for convenience as it could be used at end of INSERT query.
|
||||
bool is_first_row = getTotalRows() == 1;
|
||||
if (!in.eof())
|
||||
if (!in.eof())
|
||||
{
|
||||
/// There may be optional ',' (but not before the first row)
|
||||
if (!is_first_row && *in.position() == ',')
|
||||
|
@ -1,3 +1,4 @@
|
||||
#pragma once
|
||||
#include <DataStreams/NativeBlockInputStream.h>
|
||||
#include <DataStreams/NativeBlockOutputStream.h>
|
||||
#include <Formats/FormatFactory.h>
|
||||
|
@ -52,7 +52,7 @@ public:
|
||||
~ParallelFormattingOutputFormat() override
|
||||
{
|
||||
need_flush = true;
|
||||
if (!IOutputFormat::finalized)
|
||||
if (!IOutputFormat::finalized)
|
||||
finalize();
|
||||
finishAndWait();
|
||||
}
|
||||
@ -64,12 +64,12 @@ public:
|
||||
need_flush = true;
|
||||
}
|
||||
|
||||
void doWritePrefix() override
|
||||
void doWritePrefix() override
|
||||
{
|
||||
addChunk(Chunk{}, ProcessingUnitType::START);
|
||||
}
|
||||
|
||||
void onCancel() override
|
||||
void onCancel() override
|
||||
{
|
||||
finishAndWait();
|
||||
}
|
||||
@ -260,7 +260,7 @@ private:
|
||||
IOutputFormat::flush();
|
||||
|
||||
++collector_unit_number;
|
||||
|
||||
|
||||
{
|
||||
/// Notify other threads.
|
||||
std::lock_guard<std::mutex> lock(mutex);
|
||||
@ -292,13 +292,18 @@ private:
|
||||
auto & unit = processing_units[current_unit_number];
|
||||
assert(unit.status = READY_TO_FORMAT);
|
||||
|
||||
/// We want to preallocate memory buffer (increase capacity)
|
||||
/// and put the pointer at the beginning of the buffer
|
||||
/// FIXME: Implement reserve() method in Memory.
|
||||
unit.segment.resize(DBMS_DEFAULT_BUFFER_SIZE);
|
||||
unit.segment.resize(0);
|
||||
|
||||
unit.actual_memory_size = 0;
|
||||
BufferWithOutsideMemory<WriteBuffer> out_buffer(unit.segment);
|
||||
|
||||
auto formatter = internal_formatter_creator(out_buffer);
|
||||
|
||||
switch (unit.type)
|
||||
switch (unit.type)
|
||||
{
|
||||
case ProcessingUnitType::START :
|
||||
{
|
||||
|
@ -95,7 +95,7 @@ void ParallelParsingInputFormat::parserThreadFunction(ThreadGroupStatusPtr threa
|
||||
{
|
||||
/// Variable chunk is moved, but it is not really used in the next iteration.
|
||||
/// NOLINTNEXTLINE(bugprone-use-after-move)
|
||||
unit.chunk_ext.chunk.emplace_back(std::move(chunk));
|
||||
unit.chunk_ext.chunk.emplace_back(std::move(chunk));
|
||||
unit.chunk_ext.block_missing_values.emplace_back(parser.getMissingValues());
|
||||
}
|
||||
|
||||
|
@ -13,6 +13,12 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
class Context;
|
||||
|
||||
/**
|
||||
@ -52,7 +58,7 @@ class Context;
|
||||
class ParallelParsingInputFormat : public IInputFormat
|
||||
{
|
||||
public:
|
||||
/* Used to recreate parser on every new data piece. */
|
||||
/* Used to recreate parser on every new data piece.*/
|
||||
using InternalParserCreator = std::function<InputFormatPtr(ReadBuffer & buf)>;
|
||||
|
||||
struct Params
|
||||
|
Loading…
Reference in New Issue
Block a user