mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-16 19:32:07 +00:00
fix test
Change-Id: Ia7dbf1d762f7f054a9aa677caaaff6bfe1a42c38
This commit is contained in:
parent
e1fcdba4dd
commit
471dff6589
@ -176,6 +176,7 @@ static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> sett
|
||||
{"default_normal_view_sql_security", "INVOKER", "INVOKER", "Allows to set default `SQL SECURITY` option while creating a normal view"},
|
||||
{"mysql_map_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."},
|
||||
{"mysql_map_fixed_string_to_text_in_show_columns", false, true, "Reduce the configuration effort to connect ClickHouse with BI tools."},
|
||||
{"input_format_parquet_use_native_reader", false, false, "When reading Parquet files, to use native reader instead of arrow reader."},
|
||||
}},
|
||||
{"24.1", {{"print_pretty_type_names", false, true, "Better user experience."},
|
||||
{"input_format_json_read_bools_as_strings", false, true, "Allow to read bools as strings in JSON formats by default"},
|
||||
|
@ -38,15 +38,13 @@ public:
|
||||
template <typename TValue>
|
||||
void ALWAYS_INLINE readValue(TValue & dst)
|
||||
{
|
||||
checkAvaible(sizeof(TValue));
|
||||
dst = *(reinterpret_cast<const TValue *>(data));
|
||||
consume(sizeof(TValue));
|
||||
readBytes(&dst, sizeof(TValue));
|
||||
}
|
||||
|
||||
void ALWAYS_INLINE readBytes(void * dst, size_t bytes)
|
||||
{
|
||||
checkAvaible(bytes);
|
||||
memcpy(dst, data, bytes);
|
||||
std::copy(data, data + bytes, reinterpret_cast<Int8 *>(dst));
|
||||
consume(bytes);
|
||||
}
|
||||
|
||||
@ -68,13 +66,12 @@ public:
|
||||
100000000 * spd,
|
||||
1000000000 * spd};
|
||||
|
||||
checkAvaible(sizeof(parquet::Int96));
|
||||
auto decoded = parquet::DecodeInt96Timestamp(*reinterpret_cast<const parquet::Int96 *>(data));
|
||||
parquet::Int96 tmp;
|
||||
readValue(tmp);
|
||||
auto decoded = parquet::DecodeInt96Timestamp(tmp);
|
||||
|
||||
uint64_t scaled_nano = decoded.nanoseconds / pow10[datetime64_scale];
|
||||
dst = static_cast<Int64>(decoded.days_since_epoch * scaled_day[datetime64_scale] + scaled_nano);
|
||||
|
||||
consume(sizeof(parquet::Int96));
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -306,7 +306,7 @@ void ParquetPlainValuesReader<ColumnDecimal<DateTime64>>::readBatch(
|
||||
},
|
||||
/* repeated_visitor */ [&](size_t nest_cursor, UInt32 count)
|
||||
{
|
||||
auto col_data_pos = column_data + nest_cursor;
|
||||
auto * col_data_pos = column_data + nest_cursor;
|
||||
for (UInt32 i = 0; i < count; i++)
|
||||
{
|
||||
plain_data_buffer.readDateTime64(col_data_pos[i]);
|
||||
|
@ -97,7 +97,7 @@ public:
|
||||
* @tparam ValueGetter A callback with signature: TValue(Int32 val)
|
||||
*/
|
||||
template <typename TValue, typename ValueGetter>
|
||||
void setValues(TValue * column_data, UInt32 num_values, ValueGetter && val_getter);
|
||||
void setValues(TValue * res_values, UInt32 num_values, ValueGetter && val_getter);
|
||||
|
||||
/**
|
||||
* @brief Set the value by valid_index_steps generated in visitNullableBySteps.
|
||||
@ -106,7 +106,7 @@ public:
|
||||
*/
|
||||
template <typename TValue, typename ValueGetter>
|
||||
void setValueBySteps(
|
||||
TValue * column_data,
|
||||
TValue * res_values,
|
||||
const std::vector<UInt8> & col_data_steps,
|
||||
ValueGetter && val_getter);
|
||||
|
||||
|
@ -113,7 +113,7 @@ ColumnPtr readDictPage<ColumnDecimal<DateTime64>>(
|
||||
const parquet::ColumnDescriptor & /* col_des */,
|
||||
const DataTypePtr & data_type)
|
||||
{
|
||||
auto & datetime_type = assert_cast<const DataTypeDateTime64 &>(*data_type);
|
||||
const auto & datetime_type = assert_cast<const DataTypeDateTime64 &>(*data_type);
|
||||
auto dict_col = ColumnDecimal<DateTime64>::create(page.num_values(), datetime_type.getScale());
|
||||
auto * col_data = dict_col->getData().data();
|
||||
ParquetDataBuffer buffer(page.data(), page.size(), datetime_type.getScale());
|
||||
@ -282,7 +282,7 @@ void ParquetLeafColReader<TColumn>::degradeDictionary()
|
||||
dictionary = nullptr;
|
||||
return;
|
||||
}
|
||||
assert(dictionary && column->size());
|
||||
assert(dictionary && !column->empty());
|
||||
|
||||
null_map = std::make_unique<LazyNullMap>(reading_rows_num);
|
||||
auto col_existing = std::move(column);
|
||||
@ -372,7 +372,7 @@ void ParquetLeafColReader<TColumn>::readPage()
|
||||
dict_page.encoding() != parquet::Encoding::PLAIN_DICTIONARY
|
||||
&& dict_page.encoding() != parquet::Encoding::PLAIN))
|
||||
{
|
||||
throw new Exception(
|
||||
throw Exception(
|
||||
ErrorCodes::NOT_IMPLEMENTED, "Unsupported dictionary page encoding {}", dict_page.encoding());
|
||||
}
|
||||
LOG_DEBUG(log, "{} values in dictionary page of column {}", dict_page.num_values(), col_descriptor.name());
|
||||
|
@ -156,9 +156,6 @@ ParquetRecordReader::ParquetRecordReader(
|
||||
, row_groups_indices(std::move(row_groups_indices_))
|
||||
, left_rows(getTotalRows(*file_reader->metadata()))
|
||||
{
|
||||
// Only little endian system is supported currently
|
||||
static_assert(std::endian::native == std::endian::little);
|
||||
|
||||
log = &Poco::Logger::get("ParquetRecordReader");
|
||||
|
||||
parquet_col_indice.reserve(header.columns());
|
||||
@ -230,9 +227,9 @@ void ParquetRecordReader::loadNextRowGroup()
|
||||
Int64 ParquetRecordReader::getTotalRows(const parquet::FileMetaData & meta_data)
|
||||
{
|
||||
Int64 res = 0;
|
||||
for (size_t i = 0; i < row_groups_indices.size(); i++)
|
||||
for (auto idx : row_groups_indices)
|
||||
{
|
||||
res += meta_data.RowGroup(row_groups_indices[i])->num_rows();
|
||||
res += meta_data.RowGroup(idx)->num_rows();
|
||||
}
|
||||
return res;
|
||||
}
|
||||
|
@ -484,6 +484,14 @@ void ParquetBlockInputFormat::initializeRowGroupBatchReader(size_t row_group_bat
|
||||
|
||||
if (format_settings.parquet.use_native_reader)
|
||||
{
|
||||
#pragma clang diagnostic push
|
||||
#pragma clang diagnostic ignored "-Wunreachable-code"
|
||||
if constexpr (std::endian::native != std::endian::little)
|
||||
throw Exception(
|
||||
ErrorCodes::BAD_ARGUMENTS,
|
||||
"parquet native reader only supports little endian system currently");
|
||||
#pragma clang diagnostic pop
|
||||
|
||||
row_group_batch.native_record_reader = std::make_shared<ParquetRecordReader>(
|
||||
getPort().getHeader(),
|
||||
std::move(properties),
|
||||
|
@ -1,4 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-fasttest
|
||||
|
||||
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||
# shellcheck source=../shell_config.sh
|
||||
@ -201,8 +202,8 @@ CH_SCHEMA="\
|
||||
plain_encoding_str Nullable(String), \
|
||||
mix_encoding_str Nullable(String), \
|
||||
dict_encoding_str LowCardinality(Nullable(String)), \
|
||||
plain_encoding_dt64 Nullable(DateTime64(9)), \
|
||||
dict_encoding_dt64 Nullable(DateTime64(9)), \
|
||||
plain_encoding_dt64 Nullable(DateTime64(9, \\'UTC\\')), \
|
||||
dict_encoding_dt64 Nullable(DateTime64(9, \\'UTC\\')), \
|
||||
plain_encoding_decimal128 Nullable(Decimal(38, 3))"
|
||||
QUERY="SELECT * from file('$PAR_PATH', 'Parquet', '$CH_SCHEMA')"
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user