mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-26 09:32:01 +00:00
Merge branch 'master' of github.com:yandex/ClickHouse
This commit is contained in:
commit
c269dd2cef
@ -170,6 +170,46 @@ template <> struct NativeType<Decimal32> { using Type = Int32; };
|
|||||||
template <> struct NativeType<Decimal64> { using Type = Int64; };
|
template <> struct NativeType<Decimal64> { using Type = Int64; };
|
||||||
template <> struct NativeType<Decimal128> { using Type = Int128; };
|
template <> struct NativeType<Decimal128> { using Type = Int128; };
|
||||||
|
|
||||||
|
inline const char * getTypeName(TypeIndex idx)
|
||||||
|
{
|
||||||
|
switch (idx)
|
||||||
|
{
|
||||||
|
case TypeIndex::Nothing: return "Nothing";
|
||||||
|
case TypeIndex::UInt8: return TypeName<UInt8>::get();
|
||||||
|
case TypeIndex::UInt16: return TypeName<UInt16>::get();
|
||||||
|
case TypeIndex::UInt32: return TypeName<UInt32>::get();
|
||||||
|
case TypeIndex::UInt64: return TypeName<UInt64>::get();
|
||||||
|
case TypeIndex::UInt128: return "UInt128";
|
||||||
|
case TypeIndex::Int8: return TypeName<Int8>::get();
|
||||||
|
case TypeIndex::Int16: return TypeName<Int16>::get();
|
||||||
|
case TypeIndex::Int32: return TypeName<Int32>::get();
|
||||||
|
case TypeIndex::Int64: return TypeName<Int64>::get();
|
||||||
|
case TypeIndex::Int128: return TypeName<Int128>::get();
|
||||||
|
case TypeIndex::Float32: return TypeName<Float32>::get();
|
||||||
|
case TypeIndex::Float64: return TypeName<Float64>::get();
|
||||||
|
case TypeIndex::Date: return "Date";
|
||||||
|
case TypeIndex::DateTime: return "DateTime";
|
||||||
|
case TypeIndex::String: return TypeName<String>::get();
|
||||||
|
case TypeIndex::FixedString: return "FixedString";
|
||||||
|
case TypeIndex::Enum8: return "Enum8";
|
||||||
|
case TypeIndex::Enum16: return "Enum16";
|
||||||
|
case TypeIndex::Decimal32: return TypeName<Decimal32>::get();
|
||||||
|
case TypeIndex::Decimal64: return TypeName<Decimal64>::get();
|
||||||
|
case TypeIndex::Decimal128: return TypeName<Decimal128>::get();
|
||||||
|
case TypeIndex::UUID: return "UUID";
|
||||||
|
case TypeIndex::Array: return "Array";
|
||||||
|
case TypeIndex::Tuple: return "Tuple";
|
||||||
|
case TypeIndex::Set: return "Set";
|
||||||
|
case TypeIndex::Interval: return "Interval";
|
||||||
|
case TypeIndex::Nullable: return "Nullable";
|
||||||
|
case TypeIndex::Function: return "Function";
|
||||||
|
case TypeIndex::AggregateFunction: return "AggregateFunction";
|
||||||
|
case TypeIndex::LowCardinality: return "LowCardinality";
|
||||||
|
}
|
||||||
|
|
||||||
|
__builtin_unreachable();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/// Specialization of `std::hash` for the Decimal<T> types.
|
/// Specialization of `std::hash` for the Decimal<T> types.
|
||||||
|
@ -660,10 +660,10 @@ void FlatDictionary::setAttributeValue(Attribute & attribute, const Key id, cons
|
|||||||
break;
|
break;
|
||||||
|
|
||||||
case AttributeUnderlyingType::Decimal32:
|
case AttributeUnderlyingType::Decimal32:
|
||||||
setAttributeValueImpl<Decimal32>(attribute, id, value.get<Decimal128>());
|
setAttributeValueImpl<Decimal32>(attribute, id, value.get<Decimal32>());
|
||||||
break;
|
break;
|
||||||
case AttributeUnderlyingType::Decimal64:
|
case AttributeUnderlyingType::Decimal64:
|
||||||
setAttributeValueImpl<Decimal64>(attribute, id, value.get<Decimal128>());
|
setAttributeValueImpl<Decimal64>(attribute, id, value.get<Decimal64>());
|
||||||
break;
|
break;
|
||||||
case AttributeUnderlyingType::Decimal128:
|
case AttributeUnderlyingType::Decimal128:
|
||||||
setAttributeValueImpl<Decimal128>(attribute, id, value.get<Decimal128>());
|
setAttributeValueImpl<Decimal128>(attribute, id, value.get<Decimal128>());
|
||||||
|
@ -1,6 +1,7 @@
|
|||||||
#pragma once
|
#pragma once
|
||||||
|
|
||||||
#include <DataTypes/DataTypesNumber.h>
|
#include <DataTypes/DataTypesNumber.h>
|
||||||
|
#include <DataTypes/DataTypesDecimal.h>
|
||||||
#include <DataTypes/DataTypeArray.h>
|
#include <DataTypes/DataTypeArray.h>
|
||||||
#include <DataTypes/DataTypeString.h>
|
#include <DataTypes/DataTypeString.h>
|
||||||
#include <DataTypes/DataTypeDate.h>
|
#include <DataTypes/DataTypeDate.h>
|
||||||
@ -661,21 +662,77 @@ DECLARE_DICT_GET_TRAITS(UInt32, DataTypeDateTime)
|
|||||||
DECLARE_DICT_GET_TRAITS(UInt128, DataTypeUUID)
|
DECLARE_DICT_GET_TRAITS(UInt128, DataTypeUUID)
|
||||||
#undef DECLARE_DICT_GET_TRAITS
|
#undef DECLARE_DICT_GET_TRAITS
|
||||||
|
|
||||||
|
template <typename T> struct DictGetTraits<DataTypeDecimal<T>>
|
||||||
|
{
|
||||||
|
static constexpr bool is_dec32 = std::is_same_v<T, Decimal32>;
|
||||||
|
static constexpr bool is_dec64 = std::is_same_v<T, Decimal64>;
|
||||||
|
static constexpr bool is_dec128 = std::is_same_v<T, Decimal128>;
|
||||||
|
|
||||||
|
template <typename DictionaryType>
|
||||||
|
static void get(const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids,
|
||||||
|
DecimalPaddedPODArray<T> & out)
|
||||||
|
{
|
||||||
|
if constexpr (is_dec32) dict->getDecimal32(name, ids, out);
|
||||||
|
if constexpr (is_dec64) dict->getDecimal64(name, ids, out);
|
||||||
|
if constexpr (is_dec128) dict->getDecimal128(name, ids, out);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename DictionaryType>
|
||||||
|
static void get(const DictionaryType * dict, const std::string & name, const Columns & key_columns, const DataTypes & key_types,
|
||||||
|
DecimalPaddedPODArray<T> & out)
|
||||||
|
{
|
||||||
|
if constexpr (is_dec32) dict->getDecimal32(name, key_columns, key_types, out);
|
||||||
|
if constexpr (is_dec64) dict->getDecimal64(name, key_columns, key_types, out);
|
||||||
|
if constexpr (is_dec128) dict->getDecimal128(name, key_columns, key_types, out);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename DictionaryType>
|
||||||
|
static void get(const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids,
|
||||||
|
const PaddedPODArray<Int64> & dates, DecimalPaddedPODArray<T> & out)
|
||||||
|
{
|
||||||
|
if constexpr (is_dec32) dict->getDecimal32(name, ids, dates, out);
|
||||||
|
if constexpr (is_dec64) dict->getDecimal64(name, ids, dates, out);
|
||||||
|
if constexpr (is_dec128) dict->getDecimal128(name, ids, dates, out);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename DictionaryType, typename DefaultsType>
|
||||||
|
static void getOrDefault(const DictionaryType * dict, const std::string & name, const PaddedPODArray<UInt64> & ids,
|
||||||
|
const DefaultsType & def, DecimalPaddedPODArray<T> & out)
|
||||||
|
{
|
||||||
|
if constexpr (is_dec32) dict->getDecimal32(name, ids, def, out);
|
||||||
|
if constexpr (is_dec64) dict->getDecimal64(name, ids, def, out);
|
||||||
|
if constexpr (is_dec128) dict->getDecimal128(name, ids, def, out);
|
||||||
|
}
|
||||||
|
|
||||||
|
template <typename DictionaryType, typename DefaultsType>
|
||||||
|
static void getOrDefault(const DictionaryType * dict, const std::string & name, const Columns & key_columns,
|
||||||
|
const DataTypes & key_types, const DefaultsType & def, DecimalPaddedPODArray<T> & out)
|
||||||
|
{
|
||||||
|
if constexpr (is_dec32) dict->getDecimal32(name, key_columns, key_types, def, out);
|
||||||
|
if constexpr (is_dec64) dict->getDecimal64(name, key_columns, key_types, def, out);
|
||||||
|
if constexpr (is_dec128) dict->getDecimal128(name, key_columns, key_types, def, out);
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
|
|
||||||
template <typename DataType, typename Name>
|
template <typename DataType, typename Name>
|
||||||
class FunctionDictGet final : public IFunction
|
class FunctionDictGet final : public IFunction
|
||||||
{
|
{
|
||||||
using Type = typename DataType::FieldType;
|
using Type = typename DataType::FieldType;
|
||||||
|
using ColVec = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static constexpr auto name = Name::name;
|
static constexpr auto name = Name::name;
|
||||||
|
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context, UInt32 dec_scale = 0)
|
||||||
{
|
{
|
||||||
return std::make_shared<FunctionDictGet>(context.getExternalDictionaries());
|
return std::make_shared<FunctionDictGet>(context.getExternalDictionaries(), dec_scale);
|
||||||
}
|
}
|
||||||
|
|
||||||
FunctionDictGet(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
|
FunctionDictGet(const ExternalDictionaries & dictionaries, UInt32 dec_scale = 0)
|
||||||
|
: dictionaries(dictionaries)
|
||||||
|
, decimal_scale(dec_scale)
|
||||||
|
{}
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
|
|
||||||
@ -719,7 +776,10 @@ private:
|
|||||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
}
|
}
|
||||||
|
|
||||||
return std::make_shared<DataType>();
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
return std::make_shared<DataType>(DataType::maxPrecision(), decimal_scale);
|
||||||
|
else
|
||||||
|
return std::make_shared<DataType>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() const override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
@ -771,7 +831,11 @@ private:
|
|||||||
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
|
const auto id_col_untyped = block.getByPosition(arguments[2]).column.get();
|
||||||
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
|
if (const auto id_col = checkAndGetColumn<ColumnUInt64>(id_col_untyped))
|
||||||
{
|
{
|
||||||
auto out = ColumnVector<Type>::create(id_col->size());
|
typename ColVec::MutablePtr out;
|
||||||
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
out = ColVec::create(id_col->size(), decimal_scale);
|
||||||
|
else
|
||||||
|
out = ColVec::create(id_col->size());
|
||||||
const auto & ids = id_col->getData();
|
const auto & ids = id_col->getData();
|
||||||
auto & data = out->getData();
|
auto & data = out->getData();
|
||||||
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
|
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
|
||||||
@ -780,9 +844,21 @@ private:
|
|||||||
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
|
else if (const auto id_col_const = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
|
||||||
{
|
{
|
||||||
const PaddedPODArray<UInt64> ids(1, id_col_const->getValue<UInt64>());
|
const PaddedPODArray<UInt64> ids(1, id_col_const->getValue<UInt64>());
|
||||||
PaddedPODArray<Type> data(1);
|
|
||||||
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col_const->size(), toField(data.front()));
|
{
|
||||||
|
DecimalPaddedPODArray<Type> data(1, decimal_scale);
|
||||||
|
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
|
||||||
|
block.getByPosition(result).column =
|
||||||
|
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
|
||||||
|
id_col_const->size(), toField(data.front(), decimal_scale));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
PaddedPODArray<Type> data(1);
|
||||||
|
DictGetTraits<DataType>::get(dict, attr_name, ids, data);
|
||||||
|
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col_const->size(), toField(data.front()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
|
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
|
||||||
@ -818,7 +894,11 @@ private:
|
|||||||
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col).getColumnsCopy();
|
const auto & key_columns = static_cast<const ColumnTuple &>(*key_col).getColumnsCopy();
|
||||||
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
|
||||||
|
|
||||||
auto out = ColumnVector<Type>::create(key_columns.front()->size());
|
typename ColVec::MutablePtr out;
|
||||||
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
out = ColVec::create(key_columns.front()->size(), decimal_scale);
|
||||||
|
else
|
||||||
|
out = ColVec::create(key_columns.front()->size());
|
||||||
auto & data = out->getData();
|
auto & data = out->getData();
|
||||||
DictGetTraits<DataType>::get(dict, attr_name, key_columns, key_types, data);
|
DictGetTraits<DataType>::get(dict, attr_name, key_columns, key_types, data);
|
||||||
block.getByPosition(result).column = std::move(out);
|
block.getByPosition(result).column = std::move(out);
|
||||||
@ -855,7 +935,11 @@ private:
|
|||||||
const auto & id_col_values = getColumnDataAsPaddedPODArray(*id_col_untyped, id_col_values_storage);
|
const auto & id_col_values = getColumnDataAsPaddedPODArray(*id_col_untyped, id_col_values_storage);
|
||||||
const auto & range_col_values = getColumnDataAsPaddedPODArray(*range_col_untyped, range_col_values_storage);
|
const auto & range_col_values = getColumnDataAsPaddedPODArray(*range_col_untyped, range_col_values_storage);
|
||||||
|
|
||||||
auto out = ColumnVector<Type>::create(id_col_untyped->size());
|
typename ColVec::MutablePtr out;
|
||||||
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
out = ColVec::create(id_col_untyped->size(), decimal_scale);
|
||||||
|
else
|
||||||
|
out = ColVec::create(id_col_untyped->size());
|
||||||
auto & data = out->getData();
|
auto & data = out->getData();
|
||||||
DictGetTraits<DataType>::get(dict, attr_name, id_col_values, range_col_values, data);
|
DictGetTraits<DataType>::get(dict, attr_name, id_col_values, range_col_values, data);
|
||||||
block.getByPosition(result).column = std::move(out);
|
block.getByPosition(result).column = std::move(out);
|
||||||
@ -864,6 +948,7 @@ private:
|
|||||||
}
|
}
|
||||||
|
|
||||||
const ExternalDictionaries & dictionaries;
|
const ExternalDictionaries & dictionaries;
|
||||||
|
UInt32 decimal_scale;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct NameDictGetUInt8 { static constexpr auto name = "dictGetUInt8"; };
|
struct NameDictGetUInt8 { static constexpr auto name = "dictGetUInt8"; };
|
||||||
@ -879,6 +964,9 @@ struct NameDictGetFloat64 { static constexpr auto name = "dictGetFloat64"; };
|
|||||||
struct NameDictGetDate { static constexpr auto name = "dictGetDate"; };
|
struct NameDictGetDate { static constexpr auto name = "dictGetDate"; };
|
||||||
struct NameDictGetDateTime { static constexpr auto name = "dictGetDateTime"; };
|
struct NameDictGetDateTime { static constexpr auto name = "dictGetDateTime"; };
|
||||||
struct NameDictGetUUID { static constexpr auto name = "dictGetUUID"; };
|
struct NameDictGetUUID { static constexpr auto name = "dictGetUUID"; };
|
||||||
|
struct NameDictGetDecimal32 { static constexpr auto name = "dictGetDecimal32"; };
|
||||||
|
struct NameDictGetDecimal64 { static constexpr auto name = "dictGetDecimal64"; };
|
||||||
|
struct NameDictGetDecimal128 { static constexpr auto name = "dictGetDecimal128"; };
|
||||||
|
|
||||||
using FunctionDictGetUInt8 = FunctionDictGet<DataTypeUInt8, NameDictGetUInt8>;
|
using FunctionDictGetUInt8 = FunctionDictGet<DataTypeUInt8, NameDictGetUInt8>;
|
||||||
using FunctionDictGetUInt16 = FunctionDictGet<DataTypeUInt16, NameDictGetUInt16>;
|
using FunctionDictGetUInt16 = FunctionDictGet<DataTypeUInt16, NameDictGetUInt16>;
|
||||||
@ -893,22 +981,29 @@ using FunctionDictGetFloat64 = FunctionDictGet<DataTypeFloat64, NameDictGetFloat
|
|||||||
using FunctionDictGetDate = FunctionDictGet<DataTypeDate, NameDictGetDate>;
|
using FunctionDictGetDate = FunctionDictGet<DataTypeDate, NameDictGetDate>;
|
||||||
using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime, NameDictGetDateTime>;
|
using FunctionDictGetDateTime = FunctionDictGet<DataTypeDateTime, NameDictGetDateTime>;
|
||||||
using FunctionDictGetUUID = FunctionDictGet<DataTypeUUID, NameDictGetUUID>;
|
using FunctionDictGetUUID = FunctionDictGet<DataTypeUUID, NameDictGetUUID>;
|
||||||
|
using FunctionDictGetDecimal32 = FunctionDictGet<DataTypeDecimal<Decimal32>, NameDictGetDecimal32>;
|
||||||
|
using FunctionDictGetDecimal64 = FunctionDictGet<DataTypeDecimal<Decimal64>, NameDictGetDecimal64>;
|
||||||
|
using FunctionDictGetDecimal128 = FunctionDictGet<DataTypeDecimal<Decimal128>, NameDictGetDecimal128>;
|
||||||
|
|
||||||
|
|
||||||
template <typename DataType, typename Name>
|
template <typename DataType, typename Name>
|
||||||
class FunctionDictGetOrDefault final : public IFunction
|
class FunctionDictGetOrDefault final : public IFunction
|
||||||
{
|
{
|
||||||
using Type = typename DataType::FieldType;
|
using Type = typename DataType::FieldType;
|
||||||
|
using ColVec = std::conditional_t<IsDecimalNumber<Type>, ColumnDecimal<Type>, ColumnVector<Type>>;
|
||||||
|
|
||||||
public:
|
public:
|
||||||
static constexpr auto name = Name::name;
|
static constexpr auto name = Name::name;
|
||||||
|
|
||||||
static FunctionPtr create(const Context & context)
|
static FunctionPtr create(const Context & context, UInt32 dec_scale = 0)
|
||||||
{
|
{
|
||||||
return std::make_shared<FunctionDictGetOrDefault>(context.getExternalDictionaries());
|
return std::make_shared<FunctionDictGetOrDefault>(context.getExternalDictionaries(), dec_scale);
|
||||||
}
|
}
|
||||||
|
|
||||||
FunctionDictGetOrDefault(const ExternalDictionaries & dictionaries) : dictionaries(dictionaries) {}
|
FunctionDictGetOrDefault(const ExternalDictionaries & dictionaries, UInt32 dec_scale = 0)
|
||||||
|
: dictionaries(dictionaries)
|
||||||
|
, decimal_scale(dec_scale)
|
||||||
|
{}
|
||||||
|
|
||||||
String getName() const override { return name; }
|
String getName() const override { return name; }
|
||||||
|
|
||||||
@ -935,9 +1030,12 @@ private:
|
|||||||
|
|
||||||
if (!checkAndGetDataType<DataType>(arguments[3].get()))
|
if (!checkAndGetDataType<DataType>(arguments[3].get()))
|
||||||
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
|
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
|
||||||
+ ", must be " + String(DataType{}.getFamilyName()) + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
+ ", must be " + TypeName<Type>::get() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
return std::make_shared<DataType>();
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
return std::make_shared<DataType>(DataType::maxPrecision(), decimal_scale);
|
||||||
|
else
|
||||||
|
return std::make_shared<DataType>();
|
||||||
}
|
}
|
||||||
|
|
||||||
bool isDeterministic() const override { return false; }
|
bool isDeterministic() const override { return false; }
|
||||||
@ -999,20 +1097,28 @@ private:
|
|||||||
{
|
{
|
||||||
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
||||||
|
|
||||||
if (const auto default_col = checkAndGetColumn<ColumnVector<Type>>(default_col_untyped))
|
if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped))
|
||||||
{
|
{
|
||||||
/// vector ids, vector defaults
|
/// vector ids, vector defaults
|
||||||
auto out = ColumnVector<Type>::create(id_col->size());
|
typename ColVec::MutablePtr out;
|
||||||
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
out = ColVec::create(id_col->size(), decimal_scale);
|
||||||
|
else
|
||||||
|
out = ColVec::create(id_col->size());
|
||||||
const auto & ids = id_col->getData();
|
const auto & ids = id_col->getData();
|
||||||
auto & data = out->getData();
|
auto & data = out->getData();
|
||||||
const auto & defs = default_col->getData();
|
const auto & defs = default_col->getData();
|
||||||
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
|
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, defs, data);
|
||||||
block.getByPosition(result).column = std::move(out);
|
block.getByPosition(result).column = std::move(out);
|
||||||
}
|
}
|
||||||
else if (const auto default_col_const = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
|
else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped))
|
||||||
{
|
{
|
||||||
/// vector ids, const defaults
|
/// vector ids, const defaults
|
||||||
auto out = ColumnVector<Type>::create(id_col->size());
|
typename ColVec::MutablePtr out;
|
||||||
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
out = ColVec::create(id_col->size(), decimal_scale);
|
||||||
|
else
|
||||||
|
out = ColVec::create(id_col->size());
|
||||||
const auto & ids = id_col->getData();
|
const auto & ids = id_col->getData();
|
||||||
auto & data = out->getData();
|
auto & data = out->getData();
|
||||||
const auto def = default_col_const->template getValue<Type>();
|
const auto def = default_col_const->template getValue<Type>();
|
||||||
@ -1020,7 +1126,7 @@ private:
|
|||||||
block.getByPosition(result).column = std::move(out);
|
block.getByPosition(result).column = std::move(out);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception{"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()), ErrorCodes::ILLEGAL_COLUMN};
|
throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN};
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename DictionaryType>
|
template <typename DictionaryType>
|
||||||
@ -1030,7 +1136,7 @@ private:
|
|||||||
{
|
{
|
||||||
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
||||||
|
|
||||||
if (const auto default_col = checkAndGetColumn<ColumnVector<Type>>(default_col_untyped))
|
if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped))
|
||||||
{
|
{
|
||||||
/// const ids, vector defaults
|
/// const ids, vector defaults
|
||||||
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
|
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
|
||||||
@ -1038,24 +1144,48 @@ private:
|
|||||||
dictionary->has(ids, flags);
|
dictionary->has(ids, flags);
|
||||||
if (flags.front())
|
if (flags.front())
|
||||||
{
|
{
|
||||||
PaddedPODArray<Type> data(1);
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, Type(), data);
|
{
|
||||||
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
|
DecimalPaddedPODArray<Type> data(1, decimal_scale);
|
||||||
|
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, Type(), data);
|
||||||
|
block.getByPosition(result).column =
|
||||||
|
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
|
||||||
|
id_col->size(), toField(data.front(), decimal_scale));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
PaddedPODArray<Type> data(1);
|
||||||
|
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, Type(), data);
|
||||||
|
block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column
|
block.getByPosition(result).column = block.getByPosition(arguments[3]).column; // reuse the default column
|
||||||
}
|
}
|
||||||
else if (const auto default_col_const = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
|
else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped))
|
||||||
{
|
{
|
||||||
/// const ids, const defaults
|
/// const ids, const defaults
|
||||||
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
|
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
|
||||||
PaddedPODArray<Type> data(1);
|
|
||||||
const auto & def = default_col_const->template getValue<Type>();
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
|
{
|
||||||
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
|
DecimalPaddedPODArray<Type> data(1, decimal_scale);
|
||||||
|
const auto & def = default_col_const->template getValue<Type>();
|
||||||
|
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
|
||||||
|
block.getByPosition(result).column =
|
||||||
|
DataType(DataType::maxPrecision(), decimal_scale).createColumnConst(
|
||||||
|
id_col->size(), toField(data.front(), decimal_scale));
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
PaddedPODArray<Type> data(1);
|
||||||
|
const auto & def = default_col_const->template getValue<Type>();
|
||||||
|
DictGetTraits<DataType>::getOrDefault(dictionary, attr_name, ids, def, data);
|
||||||
|
block.getByPosition(result).column = DataType().createColumnConst(id_col->size(), toField(data.front()));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception{"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()), ErrorCodes::ILLEGAL_COLUMN};
|
throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN};
|
||||||
}
|
}
|
||||||
|
|
||||||
template <typename DictionaryType>
|
template <typename DictionaryType>
|
||||||
@ -1082,31 +1212,36 @@ private:
|
|||||||
|
|
||||||
/// @todo detect when all key columns are constant
|
/// @todo detect when all key columns are constant
|
||||||
const auto rows = key_col->size();
|
const auto rows = key_col->size();
|
||||||
auto out = ColumnVector<Type>::create(rows);
|
typename ColVec::MutablePtr out;
|
||||||
|
if constexpr (IsDataTypeDecimal<DataType>)
|
||||||
|
out = ColVec::create(rows, decimal_scale);
|
||||||
|
else
|
||||||
|
out = ColVec::create(rows);
|
||||||
auto & data = out->getData();
|
auto & data = out->getData();
|
||||||
|
|
||||||
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
|
||||||
if (const auto default_col = checkAndGetColumn<ColumnVector<Type>>(default_col_untyped))
|
if (const auto default_col = checkAndGetColumn<ColVec>(default_col_untyped))
|
||||||
{
|
{
|
||||||
/// const defaults
|
/// const defaults
|
||||||
const auto & defs = default_col->getData();
|
const auto & defs = default_col->getData();
|
||||||
|
|
||||||
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, defs, data);
|
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, defs, data);
|
||||||
}
|
}
|
||||||
else if (const auto default_col_const = checkAndGetColumnConst<ColumnVector<Type>>(default_col_untyped))
|
else if (const auto default_col_const = checkAndGetColumnConst<ColVec>(default_col_untyped))
|
||||||
{
|
{
|
||||||
const auto def = default_col_const->template getValue<Type>();
|
const auto def = default_col_const->template getValue<Type>();
|
||||||
|
|
||||||
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, def, data);
|
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, def, data);
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
throw Exception{"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()), ErrorCodes::ILLEGAL_COLUMN};
|
throw Exception{"Fourth argument of function " + getName() + " must be " + TypeName<Type>::get(), ErrorCodes::ILLEGAL_COLUMN};
|
||||||
|
|
||||||
block.getByPosition(result).column = std::move(out);
|
block.getByPosition(result).column = std::move(out);
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
const ExternalDictionaries & dictionaries;
|
const ExternalDictionaries & dictionaries;
|
||||||
|
UInt32 decimal_scale;
|
||||||
};
|
};
|
||||||
|
|
||||||
struct NameDictGetUInt8OrDefault { static constexpr auto name = "dictGetUInt8OrDefault"; };
|
struct NameDictGetUInt8OrDefault { static constexpr auto name = "dictGetUInt8OrDefault"; };
|
||||||
@ -1122,6 +1257,9 @@ struct NameDictGetFloat64OrDefault { static constexpr auto name = "dictGetFloat6
|
|||||||
struct NameDictGetDateOrDefault { static constexpr auto name = "dictGetDateOrDefault"; };
|
struct NameDictGetDateOrDefault { static constexpr auto name = "dictGetDateOrDefault"; };
|
||||||
struct NameDictGetDateTimeOrDefault { static constexpr auto name = "dictGetDateTimeOrDefault"; };
|
struct NameDictGetDateTimeOrDefault { static constexpr auto name = "dictGetDateTimeOrDefault"; };
|
||||||
struct NameDictGetUUIDOrDefault { static constexpr auto name = "dictGetUUIDOrDefault"; };
|
struct NameDictGetUUIDOrDefault { static constexpr auto name = "dictGetUUIDOrDefault"; };
|
||||||
|
struct NameDictGetDecimal32OrDefault { static constexpr auto name = "dictGetDecimal32OrDefault"; };
|
||||||
|
struct NameDictGetDecimal64OrDefault { static constexpr auto name = "dictGetDecimal64OrDefault"; };
|
||||||
|
struct NameDictGetDecimal128OrDefault { static constexpr auto name = "dictGetDecimal128OrDefault"; };
|
||||||
|
|
||||||
using FunctionDictGetUInt8OrDefault = FunctionDictGetOrDefault<DataTypeUInt8, NameDictGetUInt8OrDefault>;
|
using FunctionDictGetUInt8OrDefault = FunctionDictGetOrDefault<DataTypeUInt8, NameDictGetUInt8OrDefault>;
|
||||||
using FunctionDictGetUInt16OrDefault = FunctionDictGetOrDefault<DataTypeUInt16, NameDictGetUInt16OrDefault>;
|
using FunctionDictGetUInt16OrDefault = FunctionDictGetOrDefault<DataTypeUInt16, NameDictGetUInt16OrDefault>;
|
||||||
@ -1136,21 +1274,10 @@ using FunctionDictGetFloat64OrDefault = FunctionDictGetOrDefault<DataTypeFloat64
|
|||||||
using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault<DataTypeDate, NameDictGetDateOrDefault>;
|
using FunctionDictGetDateOrDefault = FunctionDictGetOrDefault<DataTypeDate, NameDictGetDateOrDefault>;
|
||||||
using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTime, NameDictGetDateTimeOrDefault>;
|
using FunctionDictGetDateTimeOrDefault = FunctionDictGetOrDefault<DataTypeDateTime, NameDictGetDateTimeOrDefault>;
|
||||||
using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault<DataTypeUUID, NameDictGetUUIDOrDefault>;
|
using FunctionDictGetUUIDOrDefault = FunctionDictGetOrDefault<DataTypeUUID, NameDictGetUUIDOrDefault>;
|
||||||
|
using FunctionDictGetDecimal32OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal32>, NameDictGetDecimal32OrDefault>;
|
||||||
|
using FunctionDictGetDecimal64OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal64>, NameDictGetDecimal64OrDefault>;
|
||||||
|
using FunctionDictGetDecimal128OrDefault = FunctionDictGetOrDefault<DataTypeDecimal<Decimal128>, NameDictGetDecimal128OrDefault>;
|
||||||
|
|
||||||
#define FOR_DICT_TYPES(M) \
|
|
||||||
M(UInt8) \
|
|
||||||
M(UInt16) \
|
|
||||||
M(UInt32) \
|
|
||||||
M(UInt64) \
|
|
||||||
M(Int8) \
|
|
||||||
M(Int16) \
|
|
||||||
M(Int32) \
|
|
||||||
M(Int64) \
|
|
||||||
M(Float32) \
|
|
||||||
M(Float64) \
|
|
||||||
M(Date) \
|
|
||||||
M(DateTime) \
|
|
||||||
M(UUID)
|
|
||||||
|
|
||||||
/// This variant of function derives the result type automatically.
|
/// This variant of function derives the result type automatically.
|
||||||
class FunctionDictGetNoType final : public IFunction
|
class FunctionDictGetNoType final : public IFunction
|
||||||
@ -1225,15 +1352,63 @@ private:
|
|||||||
if (attribute.name == attr_name)
|
if (attribute.name == attr_name)
|
||||||
{
|
{
|
||||||
WhichDataType dt = attribute.type;
|
WhichDataType dt = attribute.type;
|
||||||
if (dt.idx == TypeIndex::String)
|
switch (dt.idx)
|
||||||
impl = FunctionDictGetString::create(context);
|
{
|
||||||
#define DISPATCH(TYPE) \
|
case TypeIndex::String:
|
||||||
else if (dt.idx == TypeIndex::TYPE) \
|
case TypeIndex::FixedString:
|
||||||
impl = FunctionDictGet<DataType##TYPE, NameDictGet##TYPE>::create(context);
|
impl = FunctionDictGetString::create(context);
|
||||||
FOR_DICT_TYPES(DISPATCH)
|
break;
|
||||||
#undef DISPATCH
|
case TypeIndex::UInt8:
|
||||||
else
|
impl = FunctionDictGetUInt8::create(context);
|
||||||
throw Exception("Unknown dictGet type", ErrorCodes::UNKNOWN_TYPE);
|
break;
|
||||||
|
case TypeIndex::UInt16:
|
||||||
|
impl = FunctionDictGetUInt16::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::UInt32:
|
||||||
|
impl = FunctionDictGetUInt32::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::UInt64:
|
||||||
|
impl = FunctionDictGetUInt64::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int8:
|
||||||
|
impl = FunctionDictGetInt8::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int16:
|
||||||
|
impl = FunctionDictGetInt16::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int32:
|
||||||
|
impl = FunctionDictGetInt32::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int64:
|
||||||
|
impl = FunctionDictGetInt64::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Float32:
|
||||||
|
impl = FunctionDictGetFloat32::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Float64:
|
||||||
|
impl = FunctionDictGetFloat64::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Date:
|
||||||
|
impl = FunctionDictGetDate::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::DateTime:
|
||||||
|
impl = FunctionDictGetDateTime::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::UUID:
|
||||||
|
impl = FunctionDictGetUUID::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Decimal32:
|
||||||
|
impl = FunctionDictGetDecimal32::create(context, getDecimalScale(*attribute.type));
|
||||||
|
break;
|
||||||
|
case TypeIndex::Decimal64:
|
||||||
|
impl = FunctionDictGetDecimal64::create(context, getDecimalScale(*attribute.type));
|
||||||
|
break;
|
||||||
|
case TypeIndex::Decimal128:
|
||||||
|
impl = FunctionDictGetDecimal128::create(context, getDecimalScale(*attribute.type));
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw Exception("Unknown dictGet type", ErrorCodes::UNKNOWN_TYPE);
|
||||||
|
}
|
||||||
return attribute.type;
|
return attribute.type;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -1312,26 +1487,70 @@ private:
|
|||||||
const DictionaryAttribute & attribute = structure.attributes[idx];
|
const DictionaryAttribute & attribute = structure.attributes[idx];
|
||||||
if (attribute.name == attr_name)
|
if (attribute.name == attr_name)
|
||||||
{
|
{
|
||||||
|
auto arg_type = arguments[3].type;
|
||||||
WhichDataType dt = attribute.type;
|
WhichDataType dt = attribute.type;
|
||||||
if (dt.idx == TypeIndex::String)
|
|
||||||
|
if ((arg_type->getTypeId() != dt.idx) || (dt.isStringOrFixedString() && !isString(arg_type)))
|
||||||
|
throw Exception{"Illegal type " + arg_type->getName() + " of fourth argument of function " + getName() +
|
||||||
|
", must be " + getTypeName(dt.idx) + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||||
|
|
||||||
|
switch (dt.idx)
|
||||||
{
|
{
|
||||||
if (!isString(arguments[3].type))
|
case TypeIndex::String:
|
||||||
throw Exception{"Illegal type " + arguments[3].type->getName() + " of fourth argument of function " + getName() +
|
impl = FunctionDictGetStringOrDefault::create(context);
|
||||||
", must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
break;
|
||||||
impl = FunctionDictGetStringOrDefault::create(context);
|
case TypeIndex::UInt8:
|
||||||
|
impl = FunctionDictGetUInt8OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::UInt16:
|
||||||
|
impl = FunctionDictGetUInt16OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::UInt32:
|
||||||
|
impl = FunctionDictGetUInt32OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::UInt64:
|
||||||
|
impl = FunctionDictGetUInt64OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int8:
|
||||||
|
impl = FunctionDictGetInt8OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int16:
|
||||||
|
impl = FunctionDictGetInt16OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int32:
|
||||||
|
impl = FunctionDictGetInt32OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Int64:
|
||||||
|
impl = FunctionDictGetInt64OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Float32:
|
||||||
|
impl = FunctionDictGetFloat32OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Float64:
|
||||||
|
impl = FunctionDictGetFloat64OrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Date:
|
||||||
|
impl = FunctionDictGetDateOrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::DateTime:
|
||||||
|
impl = FunctionDictGetDateTimeOrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::UUID:
|
||||||
|
impl = FunctionDictGetUUIDOrDefault::create(context);
|
||||||
|
break;
|
||||||
|
case TypeIndex::Decimal32:
|
||||||
|
impl = FunctionDictGetDecimal32OrDefault::create(context, getDecimalScale(*attribute.type));
|
||||||
|
break;
|
||||||
|
case TypeIndex::Decimal64:
|
||||||
|
impl = FunctionDictGetDecimal64OrDefault::create(context, getDecimalScale(*attribute.type));
|
||||||
|
break;
|
||||||
|
case TypeIndex::Decimal128:
|
||||||
|
impl = FunctionDictGetDecimal128OrDefault::create(context, getDecimalScale(*attribute.type));
|
||||||
|
break;
|
||||||
|
default:
|
||||||
|
throw Exception("Unknown dictGetOrDefault type", ErrorCodes::UNKNOWN_TYPE);
|
||||||
}
|
}
|
||||||
#define DISPATCH(TYPE) \
|
|
||||||
else if (dt.idx == TypeIndex::TYPE) \
|
|
||||||
{ \
|
|
||||||
if (!checkAndGetDataType<DataType##TYPE>(arguments[3].type.get())) \
|
|
||||||
throw Exception{"Illegal type " + arguments[3].type->getName() + " of fourth argument of function " + getName() \
|
|
||||||
+ ", must be " + String(DataType##TYPE{}.getFamilyName()) + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; \
|
|
||||||
impl = FunctionDictGetOrDefault<DataType##TYPE, NameDictGet##TYPE ## OrDefault>::create(context); \
|
|
||||||
}
|
|
||||||
FOR_DICT_TYPES(DISPATCH)
|
|
||||||
#undef DISPATCH
|
|
||||||
else
|
|
||||||
throw Exception("Unknown dictGetOrDefault type", ErrorCodes::UNKNOWN_TYPE);
|
|
||||||
return attribute.type;
|
return attribute.type;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -57,6 +57,9 @@ if [ "$DATA_DIR_PATTERN" != "$DATA_DIR" ]; then
|
|||||||
export CLICKHOUSE_CONFIG=$DATA_DIR/etc/server-config.xml
|
export CLICKHOUSE_CONFIG=$DATA_DIR/etc/server-config.xml
|
||||||
cp $CLICKHOUSE_CONFIG_USERS $DATA_DIR/etc
|
cp $CLICKHOUSE_CONFIG_USERS $DATA_DIR/etc
|
||||||
cp -R -L $CLICKHOUSE_CONFIG_USERS_D $DATA_DIR/etc
|
cp -R -L $CLICKHOUSE_CONFIG_USERS_D $DATA_DIR/etc
|
||||||
|
cat ${CONFIG_SERVER_DIR}/ints_dictionary.xml | sed -e s!9000!$CLICKHOUSE_PORT_TCP! > $DATA_DIR/etc/ints_dictionary.xml
|
||||||
|
cat ${CONFIG_SERVER_DIR}/strings_dictionary.xml | sed -e s!9000!$CLICKHOUSE_PORT_TCP! > $DATA_DIR/etc/strings_dictionary.xml
|
||||||
|
cat ${CONFIG_SERVER_DIR}/decimals_dictionary.xml | sed -e s!9000!$CLICKHOUSE_PORT_TCP! > $DATA_DIR/etc/decimals_dictionary.xml
|
||||||
fi
|
fi
|
||||||
|
|
||||||
CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="${CLICKHOUSE_EXTRACT} --config=$CLICKHOUSE_CONFIG"}
|
CLICKHOUSE_EXTRACT_CONFIG=${CLICKHOUSE_EXTRACT_CONFIG:="${CLICKHOUSE_EXTRACT} --config=$CLICKHOUSE_CONFIG"}
|
||||||
|
1
dbms/tests/decimals_dictionary.xml
Symbolic link
1
dbms/tests/decimals_dictionary.xml
Symbolic link
@ -0,0 +1 @@
|
|||||||
|
../../docker/test/stateless/decimals_dictionary.xml
|
1
dbms/tests/ints_dictionary.xml
Symbolic link
1
dbms/tests/ints_dictionary.xml
Symbolic link
@ -0,0 +1 @@
|
|||||||
|
../../docker/test/stateless/ints_dictionary.xml
|
@ -30,5 +30,5 @@ CREATE DATABASE test_DatabaseDictionary ENGINE = Dictionary;
|
|||||||
|
|
||||||
SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables;
|
SELECT sum(ignore(*, metadata_modification_time, engine_full, create_table_query)) FROM system.tables;
|
||||||
|
|
||||||
DROP DATABASE test_DatabaseDictionary;
|
DROP DATABASE test_DatabaseDictionary; -- { serverError 48 }
|
||||||
DROP DATABASE test_DatabaseMemory;
|
DROP DATABASE test_DatabaseMemory;
|
||||||
|
45
dbms/tests/queries/0_stateless/00950_dict_get.reference
Normal file
45
dbms/tests/queries/0_stateless/00950_dict_get.reference
Normal file
@ -0,0 +1,45 @@
|
|||||||
|
dictGet flat_ints 1 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault flat_ints 1 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault flat_ints 0 42 42 42 42 42 42 42 42
|
||||||
|
dictGet hashed_ints 1 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault hashed_ints 1 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault hashed_ints 0 42 42 42 42 42 42 42 42
|
||||||
|
dictGet cache_ints 1 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault cache_ints 1 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault cache_ints 0 42 42 42 42 42 42 42 42
|
||||||
|
dictGet complex_hashed_ints (1) 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault complex_hashed_ints (1) 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault complex_hashed_ints (0) 42 42 42 42 42 42 42 42
|
||||||
|
dictGet complex_cache_ints (1) 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault complex_cache_ints (1) 1 1 1 1 1 1 1 1
|
||||||
|
dictGetOrDefault complex_cache_ints (0) 42 42 42 42 42 42 42 42
|
||||||
|
dictGet flat_strings 1 1
|
||||||
|
dictGetOrDefault flat_strings 1 1
|
||||||
|
dictGetOrDefault flat_strings 0 *
|
||||||
|
dictGet hashed_strings 1 1
|
||||||
|
dictGetOrDefault hashed_strings 1 1
|
||||||
|
dictGetOrDefault hashed_strings 0 *
|
||||||
|
dictGet cache_strings 1 1
|
||||||
|
dictGetOrDefault cache_strings 1 1
|
||||||
|
dictGetOrDefault cache_strings 0 *
|
||||||
|
dictGet complex_hashed_strings 1 1
|
||||||
|
dictGetOrDefault complex_hashed_strings 1 1
|
||||||
|
dictGetOrDefault complex_hashed_strings 0 *
|
||||||
|
dictGet complex_cache_strings 1 1
|
||||||
|
dictGetOrDefault complex_cache_strings 1 1
|
||||||
|
dictGetOrDefault complex_cache_strings 0 *
|
||||||
|
dictGet flat_decimals 1 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault flat_decimals 1 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault flat_decimals 0 42.0000 42.000000 42.0
|
||||||
|
dictGet hashed_decimals 1 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault hashed_decimals 1 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault hashed_decimals 0 42.0000 42.000000 42.0
|
||||||
|
dictGet cache_decimals 1 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault cache_decimals 1 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault cache_decimals 0 42.0000 42.000000 42.0
|
||||||
|
dictGet complex_hashed_decimals (1) 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault complex_hashed_decimals (1) 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault complex_hashed_decimals (0) 42.0000 42.000000 42.0
|
||||||
|
dictGet complex_cache_decimals (1) 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault complex_cache_decimals (1) 1.0000 1.000000 1.0
|
||||||
|
dictGetOrDefault complex_cache_decimals (0) 42.0000 42.000000 42.0
|
245
dbms/tests/queries/0_stateless/00950_dict_get.sql
Normal file
245
dbms/tests/queries/0_stateless/00950_dict_get.sql
Normal file
@ -0,0 +1,245 @@
|
|||||||
|
use default;
|
||||||
|
drop table if exists ints;
|
||||||
|
drop table if exists strings;
|
||||||
|
drop table if exists decimals;
|
||||||
|
|
||||||
|
create table ints (key UInt64, i8 Int8, i16 Int16, i32 Int32, i64 Int64, u8 UInt8, u16 UInt16, u32 UInt32, u64 UInt64) Engine = Memory;
|
||||||
|
create table strings (key UInt64, str String) Engine = Memory;
|
||||||
|
create table decimals (key UInt64, d32 Decimal32(4), d64 Decimal64(6), d128 Decimal128(1)) Engine = Memory;
|
||||||
|
|
||||||
|
insert into ints values (1, 1, 1, 1, 1, 1, 1, 1, 1);
|
||||||
|
insert into strings values (1, '1');
|
||||||
|
insert into decimals values (1, 1, 1, 1);
|
||||||
|
|
||||||
|
select 'dictGet', 'flat_ints' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGet(dict_name, 'i8', k),
|
||||||
|
dictGet(dict_name, 'i16', k),
|
||||||
|
dictGet(dict_name, 'i32', k),
|
||||||
|
dictGet(dict_name, 'i64', k),
|
||||||
|
dictGet(dict_name, 'u8', k),
|
||||||
|
dictGet(dict_name, 'u16', k),
|
||||||
|
dictGet(dict_name, 'u32', k),
|
||||||
|
dictGet(dict_name, 'u64', k);
|
||||||
|
select 'dictGetOrDefault', 'flat_ints' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
select 'dictGetOrDefault', 'flat_ints' as dict_name, toUInt64(0) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
|
||||||
|
select 'dictGet', 'hashed_ints' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGet(dict_name, 'i8', k),
|
||||||
|
dictGet(dict_name, 'i16', k),
|
||||||
|
dictGet(dict_name, 'i32', k),
|
||||||
|
dictGet(dict_name, 'i64', k),
|
||||||
|
dictGet(dict_name, 'u8', k),
|
||||||
|
dictGet(dict_name, 'u16', k),
|
||||||
|
dictGet(dict_name, 'u32', k),
|
||||||
|
dictGet(dict_name, 'u64', k);
|
||||||
|
select 'dictGetOrDefault', 'hashed_ints' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
select 'dictGetOrDefault', 'hashed_ints' as dict_name, toUInt64(0) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
|
||||||
|
select 'dictGet', 'cache_ints' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGet(dict_name, 'i8', k),
|
||||||
|
dictGet(dict_name, 'i16', k),
|
||||||
|
dictGet(dict_name, 'i32', k),
|
||||||
|
dictGet(dict_name, 'i64', k),
|
||||||
|
dictGet(dict_name, 'u8', k),
|
||||||
|
dictGet(dict_name, 'u16', k),
|
||||||
|
dictGet(dict_name, 'u32', k),
|
||||||
|
dictGet(dict_name, 'u64', k);
|
||||||
|
select 'dictGetOrDefault', 'cache_ints' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
select 'dictGetOrDefault', 'cache_ints' as dict_name, toUInt64(0) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
|
||||||
|
select 'dictGet', 'complex_hashed_ints' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGet(dict_name, 'i8', k),
|
||||||
|
dictGet(dict_name, 'i16', k),
|
||||||
|
dictGet(dict_name, 'i32', k),
|
||||||
|
dictGet(dict_name, 'i64', k),
|
||||||
|
dictGet(dict_name, 'u8', k),
|
||||||
|
dictGet(dict_name, 'u16', k),
|
||||||
|
dictGet(dict_name, 'u32', k),
|
||||||
|
dictGet(dict_name, 'u64', k);
|
||||||
|
select 'dictGetOrDefault', 'complex_hashed_ints' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
select 'dictGetOrDefault', 'complex_hashed_ints' as dict_name, tuple(toUInt64(0)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
|
||||||
|
select 'dictGet', 'complex_cache_ints' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGet(dict_name, 'i8', k),
|
||||||
|
dictGet(dict_name, 'i16', k),
|
||||||
|
dictGet(dict_name, 'i32', k),
|
||||||
|
dictGet(dict_name, 'i64', k),
|
||||||
|
dictGet(dict_name, 'u8', k),
|
||||||
|
dictGet(dict_name, 'u16', k),
|
||||||
|
dictGet(dict_name, 'u32', k),
|
||||||
|
dictGet(dict_name, 'u64', k);
|
||||||
|
select 'dictGetOrDefault', 'complex_cache_ints' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
select 'dictGetOrDefault', 'complex_cache_ints' as dict_name, tuple(toUInt64(0)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'i8', k, toInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i16', k, toInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i32', k, toInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'i64', k, toInt64(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u8', k, toUInt8(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u16', k, toUInt16(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u32', k, toUInt32(42)),
|
||||||
|
dictGetOrDefault(dict_name, 'u64', k, toUInt64(42));
|
||||||
|
|
||||||
|
--
|
||||||
|
|
||||||
|
select 'dictGet', 'flat_strings' as dict_name, toUInt64(1) as k, dictGet(dict_name, 'str', k);
|
||||||
|
select 'dictGetOrDefault', 'flat_strings' as dict_name, toUInt64(1) as k, dictGetOrDefault(dict_name, 'str', k, '*');
|
||||||
|
select 'dictGetOrDefault', 'flat_strings' as dict_name, toUInt64(0) as k, dictGetOrDefault(dict_name, 'str', k, '*');
|
||||||
|
|
||||||
|
select 'dictGet', 'hashed_strings' as dict_name, toUInt64(1) as k, dictGet(dict_name, 'str', k);
|
||||||
|
select 'dictGetOrDefault', 'hashed_strings' as dict_name, toUInt64(1) as k, dictGetOrDefault(dict_name, 'str', k, '*');
|
||||||
|
select 'dictGetOrDefault', 'hashed_strings' as dict_name, toUInt64(0) as k, dictGetOrDefault(dict_name, 'str', k, '*');
|
||||||
|
|
||||||
|
select 'dictGet', 'cache_strings' as dict_name, toUInt64(1) as k, dictGet(dict_name, 'str', k);
|
||||||
|
select 'dictGetOrDefault', 'cache_strings' as dict_name, toUInt64(1) as k, dictGetOrDefault(dict_name, 'str', k, '*');
|
||||||
|
select 'dictGetOrDefault', 'cache_strings' as dict_name, toUInt64(0) as k, dictGetOrDefault(dict_name, 'str', k, '*');
|
||||||
|
|
||||||
|
select 'dictGet', 'complex_hashed_strings' as dict_name, toUInt64(1) as k, dictGet(dict_name, 'str', tuple(k));
|
||||||
|
select 'dictGetOrDefault', 'complex_hashed_strings' as dict_name, toUInt64(1) as k, dictGetOrDefault(dict_name, 'str', tuple(k), '*');
|
||||||
|
select 'dictGetOrDefault', 'complex_hashed_strings' as dict_name, toUInt64(0) as k, dictGetOrDefault(dict_name, 'str', tuple(k), '*');
|
||||||
|
|
||||||
|
select 'dictGet', 'complex_cache_strings' as dict_name, toUInt64(1) as k, dictGet(dict_name, 'str', tuple(k));
|
||||||
|
select 'dictGetOrDefault', 'complex_cache_strings' as dict_name, toUInt64(1) as k, dictGetOrDefault(dict_name, 'str', tuple(k), '*');
|
||||||
|
select 'dictGetOrDefault', 'complex_cache_strings' as dict_name, toUInt64(0) as k, dictGetOrDefault(dict_name, 'str', tuple(k), '*');
|
||||||
|
|
||||||
|
--
|
||||||
|
|
||||||
|
select 'dictGet', 'flat_decimals' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGet(dict_name, 'd32', k),
|
||||||
|
dictGet(dict_name, 'd64', k),
|
||||||
|
dictGet(dict_name, 'd128', k);
|
||||||
|
select 'dictGetOrDefault', 'flat_decimals' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
select 'dictGetOrDefault', 'flat_decimals' as dict_name, toUInt64(0) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
|
||||||
|
select 'dictGet', 'hashed_decimals' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGet(dict_name, 'd32', k),
|
||||||
|
dictGet(dict_name, 'd64', k),
|
||||||
|
dictGet(dict_name, 'd128', k);
|
||||||
|
select 'dictGetOrDefault', 'hashed_decimals' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
select 'dictGetOrDefault', 'hashed_decimals' as dict_name, toUInt64(0) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
|
||||||
|
select 'dictGet', 'cache_decimals' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGet(dict_name, 'd32', k),
|
||||||
|
dictGet(dict_name, 'd64', k),
|
||||||
|
dictGet(dict_name, 'd128', k);
|
||||||
|
select 'dictGetOrDefault', 'cache_decimals' as dict_name, toUInt64(1) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
select 'dictGetOrDefault', 'cache_decimals' as dict_name, toUInt64(0) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
|
||||||
|
select 'dictGet', 'complex_hashed_decimals' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGet(dict_name, 'd32', k),
|
||||||
|
dictGet(dict_name, 'd64', k),
|
||||||
|
dictGet(dict_name, 'd128', k);
|
||||||
|
select 'dictGetOrDefault', 'complex_hashed_decimals' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
select 'dictGetOrDefault', 'complex_hashed_decimals' as dict_name, tuple(toUInt64(0)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
|
||||||
|
select 'dictGet', 'complex_cache_decimals' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGet(dict_name, 'd32', k),
|
||||||
|
dictGet(dict_name, 'd64', k),
|
||||||
|
dictGet(dict_name, 'd128', k);
|
||||||
|
select 'dictGetOrDefault', 'complex_cache_decimals' as dict_name, tuple(toUInt64(1)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
select 'dictGetOrDefault', 'complex_cache_decimals' as dict_name, tuple(toUInt64(0)) as k,
|
||||||
|
dictGetOrDefault(dict_name, 'd32', k, toDecimal32(42, 4)),
|
||||||
|
dictGetOrDefault(dict_name, 'd64', k, toDecimal64(42, 6)),
|
||||||
|
dictGetOrDefault(dict_name, 'd128', k, toDecimal128(42, 1));
|
||||||
|
|
||||||
|
drop table ints;
|
||||||
|
drop table strings;
|
||||||
|
drop table decimals;
|
@ -110,7 +110,7 @@
|
|||||||
<table>query_log</table>
|
<table>query_log</table>
|
||||||
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
<flush_interval_milliseconds>7500</flush_interval_milliseconds>
|
||||||
</query_log>
|
</query_log>
|
||||||
<dictionaries_config>dictionaries/dictionary_*.xml</dictionaries_config>
|
<dictionaries_config>*_dictionary.xml</dictionaries_config>
|
||||||
<compression incl="clickhouse_compression">
|
<compression incl="clickhouse_compression">
|
||||||
</compression>
|
</compression>
|
||||||
<distributed_ddl>
|
<distributed_ddl>
|
||||||
|
1
dbms/tests/strings_dictionary.xml
Symbolic link
1
dbms/tests/strings_dictionary.xml
Symbolic link
@ -0,0 +1 @@
|
|||||||
|
../../docker/test/stateless/strings_dictionary.xml
|
@ -34,6 +34,9 @@ COPY listen.xml /etc/clickhouse-server/config.d/listen.xml
|
|||||||
COPY part_log.xml /etc/clickhouse-server/config.d/part_log.xml
|
COPY part_log.xml /etc/clickhouse-server/config.d/part_log.xml
|
||||||
COPY log_queries.xml /etc/clickhouse-server/users.d/log_queries.xml
|
COPY log_queries.xml /etc/clickhouse-server/users.d/log_queries.xml
|
||||||
COPY readonly.xml /etc/clickhouse-server/users.d/readonly.xml
|
COPY readonly.xml /etc/clickhouse-server/users.d/readonly.xml
|
||||||
|
COPY ints_dictionary.xml /etc/clickhouse-server/ints_dictionary.xml
|
||||||
|
COPY strings_dictionary.xml /etc/clickhouse-server/strings_dictionary.xml
|
||||||
|
COPY decimals_dictionary.xml /etc/clickhouse-server/decimals_dictionary.xml
|
||||||
|
|
||||||
CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
CMD dpkg -i package_folder/clickhouse-common-static_*.deb; \
|
||||||
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb; \
|
dpkg -i package_folder/clickhouse-common-static-dbg_*.deb; \
|
||||||
|
197
docker/test/stateless/decimals_dictionary.xml
Normal file
197
docker/test/stateless/decimals_dictionary.xml
Normal file
@ -0,0 +1,197 @@
|
|||||||
|
<dictionaries>
|
||||||
|
<dictionary>
|
||||||
|
<name>flat_decimals</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>decimals</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<flat/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>d32</name>
|
||||||
|
<type>Decimal32(4)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d64</name>
|
||||||
|
<type>Decimal64(6)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d128</name>
|
||||||
|
<type>Decimal128(1)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>hashed_decimals</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>decimals</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<hashed/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>d32</name>
|
||||||
|
<type>Decimal32(4)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d64</name>
|
||||||
|
<type>Decimal64(6)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d128</name>
|
||||||
|
<type>Decimal128(1)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>cache_decimals</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>decimals</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<cache><size_in_cells>1000</size_in_cells></cache>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>d32</name>
|
||||||
|
<type>Decimal32(4)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d64</name>
|
||||||
|
<type>Decimal64(6)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d128</name>
|
||||||
|
<type>Decimal128(1)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_hashed_decimals</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>decimals</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_hashed/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>d32</name>
|
||||||
|
<type>Decimal32(4)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d64</name>
|
||||||
|
<type>Decimal64(6)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d128</name>
|
||||||
|
<type>Decimal128(1)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_cache_decimals</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>decimals</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_cache><size_in_cells>1000</size_in_cells></complex_key_cache>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>d32</name>
|
||||||
|
<type>Decimal32(4)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d64</name>
|
||||||
|
<type>Decimal64(6)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>d128</name>
|
||||||
|
<type>Decimal128(1)</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
</dictionaries>
|
322
docker/test/stateless/ints_dictionary.xml
Normal file
322
docker/test/stateless/ints_dictionary.xml
Normal file
@ -0,0 +1,322 @@
|
|||||||
|
<dictionaries>
|
||||||
|
<dictionary>
|
||||||
|
<name>flat_ints</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>ints</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<flat/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>i8</name>
|
||||||
|
<type>Int8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i16</name>
|
||||||
|
<type>Int16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i32</name>
|
||||||
|
<type>Int32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i64</name>
|
||||||
|
<type>Int64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u8</name>
|
||||||
|
<type>UInt8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u16</name>
|
||||||
|
<type>UInt16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u32</name>
|
||||||
|
<type>UInt32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u64</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>hashed_ints</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>ints</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<hashed/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>i8</name>
|
||||||
|
<type>Int8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i16</name>
|
||||||
|
<type>Int16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i32</name>
|
||||||
|
<type>Int32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i64</name>
|
||||||
|
<type>Int64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u8</name>
|
||||||
|
<type>UInt8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u16</name>
|
||||||
|
<type>UInt16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u32</name>
|
||||||
|
<type>UInt32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u64</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>cache_ints</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>ints</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<cache><size_in_cells>1000</size_in_cells></cache>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>i8</name>
|
||||||
|
<type>Int8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i16</name>
|
||||||
|
<type>Int16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i32</name>
|
||||||
|
<type>Int32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i64</name>
|
||||||
|
<type>Int64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u8</name>
|
||||||
|
<type>UInt8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u16</name>
|
||||||
|
<type>UInt16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u32</name>
|
||||||
|
<type>UInt32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u64</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_hashed_ints</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>ints</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_hashed/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>i8</name>
|
||||||
|
<type>Int8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i16</name>
|
||||||
|
<type>Int16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i32</name>
|
||||||
|
<type>Int32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i64</name>
|
||||||
|
<type>Int64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u8</name>
|
||||||
|
<type>UInt8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u16</name>
|
||||||
|
<type>UInt16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u32</name>
|
||||||
|
<type>UInt32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u64</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_cache_ints</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>ints</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_cache><size_in_cells>1000</size_in_cells></complex_key_cache>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>i8</name>
|
||||||
|
<type>Int8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i16</name>
|
||||||
|
<type>Int16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i32</name>
|
||||||
|
<type>Int32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>i64</name>
|
||||||
|
<type>Int64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u8</name>
|
||||||
|
<type>UInt8</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u16</name>
|
||||||
|
<type>UInt16</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u32</name>
|
||||||
|
<type>UInt32</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
<attribute>
|
||||||
|
<name>u64</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
</dictionaries>
|
209
docker/test/stateless/strings_dictionary.xml
Normal file
209
docker/test/stateless/strings_dictionary.xml
Normal file
@ -0,0 +1,209 @@
|
|||||||
|
<dictionaries>
|
||||||
|
<dictionary>
|
||||||
|
<name>flat_strings</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>strings</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<flat/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>str</name>
|
||||||
|
<type>String</type>
|
||||||
|
<null_value></null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>hashed_strings</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>strings</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<hashed/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>str</name>
|
||||||
|
<type>String</type>
|
||||||
|
<null_value></null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>cache_strings</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>strings</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<cache><size_in_cells>1000</size_in_cells></cache>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<id>
|
||||||
|
<name>key</name>
|
||||||
|
</id>
|
||||||
|
<attribute>
|
||||||
|
<name>str</name>
|
||||||
|
<type>String</type>
|
||||||
|
<null_value></null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_hashed_strings</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>strings</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_hashed/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>str</name>
|
||||||
|
<type>String</type>
|
||||||
|
<null_value></null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_cache_strings</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>strings</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_cache><size_in_cells>1000</size_in_cells></complex_key_cache>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>str</name>
|
||||||
|
<type>String</type>
|
||||||
|
<null_value></null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_hashed_strings_key</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>strings</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_hashed/>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>str</name>
|
||||||
|
<type>String</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
|
||||||
|
<dictionary>
|
||||||
|
<name>complex_cache_strings_key</name>
|
||||||
|
<source>
|
||||||
|
<clickhouse>
|
||||||
|
<host>localhost</host>
|
||||||
|
<port>9000</port>
|
||||||
|
<user>default</user>
|
||||||
|
<password></password>
|
||||||
|
<db></db>
|
||||||
|
<table>strings</table>
|
||||||
|
</clickhouse>
|
||||||
|
</source>
|
||||||
|
<lifetime>0</lifetime>
|
||||||
|
<layout>
|
||||||
|
<complex_key_cache><size_in_cells>1000</size_in_cells></complex_key_cache>
|
||||||
|
</layout>
|
||||||
|
<structure>
|
||||||
|
<key>
|
||||||
|
<attribute>
|
||||||
|
<name>str</name>
|
||||||
|
<type>String</type>
|
||||||
|
</attribute>
|
||||||
|
</key>
|
||||||
|
<attribute>
|
||||||
|
<name>key</name>
|
||||||
|
<type>UInt64</type>
|
||||||
|
<null_value>0</null_value>
|
||||||
|
</attribute>
|
||||||
|
</structure>
|
||||||
|
</dictionary>
|
||||||
|
</dictionaries>
|
Loading…
Reference in New Issue
Block a user