ClickHouse/tests/queries/0_stateless/02998_native_parquet_reader.sh
2024-07-30 11:04:41 +00:00

212 lines
8.0 KiB
Bash
Executable File

#!/usr/bin/env bash
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
PAR_PATH="$CURDIR"/data_parquet/native_parquet_reader.parquet
# the content of parquet file can be generated by following codes
# <<EndOfCodes
#
# #include <Columns/ColumnNullable.h>
# #include <Columns/ColumnString.h>
# #include <Common/tests/gtest_global_register.h>
# #include <Core/Block.h>
# #include <DataTypes/DataTypeDateTime64.h>
# #include <DataTypes/DataTypeLowCardinality.h>
# #include <DataTypes/DataTypeNullable.h>
# #include <DataTypes/DataTypeString.h>
# #include <DataTypes/DataTypesDecimal.h>
# #include <IO/WriteBufferFromFile.h>
# #include <Processors/Formats/IOutputFormat.h>
# #include <Processors/Formats/Impl/Parquet/ParquetRecordReader.h>
#
# namespace
# {
#
# using namespace DB;
#
# const UInt32 ROW_NUM = 2000;
# const UInt32 MIN_STRING_LEN = 3;
# const UInt32 MAX_STRING_LEN = 5;
#
# const UInt32 PLAIN_ENCODING_CARDINALITY = ROW_NUM * 2;
# const UInt32 MIX_ENCODING_CARDINALITY = 800;
# const UInt32 DICT_ENCODING_CARDINALITY = 20;
#
# UInt16 nextNum()
# {
# static UInt16 idx = 0;
# static UInt16 nums[] = {0, 21845, 43690};
# static size_t nums_len = sizeof(nums) / sizeof(nums[0]);
# return nums[(idx++) % nums_len]++;
# }
#
# template <typename NumericDataType>
# void generateValues(MutableColumnPtr & col, size_t num)
# {
# using FieldType = typename NumericDataType::FieldType;
#
# const size_t next_num_bytes = sizeof(nextNum());
# char bytewise_val[sizeof(FieldType)];
#
# while (col->size() < num)
# {
# for (auto bytes = 0; bytes < sizeof(FieldType); bytes += next_num_bytes)
# {
# auto tmp = nextNum();
# memcpy(bytewise_val + bytes, &tmp, std::min(next_num_bytes, sizeof(FieldType) - bytes));
# }
# if (is_decimal<FieldType>)
# {
# // clean highest 3 bits, make sure the result doest not exceed the limits of the decimal type
# if (bytewise_val[sizeof(FieldType) - 1] > 0)
# bytewise_val[sizeof(FieldType) - 1] &= 0x0f;
# else
# bytewise_val[sizeof(FieldType) - 1] |= 0xf0;
# }
# FieldType val;
# memcpy(&val, &bytewise_val, sizeof(FieldType));
# col->insert(val);
# }
# }
#
# template <>
# void generateValues<DataTypeString>(MutableColumnPtr & col, size_t num)
# {
# std::string str;
# while (col->size() < num)
# {
# auto len = MIN_STRING_LEN + nextNum() % (MAX_STRING_LEN - MIN_STRING_LEN);
# str.clear();
# for (size_t i = 0; i < len; i++)
# {
# str.push_back('a' + nextNum() % ('z' - 'a'));
# }
# col->insert(str);
# }
# }
#
# template <typename DataType>
# ColumnWithTypeAndName generateColumn(
# std::shared_ptr<DataType> ch_type,
# size_t cardinality,
# const std::string & col_name,
# const std::set<size_t> & null_indice)
# {
# DataTypePtr col_type = ch_type;
# if (!null_indice.empty())
# {
# col_type = std::make_shared<DataTypeNullable>(ch_type);
# }
#
# auto values = ch_type->createColumn();
# values->reserve(cardinality);
# generateValues<DataType>(values, cardinality);
#
# auto col = col_type->createColumn();
# col->reserve(ROW_NUM);
# for (size_t i = 0; i < ROW_NUM; i++)
# {
# if (!null_indice.empty() && null_indice.contains(i))
# {
# col->insert(Null());
# }
# else
# {
# col->insert(values->operator[](nextNum() % cardinality));
# }
# }
# return {std::move(col), col_type, col_name};
# }
#
# Block generateBlock()
# {
# ColumnsWithTypeAndName cols;
#
# // test Int32 type
# std::set<size_t> null_indice{512, 1001, 211, 392, 553, 1725};
# // Nullability is expressed by definition level, and encoded by bit packed with smallest group size of 8
# // when null value appeared. Here we make a big bit packed group with more than 1000 values.
# for (size_t i = 0; i < 170; i++)
# {
# null_indice.emplace(622 + i * 6);
# }
# cols.emplace_back(generateColumn(
# std::make_shared<DataTypeInt32>(), PLAIN_ENCODING_CARDINALITY, "plain_encoding_i32", null_indice));
# null_indice = {917, 482, 283, 580, 1926, 1667, 1971};
# cols.emplace_back(generateColumn(
# std::make_shared<DataTypeInt32>(), DICT_ENCODING_CARDINALITY, "dict_encoding_i32", null_indice));
#
# // test string type
# null_indice = {818, 928, 1958, 1141, 1553, 1407, 690, 1769};
# cols.emplace_back(generateColumn(
# std::make_shared<DataTypeString>(), PLAIN_ENCODING_CARDINALITY, "plain_encoding_str", null_indice));
# null_indice = {1441, 1747, 216, 1209, 89, 52, 536, 625};
# cols.emplace_back(generateColumn(
# std::make_shared<DataTypeString>(), MIX_ENCODING_CARDINALITY, "mix_encoding_str", null_indice));
# null_indice = {1478, 1862, 894, 1314, 1844, 243, 869, 551};
# cols.emplace_back(generateColumn(
# std::make_shared<DataTypeString>(), DICT_ENCODING_CARDINALITY, "dict_encoding_str", null_indice));
#
# // test DateTime64 type
# auto dt_type = std::make_shared<DataTypeDateTime64>(ParquetRecordReader::default_datetime64_scale);
# null_indice = {1078, 112, 1981, 795, 371, 1176, 1526, 11};
# cols.emplace_back(generateColumn(dt_type, PLAIN_ENCODING_CARDINALITY, "plain_encoding_dt64", null_indice));
# null_indice = {1734, 1153, 1893, 1205, 644, 1670, 1482, 1479};
# cols.emplace_back(generateColumn(dt_type, DICT_ENCODING_CARDINALITY, "dict_encoding_dt64", null_indice));
#
# // test Decimal128 type
# auto d128_type = std::make_shared<DataTypeDecimal128>(DecimalUtils::max_precision<Decimal128>, 3);
# null_indice = {852, 1448, 1569, 896, 1866, 1655, 100, 418};
# cols.emplace_back(generateColumn(d128_type, PLAIN_ENCODING_CARDINALITY, "plain_encoding_decimal128", null_indice));
#
# return {cols};
# }
#
# void dumpBlock(const Block & block)
# {
# WriteBufferFromFile output_buf("/tmp/ut-out.csv");
# auto out = getContext().context->getOutputFormat("CSVWithNames", output_buf, block);
# out->write(block);
# out->finalize();
# std::cerr << block.dumpStructure() << std::endl << std::endl;
# }
#
# }
#
# EndOfCodes
#
# How to generate the parquet file:
# 1. Use above C++ codes.
# Put above codes in src/Common/tests/gtest_main.cpp, add following two inlines in main function:
# tryRegisterFormats();
# dumpBlock(generateBlock());
# 2. Genetate /tmp/ut-out.csv.
# After compiled, run any test, such as "./src/unit_tests_dbms --gtest_filter=IColumn.dumpStructure",
# 3. Generate the parquet file by following spark sql
# create temporary view tv using csv options('path' '/tmp/ut-out.csv', 'header' 'true', 'nullValue' '\\N');
# insert overwrite directory "/tmp/test-parquet" using Parquet
# options('parquet.dictionary.page.size' '500')
# select /*+ COALESCE(1) */ cast(plain_encoding_i32 as int), cast(dict_encoding_i32 as int),
# plain_encoding_str, mix_encoding_str, dict_encoding_str,
# cast(plain_encoding_dt64 as timestamp), cast(dict_encoding_dt64 as timestamp),
# cast(plain_encoding_decimal128 as decimal(38, 3))
# from tv;
#
CH_SCHEMA="\
plain_encoding_i32 Nullable(Int32), \
dict_encoding_i32 Nullable(Int32), \
plain_encoding_str Nullable(String), \
mix_encoding_str Nullable(String), \
dict_encoding_str LowCardinality(Nullable(String)), \
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')"
# there may be more than on group in parquet files, unstable results may generated by multithreads
$CLICKHOUSE_LOCAL --max_threads 1 --max_parsing_threads 1 --input_format_parquet_use_native_reader true --query "$QUERY"