Merge branch 'master' into zookeeper_client_fault_injection

This commit is contained in:
Alexander Tokmakov 2022-08-19 11:12:40 +03:00 committed by GitHub
commit a90d2ad35e
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
26 changed files with 428 additions and 176 deletions

View File

@ -35,8 +35,9 @@ jobs:
GID=$(id -g "${UID}")
docker run -u "${UID}:${GID}" -e PYTHONUNBUFFERED=1 \
--volume="${GITHUB_WORKSPACE}:/ClickHouse" clickhouse/style-test \
/ClickHouse/utils/changelog/changelog.py -vv --gh-user-or-token="$GITHUB_TOKEN" \
--output="/ClickHouse/docs/changelogs/${GITHUB_TAG}.md" --jobs=5 "${GITHUB_TAG}"
/ClickHouse/utils/changelog/changelog.py -v --debug-helpers \
--gh-user-or-token="$GITHUB_TOKEN" --jobs=5 \
--output="/ClickHouse/docs/changelogs/${GITHUB_TAG}.md" "${GITHUB_TAG}"
git add "./docs/changelogs/${GITHUB_TAG}.md"
git diff HEAD
- name: Create Pull Request

View File

@ -1822,10 +1822,13 @@ Result:
Evaluate external model.
Accepts a model name and model arguments. Returns Float64.
## throwIf(x\[, custom_message\])
## throwIf(x\[, message\[, error_code\]\])
Throw an exception if the argument is non zero.
custom_message - is an optional parameter: a constant string, provides an error message
`message` - is an optional parameter: a constant string providing a custom error message
`error_code` - is an optional parameter: a constant integer providing a custom error code
To use the `error_code` argument, configuration parameter `allow_custom_error_code_in_throwif` must be enabled.
``` sql
SELECT throwIf(number = 3, 'Too many') FROM numbers(10);

View File

@ -1727,10 +1727,13 @@ SELECT joinGet(db_test.id_val,'val',toUInt32(number)) from numbers(4) SETTINGS j
Принимает на вход имя и аргументы модели. Возвращает Float64.
## throwIf(x\[, custom_message\]) {#throwifx-custom-message}
## throwIf(x\[, message\[, error_code\]\]) {#throwifx-custom-message}
Бросает исключение, если аргумент не равен нулю.
custom_message - необязательный параметр, константная строка, задает текст сообщения об ошибке.
`custom_message` - необязательный параметр, константная строка, задает текст сообщения об ошибке.
`error_code` - необязательный параметр, константное число, задает код ошибки.
Чтобы использовать аргумент `error_code`, должен быть включен параметр конфигурации `allow_custom_error_code_in_throwif`.
``` sql
SELECT throwIf(number = 3, 'Too many') FROM numbers(10);

View File

@ -408,6 +408,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(UInt64, low_cardinality_max_dictionary_size, 8192, "Maximum size (in rows) of shared global dictionary for LowCardinality type.", 0) \
M(Bool, low_cardinality_use_single_dictionary_for_part, false, "LowCardinality type serialization setting. If is true, than will use additional keys when global dictionary overflows. Otherwise, will create several shared dictionaries.", 0) \
M(Bool, decimal_check_overflow, true, "Check overflow of decimal arithmetic/comparison operations", 0) \
M(Bool, allow_custom_error_code_in_throwif, false, "Enable custom error code in function throwIf(). If true, thrown exceptions may have unexpected error codes.", 0) \
\
M(Bool, prefer_localhost_replica, true, "If it's true then queries will be always sent to local replica (if it exists). If it's false then replica to send a query will be chosen between local and remote ones according to load_balancing", 0) \
M(UInt64, max_fetch_partition_retries_count, 5, "Amount of retries while fetching partition from another host.", 0) \
@ -713,7 +714,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, input_format_orc_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format ORC", 0) \
M(Bool, input_format_arrow_skip_columns_with_unsupported_types_in_schema_inference, false, "Skip columns with unsupported types while schema inference for format Arrow", 0) \
M(String, column_names_for_schema_inference, "", "The list of column names to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \
M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column1,column2,column3,...'", 0) \
M(String, schema_inference_hints, "", "The list of column names and types to use in schema inference for formats without column names. The format: 'column_name1 column_type1, column_name2 column_type2, ...'", 0) \
M(Bool, input_format_json_read_bools_as_numbers, true, "Allow to parse bools as numbers in JSON input formats", 0) \
M(Bool, input_format_json_try_infer_numbers_from_strings, true, "Try to infer numbers from string fields while schema inference", 0) \
M(Bool, input_format_try_infer_integers, true, "Try to infer numbers from string fields while schema inference in text formats", 0) \

View File

@ -611,7 +611,7 @@ public:
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (!isNativeNumber(arguments[1].type))
throw Exception("Second argument for function " + getName() + " (delta) must be number",
throw Exception("Second argument for function " + getName() + " (delta) must be a number",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 2)
@ -627,7 +627,7 @@ public:
{
throw Exception(
"Function " + getName() + " supports 2 or 3 arguments. The 1st argument "
"must be of type Date or DateTime. The 2nd argument must be number. "
"must be of type Date or DateTime. The 2nd argument must be a number. "
"The 3rd argument (optional) must be "
"a constant string with timezone name. The timezone argument is allowed "
"only when the 1st argument has the type DateTime",

View File

@ -46,7 +46,7 @@ public:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!isNativeNumber(arguments.front()))
throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Argument for function " + getName() + " must be a number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
}

View File

@ -62,7 +62,7 @@ protected:
DataTypePtr argument_type = arguments[i].type;
if (!isNumber(argument_type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument '{}' for function {} must be number", std::string(argument_names[i]), getName());
"Argument '{}' for function {} must be a number", std::string(argument_names[i]), getName());
}
}
@ -322,7 +322,7 @@ public:
const auto& fraction_argument = arguments[argument_names.size()];
if (!isNumber(fraction_argument.type))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument 'fraction' for function {} must be number", getName());
"Argument 'fraction' for function {} must be a number", getName());
}
/// Optional precision argument

View File

@ -4,9 +4,10 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnsCommon.h>
#include <Common/ErrorCodes.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -21,55 +22,56 @@ namespace ErrorCodes
namespace
{
/// The regex-based code style check script in CI complains when it sees "ErrorCodes:: ErrorCode" (space added to avoid another match).
/// Because this expression is only used in this file, don't add some suppression mechanism to the already complex style checker, instead
/// work around by creating a namespace alias.
namespace ErrorCodeAlias = ErrorCodes;
/// Throw an exception if the argument is non zero.
class FunctionThrowIf : public IFunction
{
public:
static constexpr auto name = "throwIf";
static FunctionPtr create(ContextPtr)
{
return std::make_shared<FunctionThrowIf>();
}
String getName() const override
{
return name;
}
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionThrowIf>(context); }
explicit FunctionThrowIf(ContextPtr context_) : allow_custom_error_code_argument(context_->getSettingsRef().allow_custom_error_code_in_throwif) {}
String getName() const override { return name; }
bool isVariadic() const override { return true; }
bool isSuitableForShortCircuitArgumentsExecution(const DataTypesWithConstInfo & /*arguments*/) const override { return true; }
size_t getNumberOfArguments() const override
{
return 0;
}
size_t getNumberOfArguments() const override { return 0; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
const size_t number_of_arguments = arguments.size();
if (number_of_arguments < 1 || number_of_arguments > 2)
if (number_of_arguments < 1 || number_of_arguments > (allow_custom_error_code_argument ? 3 : 2))
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Number of arguments for function {} doesn't match: passed {}, should be 1 or 2",
getName(),
toString(number_of_arguments));
"Number of arguments for function {} doesn't match: passed {}, should be {}",
getName(), toString(number_of_arguments), allow_custom_error_code_argument ? "1 or 2 or 3" : "1 or 2");
if (!isNativeNumber(arguments[0]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Argument for function {} must be number",
getName());
"First argument of function {} must be a number (passed: {})", getName(), arguments[0]->getName());
if (number_of_arguments > 1 && !isString(arguments[1]))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Illegal type {} of argument of function {}",
arguments[1]->getName(),
getName());
"Second argument of function {} must be a string (passed: {})", getName(), arguments[1]->getName());
if (allow_custom_error_code_argument && number_of_arguments > 2)
{
WhichDataType which(arguments[2]);
if (!(which.isInt8() || which.isInt16() || which.isInt32()))
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"Third argument of function {} must be Int8, Int16 or Int32 (passed: {})", getName(), arguments[2]->getName());
}
return std::make_shared<DataTypeUInt8>();
}
bool useDefaultImplementationForConstants() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1, 2}; }
/** Prevent constant folding for FunctionThrowIf because for short circuit evaluation
* it is unsafe to evaluate this function during DAG analysis.
@ -86,36 +88,44 @@ public:
{
const auto * message_column = checkAndGetColumnConst<ColumnString>(arguments[1].column.get());
if (!message_column)
throw Exception(ErrorCodes::ILLEGAL_COLUMN,
"Second argument for function {} must be constant String",
getName());
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Second argument for function {} must be constant String", getName());
custom_message = message_column->getValue<String>();
}
std::optional<ErrorCodeAlias::ErrorCode> custom_error_code;
if (allow_custom_error_code_argument && arguments.size() == 3)
{
if (!isColumnConst(*(arguments[2].column)))
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Third argument for function {} must be constant number", getName());
custom_error_code = arguments[2].column->getInt(0);
}
auto first_argument_column = arguments.front().column;
const auto * in = first_argument_column.get();
ColumnPtr res;
if (!((res = execute<UInt8>(in, custom_message))
|| (res = execute<UInt16>(in, custom_message))
|| (res = execute<UInt32>(in, custom_message))
|| (res = execute<UInt64>(in, custom_message))
|| (res = execute<Int8>(in, custom_message))
|| (res = execute<Int16>(in, custom_message))
|| (res = execute<Int32>(in, custom_message))
|| (res = execute<Int64>(in, custom_message))
|| (res = execute<Float32>(in, custom_message))
|| (res = execute<Float64>(in, custom_message))))
if (!((res = execute<UInt8>(in, custom_message, custom_error_code))
|| (res = execute<UInt16>(in, custom_message, custom_error_code))
|| (res = execute<UInt32>(in, custom_message, custom_error_code))
|| (res = execute<UInt64>(in, custom_message, custom_error_code))
|| (res = execute<Int8>(in, custom_message, custom_error_code))
|| (res = execute<Int16>(in, custom_message, custom_error_code))
|| (res = execute<Int32>(in, custom_message, custom_error_code))
|| (res = execute<Int64>(in, custom_message, custom_error_code))
|| (res = execute<Float32>(in, custom_message, custom_error_code))
|| (res = execute<Float64>(in, custom_message, custom_error_code))))
{
throw Exception{"Illegal column " + in->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
}
return res;
}
private:
template <typename T>
ColumnPtr execute(const IColumn * in_untyped, const std::optional<String> & message) const
ColumnPtr execute(const IColumn * in_untyped, const std::optional<String> & message, const std::optional<ErrorCodeAlias::ErrorCode> & error_code) const
{
const auto * in = checkAndGetColumn<ColumnVector<T>>(in_untyped);
@ -127,8 +137,9 @@ public:
const auto & in_data = in->getData();
if (!memoryIsZero(in_data.data(), 0, in_data.size() * sizeof(in_data[0])))
{
throw Exception(ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO,
message.value_or("Value passed to '" + getName() + "' function is non zero"));
throw Exception(
error_code.value_or(ErrorCodes::FUNCTION_THROW_IF_VALUE_IS_NON_ZERO),
message.value_or("Value passed to '" + getName() + "' function is non-zero"));
}
size_t result_size = in_untyped->size();
@ -139,6 +150,8 @@ public:
return nullptr;
}
bool allow_custom_error_code_argument;
};
}

View File

@ -70,6 +70,7 @@ namespace ErrorCodes
extern const int THERE_IS_NO_COLUMN;
extern const int UNKNOWN_EXCEPTION;
extern const int INCORRECT_NUMBER_OF_COLUMNS;
extern const int INCORRECT_DATA;
}
/// Inserts numeric data right into internal column data to reduce an overhead
@ -266,6 +267,9 @@ static ColumnWithTypeAndName readColumnWithDecimalData(std::shared_ptr<arrow::Ch
/// Creates a null bytemap from arrow's null bitmap
static ColumnPtr readByteMapFromArrowColumn(std::shared_ptr<arrow::ChunkedArray> & arrow_column)
{
if (!arrow_column->null_count())
return ColumnUInt8::create(arrow_column->length(), 0);
auto nullmap_column = ColumnUInt8::create();
PaddedPODArray<UInt8> & bytemap_data = assert_cast<ColumnVector<UInt8> &>(*nullmap_column).getData();
bytemap_data.reserve(arrow_column->length());
@ -298,14 +302,121 @@ static ColumnPtr readOffsetsFromArrowListColumn(std::shared_ptr<arrow::ChunkedAr
return offsets_column;
}
static ColumnPtr readColumnWithIndexesData(std::shared_ptr<arrow::ChunkedArray> & arrow_column)
/*
* Arrow Dictionary and ClickHouse LowCardinality types are a bit different.
* Dictionary(Nullable(X)) in ArrowColumn format is composed of a nullmap, dictionary and an index.
* It doesn't have the concept of null or default values.
* An empty string is just a regular value appended at any position of the dictionary.
* Null values have an index of 0, but it should be ignored since the nullmap will return null.
* In ClickHouse LowCardinality, it's different. The dictionary contains null (if dictionary type is Nullable)
* and default values at the beginning. [default, ...] when default values have index of 0 or [null, default, ...]
* when null values have an index of 0 and default values have an index of 1.
* So, we should remap indexes while converting Arrow Dictionary to ClickHouse LowCardinality
* */
template <typename NumericType, typename VectorType = ColumnVector<NumericType>>
static ColumnWithTypeAndName readColumnWithIndexesDataImpl(std::shared_ptr<arrow::ChunkedArray> & arrow_column, const String & column_name, Int64 default_value_index, NumericType dict_size, bool is_nullable)
{
auto internal_type = std::make_shared<DataTypeNumber<NumericType>>();
auto internal_column = internal_type->createColumn();
auto & column_data = static_cast<VectorType &>(*internal_column).getData();
column_data.reserve(arrow_column->length());
NumericType shift = is_nullable ? 2 : 1;
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
{
std::shared_ptr<arrow::Array> chunk = arrow_column->chunk(chunk_i);
if (chunk->length() == 0)
continue;
/// buffers[0] is a null bitmap and buffers[1] are actual values
std::shared_ptr<arrow::Buffer> buffer = chunk->data()->buffers[1];
const auto * data = reinterpret_cast<const NumericType *>(buffer->data());
/// Check that indexes are correct (protection against corrupted files)
for (int64_t i = 0; i != chunk->length(); ++i)
{
if (data[i] < 0 || data[i] >= dict_size)
throw Exception(ErrorCodes::INCORRECT_DATA, "Index {} in Dictionary column is out of bounds, dictionary size is {}", Int64(data[i]), UInt64(dict_size));
}
/// If dictionary type is not nullable and arrow dictionary contains default type
/// at 0 index, we don't need to remap anything (it's the case when this data
/// was generated by ClickHouse)
if (!is_nullable && default_value_index == 0)
{
column_data.insert_assume_reserved(data, data + chunk->length());
}
/// If dictionary don't contain default value, we should move all indexes
/// to the right one or two (if dictionary is Nullable) positions
/// Example:
/// Dictionary:
/// dict: ["one", "two"]
/// indexes: [0, 1, 0]
/// LowCardinality:
/// dict: ["", "one", "two"]
/// indexes: [1, 2, 1]
/// LowCardinality(Nullable):
/// dict: [null, "", "one", "two"]
/// indexes: [2, 3, 2]
else if (default_value_index == -1)
{
for (int64_t i = 0; i != chunk->length(); ++i)
{
if (chunk->IsNull(i))
column_data.push_back(0);
else
column_data.push_back(data[i] + shift);
}
}
/// If dictionary contains default value, we change all indexes of it to
/// 0 or 1 (if dictionary type is Nullable) and move all indexes
/// that are less then default value index to the right one or two
/// (if dictionary is Nullable) position and all indexes that are
/// greater then default value index zero or one (if dictionary is Nullable)
/// positions.
/// Example:
/// Dictionary:
/// dict: ["one", "two", "", "three"]
/// indexes: [0, 1, 2, 3, 0]
/// LowCardinality :
/// dict: ["", "one", "two", "three"]
/// indexes: [1, 2, 0, 3, 1]
/// LowCardinality(Nullable):
/// dict: [null, "", "one", "two", "three"]
/// indexes: [2, 3, 1, 4, 2]
else
{
NumericType new_default_index = is_nullable ? 1 : 0;
NumericType default_index = NumericType(default_value_index);
for (int64_t i = 0; i != chunk->length(); ++i)
{
if (chunk->IsNull(i))
column_data.push_back(0);
else
{
NumericType value = data[i];
if (value == default_index)
value = new_default_index;
else if (value < default_index)
value += shift;
else
value += shift - 1;
column_data.push_back(value);
}
}
}
}
return {std::move(internal_column), std::move(internal_type), column_name};
}
static ColumnPtr readColumnWithIndexesData(std::shared_ptr<arrow::ChunkedArray> & arrow_column, Int64 default_value_index, UInt64 dict_size, bool is_nullable)
{
switch (arrow_column->type()->id())
{
# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \
case ARROW_NUMERIC_TYPE: \
{ \
return readColumnWithNumericData<CPP_NUMERIC_TYPE>(arrow_column, "").column; \
return readColumnWithIndexesDataImpl<CPP_NUMERIC_TYPE>(arrow_column, "", default_value_index, dict_size, is_nullable).column; \
}
FOR_ARROW_INDEXES_TYPES(DISPATCH)
# undef DISPATCH
@ -327,85 +438,25 @@ static std::shared_ptr<arrow::ChunkedArray> getNestedArrowColumn(std::shared_ptr
return std::make_shared<arrow::ChunkedArray>(array_vector);
}
static ColumnWithTypeAndName createLCColumnFromArrowDictionaryValues(
const std::shared_ptr<ColumnWithTypeAndName> & dict_values,
const ColumnPtr & indexes_column,
const String & column_name
)
{
auto lc_type = std::make_shared<DataTypeLowCardinality>(dict_values->type);
auto lc_column = lc_type->createColumn();
for (auto i = 0u; i < indexes_column->size(); i++)
{
Field f;
dict_values->column->get(indexes_column->getUInt(i), f);
lc_column->insert(f);
}
return {std::move(lc_column), std::move(lc_type), column_name};
}
/*
* Dictionary(Nullable(X)) in ArrowColumn format is composed of a nullmap, dictionary and an index.
* It doesn't have the concept of null or default values.
* An empty string is just a regular value appended at any position of the dictionary.
* Null values have an index of 0, but it should be ignored since the nullmap will return null.
* In ClickHouse LowCardinality, it's different. The dictionary contains null and default values at the beginning.
* [null, default, ...]. Therefore, null values have an index of 0 and default values have an index of 1.
* No nullmap is used.
* */
static ColumnWithTypeAndName createLCOfNullableColumnFromArrowDictionaryValues(
const std::shared_ptr<ColumnWithTypeAndName> & dict_values,
const ColumnPtr & indexes_column,
const ColumnPtr & nullmap_column,
const String & column_name
)
{
/*
* ArrowColumn format handles nulls by maintaining a nullmap column, there is no nullable type.
* Therefore, dict_values->type is the actual data type/ non-nullable. It needs to be transformed into nullable
* so LC column is created from nullable type and a null value at the beginning of the collection
* is automatically added.
* */
auto lc_type = std::make_shared<DataTypeLowCardinality>(makeNullable(dict_values->type));
auto lc_column = lc_type->createColumn();
for (auto i = 0u; i < indexes_column->size(); i++)
{
if (nullmap_column && nullmap_column->getBool(i))
{
lc_column->insertDefault();
}
else
{
Field f;
dict_values->column->get(indexes_column->getUInt(i), f);
lc_column->insert(f);
}
}
return {std::move(lc_column), std::move(lc_type), column_name};
}
static ColumnWithTypeAndName readColumnFromArrowColumn(
std::shared_ptr<arrow::ChunkedArray> & arrow_column,
const std::string & column_name,
const std::string & format_name,
bool is_nullable,
std::unordered_map<String, std::shared_ptr<ColumnWithTypeAndName>> & dictionary_values,
bool read_ints_as_dates,
std::unordered_map<String, ArrowColumnToCHColumn::DictionaryInfo> & dictionary_infos,
bool allow_null_type,
bool skip_columns_with_unsupported_types,
bool & skipped)
bool & skipped,
DataTypePtr type_hint = nullptr)
{
if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST
if (!is_nullable && (arrow_column->null_count() || (type_hint && type_hint->isNullable())) && arrow_column->type()->id() != arrow::Type::LIST
&& arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT &&
arrow_column->type()->id() != arrow::Type::DICTIONARY)
{
auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
DataTypePtr nested_type_hint;
if (type_hint)
nested_type_hint = removeNullable(type_hint);
auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
if (skipped)
return {};
auto nullmap_column = readByteMapFromArrowColumn(arrow_column);
@ -435,14 +486,14 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
case arrow::Type::UINT16:
{
auto column = readColumnWithNumericData<UInt16>(arrow_column, column_name);
if (read_ints_as_dates)
if (type_hint && (isDateOrDate32(type_hint) || isDateTime(type_hint) || isDateTime64(type_hint)))
column.type = std::make_shared<DataTypeDate>();
return column;
}
case arrow::Type::UINT32:
{
auto column = readColumnWithNumericData<UInt32>(arrow_column, column_name);
if (read_ints_as_dates)
if (type_hint && (isDateOrDate32(type_hint) || isDateTime(type_hint) || isDateTime64(type_hint)))
column.type = std::make_shared<DataTypeDateTime>();
return column;
}
@ -454,8 +505,15 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
return readColumnWithDecimalData<arrow::Decimal256Array>(arrow_column, column_name);
case arrow::Type::MAP:
{
DataTypePtr nested_type_hint;
if (type_hint)
{
const auto * map_type_hint = typeid_cast<const DataTypeMap *>(type_hint.get());
if (map_type_hint)
nested_type_hint = assert_cast<const DataTypeArray *>(map_type_hint->getNestedType().get())->getNestedType();
}
auto arrow_nested_column = getNestedArrowColumn(arrow_column);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
if (skipped)
return {};
@ -469,8 +527,15 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
}
case arrow::Type::LIST:
{
DataTypePtr nested_type_hint;
if (type_hint)
{
const auto * array_type_hint = typeid_cast<const DataTypeArray *>(type_hint.get());
if (array_type_hint)
nested_type_hint = array_type_hint->getNestedType();
}
auto arrow_nested_column = getNestedArrowColumn(arrow_column);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
if (skipped)
return {};
auto offsets_column = readOffsetsFromArrowListColumn(arrow_column);
@ -493,11 +558,25 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
Columns tuple_elements;
DataTypes tuple_types;
std::vector<String> tuple_names;
const auto * tuple_type_hint = type_hint ? typeid_cast<const DataTypeTuple *>(type_hint.get()) : nullptr;
for (int i = 0; i != arrow_struct_type->num_fields(); ++i)
{
auto field_name = arrow_struct_type->field(i)->name();
DataTypePtr nested_type_hint;
if (tuple_type_hint)
{
if (tuple_type_hint->haveExplicitNames())
{
auto pos = tuple_type_hint->tryGetPositionByName(field_name);
if (pos)
nested_type_hint = tuple_type_hint->getElement(*pos);
}
else if (size_t(i) < tuple_type_hint->getElements().size())
nested_type_hint = tuple_type_hint->getElement(i);
}
auto nested_arrow_column = std::make_shared<arrow::ChunkedArray>(nested_arrow_columns[i]);
auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
auto element = readColumnFromArrowColumn(nested_arrow_column, field_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped, nested_type_hint);
if (skipped)
return {};
tuple_elements.emplace_back(std::move(element.column));
@ -511,9 +590,11 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
}
case arrow::Type::DICTIONARY:
{
auto & dict_values = dictionary_values[column_name];
auto & dict_info = dictionary_infos[column_name];
const auto is_lc_nullable = arrow_column->null_count() > 0 || (type_hint && type_hint->isLowCardinalityNullable());
/// Load dictionary values only once and reuse it.
if (!dict_values)
if (!dict_info.values)
{
arrow::ArrayVector dict_array;
for (size_t chunk_i = 0, num_chunks = static_cast<size_t>(arrow_column->num_chunks()); chunk_i < num_chunks; ++chunk_i)
@ -522,8 +603,22 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
dict_array.emplace_back(dict_chunk.dictionary());
}
auto arrow_dict_column = std::make_shared<arrow::ChunkedArray>(dict_array);
auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values, read_ints_as_dates, allow_null_type, skip_columns_with_unsupported_types, skipped);
dict_values = std::make_shared<ColumnWithTypeAndName>(std::move(dict_column));
auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_infos, allow_null_type, skip_columns_with_unsupported_types, skipped);
for (size_t i = 0; i != dict_column.column->size(); ++i)
{
if (dict_column.column->isDefaultAt(i))
{
dict_info.default_value_index = i;
break;
}
}
auto lc_type = std::make_shared<DataTypeLowCardinality>(is_lc_nullable ? makeNullable(dict_column.type) : dict_column.type);
auto tmp_lc_column = lc_type->createColumn();
auto tmp_dict_column = IColumn::mutate(assert_cast<ColumnLowCardinality *>(tmp_lc_column.get())->getDictionaryPtr());
dynamic_cast<IColumnUnique *>(tmp_dict_column.get())->uniqueInsertRangeFrom(*dict_column.column, 0, dict_column.column->size());
dict_column.column = std::move(tmp_dict_column);
dict_info.values = std::make_shared<ColumnWithTypeAndName>(std::move(dict_column));
dict_info.dictionary_size = arrow_dict_column->length();
}
arrow::ArrayVector indexes_array;
@ -534,20 +629,10 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
}
auto arrow_indexes_column = std::make_shared<arrow::ChunkedArray>(indexes_array);
auto indexes_column = readColumnWithIndexesData(arrow_indexes_column);
const auto contains_null = arrow_column->null_count() > 0;
if (contains_null)
{
auto nullmap_column = readByteMapFromArrowColumn(arrow_column);
return createLCOfNullableColumnFromArrowDictionaryValues(dict_values, indexes_column, nullmap_column, column_name);
}
else
{
return createLCColumnFromArrowDictionaryValues(dict_values, indexes_column, column_name);
}
auto indexes_column = readColumnWithIndexesData(arrow_indexes_column, dict_info.default_value_index, dict_info.dictionary_size, is_lc_nullable);
auto lc_column = ColumnLowCardinality::create(dict_info.values->column, indexes_column);
auto lc_type = std::make_shared<DataTypeLowCardinality>(is_lc_nullable ? makeNullable(dict_info.values->type) : dict_info.values->type);
return {std::move(lc_column), std::move(lc_type), column_name};
}
# define DISPATCH(ARROW_NUMERIC_TYPE, CPP_NUMERIC_TYPE) \
case ARROW_NUMERIC_TYPE: \
@ -623,13 +708,13 @@ Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(
arrow::ArrayVector array_vector = {arrow_array};
auto arrow_column = std::make_shared<arrow::ChunkedArray>(array_vector);
std::unordered_map<std::string, std::shared_ptr<ColumnWithTypeAndName>> dict_values;
std::unordered_map<std::string, DictionaryInfo> dict_infos;
bool skipped = false;
bool allow_null_type = false;
if (hint_header && hint_header->has(field->name()) && hint_header->getByName(field->name()).type->isNullable())
allow_null_type = true;
ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(
arrow_column, field->name(), format_name, false, dict_values, false, allow_null_type, skip_columns_with_unsupported_types, skipped);
arrow_column, field->name(), format_name, false, dict_infos, allow_null_type, skip_columns_with_unsupported_types, skipped);
if (!skipped)
sample_columns.emplace_back(std::move(sample_column));
}
@ -700,9 +785,17 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
{
if (!nested_tables.contains(search_nested_table_name))
{
NamesAndTypesList nested_columns;
for (const auto & name_and_type : header.getNamesAndTypesList())
{
if (name_and_type.name.starts_with(nested_table_name + "."))
nested_columns.push_back(name_and_type);
}
auto nested_table_type = Nested::collect(nested_columns).front().type;
std::shared_ptr<arrow::ChunkedArray> arrow_column = name_to_column_ptr[search_nested_table_name];
ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(
arrow_column, nested_table_name, format_name, false, dictionary_values, true, true, false, skipped)};
arrow_column, nested_table_name, format_name, false, dictionary_infos, true, false, skipped, nested_table_type)};
BlockPtr block_ptr = std::make_shared<Block>(cols);
auto column_extractor = std::make_shared<NestedColumnExtractHelper>(*block_ptr, case_insensitive_matching);
nested_tables[search_nested_table_name] = {block_ptr, column_extractor};
@ -735,7 +828,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
{
auto arrow_column = name_to_column_ptr[search_column_name];
column = readColumnFromArrowColumn(
arrow_column, header_column.name, format_name, false, dictionary_values, true, true, false, skipped);
arrow_column, header_column.name, format_name, false, dictionary_infos, true, false, skipped, header_column.type);
}
try

View File

@ -44,6 +44,14 @@ public:
const Block * hint_header = nullptr,
bool ignore_case = false);
struct DictionaryInfo
{
std::shared_ptr<ColumnWithTypeAndName> values;
Int64 default_value_index = -1;
UInt64 dictionary_size;
};
private:
const Block & header;
const std::string format_name;
@ -55,7 +63,7 @@ private:
/// Map {column name : dictionary column}.
/// To avoid converting dictionary from Arrow Dictionary
/// to LowCardinality every chunk we save it and reuse.
std::unordered_map<std::string, std::shared_ptr<ColumnWithTypeAndName>> dictionary_values;
std::unordered_map<std::string, DictionaryInfo> dictionary_infos;
};
}

View File

@ -235,27 +235,30 @@ namespace DB
}
template<typename T>
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end)
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end, bool shift)
{
const PaddedPODArray<T> & data = assert_cast<const ColumnVector<T> *>(column.get())->getData();
PaddedPODArray<Int64> result;
result.reserve(end - start);
std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value); });
if (shift)
std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value) - 1; });
else
std::transform(data.begin() + start, data.begin() + end, std::back_inserter(result), [](T value) { return Int64(value); });
return result;
}
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end)
static PaddedPODArray<Int64> extractIndexesImpl(ColumnPtr column, size_t start, size_t end, bool shift)
{
switch (column->getDataType())
{
case TypeIndex::UInt8:
return extractIndexesImpl<UInt8>(column, start, end);
return extractIndexesImpl<UInt8>(column, start, end, shift);
case TypeIndex::UInt16:
return extractIndexesImpl<UInt16>(column, start, end);
return extractIndexesImpl<UInt16>(column, start, end, shift);
case TypeIndex::UInt32:
return extractIndexesImpl<UInt32>(column, start, end);
return extractIndexesImpl<UInt32>(column, start, end, shift);
case TypeIndex::UInt64:
return extractIndexesImpl<UInt64>(column, start, end);
return extractIndexesImpl<UInt64>(column, start, end, shift);
default:
throw Exception(fmt::format("Indexes column must be ColumnUInt, got {}.", column->getName()),
ErrorCodes::LOGICAL_ERROR);
@ -267,7 +270,7 @@ namespace DB
const String & column_name,
ColumnPtr & column,
const std::shared_ptr<const IDataType> & column_type,
const PaddedPODArray<UInt8> * null_bytemap,
const PaddedPODArray<UInt8> *,
arrow::ArrayBuilder * array_builder,
String format_name,
size_t start,
@ -278,6 +281,7 @@ namespace DB
const auto * column_lc = assert_cast<const ColumnLowCardinality *>(column.get());
arrow::DictionaryBuilder<ValueType> * builder = assert_cast<arrow::DictionaryBuilder<ValueType> *>(array_builder);
auto & dict_values = dictionary_values[column_name];
bool is_nullable = column_type->isLowCardinalityNullable();
/// Convert dictionary from LowCardinality to Arrow dictionary only once and then reuse it.
if (!dict_values)
@ -288,9 +292,9 @@ namespace DB
arrow::Status status = MakeBuilder(pool, value_type, &values_builder);
checkStatus(status, column->getName(), format_name);
auto dict_column = column_lc->getDictionary().getNestedColumn();
const auto & dict_type = assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, 0, dict_column->size(), output_string_as_string, dictionary_values);
auto dict_column = column_lc->getDictionary().getNestedNotNullableColumn();
const auto & dict_type = removeNullable(assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType());
fillArrowArray(column_name, dict_column, dict_type, nullptr, values_builder.get(), format_name, is_nullable, dict_column->size(), output_string_as_string, dictionary_values);
status = values_builder->Finish(&dict_values);
checkStatus(status, column->getName(), format_name);
}
@ -300,15 +304,14 @@ namespace DB
/// AppendIndices in DictionaryBuilder works only with int64_t data, so we cannot use
/// fillArrowArray here and should copy all indexes to int64_t container.
auto indexes = extractIndexesImpl(column_lc->getIndexesPtr(), start, end);
auto indexes = extractIndexesImpl(column_lc->getIndexesPtr(), start, end, is_nullable);
const uint8_t * arrow_null_bytemap_raw_ptr = nullptr;
PaddedPODArray<uint8_t> arrow_null_bytemap;
if (null_bytemap)
if (column_type->isLowCardinalityNullable())
{
/// Invert values since Arrow interprets 1 as a non-null value, while CH as a null
arrow_null_bytemap.reserve(end - start);
for (size_t i = start; i < end; ++i)
arrow_null_bytemap.emplace_back(!(*null_bytemap)[i]);
arrow_null_bytemap.emplace_back(!column_lc->isNullAt(i));
arrow_null_bytemap_raw_ptr = arrow_null_bytemap.data();
}
@ -680,7 +683,7 @@ namespace DB
{
auto nested_type = assert_cast<const DataTypeLowCardinality *>(column_type.get())->getDictionaryType();
const auto * lc_column = assert_cast<const ColumnLowCardinality *>(column.get());
const auto & nested_column = lc_column->getDictionaryPtr();
const auto & nested_column = lc_column->getDictionary().getNestedColumn();
const auto & indexes_column = lc_column->getIndexesPtr();
return arrow::dictionary(
getArrowTypeForLowCardinalityIndexes(indexes_column),

View File

@ -211,7 +211,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise
"Assing to assignees of the original PR: %s",
", ".join(user.login for user in self.pr.assignees),
)
self.cherrypick_pr.add_to_assignees(self.pr.assignees)
self.cherrypick_pr.add_to_assignees(*self.pr.assignees)
logging.info("Assign to the author of the original PR: %s", self.pr.user.login)
self.cherrypick_pr.add_to_assignees(self.pr.user)
@ -249,7 +249,7 @@ Merge it only if you intend to backport changes to the target branch, otherwise
"Assing to assignees of the original PR: %s",
", ".join(user.login for user in self.pr.assignees),
)
self.cherrypick_pr.add_to_assignees(self.pr.assignees)
self.cherrypick_pr.add_to_assignees(*self.pr.assignees)
logging.info("Assign to the author of the original PR: %s", self.pr.user.login)
self.backport_pr.add_to_assignees(self.pr.user)

View File

@ -98,7 +98,7 @@ class Packages:
class S3:
template = (
f"{S3_DOWNLOAD}"
f"{S3_DOWNLOAD}/"
# "clickhouse-builds/"
f"{S3_BUILDS_BUCKET}/"
# "33333/" or "21.11/" from --release, if pull request is omitted

View File

@ -0,0 +1,10 @@
<test>
<create_query>CREATE TABLE test (uint32 UInt32, n_uint32 Nullable(UInt32), lc LowCardinality(String)) ENGINE=File(Arrow) SETTINGS output_format_arrow_low_cardinality_as_dictionary=1</create_query>
<fill_query>insert into test select number, number, toString(number % 10000) from numbers(10000000)</fill_query>
<query>SELECT uint32 from test format Null</query>
<query>SELECT n_uint32 from test format Null</query>
<query>SELECT lc from test format Null</query>
<drop_query>DROP TABLE IF EXISTS test</drop_query>
</test>

View File

@ -1,3 +1,13 @@
1
1
1000000
1
1
1
1
1
1
1
1
1
1
1000

View File

@ -4,9 +4,43 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
default_exception_message="Value passed to 'throwIf' function is non zero"
custom_exception_message="Number equals 1000000"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000) FROM system.numbers" 2>&1 | grep -cF "$default_exception_message"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000000, '$custom_exception_message') FROM system.numbers" 2>&1 | grep -v '^(query: ' | grep -cF "$custom_exception_message"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000000) AS x FROM numbers(1000000))" 2>&1
default_exception_message="Value passed to 'throwIf' function is non-zero"
custom_exception_message="Number equals 1000"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000) FROM system.numbers" 2>&1 \
| grep -cF "$default_exception_message"
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000, '$custom_exception_message') FROM system.numbers" 2>&1 \
| grep -v '^(query: ' | grep -cF "$custom_exception_message"
# Custom error code arguments are not enabled via configuration.
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000, '$custom_exception_message', 1) FROM system.numbers" 2>&1 \
| grep -v '^(query: ' | grep -c "Number of arguments for function throwIf doesn't match: passed 3, should be 1 or 2"
# Custom error code argument enabled but using the wrong type.
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000, '$custom_exception_message', 1) FROM system.numbers SETTINGS allow_custom_error_code_in_throwif=true" 2>&1 \
| grep -v '^(query: ' | grep -c "Third argument of function throwIf must be Int8, Int16 or Int32 (passed: UInt8)"
# Normal error code + some weird ones.
# Internal error codes use the upper half of 32-bit int.
custom_error_codes=(
"42"
"0" # OK
"101" # UNEXPECTED_PACKET_FROM_CLIENT (interpreted by client)
"102" # UNEXPECTED_PACKET_FROM_SERVER (interpreted by client)
"1001" # STD_EXCEPTION
"1002" # UNKNOWN_EXCEPTION
"999999" # Unused error code.
"-1") # Also unused. Weird but we should allow throwing negative errors.
for ec in "${custom_error_codes[@]}"
do
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT throwIf(number = 1000, '$custom_exception_message', toInt32($ec)) FROM system.numbers SETTINGS allow_custom_error_code_in_throwif=true" 2>&1 \
| grep -v '^(query: ' | grep -c "Code: $ec.*$custom_exception_message"
done
${CLICKHOUSE_CLIENT} --server_logs_file /dev/null --query="SELECT sum(x = 0) FROM (SELECT throwIf(number = 1000) AS x FROM numbers(1000))" 2>&1

View File

@ -10,6 +10,6 @@ $CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS check;"
$CLICKHOUSE_CLIENT --query="CREATE TABLE check (x UInt64, y UInt64 DEFAULT throwIf(x > 1500000)) ENGINE = Memory;"
seq 1 2000000 | $CLICKHOUSE_CLIENT --query="INSERT INTO check(x) FORMAT TSV" 2>&1 | grep -q "Value passed to 'throwIf' function is non zero." && echo 'OK' || echo 'FAIL' ||:
seq 1 2000000 | $CLICKHOUSE_CLIENT --query="INSERT INTO check(x) FORMAT TSV" 2>&1 | grep -q "Value passed to 'throwIf' function is non-zero." && echo 'OK' || echo 'FAIL' ||:
$CLICKHOUSE_CLIENT --query="DROP TABLE check;"

View File

@ -0,0 +1,32 @@
dict LowCardinality(Nullable(String))
one
two
three
one
two
dict LowCardinality(Nullable(String))
one
two
three
one
three
dict LowCardinality(Nullable(String))
one
two
three
one
two
three
lc LowCardinality(Nullable(String))
OK
dict LowCardinality(Nullable(String))
one
two
three
one
\N
three

View File

@ -0,0 +1,29 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
mkdir -p $USER_FILES_PATH/test_02383
cp $CURDIR/data_arrow/dictionary*.arrow $USER_FILES_PATH/test_02383/
cp $CURDIR/data_arrow/corrupted.arrow $USER_FILES_PATH/test_02383/
cp $CURDIR/data_arrow/dict_with_nulls.arrow $USER_FILES_PATH/test_02383/
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary1.arrow')"
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary1.arrow')"
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary2.arrow')"
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary2.arrow')"
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dictionary3.arrow')"
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dictionary3.arrow')"
$CLICKHOUSE_CLIENT -q "desc file('test_02383/corrupted.arrow')"
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/corrupted.arrow')" 2>&1 | grep -F -q "INCORRECT_DATA" && echo OK || echo FAIL
$CLICKHOUSE_CLIENT -q "desc file('test_02383/dict_with_nulls.arrow')"
$CLICKHOUSE_CLIENT -q "select * from file('test_02383/dict_with_nulls.arrow')"
rm -rf $USER_FILES_PATH/test_02383

View File

@ -0,0 +1,4 @@
lc LowCardinality(Nullable(String))
abc
lc LowCardinality(Nullable(String))
abc

View File

@ -0,0 +1,8 @@
-- Tags: no-fasttest
insert into function file(02384_data.arrow) select toLowCardinality(toNullable('abc')) as lc settings output_format_arrow_low_cardinality_as_dictionary=1, output_format_arrow_string_as_string=0, engine_file_truncate_on_insert=1;
desc file(02384_data.arrow);
select * from file(02384_data.arrow);
insert into function file(02384_data.arrow) select toLowCardinality(toNullable('abc')) as lc settings output_format_arrow_low_cardinality_as_dictionary=1, output_format_arrow_string_as_string=1, engine_file_truncate_on_insert=1;
desc file(02384_data.arrow);
select * from file(02384_data.arrow);

Binary file not shown.

Binary file not shown.

Binary file not shown.

Binary file not shown.