mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Remove Impl including from .h file
This commit is contained in:
parent
83c843a20b
commit
a4ef60e230
@ -1,4 +1,5 @@
|
||||
#include "ArrowBlockInputFormat.h"
|
||||
|
||||
#if USE_ARROW
|
||||
|
||||
#include <Formats/FormatFactory.h>
|
||||
@ -22,7 +23,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
ArrowBlockInputFormat::ArrowBlockInputFormat(ReadBuffer & in_, const Block & header_, bool stream_)
|
||||
: IInputFormat(header_, in_), stream{stream_}
|
||||
: IInputFormat(header_, in_), stream{stream_}, arrow_column_to_ch_column(std::make_unique<ArrowColumnToCHColumn>())
|
||||
{
|
||||
}
|
||||
|
||||
@ -63,7 +64,7 @@ Chunk ArrowBlockInputFormat::generate()
|
||||
|
||||
++record_batch_current;
|
||||
|
||||
arrow_column_to_ch_column.arrowTableToCHChunk(res, *table_result, header, "Arrow");
|
||||
arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, header, "Arrow");
|
||||
|
||||
return res;
|
||||
}
|
||||
|
@ -3,7 +3,6 @@
|
||||
#if USE_ARROW
|
||||
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Formats/Impl/ArrowColumnToCHColumn.h>
|
||||
|
||||
namespace arrow { class RecordBatchReader; }
|
||||
namespace arrow::ipc { class RecordBatchFileReader; }
|
||||
@ -12,6 +11,7 @@ namespace DB
|
||||
{
|
||||
|
||||
class ReadBuffer;
|
||||
class ArrowColumnToCHColumn;
|
||||
|
||||
class ArrowBlockInputFormat : public IInputFormat
|
||||
{
|
||||
@ -33,7 +33,7 @@ private:
|
||||
// The following fields are used only for Arrow format
|
||||
std::shared_ptr<arrow::ipc::RecordBatchFileReader> file_reader;
|
||||
|
||||
ArrowColumnToCHColumn arrow_column_to_ch_column;
|
||||
std::unique_ptr<ArrowColumnToCHColumn> arrow_column_to_ch_column;
|
||||
|
||||
int record_batch_total = 0;
|
||||
int record_batch_current = 0;
|
||||
|
@ -18,7 +18,11 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
ArrowBlockOutputFormat::ArrowBlockOutputFormat(WriteBuffer & out_, const Block & header_, bool stream_, const FormatSettings & format_settings_)
|
||||
: IOutputFormat(header_, out_), stream{stream_}, format_settings{format_settings_}, arrow_ostream{std::make_shared<ArrowBufferedOutputStream>(out_)}
|
||||
: IOutputFormat(header_, out_)
|
||||
, stream{stream_}
|
||||
, format_settings{format_settings_}
|
||||
, arrow_ostream{std::make_shared<ArrowBufferedOutputStream>(out_)}
|
||||
, ch_column_to_arrow_column(std::make_unique<CHColumnToArrowColumn>())
|
||||
{
|
||||
}
|
||||
|
||||
@ -28,7 +32,7 @@ void ArrowBlockOutputFormat::consume(Chunk chunk)
|
||||
const size_t columns_num = chunk.getNumColumns();
|
||||
std::shared_ptr<arrow::Table> arrow_table;
|
||||
|
||||
ch_column_to_arrow_column.chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow", format_settings.arrow.low_cardinality_as_dictionary);
|
||||
ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Arrow", format_settings.arrow.low_cardinality_as_dictionary);
|
||||
|
||||
if (!writer)
|
||||
prepareWriter(arrow_table->schema());
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#include <Formats/FormatSettings.h>
|
||||
#include <Processors/Formats/IOutputFormat.h>
|
||||
#include <Processors/Formats/Impl/CHColumnToArrowColumn.h>
|
||||
#include "ArrowBufferedStreams.h"
|
||||
|
||||
namespace arrow { class Schema; }
|
||||
@ -13,6 +12,8 @@ namespace arrow::ipc { class RecordBatchWriter; }
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class CHColumnToArrowColumn;
|
||||
|
||||
class ArrowBlockOutputFormat : public IOutputFormat
|
||||
{
|
||||
public:
|
||||
@ -29,7 +30,7 @@ private:
|
||||
const FormatSettings format_settings;
|
||||
std::shared_ptr<ArrowBufferedOutputStream> arrow_ostream;
|
||||
std::shared_ptr<arrow::ipc::RecordBatchWriter> writer;
|
||||
CHColumnToArrowColumn ch_column_to_arrow_column;
|
||||
std::unique_ptr<CHColumnToArrowColumn> ch_column_to_arrow_column;
|
||||
|
||||
void prepareWriter(const std::shared_ptr<arrow::Schema> & schema);
|
||||
};
|
||||
|
@ -26,7 +26,7 @@ namespace ErrorCodes
|
||||
throw Exception(_s.ToString(), ErrorCodes::BAD_ARGUMENTS); \
|
||||
} while (false)
|
||||
|
||||
ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInputFormat(std::move(header_), in_)
|
||||
ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_) : IInputFormat(std::move(header_), in_), arrow_column_to_ch_column(std::make_unique<ArrowColumnToCHColumn>())
|
||||
{
|
||||
}
|
||||
|
||||
@ -54,7 +54,7 @@ Chunk ORCBlockInputFormat::generate()
|
||||
|
||||
++stripe_current;
|
||||
|
||||
arrow_column_to_ch_column.arrowTableToCHChunk(res, *table_result, header, "ORC");
|
||||
arrow_column_to_ch_column->arrowTableToCHChunk(res, *table_result, header, "ORC");
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -3,12 +3,14 @@
|
||||
#if USE_ORC
|
||||
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Formats/Impl/ArrowColumnToCHColumn.h>
|
||||
|
||||
namespace arrow::adapters::orc { class ORCFileReader; }
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ArrowColumnToCHColumn;
|
||||
|
||||
class ORCBlockInputFormat : public IInputFormat
|
||||
{
|
||||
public:
|
||||
@ -27,7 +29,7 @@ private:
|
||||
|
||||
std::unique_ptr<arrow::adapters::orc::ORCFileReader> file_reader;
|
||||
|
||||
ArrowColumnToCHColumn arrow_column_to_ch_column;
|
||||
std::unique_ptr<ArrowColumnToCHColumn> arrow_column_to_ch_column;
|
||||
|
||||
int stripe_total = 0;
|
||||
|
||||
|
@ -31,7 +31,7 @@ namespace ErrorCodes
|
||||
} while (false)
|
||||
|
||||
ParquetBlockInputFormat::ParquetBlockInputFormat(ReadBuffer & in_, Block header_)
|
||||
: IInputFormat(std::move(header_), in_)
|
||||
: IInputFormat(std::move(header_), in_), arrow_column_to_ch_column(std::make_unique<ArrowColumnToCHColumn>())
|
||||
{
|
||||
}
|
||||
|
||||
@ -54,7 +54,7 @@ Chunk ParquetBlockInputFormat::generate()
|
||||
|
||||
++row_group_current;
|
||||
|
||||
arrow_column_to_ch_column.arrowTableToCHChunk(res, table, header, "Parquet");
|
||||
arrow_column_to_ch_column->arrowTableToCHChunk(res, table, header, "Parquet");
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -4,7 +4,6 @@
|
||||
#if USE_PARQUET
|
||||
|
||||
#include <Processors/Formats/IInputFormat.h>
|
||||
#include <Processors/Formats/Impl/ArrowColumnToCHColumn.h>
|
||||
|
||||
namespace parquet::arrow { class FileReader; }
|
||||
|
||||
@ -13,6 +12,8 @@ namespace arrow { class Buffer; }
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class ArrowColumnToCHColumn;
|
||||
|
||||
class ParquetBlockInputFormat : public IInputFormat
|
||||
{
|
||||
public:
|
||||
@ -33,7 +34,7 @@ private:
|
||||
int row_group_total = 0;
|
||||
// indices of columns to read from Parquet file
|
||||
std::vector<int> column_indices;
|
||||
ArrowColumnToCHColumn arrow_column_to_ch_column;
|
||||
std::unique_ptr<ArrowColumnToCHColumn> arrow_column_to_ch_column;
|
||||
int row_group_current = 0;
|
||||
};
|
||||
|
||||
|
@ -25,7 +25,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
ParquetBlockOutputFormat::ParquetBlockOutputFormat(WriteBuffer & out_, const Block & header_, const FormatSettings & format_settings_)
|
||||
: IOutputFormat(header_, out_), format_settings{format_settings_}
|
||||
: IOutputFormat(header_, out_), format_settings{format_settings_}, ch_column_to_arrow_column(std::make_unique<CHColumnToArrowColumn>())
|
||||
{
|
||||
}
|
||||
|
||||
@ -35,7 +35,7 @@ void ParquetBlockOutputFormat::consume(Chunk chunk)
|
||||
const size_t columns_num = chunk.getNumColumns();
|
||||
std::shared_ptr<arrow::Table> arrow_table;
|
||||
|
||||
ch_column_to_arrow_column.chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet");
|
||||
ch_column_to_arrow_column->chChunkToArrowTable(arrow_table, header, chunk, columns_num, "Parquet");
|
||||
|
||||
if (!file_writer)
|
||||
{
|
||||
|
@ -4,7 +4,6 @@
|
||||
#if USE_PARQUET
|
||||
# include <Processors/Formats/IOutputFormat.h>
|
||||
# include <Formats/FormatSettings.h>
|
||||
# include <Processors/Formats/Impl/CHColumnToArrowColumn.h>
|
||||
|
||||
namespace arrow
|
||||
{
|
||||
@ -22,6 +21,9 @@ namespace arrow
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class CHColumnToArrowColumn;
|
||||
|
||||
class ParquetBlockOutputFormat : public IOutputFormat
|
||||
{
|
||||
public:
|
||||
@ -37,7 +39,7 @@ private:
|
||||
const FormatSettings format_settings;
|
||||
|
||||
std::unique_ptr<parquet::arrow::FileWriter> file_writer;
|
||||
CHColumnToArrowColumn ch_column_to_arrow_column;
|
||||
std::unique_ptr<CHColumnToArrowColumn> ch_column_to_arrow_column;
|
||||
};
|
||||
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user