ClickHouse/src/Processors/Formats/Impl/ORCBlockInputFormat.cpp

231 lines
7.3 KiB
C++
Raw Normal View History

2019-08-21 14:19:47 +00:00
#include "ORCBlockInputFormat.h"
#include <boost/algorithm/string/case_conv.hpp>
2019-08-21 14:19:47 +00:00
#if USE_ORC
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
2019-08-21 14:19:47 +00:00
#include <IO/ReadBufferFromMemory.h>
#include <IO/WriteHelpers.h>
#include <IO/copyData.h>
#include "ArrowBufferedStreams.h"
2019-08-21 14:19:47 +00:00
#include "ArrowColumnToCHColumn.h"
#include <DataTypes/NestedUtils.h>
2019-08-21 14:19:47 +00:00
namespace DB
{
2020-02-25 18:20:08 +00:00
2020-02-25 18:10:48 +00:00
namespace ErrorCodes
{
2020-05-03 23:19:56 +00:00
extern const int BAD_ARGUMENTS;
2020-02-25 18:10:48 +00:00
extern const int CANNOT_READ_ALL_DATA;
}
2019-08-21 14:19:47 +00:00
ORCBlockInputFormat::ORCBlockInputFormat(ReadBuffer & in_, Block header_, const FormatSettings & format_settings_)
: IInputFormat(std::move(header_), in_), format_settings(format_settings_), skip_stripes(format_settings.orc.skip_stripes)
2020-02-25 18:20:08 +00:00
{
}
2019-08-21 14:19:47 +00:00
2020-02-25 18:20:08 +00:00
Chunk ORCBlockInputFormat::generate()
{
block_missing_values.clear();
2019-08-21 14:19:47 +00:00
if (!file_reader)
prepareReader();
2021-12-27 19:42:56 +00:00
if (is_stopped)
return {};
for (; stripe_current < stripe_total && skip_stripes.contains(stripe_current); ++stripe_current)
;
2019-08-21 14:19:47 +00:00
if (stripe_current >= stripe_total)
return {};
auto batch_result = file_reader->ReadStripe(stripe_current, include_indices);
if (!batch_result.ok())
throw ParsingException(ErrorCodes::CANNOT_READ_ALL_DATA, "Failed to create batch reader: {}", batch_result.status().ToString());
2020-05-02 19:40:50 +00:00
auto batch = batch_result.ValueOrDie();
if (!batch)
return {};
auto table_result = arrow::Table::FromRecordBatches({batch});
if (!table_result.ok())
throw ParsingException(
ErrorCodes::CANNOT_READ_ALL_DATA, "Error while reading batch of ORC data: {}", table_result.status().ToString());
auto table = table_result.ValueOrDie();
if (!table || !table->num_rows())
return {};
2021-10-31 19:53:24 +00:00
++stripe_current;
Chunk res;
arrow_column_to_ch_column->arrowTableToCHChunk(res, table);
2021-12-02 08:14:25 +00:00
/// If defaults_for_omitted_fields is true, calculate the default values from default expression for omitted fields.
/// Otherwise fill the missing columns with zero values of its type.
if (format_settings.defaults_for_omitted_fields)
2022-05-06 16:48:48 +00:00
for (const auto & column_idx : missing_columns)
block_missing_values.setBits(column_idx, res.getNumRows());
2020-02-25 18:20:08 +00:00
return res;
}
void ORCBlockInputFormat::resetParser()
{
IInputFormat::resetParser();
file_reader.reset();
include_indices.clear();
block_missing_values.clear();
}
const BlockMissingValues & ORCBlockInputFormat::getMissingValues() const
{
return block_missing_values;
}
2021-05-20 13:47:12 +00:00
static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
{
if (type->id() == arrow::Type::LIST)
return countIndicesForType(static_cast<arrow::ListType *>(type.get())->value_type()) + 1;
if (type->id() == arrow::Type::STRUCT)
{
2021-05-27 19:01:06 +00:00
int indices = 1;
auto * struct_type = static_cast<arrow::StructType *>(type.get());
for (int i = 0; i != struct_type->num_fields(); ++i)
indices += countIndicesForType(struct_type->field(i)->type());
2021-05-27 19:01:06 +00:00
return indices;
}
2021-05-27 19:01:06 +00:00
if (type->id() == arrow::Type::MAP)
{
auto * map_type = static_cast<arrow::MapType *>(type.get());
2021-05-27 19:10:45 +00:00
return countIndicesForType(map_type->key_type()) + countIndicesForType(map_type->item_type());
2021-05-27 19:01:06 +00:00
}
return 1;
}
static void getFileReaderAndSchema(
ReadBuffer & in,
std::unique_ptr<arrow::adapters::orc::ORCFileReader> & file_reader,
std::shared_ptr<arrow::Schema> & schema,
const FormatSettings & format_settings,
std::atomic<int> & is_stopped)
{
auto arrow_file = asArrowFile(in, format_settings, is_stopped, "ORC", ORC_MAGIC_BYTES);
2021-12-27 19:42:56 +00:00
if (is_stopped)
return;
auto result = arrow::adapters::orc::ORCFileReader::Open(arrow_file, arrow::default_memory_pool());
2021-11-30 16:39:29 +00:00
if (!result.ok())
throw Exception(result.status().ToString(), ErrorCodes::BAD_ARGUMENTS);
2021-12-01 11:19:24 +00:00
file_reader = std::move(result).ValueOrDie();
2021-11-30 16:39:29 +00:00
auto read_schema_result = file_reader->ReadSchema();
if (!read_schema_result.ok())
throw Exception(read_schema_result.status().ToString(), ErrorCodes::BAD_ARGUMENTS);
schema = std::move(read_schema_result).ValueOrDie();
}
void ORCBlockInputFormat::prepareReader()
{
2022-03-11 09:26:13 +00:00
std::shared_ptr<arrow::Schema> schema;
getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped);
if (is_stopped)
return;
stripe_total = file_reader->NumberOfStripes();
stripe_current = 0;
arrow_column_to_ch_column = std::make_unique<ArrowColumnToCHColumn>(
2022-03-21 07:50:17 +00:00
getPort().getHeader(),
"ORC",
format_settings.orc.import_nested,
format_settings.orc.allow_missing_columns,
format_settings.orc.case_insensitive_column_matching);
2021-12-02 08:14:25 +00:00
missing_columns = arrow_column_to_ch_column->getMissingColumns(*schema);
2022-03-23 15:02:19 +00:00
const bool ignore_case = format_settings.orc.case_insensitive_column_matching;
std::unordered_set<String> nested_table_names;
if (format_settings.orc.import_nested)
2022-03-23 15:02:19 +00:00
nested_table_names = Nested::getAllTableNames(getPort().getHeader(), ignore_case);
2021-06-07 15:15:58 +00:00
/// In ReadStripe column indices should be started from 1,
/// because 0 indicates to select all columns.
int index = 1;
for (int i = 0; i < schema->num_fields(); ++i)
{
/// LIST type require 2 indices, STRUCT - the number of elements + 1,
/// so we should recursively count the number of indices we need for this type.
int indexes_count = countIndicesForType(schema->field(i)->type());
const auto & name = schema->field(i)->name();
2022-03-23 15:02:19 +00:00
if (getPort().getHeader().has(name, ignore_case) || nested_table_names.contains(ignore_case ? boost::to_lower_copy(name) : name))
{
for (int j = 0; j != indexes_count; ++j)
include_indices.push_back(index + j);
}
2022-03-21 07:47:37 +00:00
index += indexes_count;
}
2020-02-25 18:20:08 +00:00
}
2022-03-21 07:50:17 +00:00
ORCSchemaReader::ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: ISchemaReader(in_), format_settings(format_settings_)
{
}
NamesAndTypesList ORCSchemaReader::readSchema()
{
std::unique_ptr<arrow::adapters::orc::ORCFileReader> file_reader;
std::shared_ptr<arrow::Schema> schema;
std::atomic<int> is_stopped = 0;
getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped);
auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(
*schema, "ORC", format_settings.orc.skip_columns_with_unsupported_types_in_schema_inference);
return getNamesAndRecursivelyNullableTypes(header);
}
void registerInputFormatORC(FormatFactory & factory)
2020-02-25 18:20:08 +00:00
{
2021-10-11 16:11:50 +00:00
factory.registerInputFormat(
2020-02-25 18:20:08 +00:00
"ORC",
[](ReadBuffer &buf,
const Block &sample,
const RowInputFormatParams &,
const FormatSettings & settings)
2020-05-03 23:19:56 +00:00
{
return std::make_shared<ORCBlockInputFormat>(buf, sample, settings);
2020-02-25 18:20:08 +00:00
});
2021-03-30 21:25:37 +00:00
factory.markFormatAsColumnOriented("ORC");
2020-02-25 18:20:08 +00:00
}
2019-08-21 14:19:47 +00:00
void registerORCSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader(
"ORC",
[](ReadBuffer & buf, const FormatSettings & settings)
{
return std::make_shared<ORCSchemaReader>(buf, settings);
}
);
}
2019-08-21 14:19:47 +00:00
}
#else
namespace DB
{
class FormatFactory;
2021-10-11 16:11:50 +00:00
void registerInputFormatORC(FormatFactory &)
2019-08-21 14:19:47 +00:00
{
}
void registerORCSchemaReader(FormatFactory &)
{
}
2019-08-21 14:19:47 +00:00
}
#endif