Cleanup semantics of data types [#CLICKHOUSE-2].

This commit is contained in:
Alexey Milovidov 2017-12-09 10:32:32 +03:00 committed by alexey-milovidov
parent d2cfda4d9b
commit 7f6c48d5de
21 changed files with 91 additions and 177 deletions

View File

@ -196,7 +196,7 @@ public:
if (!arguments.front()->isNullable())
throw Exception("Logical error: not nullable data type is passed to AggregateFunctionNullUnary", ErrorCodes::LOGICAL_ERROR);
this->nested_function->setArguments({static_cast<const DataTypeNullable &>(*arguments.front()).getNestedType()});
this->nested_function->setArguments({removeNullable(arguments.front())});
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override
@ -246,15 +246,7 @@ public:
for (size_t i = 0; i < number_of_arguments; ++i)
{
is_nullable[i] = arguments[i]->isNullable();
if (is_nullable[i])
{
const DataTypeNullable & nullable_type = static_cast<const DataTypeNullable &>(*arguments[i]);
const DataTypePtr & nested_type = nullable_type.getNestedType();
nested_args[i] = nested_type;
}
else
nested_args[i] = arguments[i];
nested_args[i] = removeNullable(arguments[i]);
}
this->nested_function->setArguments(nested_args);

View File

@ -136,8 +136,8 @@ public:
ColumnVector(const size_t n) : data{n} {}
ColumnVector(const size_t n, const value_type x) : data{n, x} {}
bool isNumeric() const override { return IsNumber<T>::value; }
bool isFixed() const override { return IsNumber<T>::value; }
bool isNumeric() const override { return IsNumber<T>; }
bool isFixed() const override { return IsNumber<T>; }
size_t sizeOfField() const override { return sizeof(T); }

View File

@ -42,21 +42,6 @@ public:
/// Name of a Column kind, without parameters (example: FixedString, Array).
virtual const char * getFamilyName() const = 0;
/// Column is vector of numbers or numeric constant.
virtual bool isNumeric() const { return false; }
/// Is this column numeric and not nullable?
virtual bool isNumericNotNullable() const { return isNumeric(); }
/// Column stores a constant value.
virtual bool isConst() const { return false; }
/// Is this column a container for nullable values?
virtual bool isNullable() const { return false; }
/// Is this a null column?
virtual bool isNull() const { return false; }
/** If column isn't constant, returns nullptr (or itself).
* If column is constant, transforms constant to full column (if column type allows such tranform) and return it.
* Special case:
@ -65,12 +50,6 @@ public:
*/
virtual ColumnPtr convertToFullColumnIfConst() const { return {}; }
/// Values in column have equal size in memory.
virtual bool isFixed() const { return false; }
/// If column isFixed(), returns size of value.
virtual size_t sizeOfField() const { throw Exception("Cannot get sizeOfField() for column " + getName(), ErrorCodes::CANNOT_GET_SIZE_OF_FIELD); }
/// Creates the same column with the same data.
virtual ColumnPtr clone() const { return cut(0, size()); }
@ -271,6 +250,31 @@ public:
using ColumnCallback = std::function<void(ColumnPtr&)>;
virtual void forEachSubcolumn(ColumnCallback) {}
/// Various properties on behaviour of column type.
/// Column is vector of numbers or numeric constant.
virtual bool isNumeric() const { return false; }
/// Is this column numeric and not nullable?
virtual bool isNumericNotNullable() const { return isNumeric(); }
/// Column stores a constant value.
virtual bool isConst() const { return false; }
/// Is this column a container for nullable values?
virtual bool isNullable() const { return false; }
/// Is this a null column?
virtual bool isNull() const { return false; }
/// Values in column have equal size in memory.
virtual bool isFixed() const { return false; }
/// If column isFixed(), returns size of value.
virtual size_t sizeOfField() const { throw Exception("Cannot get sizeOfField() for column " + getName(), ErrorCodes::CANNOT_GET_SIZE_OF_FIELD); }
virtual ~IColumn() {}
String dumpStructure() const;

View File

@ -66,7 +66,7 @@ template <typename T> bool inline operator> (T a, const UInt128 b) { return UIn
template <typename T> bool inline operator<= (T a, const UInt128 b) { return UInt128(a) <= b; }
template <typename T> bool inline operator< (T a, const UInt128 b) { return UInt128(a) < b; }
template <> struct IsNumber<UInt128> { static constexpr bool value = true; };
template <> bool IsNumber<UInt128> = true;
template <> struct TypeName<UInt128> { static const char * get() { return "UInt128"; } };
struct UInt128Hash

View File

@ -3,16 +3,14 @@
#include <string>
#include <vector>
#include <Poco/Types.h>
#include <common/strong_typedef.h>
namespace DB
{
/** Data types for representing values from a database in RAM.
*/
/// Data types for representing elementary values from a database in RAM.
STRONG_TYPEDEF(char, Null);
struct Null {};
using UInt8 = Poco::UInt8;
using UInt16 = Poco::UInt16;
@ -28,39 +26,24 @@ using Float32 = float;
using Float64 = double;
using String = std::string;
using Strings = std::vector<String>;
/// Ordinary types with nullability.
template <typename T> struct Nullable { using Type = T; };
/// Get a non-nullable type.
template <typename T> struct RemoveNullable { using Type = T; };
template <typename T> struct RemoveNullable<Nullable<T>> { using Type = T; };
/// Check if a type is nullable.
template <typename T> struct IsNullable { static constexpr bool value = false; };
template <typename T> struct IsNullable<Nullable<T>> { static constexpr bool value = true; };
template <typename T> struct IsNumber { static constexpr bool value = false; };
template <typename T> struct IsNumber<Nullable<T>> { static constexpr bool value = IsNumber<T>::value; };
template <> struct IsNumber<UInt8> { static constexpr bool value = true; };
template <> struct IsNumber<UInt16> { static constexpr bool value = true; };
template <> struct IsNumber<UInt32> { static constexpr bool value = true; };
template <> struct IsNumber<UInt64> { static constexpr bool value = true; };
template <> struct IsNumber<Int8> { static constexpr bool value = true; };
template <> struct IsNumber<Int16> { static constexpr bool value = true; };
template <> struct IsNumber<Int32> { static constexpr bool value = true; };
template <> struct IsNumber<Int64> { static constexpr bool value = true; };
template <> struct IsNumber<Float32> { static constexpr bool value = true; };
template <> struct IsNumber<Float64> { static constexpr bool value = true; };
/** Note that for types not used in DB, IsNumber is false.
*/
template <typename T> bool IsNumber = false;
template <> bool IsNumber<UInt8> = true;
template <> bool IsNumber<UInt16> = true;
template <> bool IsNumber<UInt32> = true;
template <> bool IsNumber<UInt64> = true;
template <> bool IsNumber<Int8> = true;
template <> bool IsNumber<Int16> = true;
template <> bool IsNumber<Int32> = true;
template <> bool IsNumber<Int64> = true;
template <> bool IsNumber<Float32> = true;
template <> bool IsNumber<Float64> = true;
template <typename T> struct TypeName;
template <typename T> struct TypeName<Nullable<T>> { static const char * get() { return "Nullable"; } };
template <> struct TypeName<Null> { static const char * get() { return "Null"; } };
template <> struct TypeName<Nullable<void>> : TypeName<Null> {};
template <> struct TypeName<UInt8> { static const char * get() { return "UInt8"; } };
template <> struct TypeName<UInt16> { static const char * get() { return "UInt16"; } };
@ -75,4 +58,7 @@ template <> struct TypeName<Float64> { static const char * get() { return "Float
template <> struct TypeName<String> { static const char * get() { return "String"; } };
/// Not a data type in database, defined just for convenience.
using Strings = std::vector<String>;
}

View File

@ -58,7 +58,7 @@ void AddingDefaultBlockOutputStream::write(const DB::Block & block)
if (offset_columns.count(offsets_name))
{
ColumnPtr offsets_column = offset_columns[offsets_name];
DataTypePtr nested_type = typeid_cast<DataTypeArray &>(*column_to_add.type).getNestedType();
DataTypePtr nested_type = typeid_cast<const DataTypeArray &>(*column_to_add.type).getNestedType();
UInt64 nested_rows = rows ? get<UInt64>((*offsets_column)[rows - 1]) : 0;
ColumnPtr nested_column = nested_type->createConstColumn(nested_rows, nested_type->getDefault())->convertToFullColumnIfConst();

View File

@ -8,13 +8,6 @@
namespace DB
{
static DataTypePtr removeNullable(const DataTypePtr & type)
{
if (type->isNullable())
return typeid_cast<DataTypeNullable *>(type.get())->getNestedType();
return type;
}
bool isConvertableTypes(const DataTypePtr & from, const DataTypePtr & to)
{
auto from_nn = removeNullable(from);

View File

@ -303,4 +303,19 @@ void registerDataTypeNullable(DataTypeFactory & factory)
factory.registerDataType("Nullable", create);
}
DataTypePtr makeNullable(const DataTypePtr & type)
{
if (type->isNullable())
return type;
return std::make_shared<DataTypeNullable>(type);
}
DataTypePtr removeNullable(const DataTypePtr & type)
{
if (type->isNullable())
return static_cast<const DataTypeNullable &>(*type).getNestedType();
return type;
}
}

View File

@ -86,4 +86,8 @@ private:
DataTypePtr nested_data_type;
};
DataTypePtr makeNullable(const DataTypePtr & type);
DataTypePtr removeNullable(const DataTypePtr & type);
}

View File

@ -1,17 +0,0 @@
#include <DataTypes/DataTypeTraits.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB { namespace DataTypeTraits {
const DataTypePtr & removeNullable(const DataTypePtr & type)
{
if (type->isNullable())
{
const auto & nullable_type = static_cast<const DataTypeNullable &>(*type);
return nullable_type.getNestedType();
}
else
return type;
}
}}

View File

@ -1,18 +0,0 @@
#pragma once
#include <DataTypes/IDataType.h>
namespace DB
{
namespace DataTypeTraits
{
/// If the input type is nullable, return its nested type.
/// Otherwise it is an identity mapping.
const DataTypePtr & removeNullable(const DataTypePtr & type);
}
}

View File

@ -67,8 +67,8 @@ add_headers_and_sources(clickhouse_functions .)
add_headers_and_sources(clickhouse_functions ./Conditional)
add_headers_and_sources(clickhouse_functions ${ClickHouse_BINARY_DIR}/dbms/src/Functions)
list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp DataTypeTraits.cpp FunctionHelpers.cpp)
list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h DataTypeTraits.h FunctionHelpers.h)
list(REMOVE_ITEM clickhouse_functions_sources IFunction.cpp FunctionFactory.cpp FunctionHelpers.cpp)
list(REMOVE_ITEM clickhouse_functions_headers IFunction.h FunctionFactory.h FunctionHelpers.h)
add_library(clickhouse_functions ${clickhouse_functions_sources})
target_link_libraries(clickhouse_functions dbms)

View File

@ -484,15 +484,6 @@ template <> struct IsIntegral<DataTypeInt16> { static constexpr auto value = tru
template <> struct IsIntegral<DataTypeInt32> { static constexpr auto value = true; };
template <> struct IsIntegral<DataTypeInt64> { static constexpr auto value = true; };
template <typename DataType> struct IsFloating { static constexpr auto value = false; };
template <> struct IsFloating<DataTypeFloat32> { static constexpr auto value = true; };
template <> struct IsFloating<DataTypeFloat64> { static constexpr auto value = true; };
template <typename DataType> struct IsNumeric
{
static constexpr auto value = IsIntegral<DataType>::value || IsFloating<DataType>::value;
};
template <typename DataType> struct IsDateOrDateTime { static constexpr auto value = false; };
template <> struct IsDateOrDateTime<DataTypeDate> { static constexpr auto value = true; };
template <> struct IsDateOrDateTime<DataTypeDateTime> { static constexpr auto value = true; };

View File

@ -15,7 +15,6 @@
#include <Columns/ColumnNullable.h>
#include <Functions/IFunction.h>
#include <DataTypes/DataTypeTraits.h>
#include <Functions/ObjectPool.h>
#include <Functions/FunctionHelpers.h>
#include <Common/StringUtils.h>
@ -1024,8 +1023,8 @@ public:
if (!arguments[1]->onlyNull())
{
const IDataType * observed_type0 = DataTypeTraits::removeNullable(array_type->getNestedType()).get();
const IDataType * observed_type1 = DataTypeTraits::removeNullable(arguments[1]).get();
DataTypePtr observed_type0 = removeNullable(array_type->getNestedType());
DataTypePtr observed_type1 = removeNullable(arguments[1]);
if (!(observed_type0->isNumber() && observed_type1->isNumber())
&& !observed_type0->equals(*observed_type1))

View File

@ -216,7 +216,7 @@ DataTypePtr FunctionMultiIf::getReturnTypeImpl(const DataTypes & args) const
DataTypePtr common_type_of_branches = getLeastCommonType(types_of_branches);
return have_nullable_condition
? makeNullableDataTypeIfNot(common_type_of_branches)
? makeNullable(common_type_of_branches)
: common_type_of_branches;
}

View File

@ -110,15 +110,6 @@ public:
};
inline const DataTypePtr makeNullableDataTypeIfNot(const DataTypePtr & type)
{
if (type->isNullable())
return type;
return std::make_shared<DataTypeNullable>(type);
}
class FunctionIf : public IFunction
{
public:
@ -636,14 +627,6 @@ private:
return column;
}
static const DataTypePtr getNestedDataType(const DataTypePtr & type)
{
if (type->isNullable())
return static_cast<const DataTypeNullable &>(*type).getNestedType();
return type;
}
bool executeForNullableThenElse(Block & block, const ColumnNumbers & arguments, size_t result)
{
const ColumnWithTypeAndName & arg_cond = block.getByPosition(arguments[0]);
@ -698,17 +681,17 @@ private:
arg_cond,
{
getNestedColumn(arg_then.column),
getNestedDataType(arg_then.type),
removeNullable(arg_then.type),
""
},
{
getNestedColumn(arg_else.column),
getNestedDataType(arg_else.type),
removeNullable(arg_else.type),
""
},
{
nullptr,
getNestedDataType(block.getByPosition(result).type),
removeNullable(block.getByPosition(result).type),
""
}
});
@ -834,8 +817,8 @@ public:
return arguments[0];
if (arguments[0]->isNullable())
return makeNullableDataTypeIfNot(getReturnTypeImpl({
getNestedDataType(arguments[0]), arguments[1], arguments[2]}));
return makeNullable(getReturnTypeImpl({
removeNullable(arguments[0]), arguments[1], arguments[2]}));
if (!checkDataType<DataTypeUInt8>(arguments[0].get()))
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument (condition) of function if. Must be UInt8.",

View File

@ -98,14 +98,6 @@ void FunctionIsNotNull::executeImpl(Block & block, const ColumnNumbers & argumen
/// Implementation of coalesce.
static const DataTypePtr getNestedDataType(const DataTypePtr & type)
{
if (type->isNullable())
return static_cast<const DataTypeNullable &>(*type).getNestedType();
return type;
}
FunctionPtr FunctionCoalesce::create(const Context & context)
{
return std::make_shared<FunctionCoalesce>(context);
@ -144,7 +136,7 @@ DataTypePtr FunctionCoalesce::getReturnTypeImpl(const DataTypes & arguments) con
else
{
new_args.push_back(std::make_shared<DataTypeUInt8>());
new_args.push_back(getNestedDataType(filtered_args[i]));
new_args.push_back(removeNullable(filtered_args[i]));
}
}
@ -157,7 +149,7 @@ DataTypePtr FunctionCoalesce::getReturnTypeImpl(const DataTypes & arguments) con
/// if last argument is not nullable, result should be also not nullable
if (!new_args.back()->isNullable() && res->isNullable())
res = getNestedDataType(res);
res = removeNullable(res);
return res;
}
@ -202,7 +194,7 @@ void FunctionCoalesce::executeImpl(Block & block, const ColumnNumbers & argument
{
temp_block.insert({nullptr, std::make_shared<DataTypeUInt8>(), ""});
is_not_null.execute(temp_block, {filtered_args[i]}, res_pos);
temp_block.insert({nullptr, getNestedDataType(block.getByPosition(filtered_args[i]).type), ""});
temp_block.insert({nullptr, removeNullable(block.getByPosition(filtered_args[i]).type), ""});
assume_not_null.execute(temp_block, {filtered_args[i]}, res_pos + 1);
multi_if_args.push_back(res_pos);
@ -254,7 +246,7 @@ DataTypePtr FunctionIfNull::getReturnTypeImpl(const DataTypes & arguments) const
if (!arguments[0]->isNullable())
return arguments[0];
return FunctionIf{}.getReturnTypeImpl({std::make_shared<DataTypeUInt8>(), getNestedDataType(arguments[0]), arguments[1]});
return FunctionIf{}.getReturnTypeImpl({std::make_shared<DataTypeUInt8>(), removeNullable(arguments[0]), arguments[1]});
}
void FunctionIfNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
@ -280,7 +272,7 @@ void FunctionIfNull::executeImpl(Block & block, const ColumnNumbers & arguments,
size_t is_not_null_pos = temp_block.columns();
temp_block.insert({nullptr, std::make_shared<DataTypeUInt8>(), ""});
size_t assume_not_null_pos = temp_block.columns();
temp_block.insert({nullptr, getNestedDataType(block.getByPosition(arguments[0]).type), ""});
temp_block.insert({nullptr, removeNullable(block.getByPosition(arguments[0]).type), ""});
FunctionIsNotNull{}.execute(temp_block, {arguments[0]}, is_not_null_pos);
FunctionAssumeNotNull{}.execute(temp_block, {arguments[0]}, assume_not_null_pos);
@ -308,7 +300,7 @@ std::string FunctionNullIf::getName() const
DataTypePtr FunctionNullIf::getReturnTypeImpl(const DataTypes & arguments) const
{
return FunctionIf{}.getReturnTypeImpl({std::make_shared<DataTypeUInt8>(), makeNullableDataTypeIfNot(arguments[0]), arguments[0]});
return FunctionIf{}.getReturnTypeImpl({std::make_shared<DataTypeUInt8>(), makeNullable(arguments[0]), arguments[0]});
}
void FunctionNullIf::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)
@ -352,7 +344,7 @@ std::string FunctionAssumeNotNull::getName() const
DataTypePtr FunctionAssumeNotNull::getReturnTypeImpl(const DataTypes & arguments) const
{
return getNestedDataType(arguments[0]);
return removeNullable(arguments[0]);
}
void FunctionAssumeNotNull::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result)

View File

@ -18,8 +18,6 @@
#include <Core/TypeListNumber.h>
#include <Common/FieldVisitors.h>
#include <DataTypes/DataTypeTraits.h>
/** These methods are intended for implementation of functions, that
* copy ranges from one or more columns to another column.
*

View File

@ -862,14 +862,8 @@ void Join::checkTypesOfKeys(const Block & block_left, const Block & block_right)
{
/// Compare up to Nullability.
IDataType * left_type = block_left.getByName(key_names_left[i]).type.get();
IDataType * right_type = block_right.getByName(key_names_right[i]).type.get();
if (left_type->isNullable())
left_type = static_cast<const DataTypeNullable &>(*left_type).getNestedType().get();
if (right_type->isNullable())
right_type = static_cast<const DataTypeNullable &>(*right_type).getNestedType().get();
DataTypePtr left_type = removeNullable(block_left.getByName(key_names_left[i]).type);
DataTypePtr right_type = removeNullable(block_right.getByName(key_names_right[i]).type);
if (!left_type->equals(*right_type))
throw Exception("Type mismatch of columns to JOIN by: "

View File

@ -11,7 +11,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeTraits.h>
#include <DataTypes/DataTypeNullable.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
@ -367,8 +367,7 @@ ColumnPtr Set::execute(const Block & block, bool negative) const
{
key_columns.push_back(block.safeGetByPosition(i).column.get());
if (DataTypeTraits::removeNullable(data_types[i])->getName() !=
DataTypeTraits::removeNullable(block.safeGetByPosition(i).type)->getName())
if (removeNullable(data_types[i])->equals(*removeNullable(block.safeGetByPosition(i).type)))
throw Exception("Types of column " + toString(i + 1) + " in section IN don't match: "
+ data_types[i]->getName() + " on the right, " + block.safeGetByPosition(i).type->getName() +
" on the left.", ErrorCodes::TYPE_MISMATCH);

View File

@ -12,7 +12,6 @@
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTraits.h>
#include <Core/AccurateComparison.h>
#include <Common/FieldVisitors.h>
@ -166,14 +165,14 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type)
{
if (src.getType() == Field::Types::Array)
{
const IDataType & nested_type = *DataTypeTraits::removeNullable(type_array->getNestedType());
const DataTypePtr nested_type = removeNullable(type_array->getNestedType());
const Array & src_arr = src.get<Array>();
size_t src_arr_size = src_arr.size();
Array res(src_arr_size);
for (size_t i = 0; i < src_arr_size; ++i)
res[i] = convertFieldToType(src_arr[i], nested_type);
res[i] = convertFieldToType(src_arr[i], *nested_type);
return res;
}