mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-25 17:12:03 +00:00
Merge remote-tracking branch 'upstream/master' into fix14
This commit is contained in:
commit
a1b4605314
@ -67,7 +67,9 @@ option (USE_STATIC_LIBRARIES "Set to FALSE to use shared libraries" ON)
|
||||
option (MAKE_STATIC_LIBRARIES "Set to FALSE to make shared libraries" ${USE_STATIC_LIBRARIES})
|
||||
if (NOT MAKE_STATIC_LIBRARIES)
|
||||
option (SPLIT_SHARED_LIBRARIES "DEV ONLY. Keep all internal libs as separate .so for faster linking" OFF)
|
||||
option (CLICKHOUSE_SPLIT_BINARY "Make several binaries instead one bundled (clickhouse-server, clickhouse-client, ... )" OFF)
|
||||
endif ()
|
||||
|
||||
if (SPLIT_SHARED_LIBRARIES)
|
||||
set (SPLIT_SHARED SHARED)
|
||||
endif ()
|
||||
|
@ -35,7 +35,7 @@ public:
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->isNumeric())
|
||||
if (!argument->isNumber())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
@ -52,8 +52,9 @@ public:
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->behavesAsNumber())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
if (!argument->canBeUsedInBitOperations())
|
||||
throw Exception("The type " + argument->getName() + " of argument for aggregate function " + getName()
|
||||
+ " is illegal, because it cannot be used in bitwise operations",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
|
@ -76,7 +76,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
if (arg_type->isNullable())
|
||||
{
|
||||
has_nullable_types = true;
|
||||
if (arg_type->isNull())
|
||||
if (arg_type->onlyNull())
|
||||
{
|
||||
has_null_types = true;
|
||||
break;
|
||||
|
@ -67,7 +67,7 @@ public:
|
||||
|
||||
void setArgumentsImpl(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments.at(1)->behavesAsNumber()) /// TODO filter out floating point types.
|
||||
if (!arguments.at(1)->canBeUsedAsNonNegativeArrayIndex())
|
||||
throw Exception("Second argument of aggregate function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
type = arguments.front();
|
||||
@ -98,6 +98,8 @@ public:
|
||||
if (params.size() == 2)
|
||||
{
|
||||
length_to_resize = applyVisitor(FieldVisitorConvertToNumber<UInt64>(), params[1]);
|
||||
if (length_to_resize > AGGREGATE_FUNCTION_GROUP_ARRAY_INSERT_AT_MAX_SIZE)
|
||||
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -22,23 +22,15 @@ class AggregateFunctionGroupUniqArrayDateTime : public AggregateFunctionGroupUni
|
||||
|
||||
static IAggregateFunction * createWithExtraTypes(const IDataType & argument_type)
|
||||
{
|
||||
if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionGroupUniqArrayDate;
|
||||
else if (typeid_cast<const DataTypeDateTime *>(&argument_type)) return new AggregateFunctionGroupUniqArrayDateTime;
|
||||
if (typeid_cast<const DataTypeDate *>(&argument_type)) return new AggregateFunctionGroupUniqArrayDate;
|
||||
else if (typeid_cast<const DataTypeDateTime *>(&argument_type)) return new AggregateFunctionGroupUniqArrayDateTime;
|
||||
else
|
||||
{
|
||||
/// Check that we can use plain version of AggreagteFunctionGroupUniqArrayGeneric
|
||||
if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
if (argument_type.isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||
return new AggreagteFunctionGroupUniqArrayGeneric<true>;
|
||||
|
||||
auto * array_type = typeid_cast<const DataTypeArray *>(&argument_type);
|
||||
if (array_type)
|
||||
{
|
||||
auto nested_type = array_type->getNestedType();
|
||||
if (nested_type->isNumeric() || typeid_cast<DataTypeFixedString *>(nested_type.get()))
|
||||
return new AggreagteFunctionGroupUniqArrayGeneric<true>;
|
||||
}
|
||||
|
||||
return new AggreagteFunctionGroupUniqArrayGeneric<false>;
|
||||
else
|
||||
return new AggreagteFunctionGroupUniqArrayGeneric<false>;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -85,7 +85,7 @@ public:
|
||||
DataTypePtr getReturnType() const override
|
||||
{
|
||||
return result_is_nullable
|
||||
? std::make_shared<DataTypeNullable>(nested_function->getReturnType())
|
||||
? makeNullable(nested_function->getReturnType())
|
||||
: nested_function->getReturnType();
|
||||
}
|
||||
|
||||
@ -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);
|
||||
|
@ -58,10 +58,10 @@ public:
|
||||
{
|
||||
type = returns_float ? std::make_shared<DataTypeFloat64>() : arguments[0];
|
||||
|
||||
if (!arguments[1]->isNumeric())
|
||||
if (!arguments[1]->isNumber() && !arguments[1]->isDateOrDateTime())
|
||||
throw Exception{
|
||||
"Invalid type of second argument to function " + getName() +
|
||||
", got " + arguments[1]->getName() + ", expected numeric",
|
||||
", got " + arguments[1]->getName() + ", expected numeric or Date or DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
@ -139,10 +139,10 @@ public:
|
||||
{
|
||||
type = returns_float ? std::make_shared<DataTypeFloat64>() : arguments[0];
|
||||
|
||||
if (!arguments[1]->isNumeric())
|
||||
if (!arguments[1]->isNumber() && !arguments[1]->isDateOrDateTime())
|
||||
throw Exception{
|
||||
"Invalid type of second argument to function " + getName() +
|
||||
", got " + arguments[1]->getName() + ", expected numeric",
|
||||
", got " + arguments[1]->getName() + ", expected numeric or Date or DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
|
@ -124,7 +124,7 @@ public:
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->behavesAsNumber())
|
||||
if (!argument->isNumber())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -381,11 +381,11 @@ public:
|
||||
|
||||
void setArgumentsImpl(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->behavesAsNumber())
|
||||
if (!arguments[0]->isNumber())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument to function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->behavesAsNumber())
|
||||
if (!arguments[1]->isNumber())
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument to function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
@ -33,7 +33,7 @@ public:
|
||||
|
||||
void setArgument(const DataTypePtr & argument)
|
||||
{
|
||||
if (!argument->behavesAsNumber())
|
||||
if (!argument->isSummable())
|
||||
throw Exception("Illegal type " + argument->getName() + " of argument for aggregate function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
@ -27,18 +27,10 @@ static IAggregateFunction * createWithExtraTypes(const IDataType & argument_type
|
||||
else
|
||||
{
|
||||
/// Check that we can use plain version of AggregateFunctionTopKGeneric
|
||||
if (typeid_cast<const DataTypeString*>(&argument_type) || typeid_cast<const DataTypeFixedString*>(&argument_type))
|
||||
if (argument_type.isValueUnambiguouslyRepresentedInContiguousMemoryRegion())
|
||||
return new AggregateFunctionTopKGeneric<true>;
|
||||
|
||||
auto * array_type = typeid_cast<const DataTypeArray *>(&argument_type);
|
||||
if (array_type)
|
||||
{
|
||||
auto nested_type = array_type->getNestedType();
|
||||
if (nested_type->isNumeric() || typeid_cast<DataTypeFixedString *>(nested_type.get()))
|
||||
return new AggregateFunctionTopKGeneric<true>;
|
||||
}
|
||||
|
||||
return new AggregateFunctionTopKGeneric<false>;
|
||||
else
|
||||
return new AggregateFunctionTopKGeneric<false>;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -72,6 +72,10 @@ public:
|
||||
throw Exception("Too large parameter for aggregate function " + getName() + ". Maximum: " + toString(TOP_K_MAX_SIZE),
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
if (k == 0)
|
||||
throw Exception("Parameter 0 is illegal for aggregate function " + getName(),
|
||||
ErrorCodes::ARGUMENT_OUT_OF_BOUND);
|
||||
|
||||
threshold = k;
|
||||
reserved = TOP_K_LOAD_FACTOR * k;
|
||||
}
|
||||
|
@ -54,7 +54,7 @@ void AnalyzeResultOfQuery::process(ASTPtr & ast, const Context & context, Execut
|
||||
const TypeAndConstantInference::ExpressionInfo & info = it->second;
|
||||
|
||||
result.insert(ColumnWithTypeAndName(
|
||||
info.is_constant_expression ? info.data_type->createConstColumn(1, info.value) : nullptr,
|
||||
info.is_constant_expression ? info.data_type->createColumnConst(1, info.value) : nullptr,
|
||||
info.data_type,
|
||||
std::move(name)));
|
||||
}
|
||||
|
@ -39,7 +39,7 @@ static bool isInjectiveFunction(
|
||||
const TypeAndConstantInference::ExpressionInfo & child_info = all_info.at(child_name);
|
||||
|
||||
block_with_constants.insert(ColumnWithTypeAndName(
|
||||
child_info.is_constant_expression ? child_info.data_type->createConstColumn(1, child_info.value) : nullptr,
|
||||
child_info.is_constant_expression ? child_info.data_type->createColumnConst(1, child_info.value) : nullptr,
|
||||
child_info.data_type,
|
||||
child_name));
|
||||
}
|
||||
|
@ -44,7 +44,7 @@ namespace
|
||||
|
||||
Field getValueFromConstantColumn(const ColumnPtr & column)
|
||||
{
|
||||
if (!column->isConst())
|
||||
if (!column->isColumnConst())
|
||||
throw Exception("Logical error: expected that column is constant", ErrorCodes::LOGICAL_ERROR);
|
||||
if (column->size() != 1)
|
||||
throw Exception("Logical error: expected that column with constant has single element", ErrorCodes::LOGICAL_ERROR);
|
||||
@ -221,7 +221,7 @@ void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantIn
|
||||
String child_name = child->getColumnName();
|
||||
const TypeAndConstantInference::ExpressionInfo & child_info = info.at(child_name);
|
||||
columns_for_analysis.emplace_back(
|
||||
child_info.is_constant_expression ? child_info.data_type->createConstColumn(1, child_info.value) : nullptr,
|
||||
child_info.is_constant_expression ? child_info.data_type->createColumnConst(1, child_info.value) : nullptr,
|
||||
child_info.data_type,
|
||||
child_name);
|
||||
|
||||
@ -249,7 +249,7 @@ void processFunction(const String & column_name, ASTPtr & ast, TypeAndConstantIn
|
||||
function_ptr->execute(block_with_constants, argument_numbers, result_position);
|
||||
|
||||
const auto & result_column = block_with_constants.getByPosition(result_position).column;
|
||||
if (result_column->isConst())
|
||||
if (result_column->isColumnConst())
|
||||
{
|
||||
expression_info.is_constant_expression = true;
|
||||
expression_info.value = (*result_column)[0];
|
||||
|
@ -77,6 +77,10 @@ int Collator::compare(const char * str1, size_t length1, const char * str2, size
|
||||
*/
|
||||
return compare_result;
|
||||
#else
|
||||
(void)str1;
|
||||
(void)length1;
|
||||
(void)str2;
|
||||
(void)length2;
|
||||
return 0;
|
||||
#endif
|
||||
}
|
||||
|
@ -99,8 +99,6 @@ public:
|
||||
std::string getName() const override { return "AggregateFunction(" + func->getName() + ")"; }
|
||||
const char * getFamilyName() const override { return "AggregateFunction"; }
|
||||
|
||||
size_t sizeOfField() const override { return sizeof(getData()[0]); }
|
||||
|
||||
size_t size() const override
|
||||
{
|
||||
return getData().size();
|
||||
|
@ -45,7 +45,7 @@ ColumnArray::ColumnArray(ColumnPtr nested_column, ColumnPtr offsets_column)
|
||||
}
|
||||
|
||||
/** NOTE
|
||||
* Arrays with constant value are possible and used in implementation of higher order functions and in ARRAY JOIN.
|
||||
* Arrays with constant value are possible and used in implementation of higher order functions (see FunctionReplicate).
|
||||
* But in most cases, arrays with constant value are unexpected and code will work wrong. Use with caution.
|
||||
*/
|
||||
}
|
||||
@ -152,10 +152,10 @@ void ColumnArray::insertData(const char * pos, size_t length)
|
||||
/** Similarly - only for arrays of fixed length values.
|
||||
*/
|
||||
IColumn * data_ = data.get();
|
||||
if (!data_->isFixed())
|
||||
if (!data_->isFixedAndContiguous())
|
||||
throw Exception("Method insertData is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
size_t field_size = data_->sizeOfField();
|
||||
size_t field_size = data_->sizeOfValueIfFixed();
|
||||
|
||||
const char * end = pos + length;
|
||||
size_t elems = 0;
|
||||
@ -321,19 +321,13 @@ bool ColumnArray::hasEqualOffsets(const ColumnArray & other) const
|
||||
ColumnPtr ColumnArray::convertToFullColumnIfConst() const
|
||||
{
|
||||
ColumnPtr new_data;
|
||||
ColumnPtr new_offsets;
|
||||
|
||||
if (auto full_column = getData().convertToFullColumnIfConst())
|
||||
new_data = full_column;
|
||||
else
|
||||
new_data = data;
|
||||
|
||||
if (auto full_column = offsets->convertToFullColumnIfConst())
|
||||
new_offsets = full_column;
|
||||
else
|
||||
new_offsets = offsets;
|
||||
|
||||
return std::make_shared<ColumnArray>(new_data, new_offsets);
|
||||
return std::make_shared<ColumnArray>(new_data, offsets);
|
||||
}
|
||||
|
||||
|
||||
|
@ -22,12 +22,6 @@ ColumnConst::ColumnConst(ColumnPtr data_, size_t s)
|
||||
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
|
||||
}
|
||||
|
||||
bool ColumnConst::isNull() const
|
||||
{
|
||||
const ColumnNullable * column_nullable = typeid_cast<const ColumnNullable *>(data.get());
|
||||
return column_nullable && column_nullable->isNullAt(0);
|
||||
}
|
||||
|
||||
ColumnPtr ColumnConst::convertToFullColumn() const
|
||||
{
|
||||
return data->replicate(Offsets_t(1, s));
|
||||
|
@ -28,11 +28,6 @@ private:
|
||||
public:
|
||||
ColumnConst(ColumnPtr data, size_t s);
|
||||
|
||||
bool isConst() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
ColumnPtr convertToFullColumn() const;
|
||||
|
||||
ColumnPtr convertToFullColumnIfConst() const override
|
||||
@ -50,33 +45,6 @@ public:
|
||||
return "Const";
|
||||
}
|
||||
|
||||
bool isNumeric() const override
|
||||
{
|
||||
return data->isNumeric();
|
||||
}
|
||||
|
||||
bool isNumericNotNullable() const override
|
||||
{
|
||||
return data->isNumericNotNullable();
|
||||
}
|
||||
|
||||
bool isNullable() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
bool isNull() const override;
|
||||
|
||||
bool isFixed() const override
|
||||
{
|
||||
return data->isFixed();
|
||||
}
|
||||
|
||||
size_t sizeOfField() const override
|
||||
{
|
||||
return data->sizeOfField();
|
||||
}
|
||||
|
||||
ColumnPtr cloneResized(size_t new_size) const override
|
||||
{
|
||||
return std::make_shared<ColumnConst>(data, new_size);
|
||||
@ -122,6 +90,11 @@ public:
|
||||
return data->getInt(0);
|
||||
}
|
||||
|
||||
bool isNullAt(size_t) const override
|
||||
{
|
||||
return data->isNullAt(0);
|
||||
}
|
||||
|
||||
void insertRangeFrom(const IColumn &, size_t /*start*/, size_t length) override
|
||||
{
|
||||
s += length;
|
||||
@ -253,6 +226,12 @@ public:
|
||||
callback(data);
|
||||
}
|
||||
|
||||
bool onlyNull() const override { return data->isNullAt(0); }
|
||||
bool isColumnConst() const override { return true; }
|
||||
bool isNumeric() const override { return data->isNumeric(); }
|
||||
bool isFixedAndContiguous() const override { return data->isFixedAndContiguous(); }
|
||||
bool valuesHaveFixedSize() const override { return data->valuesHaveFixedSize(); }
|
||||
size_t sizeOfValueIfFixed() const override { return data->sizeOfValueIfFixed(); }
|
||||
|
||||
/// Not part of the common interface.
|
||||
|
||||
|
@ -43,16 +43,6 @@ public:
|
||||
return chars.size() / n;
|
||||
}
|
||||
|
||||
size_t sizeOfField() const override
|
||||
{
|
||||
return n;
|
||||
}
|
||||
|
||||
bool isFixed() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
size_t byteSize() const override
|
||||
{
|
||||
return chars.size() + sizeof(n);
|
||||
@ -131,6 +121,12 @@ public:
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool isFixedAndContiguous() const override { return true; }
|
||||
size_t sizeOfValueIfFixed() const override { return n; }
|
||||
|
||||
|
||||
/// Specialized part of interface, not from IColumn.
|
||||
|
||||
Chars_t & getChars() { return chars; }
|
||||
|
@ -13,6 +13,8 @@ public:
|
||||
|
||||
const char * getFamilyName() const override { return "Nothing"; }
|
||||
ColumnPtr cloneDummy(size_t s) const override { return std::make_shared<ColumnNothing>(s); };
|
||||
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -3,9 +3,7 @@
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnArray.h>
|
||||
#include <Columns/ColumnTuple.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
#include <DataStreams/ColumnGathererStream.h>
|
||||
|
||||
|
||||
@ -23,46 +21,18 @@ namespace ErrorCodes
|
||||
ColumnNullable::ColumnNullable(ColumnPtr nested_column_, ColumnPtr null_map_)
|
||||
: nested_column{nested_column_}, null_map{null_map_}
|
||||
{
|
||||
if (nested_column->isNullable())
|
||||
throw Exception{"A nullable column cannot contain another nullable column", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
/// TODO Also check for Nullable(Array(...)). But they are occasionally used somewhere in tests.
|
||||
|
||||
if (typeid_cast<const ColumnTuple *>(nested_column.get()))
|
||||
throw Exception{"Nullable(Tuple(...)) is illegal", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
if (typeid_cast<const ColumnAggregateFunction *>(nested_column.get()))
|
||||
throw Exception{"Nullable(AggregateFunction(...)) is illegal", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
/// ColumnNullable cannot have constant nested column. But constant argument could be passed. Materialize it.
|
||||
if (auto nested_column_materialized = nested_column->convertToFullColumnIfConst())
|
||||
nested_column = nested_column_materialized;
|
||||
|
||||
if (null_map->isConst())
|
||||
if (!nested_column->canBeInsideNullable())
|
||||
throw Exception{getName() + " cannot be inside Nullable column", ErrorCodes::ILLEGAL_COLUMN};
|
||||
|
||||
if (null_map->isColumnConst())
|
||||
throw Exception{"ColumnNullable cannot have constant null map", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
|
||||
size_t ColumnNullable::sizeOfField() const
|
||||
{
|
||||
if (nested_column->isFixed())
|
||||
return getNullMapConcreteColumn().sizeOfField() + nested_column->sizeOfField();
|
||||
|
||||
throw Exception("Cannot get sizeOfField() for column " + getName(), ErrorCodes::CANNOT_GET_SIZE_OF_FIELD);
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr ColumnNullable::convertToFullColumnIfConst() const
|
||||
{
|
||||
ColumnPtr new_col_holder;
|
||||
|
||||
if (auto full_col = nested_column->convertToFullColumnIfConst())
|
||||
new_col_holder = std::make_shared<ColumnNullable>(full_col, null_map);
|
||||
|
||||
return new_col_holder;
|
||||
}
|
||||
|
||||
|
||||
void ColumnNullable::updateHashWithValue(size_t n, SipHash & hash) const
|
||||
{
|
||||
const auto & arr = getNullMap();
|
||||
@ -466,4 +436,16 @@ void ColumnNullable::checkConsistency() const
|
||||
ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT);
|
||||
}
|
||||
|
||||
|
||||
ColumnPtr makeNullable(const ColumnPtr & column)
|
||||
{
|
||||
if (column->isColumnNullable())
|
||||
return column;
|
||||
|
||||
if (column->isColumnConst())
|
||||
return std::make_shared<ColumnConst>(makeNullable(static_cast<ColumnConst &>(*column).getDataColumnPtr()), column->size());
|
||||
|
||||
return std::make_shared<ColumnNullable>(column, std::make_shared<ColumnUInt8>(column->size(), 0));
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -24,14 +24,9 @@ public:
|
||||
ColumnNullable(ColumnPtr nested_column_, ColumnPtr null_map_);
|
||||
const char * getFamilyName() const override { return "Nullable"; }
|
||||
std::string getName() const override { return "Nullable(" + nested_column->getName() + ")"; }
|
||||
bool isNumeric() const override { return nested_column->isNumeric(); }
|
||||
bool isNumericNotNullable() const override { return false; }
|
||||
bool isFixed() const override { return nested_column->isFixed(); }
|
||||
size_t sizeOfField() const override;
|
||||
bool isNullable() const override { return true; }
|
||||
ColumnPtr cloneResized(size_t size) const override;
|
||||
size_t size() const override { return nested_column->size(); }
|
||||
bool isNullAt(size_t n) const { return static_cast<const ColumnUInt8 &>(*null_map).getData()[n] != 0;}
|
||||
bool isNullAt(size_t n) const override { return static_cast<const ColumnUInt8 &>(*null_map).getData()[n] != 0;}
|
||||
Field operator[](size_t n) const override;
|
||||
void get(size_t n, Field & res) const override;
|
||||
UInt64 get64(size_t n) const override { return nested_column->get64(n); }
|
||||
@ -58,7 +53,6 @@ public:
|
||||
size_t byteSize() const override;
|
||||
size_t allocatedBytes() const override;
|
||||
ColumnPtr replicate(const Offsets_t & replicate_offsets) const override;
|
||||
ColumnPtr convertToFullColumnIfConst() const override;
|
||||
void updateHashWithValue(size_t n, SipHash & hash) const override;
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
@ -75,6 +69,12 @@ public:
|
||||
callback(null_map);
|
||||
}
|
||||
|
||||
bool isColumnNullable() const override { return true; }
|
||||
bool isFixedAndContiguous() const override { return false; }
|
||||
bool valuesHaveFixedSize() const override { return nested_column->valuesHaveFixedSize(); }
|
||||
size_t sizeOfValueIfFixed() const override { return null_map->sizeOfValueIfFixed() + nested_column->sizeOfValueIfFixed(); }
|
||||
|
||||
|
||||
/// Return the column that represents values.
|
||||
ColumnPtr & getNestedColumn() { return nested_column; }
|
||||
const ColumnPtr & getNestedColumn() const { return nested_column; }
|
||||
@ -109,4 +109,7 @@ private:
|
||||
void applyNullMapImpl(const ColumnUInt8 & map);
|
||||
};
|
||||
|
||||
|
||||
ColumnPtr makeNullable(const ColumnPtr & column);
|
||||
|
||||
}
|
||||
|
@ -19,9 +19,6 @@ class ColumnSet final : public IColumnDummy
|
||||
public:
|
||||
ColumnSet(size_t s_, const ConstSetPtr & data_) : IColumnDummy(s_), data(data_) {}
|
||||
|
||||
/// The column is not a constant. Otherwise, the column will be used in calculations in ExpressionActions::prepare, when a set from subquery is not ready yet.
|
||||
bool isConst() const override { return false; }
|
||||
|
||||
const char * getFamilyName() const override { return "Set"; }
|
||||
ColumnPtr cloneDummy(size_t s_) const override { return std::make_shared<ColumnSet>(s_, data); }
|
||||
|
||||
|
@ -244,6 +244,10 @@ public:
|
||||
|
||||
void getExtremes(Field & min, Field & max) const override;
|
||||
|
||||
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
|
||||
Chars_t & getChars() { return chars; }
|
||||
const Chars_t & getChars() const { return chars; }
|
||||
|
||||
|
@ -9,6 +9,7 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int CANNOT_INSERT_VALUE_OF_DIFFERENT_SIZE_INTO_TUPLE;
|
||||
}
|
||||
@ -32,6 +33,9 @@ std::string ColumnTuple::getName() const
|
||||
|
||||
ColumnTuple::ColumnTuple(const Columns & columns) : columns(columns)
|
||||
{
|
||||
for (const auto & column : columns)
|
||||
if (column->isColumnConst())
|
||||
throw Exception{"ColumnTuple cannot have ColumnConst as its element", ErrorCodes::ILLEGAL_COLUMN};
|
||||
}
|
||||
|
||||
ColumnPtr ColumnTuple::cloneEmpty() const
|
||||
@ -280,18 +284,6 @@ size_t ColumnTuple::allocatedBytes() const
|
||||
return res;
|
||||
}
|
||||
|
||||
ColumnPtr ColumnTuple::convertToFullColumnIfConst() const
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
Columns new_columns(columns);
|
||||
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
if (auto converted = columns[i]->convertToFullColumnIfConst())
|
||||
new_columns[i] = converted;
|
||||
|
||||
return std::make_shared<ColumnTuple>(new_columns);
|
||||
}
|
||||
|
||||
void ColumnTuple::getExtremes(Field & min, Field & max) const
|
||||
{
|
||||
const size_t tuple_size = columns.size();
|
||||
|
@ -57,7 +57,6 @@ public:
|
||||
void reserve(size_t n) override;
|
||||
size_t byteSize() const override;
|
||||
size_t allocatedBytes() const override;
|
||||
ColumnPtr convertToFullColumnIfConst() const override;
|
||||
void forEachSubcolumn(ColumnCallback callback) override;
|
||||
|
||||
const Columns & getColumns() const { return columns; }
|
||||
|
@ -136,10 +136,7 @@ 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; }
|
||||
|
||||
size_t sizeOfField() const override { return sizeof(T); }
|
||||
bool isNumeric() const override { return IsNumber<T>; }
|
||||
|
||||
size_t size() const override
|
||||
{
|
||||
@ -253,6 +250,13 @@ public:
|
||||
|
||||
void gather(ColumnGathererStream & gatherer_stream) override;
|
||||
|
||||
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool isFixedAndContiguous() const override { return true; }
|
||||
size_t sizeOfValueIfFixed() const override { return sizeof(T); }
|
||||
|
||||
|
||||
/** More efficient methods of manipulation - to manipulate with data directly. */
|
||||
Container_t & getData()
|
||||
{
|
||||
|
@ -42,35 +42,11 @@ 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:
|
||||
* If column is composed from several other columns (tuple for example), and contains both constant and full columns,
|
||||
* then each constant column is transformed, and final result is returned.
|
||||
*/
|
||||
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()); }
|
||||
|
||||
@ -127,6 +103,8 @@ public:
|
||||
throw Exception("Method getInt is not supported for " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
virtual bool isNullAt(size_t /*n*/) const { return false; }
|
||||
|
||||
/// Removes all elements outside of specified range.
|
||||
/// Is used in LIMIT operation, for example.
|
||||
virtual ColumnPtr cut(size_t start, size_t length) const
|
||||
@ -150,7 +128,7 @@ public:
|
||||
|
||||
/// Appends data located in specified memory chunk if it is possible (throws an exception if it cannot be implemented).
|
||||
/// Is used to optimize some computations (in aggregation, for example).
|
||||
/// Parameter length could be ignored if column isFixed().
|
||||
/// Parameter length could be ignored if column values have fixed size.
|
||||
virtual void insertData(const char * pos, size_t length) = 0;
|
||||
|
||||
/// Like getData, but has special behavior for columns that contain variable-length strings.
|
||||
@ -266,13 +244,73 @@ public:
|
||||
/// Zero, if could be determined.
|
||||
virtual size_t allocatedBytes() const = 0;
|
||||
|
||||
/// If the column contains subcolumns (such as Array, Nullable, etc), enumerate them.
|
||||
/// Shallow: doesn't do recursive calls.
|
||||
/// If the column contains subcolumns (such as Array, Nullable, etc), do callback on them.
|
||||
/// Shallow: doesn't do recursive calls; don't do call for itself.
|
||||
using ColumnCallback = std::function<void(ColumnPtr&)>;
|
||||
virtual void forEachSubcolumn(ColumnCallback) {}
|
||||
|
||||
/** Some columns can contain another columns inside.
|
||||
* So, we have a tree of columns. But not all combinations are possible.
|
||||
* There are the following rules:
|
||||
*
|
||||
* ColumnConst may be only at top. It cannot be inside any column.
|
||||
* ColumnNullable can contain only simple columns.
|
||||
*/
|
||||
|
||||
/// Various properties on behaviour of column type.
|
||||
|
||||
/// Is this column a container for Nullable values? It's true only for ColumnNullable.
|
||||
/// Note that ColumnConst(ColumnNullable(...)) is not considered.
|
||||
virtual bool isColumnNullable() const { return false; }
|
||||
|
||||
/// Column stores a constant value. It's true only for ColumnConst wrapper.
|
||||
virtual bool isColumnConst() const { return false; }
|
||||
|
||||
/// It's a special kind of column, that contain single value, but is not a ColumnConst.
|
||||
virtual bool isDummy() const { return false; }
|
||||
|
||||
/** Memory layout properties.
|
||||
*
|
||||
* Each value of a column can be placed in memory contiguously or not.
|
||||
*
|
||||
* Example: simple columns like UInt64 or FixedString store their values contiguously in single memory buffer.
|
||||
*
|
||||
* Example: Tuple store values of each component in separate subcolumn, so the values of Tuples with at least two components are not contiguous.
|
||||
* Another example is Nullable. Each value have null flag, that is stored separately, so the value is not contiguous in memory.
|
||||
*
|
||||
* There are some important cases, when values are not stored contiguously, but for each value, you can get contiguous memory segment,
|
||||
* that will unambiguously identify the value. In this case, methods getDataAt and insertData are implemented.
|
||||
* Example: String column: bytes of strings are stored concatenated in one memory buffer
|
||||
* and offsets to that buffer are stored in another buffer. The same is for Array of fixed-size contiguous elements.
|
||||
*
|
||||
* To avoid confusion between these cases, we don't have isContiguous method.
|
||||
*/
|
||||
|
||||
/// Values in column have fixed size (including the case when values span many memory segments).
|
||||
virtual bool valuesHaveFixedSize() const { return isFixedAndContiguous(); }
|
||||
|
||||
/// Values in column are represented as continuous memory segment of fixed size. Implies valuesHaveFixedSize.
|
||||
virtual bool isFixedAndContiguous() const { return false; }
|
||||
|
||||
/// If valuesHaveFixedSize, returns size of value, otherwise throw an exception.
|
||||
virtual size_t sizeOfValueIfFixed() const { throw Exception("Values of column " + getName() + " are not fixed size.", ErrorCodes::CANNOT_GET_SIZE_OF_FIELD); }
|
||||
|
||||
/// Column is ColumnVector of numbers or ColumnConst of it. Note that Nullable columns are not numeric.
|
||||
/// Implies isFixedAndContiguous.
|
||||
virtual bool isNumeric() const { return false; }
|
||||
|
||||
/// If the only value column can contain is NULL.
|
||||
/// Does not imply type of object, because it can be ColumnNullable(ColumnNothing) or ColumnConst(ColumnNullable(ColumnNothing))
|
||||
virtual bool onlyNull() const { return false; }
|
||||
|
||||
/// Can be inside ColumnNullable.
|
||||
virtual bool canBeInsideNullable() const { return false; }
|
||||
|
||||
|
||||
virtual ~IColumn() {}
|
||||
|
||||
/** Print column name, size, and recursively print all subcolumns.
|
||||
*/
|
||||
String dumpStructure() const;
|
||||
|
||||
protected:
|
||||
|
@ -130,6 +130,11 @@ public:
|
||||
s += delta;
|
||||
}
|
||||
|
||||
bool isDummy() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
private:
|
||||
size_t s;
|
||||
};
|
||||
|
@ -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 <> constexpr bool IsNumber<UInt128> = true;
|
||||
template <> struct TypeName<UInt128> { static const char * get() { return "UInt128"; } };
|
||||
|
||||
struct UInt128Hash
|
||||
|
@ -65,6 +65,13 @@ public:
|
||||
ColumnWithTypeAndName & getByName(const std::string & name);
|
||||
const ColumnWithTypeAndName & getByName(const std::string & name) const;
|
||||
|
||||
Container::iterator begin() { return data.begin(); }
|
||||
Container::iterator end() { return data.end(); }
|
||||
Container::const_iterator begin() const { return data.begin(); }
|
||||
Container::const_iterator end() const { return data.end(); }
|
||||
Container::const_iterator cbegin() const { return data.cbegin(); }
|
||||
Container::const_iterator cend() const { return data.cend(); }
|
||||
|
||||
bool has(const std::string & name) const;
|
||||
|
||||
size_t getPositionByName(const std::string & name) const;
|
||||
|
@ -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> constexpr bool IsNumber = false;
|
||||
|
||||
template <> constexpr bool IsNumber<UInt8> = true;
|
||||
template <> constexpr bool IsNumber<UInt16> = true;
|
||||
template <> constexpr bool IsNumber<UInt32> = true;
|
||||
template <> constexpr bool IsNumber<UInt64> = true;
|
||||
template <> constexpr bool IsNumber<Int8> = true;
|
||||
template <> constexpr bool IsNumber<Int16> = true;
|
||||
template <> constexpr bool IsNumber<Int32> = true;
|
||||
template <> constexpr bool IsNumber<Int64> = true;
|
||||
template <> constexpr bool IsNumber<Float32> = true;
|
||||
template <> constexpr 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>;
|
||||
|
||||
}
|
||||
|
@ -33,8 +33,7 @@ std::ostream & operator<<(std::ostream & stream, const DB::NameAndTypePair & wha
|
||||
|
||||
std::ostream & operator<<(std::ostream & stream, const DB::IDataType & what)
|
||||
{
|
||||
stream << "IDataType(name = " << what.getName() << ", default = " << what.getDefault() << ", isNullable = " << what.isNullable()
|
||||
<< ", isNumeric = " << what.isNumeric() << ", behavesAsNumber = " << what.behavesAsNumber() << ")";
|
||||
stream << "IDataType(name = " << what.getName() << ", default = " << what.getDefault();
|
||||
return stream;
|
||||
}
|
||||
|
||||
|
@ -38,7 +38,7 @@ protected:
|
||||
if (!res)
|
||||
return res;
|
||||
|
||||
res.insert({data_type->createConstColumn(res.rows(), value)->convertToFullColumnIfConst(), data_type, column_name});
|
||||
res.insert({data_type->createColumnConst(res.rows(), value)->convertToFullColumnIfConst(), data_type, column_name});
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -58,10 +58,10 @@ 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();
|
||||
ColumnPtr nested_column = nested_type->createColumnConst(nested_rows, nested_type->getDefault())->convertToFullColumnIfConst();
|
||||
column_to_add.column = std::make_shared<ColumnArray>(nested_column, offsets_column);
|
||||
}
|
||||
else
|
||||
@ -69,7 +69,7 @@ void AddingDefaultBlockOutputStream::write(const DB::Block & block)
|
||||
/** It is necessary to turn a constant column into a full column, since in part of blocks (from other parts),
|
||||
* it can be full (or the interpreter may decide that it is constant everywhere).
|
||||
*/
|
||||
column_to_add.column = column_to_add.type->createConstColumn(rows, column_to_add.type->getDefault())->convertToFullColumnIfConst();
|
||||
column_to_add.column = column_to_add.type->createColumnConst(rows, column_to_add.type->getDefault())->convertToFullColumnIfConst();
|
||||
}
|
||||
|
||||
res.insert(std::move(column_to_add));
|
||||
|
@ -3,7 +3,6 @@
|
||||
|
||||
#include <DataStreams/verbosePrintString.h>
|
||||
#include <DataStreams/CSVRowInputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -226,9 +225,9 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
|
||||
if (curr_position < prev_position)
|
||||
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (data_types[i]->isNumeric())
|
||||
if (data_types[i]->isNumber() || data_types[i]->isDateOrDateTime())
|
||||
{
|
||||
/// An empty string instead of a number.
|
||||
/// An empty string instead of a value.
|
||||
if (curr_position == prev_position)
|
||||
{
|
||||
out << "ERROR: text ";
|
||||
@ -254,7 +253,7 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
|
||||
|
||||
out << "\n";
|
||||
|
||||
if (data_types[i]->isNumeric())
|
||||
if (data_types[i]->haveMaximumSizeOfValue())
|
||||
{
|
||||
if (*curr_position != '\n' && *curr_position != '\r' && *curr_position != delimiter)
|
||||
{
|
||||
|
@ -1,9 +1,5 @@
|
||||
#include <DataStreams/CastTypeBlockInputStream.h>
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <Functions/FunctionFactory.h>
|
||||
#include <Functions/IFunction.h>
|
||||
#include <Interpreters/castColumn.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -14,7 +10,7 @@ CastTypeBlockInputStream::CastTypeBlockInputStream(
|
||||
const Context & context_,
|
||||
const BlockInputStreamPtr & input_,
|
||||
const Block & reference_definition_)
|
||||
: context(context_), ref_defenition(reference_definition_)
|
||||
: context(context_), ref_definition(reference_definition_)
|
||||
{
|
||||
children.emplace_back(input_);
|
||||
}
|
||||
@ -46,27 +42,16 @@ Block CastTypeBlockInputStream::readImpl()
|
||||
return block;
|
||||
|
||||
size_t num_columns = block.columns();
|
||||
Block res;
|
||||
Block res = block;
|
||||
|
||||
for (size_t col = 0; col < num_columns; ++col)
|
||||
{
|
||||
const auto & src_column = block.getByPosition(col);
|
||||
auto it = cast_description.find(col);
|
||||
|
||||
if (it == cast_description.end())
|
||||
if (cast_description.end() != it)
|
||||
{
|
||||
// Leave the same column
|
||||
res.insert(src_column);
|
||||
}
|
||||
else
|
||||
{
|
||||
CastElement & cast_element = it->second;
|
||||
size_t tmp_col = cast_element.tmp_col_offset;
|
||||
|
||||
tmp_conversion_block.getByPosition(tmp_col).column = src_column.column;
|
||||
cast_element.function->execute(tmp_conversion_block, ColumnNumbers{tmp_col, tmp_col + 1}, tmp_col + 2);
|
||||
|
||||
res.insert(tmp_conversion_block.getByPosition(tmp_col + 2));
|
||||
auto & elem = res.getByPosition(col);
|
||||
elem.column = castColumn(elem, it->second, context);
|
||||
elem.type = it->second;
|
||||
}
|
||||
}
|
||||
|
||||
@ -74,46 +59,21 @@ Block CastTypeBlockInputStream::readImpl()
|
||||
}
|
||||
|
||||
|
||||
CastTypeBlockInputStream::CastElement::CastElement(std::shared_ptr<IFunction> && function_, size_t tmp_col_offset_)
|
||||
: function(std::move(function_)), tmp_col_offset(tmp_col_offset_) {}
|
||||
|
||||
|
||||
void CastTypeBlockInputStream::initialize(const Block & src_block)
|
||||
{
|
||||
for (size_t src_col = 0; src_col < src_block.columns(); ++src_col)
|
||||
for (size_t src_col = 0, num_columns = src_block.columns(); src_col < num_columns; ++src_col)
|
||||
{
|
||||
const auto & src_column = src_block.getByPosition(src_col);
|
||||
|
||||
/// Skip, if it is a problem, it will be detected on the next pipeline stage
|
||||
if (!ref_defenition.has(src_column.name))
|
||||
if (!ref_definition.has(src_column.name))
|
||||
continue;
|
||||
|
||||
const auto & ref_column = ref_defenition.getByName(src_column.name);
|
||||
const auto & ref_column = ref_definition.getByName(src_column.name);
|
||||
|
||||
/// Force conversion if source and destination types is different.
|
||||
if (!ref_column.type->equals(*src_column.type))
|
||||
{
|
||||
ColumnWithTypeAndName res_type_name_column(DataTypeString().createConstColumn(1, ref_column.type->getName()), std::make_shared<DataTypeString>(), "");
|
||||
ColumnWithTypeAndName res_blank_column(nullptr, ref_column.type->clone(), src_column.name);
|
||||
|
||||
/// Prepares function to execution
|
||||
auto cast_function = FunctionFactory::instance().get("CAST", context);
|
||||
{
|
||||
DataTypePtr unused_return_type;
|
||||
std::vector<ExpressionAction> unused_prerequisites;
|
||||
ColumnsWithTypeAndName arguments{src_column, res_type_name_column};
|
||||
cast_function->getReturnTypeAndPrerequisites(arguments, unused_return_type, unused_prerequisites);
|
||||
}
|
||||
|
||||
/// Prefill arguments and result column for current CAST
|
||||
tmp_conversion_block.insert(src_column);
|
||||
tmp_conversion_block.insert(res_type_name_column);
|
||||
tmp_conversion_block.insert(res_blank_column);
|
||||
|
||||
/// Index of src_column blank in tmp_conversion_block
|
||||
size_t tmp_col_offset = cast_description.size() * 3;
|
||||
cast_description.emplace(src_col, CastElement(std::move(cast_function), tmp_col_offset));
|
||||
}
|
||||
cast_description.emplace(src_col, ref_column.type);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <unordered_map>
|
||||
#include <DataStreams/IProfilingBlockInputStream.h>
|
||||
|
||||
|
||||
@ -25,29 +26,15 @@ protected:
|
||||
|
||||
private:
|
||||
const Context & context;
|
||||
Block ref_defenition;
|
||||
Block ref_definition;
|
||||
|
||||
/// Initializes cast_description and prepares tmp_conversion_block
|
||||
void initialize(const Block & src_block);
|
||||
bool initialized = false;
|
||||
|
||||
struct CastElement
|
||||
{
|
||||
/// Prepared function to do conversion
|
||||
std::shared_ptr<IFunction> function;
|
||||
/// Position of first function argument in tmp_conversion_block
|
||||
size_t tmp_col_offset;
|
||||
|
||||
CastElement(std::shared_ptr<IFunction> && function_, size_t tmp_col_offset_);
|
||||
};
|
||||
|
||||
/// Describes required conversions on source block
|
||||
/// Contains column numbers in source block that should be converted
|
||||
std::map<size_t, CastElement> cast_description;
|
||||
|
||||
/// Auxiliary block, stores prefilled arguments and result for each CAST function in cast_description
|
||||
/// 3 columns are allocated for each conversion: [blank of source column, column with res type name, blank of res column]
|
||||
Block tmp_conversion_block;
|
||||
std::unordered_map<size_t, DataTypePtr> cast_description;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -146,7 +146,7 @@ ConstColumnPlainPtrs DistinctBlockInputStream::getKeyColumns(const Block & block
|
||||
: block.getByName(columns_names[i]).column;
|
||||
|
||||
/// Ignore all constant columns.
|
||||
if (!column->isConst())
|
||||
if (!column->isColumnConst())
|
||||
column_ptrs.emplace_back(column.get());
|
||||
}
|
||||
|
||||
|
@ -172,7 +172,7 @@ ConstColumnPlainPtrs DistinctSortedBlockInputStream::getKeyColumns(const Block &
|
||||
: block.getByName(columns_names[i]).column;
|
||||
|
||||
/// Ignore all constant columns.
|
||||
if (!column->isConst())
|
||||
if (!column->isColumnConst())
|
||||
column_ptrs.emplace_back(column.get());
|
||||
}
|
||||
|
||||
|
@ -55,11 +55,11 @@ const Block & FilterBlockInputStream::getTotals()
|
||||
|
||||
static void analyzeConstantFilter(const IColumn & column, bool & filter_always_false, bool & filter_always_true)
|
||||
{
|
||||
if (column.isNull())
|
||||
if (column.onlyNull())
|
||||
{
|
||||
filter_always_false = true;
|
||||
}
|
||||
else if (column.isConst())
|
||||
else if (column.isColumnConst())
|
||||
{
|
||||
if (static_cast<const ColumnConst &>(column).getValue<UInt8>())
|
||||
filter_always_true = true;
|
||||
@ -119,22 +119,13 @@ Block FilterBlockInputStream::readImpl()
|
||||
|
||||
size_t columns = res.columns();
|
||||
ColumnPtr column = res.safeGetByPosition(filter_column).column;
|
||||
bool is_nullable_column = column->isNullable();
|
||||
|
||||
auto init_observed_column = [&column, &is_nullable_column]()
|
||||
{
|
||||
if (is_nullable_column)
|
||||
{
|
||||
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*column.get());
|
||||
return nullable_col.getNestedColumn().get();
|
||||
}
|
||||
else
|
||||
return column.get();
|
||||
};
|
||||
IColumn * observed_column = column.get();
|
||||
bool is_nullable_column = observed_column->isColumnNullable();
|
||||
if (is_nullable_column)
|
||||
observed_column = static_cast<const ColumnNullable &>(*column.get()).getNestedColumn().get();
|
||||
|
||||
IColumn * observed_column = init_observed_column();
|
||||
|
||||
const ColumnUInt8 * column_vec = typeid_cast<const ColumnUInt8 *>(observed_column);
|
||||
ColumnUInt8 * column_vec = typeid_cast<ColumnUInt8 *>(observed_column);
|
||||
if (!column_vec)
|
||||
{
|
||||
/** It happens that at the stage of analysis of expressions (in sample_block) the columns-constants have not been calculated yet,
|
||||
@ -157,30 +148,20 @@ Block FilterBlockInputStream::readImpl()
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_COLUMN_FOR_FILTER);
|
||||
}
|
||||
|
||||
IColumn::Filter & filter = column_vec->getData();
|
||||
|
||||
if (is_nullable_column)
|
||||
{
|
||||
/// Exclude the entries of the filter column that actually are NULL values.
|
||||
|
||||
/// Access the filter content.
|
||||
ColumnNullable & nullable_col = static_cast<ColumnNullable &>(*column);
|
||||
auto & nested_col = nullable_col.getNestedColumn();
|
||||
auto & actual_col = static_cast<ColumnUInt8 &>(*nested_col);
|
||||
auto & filter_col = actual_col.getData();
|
||||
const NullMap & null_map = static_cast<ColumnNullable &>(*column).getNullMap();
|
||||
|
||||
/// Access the null values byte map content.
|
||||
ColumnPtr & null_map = nullable_col.getNullMapColumn();
|
||||
ColumnUInt8 & content = static_cast<ColumnUInt8 &>(*null_map);
|
||||
auto & data = content.getData();
|
||||
|
||||
for (size_t i = 0; i < data.size(); ++i)
|
||||
{
|
||||
if (data[i] != 0)
|
||||
filter_col[i] = 0;
|
||||
}
|
||||
IColumn::Filter & filter = column_vec->getData();
|
||||
for (size_t i = 0, size = null_map.size(); i < size; ++i)
|
||||
if (null_map[i])
|
||||
filter[i] = 0;
|
||||
}
|
||||
|
||||
const IColumn::Filter & filter = column_vec->getData();
|
||||
|
||||
/** Let's find out how many rows will be in result.
|
||||
* To do this, we filter out the first non-constant column
|
||||
* or calculate number of set bytes in the filter.
|
||||
@ -188,7 +169,7 @@ Block FilterBlockInputStream::readImpl()
|
||||
size_t first_non_constant_column = 0;
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
if (!res.safeGetByPosition(i).column->isConst())
|
||||
if (!res.safeGetByPosition(i).column->isColumnConst())
|
||||
{
|
||||
first_non_constant_column = i;
|
||||
|
||||
@ -217,7 +198,7 @@ Block FilterBlockInputStream::readImpl()
|
||||
if (filtered_rows == filter.size())
|
||||
{
|
||||
/// Replace the column with the filter by a constant.
|
||||
res.safeGetByPosition(filter_column).column = res.safeGetByPosition(filter_column).type->createConstColumn(filtered_rows, UInt64(1));
|
||||
res.safeGetByPosition(filter_column).column = res.safeGetByPosition(filter_column).type->createColumnConst(filtered_rows, UInt64(1));
|
||||
/// No need to touch the rest of the columns.
|
||||
return res;
|
||||
}
|
||||
@ -234,14 +215,14 @@ Block FilterBlockInputStream::readImpl()
|
||||
/// Example:
|
||||
/// SELECT materialize(100) AS x WHERE x
|
||||
/// will work incorrectly.
|
||||
current_column.column = current_column.type->createConstColumn(filtered_rows, UInt64(1));
|
||||
current_column.column = current_column.type->createColumnConst(filtered_rows, UInt64(1));
|
||||
continue;
|
||||
}
|
||||
|
||||
if (i == first_non_constant_column)
|
||||
continue;
|
||||
|
||||
if (current_column.column->isConst())
|
||||
if (current_column.column->isColumnConst())
|
||||
current_column.column = current_column.column->cut(0, filtered_rows);
|
||||
else
|
||||
current_column.column = current_column.column->filter(filter, -1);
|
||||
|
@ -12,11 +12,11 @@ JSONRowOutputStream::JSONRowOutputStream(WriteBuffer & ostr_, const Block & samp
|
||||
NamesAndTypesList columns(sample_.getColumnsList());
|
||||
fields.assign(columns.begin(), columns.end());
|
||||
|
||||
bool have_non_numeric_columns = false;
|
||||
bool need_validate_utf8 = false;
|
||||
for (size_t i = 0; i < sample_.columns(); ++i)
|
||||
{
|
||||
if (!sample_.getByPosition(i).type->isNumeric())
|
||||
have_non_numeric_columns = true;
|
||||
if (!sample_.getByPosition(i).type->textCanContainOnlyValidUTF8())
|
||||
need_validate_utf8 = true;
|
||||
|
||||
WriteBufferFromOwnString out;
|
||||
writeJSONString(fields[i].name, out);
|
||||
@ -24,7 +24,7 @@ JSONRowOutputStream::JSONRowOutputStream(WriteBuffer & ostr_, const Block & samp
|
||||
fields[i].name = out.str();
|
||||
}
|
||||
|
||||
if (have_non_numeric_columns)
|
||||
if (need_validate_utf8)
|
||||
{
|
||||
validating_ostr = std::make_unique<WriteBufferValidUTF8>(dst_ostr);
|
||||
ostr = validating_ostr.get();
|
||||
|
@ -66,7 +66,7 @@ ConstColumnPlainPtrs LimitByBlockInputStream::getKeyColumns(Block & block) const
|
||||
auto & column = block.getByName(name).column;
|
||||
|
||||
/// Ignore all constant columns.
|
||||
if (!column->isConst())
|
||||
if (!column->isColumnConst())
|
||||
column_ptrs.emplace_back(column.get());
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,7 @@ static void removeConstantsFromBlock(Block & block)
|
||||
size_t i = 0;
|
||||
while (i < columns)
|
||||
{
|
||||
if (block.getByPosition(i).column->isConst())
|
||||
if (block.getByPosition(i).column->isColumnConst())
|
||||
{
|
||||
block.erase(i);
|
||||
--columns;
|
||||
@ -40,9 +40,9 @@ static void removeConstantsFromSortDescription(const Block & sample_block, SortD
|
||||
[&](const SortColumnDescription & elem)
|
||||
{
|
||||
if (!elem.column_name.empty())
|
||||
return sample_block.getByName(elem.column_name).column->isConst();
|
||||
return sample_block.getByName(elem.column_name).column->isColumnConst();
|
||||
else
|
||||
return sample_block.safeGetByPosition(elem.column_number).column->isConst();
|
||||
return sample_block.safeGetByPosition(elem.column_number).column->isColumnConst();
|
||||
}), description.end());
|
||||
}
|
||||
|
||||
@ -57,7 +57,7 @@ static void enrichBlockWithConstants(Block & block, const Block & sample_block)
|
||||
for (size_t i = 0; i < columns; ++i)
|
||||
{
|
||||
const auto & col_type_name = sample_block.getByPosition(i);
|
||||
if (col_type_name.column->isConst())
|
||||
if (col_type_name.column->isColumnConst())
|
||||
block.insert(i, {col_type_name.column->cloneResized(rows), col_type_name.type, col_type_name.name});
|
||||
}
|
||||
}
|
||||
|
@ -2,7 +2,6 @@
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Columns/ColumnsCommon.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataStreams/isConvertableTypes.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -107,33 +106,23 @@ void NullableAdapterBlockInputStream::buildActions(
|
||||
const auto & in_elem = in_sample.getByPosition(i);
|
||||
const auto & out_elem = out_sample.getByPosition(i);
|
||||
|
||||
if (isConvertableTypes(in_elem.type, out_elem.type))
|
||||
{
|
||||
bool is_in_nullable = in_elem.type->isNullable();
|
||||
bool is_out_nullable = out_elem.type->isNullable();
|
||||
bool is_in_nullable = in_elem.type->isNullable();
|
||||
bool is_out_nullable = out_elem.type->isNullable();
|
||||
|
||||
if (is_in_nullable && !is_out_nullable)
|
||||
actions.push_back(TO_ORDINARY);
|
||||
else if (!is_in_nullable && is_out_nullable)
|
||||
actions.push_back(TO_NULLABLE);
|
||||
else
|
||||
actions.push_back(NONE);
|
||||
|
||||
if (in_elem.name != out_elem.name)
|
||||
rename.emplace_back(std::make_optional(out_elem.name));
|
||||
else
|
||||
rename.emplace_back();
|
||||
|
||||
if (actions.back() != NONE || rename.back())
|
||||
must_transform = true;
|
||||
}
|
||||
if (is_in_nullable && !is_out_nullable)
|
||||
actions.push_back(TO_ORDINARY);
|
||||
else if (!is_in_nullable && is_out_nullable)
|
||||
actions.push_back(TO_NULLABLE);
|
||||
else
|
||||
{
|
||||
throw Exception{String("Types must be the same for columns at same position. ")
|
||||
+ "Column " + in_elem.name + " has type " + in_elem.type->getName()
|
||||
+ ", but column " + out_elem.name + " has type " + out_elem.type->getName(),
|
||||
ErrorCodes::TYPE_MISMATCH};
|
||||
}
|
||||
actions.push_back(NONE);
|
||||
|
||||
if (in_elem.name != out_elem.name)
|
||||
rename.emplace_back(std::make_optional(out_elem.name));
|
||||
else
|
||||
rename.emplace_back();
|
||||
|
||||
if (actions.back() != NONE || rename.back())
|
||||
must_transform = true;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -49,29 +49,17 @@ void PrettyBlockOutputStream::calculateWidths(const Block & block, WidthsPerColu
|
||||
{
|
||||
const ColumnWithTypeAndName & elem = block.getByPosition(i);
|
||||
|
||||
if (!elem.column->isConst())
|
||||
{
|
||||
widths[i].resize(rows);
|
||||
widths[i].resize(rows);
|
||||
|
||||
for (size_t j = 0; j < rows; ++j)
|
||||
{
|
||||
{
|
||||
WriteBufferFromString out(serialized_value);
|
||||
elem.type->serializeTextEscaped(*elem.column, j, out);
|
||||
}
|
||||
|
||||
widths[i][j] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size());
|
||||
max_widths[i] = std::max(max_widths[i], widths[i][j]);
|
||||
}
|
||||
}
|
||||
else
|
||||
for (size_t j = 0; j < rows; ++j)
|
||||
{
|
||||
{
|
||||
WriteBufferFromString out(serialized_value);
|
||||
elem.type->serializeTextEscaped(*elem.column->cut(0, 1)->convertToFullColumnIfConst(), 0, out);
|
||||
|
||||
max_widths[i] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size());
|
||||
elem.type->serializeTextEscaped(*elem.column, j, out);
|
||||
}
|
||||
|
||||
widths[i][j] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size());
|
||||
max_widths[i] = std::max(max_widths[i], widths[i][j]);
|
||||
}
|
||||
|
||||
/// And also calculate widths for names of columns.
|
||||
@ -158,7 +146,7 @@ void PrettyBlockOutputStream::write(const Block & block)
|
||||
if (!no_escapes)
|
||||
writeCString("\033[1m", ostr);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
if (col.type->shouldAlignRightInPrettyFormats())
|
||||
{
|
||||
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
|
||||
writeChar(' ', ostr);
|
||||
@ -212,7 +200,7 @@ void PrettyBlockOutputStream::writeValueWithPadding(const ColumnWithTypeAndName
|
||||
writeChar(' ', ostr);
|
||||
};
|
||||
|
||||
if (elem.type->isNumeric())
|
||||
if (elem.type->shouldAlignRightInPrettyFormats())
|
||||
{
|
||||
writePadding();
|
||||
elem.type->serializeTextEscaped(*elem.column.get(), row_num, ostr);
|
||||
|
@ -27,7 +27,7 @@ void PrettyCompactBlockOutputStream::writeHeader(
|
||||
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(i);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
if (col.type->shouldAlignRightInPrettyFormats())
|
||||
{
|
||||
for (size_t k = 0; k < max_widths[i] - name_widths[i]; ++k)
|
||||
writeCString("─", ostr);
|
||||
|
@ -37,7 +37,7 @@ void PrettySpaceBlockOutputStream::write(const Block & block)
|
||||
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(i);
|
||||
|
||||
if (col.type->isNumeric())
|
||||
if (col.type->shouldAlignRightInPrettyFormats())
|
||||
{
|
||||
for (ssize_t k = 0; k < std::max(static_cast<ssize_t>(0), static_cast<ssize_t>(max_widths[i] - name_widths[i])); ++k)
|
||||
writeChar(' ', ostr);
|
||||
|
@ -151,8 +151,7 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Leave only numeric types. Note that dates and datetime here are not considered such.
|
||||
if (!column.type->behavesAsNumber())
|
||||
if (!column.type->isSummable())
|
||||
{
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
continue;
|
||||
@ -222,16 +221,14 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
|| endsWith(name, "Key")
|
||||
|| endsWith(name, "Type"))
|
||||
{
|
||||
if (!nested_type.isNumeric()
|
||||
|| nested_type.getName() == "Float32"
|
||||
|| nested_type.getName() == "Float64")
|
||||
if (!nested_type.isValueRepresentedByInteger())
|
||||
break;
|
||||
|
||||
map_desc.key_col_nums.push_back(*column_num_it);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!nested_type.behavesAsNumber())
|
||||
if (!nested_type.isSummable())
|
||||
break;
|
||||
|
||||
map_desc.val_col_nums.push_back(*column_num_it);
|
||||
|
@ -4,7 +4,6 @@
|
||||
|
||||
#include <DataStreams/TabSeparatedRowInputStream.h>
|
||||
#include <DataStreams/verbosePrintString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -193,9 +192,9 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
|
||||
if (curr_position < prev_position)
|
||||
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (data_types[i]->isNumeric())
|
||||
if (data_types[i]->isNumber() || data_types[i]->isDateOrDateTime())
|
||||
{
|
||||
/// An empty string instead of a number.
|
||||
/// An empty string instead of a value.
|
||||
if (curr_position == prev_position)
|
||||
{
|
||||
out << "ERROR: text ";
|
||||
@ -221,7 +220,7 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(Block & block,
|
||||
|
||||
out << "\n";
|
||||
|
||||
if (data_types[i]->isNumeric())
|
||||
if (data_types[i]->haveMaximumSizeOfValue())
|
||||
{
|
||||
if (*curr_position != '\n' && *curr_position != '\t')
|
||||
{
|
||||
|
@ -3,6 +3,7 @@
|
||||
#include <Interpreters/AggregateDescription.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <Columns/ColumnNullable.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
namespace DB
|
||||
@ -117,7 +118,11 @@ Block TotalsHavingBlockInputStream::readImpl()
|
||||
if (auto converted = filter_column_ptr->convertToFullColumnIfConst())
|
||||
filter_column_ptr = converted;
|
||||
|
||||
ColumnUInt8 * filter_column = typeid_cast<ColumnUInt8 *>(&*filter_column_ptr);
|
||||
bool filter_is_nullable = filter_column_ptr->isColumnNullable();
|
||||
ColumnUInt8 * filter_column = filter_is_nullable
|
||||
? typeid_cast<ColumnUInt8 *>(static_cast<ColumnNullable *>(filter_column_ptr.get())->getNestedColumn().get())
|
||||
: typeid_cast<ColumnUInt8 *>(&*filter_column_ptr);
|
||||
|
||||
if (!filter_column)
|
||||
throw Exception("Filter column must have type UInt8, found " +
|
||||
finalized.safeGetByPosition(filter_column_pos).type->getName(),
|
||||
@ -125,6 +130,14 @@ Block TotalsHavingBlockInputStream::readImpl()
|
||||
|
||||
IColumn::Filter & filter = filter_column->getData();
|
||||
|
||||
if (filter_column_ptr->isColumnNullable())
|
||||
{
|
||||
const NullMap & null_map = static_cast<ColumnNullable *>(filter_column_ptr.get())->getNullMap();
|
||||
for (size_t i = 0, size = null_map.size(); i < size; ++i)
|
||||
if (null_map[i])
|
||||
filter[i] = 0;
|
||||
}
|
||||
|
||||
/// Add values to `totals` (if it was not already done).
|
||||
if (totals_mode == TotalsMode::BEFORE_HAVING)
|
||||
addToTotals(current_totals, block, nullptr);
|
||||
|
@ -4,7 +4,6 @@
|
||||
#include <Parsers/TokenIterator.h>
|
||||
#include <Parsers/ExpressionListParsers.h>
|
||||
#include <DataStreams/ValuesRowInputStream.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <Common/FieldVisitors.h>
|
||||
#include <Core/Block.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
@ -1,7 +1,5 @@
|
||||
#include <cmath>
|
||||
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <DataStreams/VerticalRowOutputStream.h>
|
||||
|
@ -13,11 +13,11 @@ XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample
|
||||
fields.assign(columns.begin(), columns.end());
|
||||
field_tag_names.resize(sample_.columns());
|
||||
|
||||
bool have_non_numeric_columns = false;
|
||||
bool need_validate_utf8 = false;
|
||||
for (size_t i = 0; i < sample_.columns(); ++i)
|
||||
{
|
||||
if (!sample_.getByPosition(i).type->isNumeric())
|
||||
have_non_numeric_columns = true;
|
||||
if (!sample_.getByPosition(i).type->textCanContainOnlyValidUTF8())
|
||||
need_validate_utf8 = true;
|
||||
|
||||
/// As element names, we will use the column name if it has a valid form, or "field", otherwise.
|
||||
/// The condition below is more strict than the XML standard requires.
|
||||
@ -43,7 +43,7 @@ XMLRowOutputStream::XMLRowOutputStream(WriteBuffer & ostr_, const Block & sample
|
||||
: "field";
|
||||
}
|
||||
|
||||
if (have_non_numeric_columns)
|
||||
if (need_validate_utf8)
|
||||
{
|
||||
validating_ostr = std::make_unique<WriteBufferValidUTF8>(dst_ostr);
|
||||
ostr = validating_ostr.get();
|
||||
|
@ -1,35 +0,0 @@
|
||||
#include <DataStreams/isConvertableTypes.h>
|
||||
|
||||
#include <DataTypes/DataTypeEnum.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
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);
|
||||
auto to_nn = removeNullable(to);
|
||||
|
||||
if ( dynamic_cast<const IDataTypeEnum *>(to_nn.get()) &&
|
||||
!dynamic_cast<const IDataTypeEnum *>(from_nn.get()))
|
||||
{
|
||||
if (typeid_cast<const DataTypeString *>(from_nn.get()))
|
||||
return true;
|
||||
if (from_nn->isNumeric())
|
||||
return true;
|
||||
}
|
||||
|
||||
return from_nn->equals(*to_nn);
|
||||
}
|
||||
|
||||
}
|
@ -1,11 +0,0 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/IDataType.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Check that type 'from' can be implicitly converted to type 'to'.
|
||||
bool isConvertableTypes(const DataTypePtr & from, const DataTypePtr & to);
|
||||
|
||||
}
|
@ -1,5 +1,4 @@
|
||||
#include <DataStreams/materializeBlock.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -8,13 +8,6 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int THERE_IS_NO_DEFAULT_VALUE;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
|
||||
/** Type - the state of the aggregate function.
|
||||
* Type parameters is an aggregate function, the types of its arguments, and its parameters (for parametric aggregate functions).
|
||||
*/
|
||||
@ -40,6 +33,8 @@ public:
|
||||
|
||||
const char * getFamilyName() const override { return "AggregateFunction"; }
|
||||
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
DataTypePtr getReturnType() const { return function->getReturnType(); };
|
||||
DataTypes getArgumentsDataTypes() const { return argument_types; }
|
||||
|
||||
@ -67,6 +62,10 @@ public:
|
||||
ColumnPtr createColumn() const override;
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return false; }
|
||||
};
|
||||
|
||||
|
||||
|
@ -28,7 +28,6 @@ namespace ErrorCodes
|
||||
DataTypeArray::DataTypeArray(const DataTypePtr & nested_)
|
||||
: nested{nested_}
|
||||
{
|
||||
offsets = std::make_shared<DataTypeNumber<ColumnArray::Offset_t>>();
|
||||
}
|
||||
|
||||
|
||||
@ -171,7 +170,7 @@ void DataTypeArray::serializeBinaryBulkWithMultipleStreams(
|
||||
if (position_independent_encoding)
|
||||
serializeArraySizesPositionIndependent(column, *stream, offset, limit);
|
||||
else
|
||||
offsets->serializeBinaryBulk(*column_array.getOffsetsColumn(), *stream, offset, limit);
|
||||
DataTypeNumber<ColumnArray::Offset_t>().serializeBinaryBulk(*column_array.getOffsetsColumn(), *stream, offset, limit);
|
||||
}
|
||||
|
||||
/// Then serialize contents of arrays.
|
||||
@ -217,7 +216,7 @@ void DataTypeArray::deserializeBinaryBulkWithMultipleStreams(
|
||||
if (position_independent_encoding)
|
||||
deserializeArraySizesPositionIndependent(column, *stream, limit);
|
||||
else
|
||||
offsets->deserializeBinaryBulk(*column_array.getOffsetsColumn(), *stream, limit, 0);
|
||||
DataTypeNumber<ColumnArray::Offset_t>().deserializeBinaryBulk(*column_array.getOffsetsColumn(), *stream, limit, 0);
|
||||
}
|
||||
|
||||
path.back() = Substream::ArrayElements;
|
||||
|
@ -12,8 +12,6 @@ class DataTypeArray final : public IDataType
|
||||
private:
|
||||
/// The type of array elements.
|
||||
DataTypePtr nested;
|
||||
/// Type of offsets.
|
||||
DataTypePtr offsets;
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
@ -90,8 +88,18 @@ public:
|
||||
|
||||
Field getDefault() const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return true; }
|
||||
bool cannotBeStoredInTables() const override { return nested->cannotBeStoredInTables(); }
|
||||
bool textCanContainOnlyValidUTF8() const override { return nested->textCanContainOnlyValidUTF8(); }
|
||||
bool canBeComparedWithCollation() const override { return nested->canBeComparedWithCollation(); }
|
||||
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override
|
||||
{
|
||||
return nested->isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion();
|
||||
}
|
||||
|
||||
const DataTypePtr & getNestedType() const { return nested; }
|
||||
const DataTypePtr & getOffsetsType() const { return offsets; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,8 +9,6 @@ namespace DB
|
||||
class DataTypeDate final : public DataTypeNumberBase<UInt16>
|
||||
{
|
||||
public:
|
||||
bool behavesAsNumber() const override { return false; }
|
||||
|
||||
const char * getFamilyName() const override { return "Date"; }
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeDate>(); }
|
||||
|
||||
@ -23,6 +21,10 @@ public:
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override;
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool isDateOrDateTime() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -33,8 +33,6 @@ class DataTypeDateTime final : public DataTypeNumberBase<UInt32>
|
||||
public:
|
||||
DataTypeDateTime(const std::string & time_zone_name = "");
|
||||
|
||||
bool behavesAsNumber() const override { return false; }
|
||||
|
||||
const char * getFamilyName() const override { return "DateTime"; }
|
||||
std::string getName() const override;
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeDateTime>(); }
|
||||
@ -49,6 +47,10 @@ public:
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override;
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool isDateOrDateTime() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
const DateLUTImpl & getTimeZone() const { return time_zone; }
|
||||
|
||||
private:
|
||||
|
@ -5,6 +5,8 @@
|
||||
#include <Parsers/ASTFunction.h>
|
||||
#include <Parsers/ASTLiteral.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/UTF8Helpers.h>
|
||||
#include <Poco/UTF8Encoding.h>
|
||||
|
||||
#include <limits>
|
||||
|
||||
@ -247,6 +249,28 @@ void DataTypeEnum<Type>::insertDefaultInto(IColumn & column) const
|
||||
static_cast<ColumnType &>(column).getData().push_back(values.front().second);
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
bool DataTypeEnum<Type>::textCanContainOnlyValidUTF8() const
|
||||
{
|
||||
for (const auto & elem : values)
|
||||
{
|
||||
const char * pos = elem.first.data();
|
||||
const char * end = pos + elem.first.size();
|
||||
while (pos < end)
|
||||
{
|
||||
size_t length = UTF8::seqLength(*pos);
|
||||
if (pos + length > end)
|
||||
return false;
|
||||
|
||||
if (Poco::UTF8Encoding::isLegal(reinterpret_cast<const unsigned char *>(pos), length))
|
||||
pos += length;
|
||||
else
|
||||
return false;
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
static void checkOverflow(Int64 value)
|
||||
{
|
||||
|
@ -16,12 +16,22 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
class IDataTypeEnum : public IDataType
|
||||
{
|
||||
public:
|
||||
|
||||
virtual Field castToName(const Field & value_or_name) const = 0;
|
||||
virtual Field castToValue(const Field & value_or_name) const = 0;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isValueRepresentedByNumber() const override { return true; }
|
||||
bool isValueRepresentedByInteger() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool isEnum() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
|
||||
@ -54,8 +64,6 @@ public:
|
||||
const Values & getValues() const { return values; }
|
||||
std::string getName() const override { return name; }
|
||||
const char * getFamilyName() const override;
|
||||
bool isNumeric() const override { return true; }
|
||||
bool behavesAsNumber() const override { return true; }
|
||||
|
||||
const StringRef & getNameForValue(const FieldType & value) const
|
||||
{
|
||||
@ -80,7 +88,6 @@ public:
|
||||
}
|
||||
|
||||
Field castToName(const Field & value_or_name) const override;
|
||||
|
||||
Field castToValue(const Field & value_or_name) const override;
|
||||
|
||||
DataTypePtr clone() const override;
|
||||
@ -103,12 +110,13 @@ public:
|
||||
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, const size_t offset, size_t limit) const override;
|
||||
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, const size_t limit, const double avg_value_size_hint) const override;
|
||||
|
||||
size_t getSizeOfField() const override { return sizeof(FieldType); }
|
||||
|
||||
ColumnPtr createColumn() const override { return std::make_shared<ColumnType>(); }
|
||||
|
||||
Field getDefault() const override;
|
||||
void insertDefaultInto(IColumn & column) const override;
|
||||
|
||||
bool textCanContainOnlyValidUTF8() const override;
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(Field); }
|
||||
};
|
||||
|
||||
|
||||
|
@ -16,6 +16,7 @@ private:
|
||||
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
bool isParametric() const override { return true; }
|
||||
|
||||
/// Some types could be still unknown.
|
||||
DataTypeExpression(const DataTypes & argument_types_ = DataTypes(), const DataTypePtr & return_type_ = nullptr)
|
||||
|
@ -70,6 +70,15 @@ public:
|
||||
{
|
||||
return String();
|
||||
}
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isFixedString() const override { return true; };
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return n; }
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -56,10 +56,11 @@ public:
|
||||
std::string getName() const override { return std::string("Interval") + kindToString(); }
|
||||
const char * getFamilyName() const override { return "Interval"; }
|
||||
|
||||
bool behavesAsNumber() const override { return false; }
|
||||
bool notForTables() const override { return true; }
|
||||
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeInterval>(kind); }
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool cannotBeStoredInTables() const override { return true; }
|
||||
bool isCategorial() const override { return false; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -29,6 +29,8 @@ public:
|
||||
return std::make_shared<DataTypeNested>(nested);
|
||||
}
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
|
||||
const NamesAndTypesListPtr & getNestedTypesList() const { return nested; }
|
||||
|
||||
static std::string concatenateNestedName(const std::string & nested_table_name, const std::string & nested_field_name);
|
||||
|
@ -16,7 +16,6 @@ public:
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
const char * getFamilyName() const override { return "Nothing"; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
DataTypePtr clone() const override
|
||||
{
|
||||
@ -28,6 +27,12 @@ public:
|
||||
/// These methods read and write zero bytes just to allow to figure out size of column.
|
||||
void serializeBinaryBulk(const IColumn & column, WriteBuffer & ostr, size_t offset, size_t limit) const override;
|
||||
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return 0; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -31,7 +31,7 @@ DataTypeNullable::DataTypeNullable(const DataTypePtr & nested_data_type_)
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeNullable::isNull() const
|
||||
bool DataTypeNullable::onlyNull() const
|
||||
{
|
||||
return typeid_cast<const DataTypeNothing *>(nested_data_type.get());
|
||||
}
|
||||
@ -281,6 +281,12 @@ ColumnPtr DataTypeNullable::createColumn() const
|
||||
}
|
||||
|
||||
|
||||
size_t DataTypeNullable::getSizeOfValueInMemory() const
|
||||
{
|
||||
throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (arguments->children.size() != 1)
|
||||
@ -297,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;
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -16,14 +16,6 @@ public:
|
||||
DataTypeNullable(const DataTypePtr & nested_data_type_);
|
||||
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
|
||||
const char * getFamilyName() const override { return "Nullable"; }
|
||||
bool isNullable() const override { return true; }
|
||||
bool isNull() const override;
|
||||
|
||||
bool isNumeric() const override { return nested_data_type->isNumeric(); } /// TODO Absolutely wrong.
|
||||
bool isNumericNotNullable() const override { return false; }
|
||||
bool behavesAsNumber() const override { return nested_data_type->behavesAsNumber(); } /// TODO Absolutely wrong.
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeNullable>(nested_data_type->clone()); }
|
||||
|
||||
void enumerateStreams(StreamCallback callback, SubstreamPath path) const override;
|
||||
@ -73,7 +65,20 @@ public:
|
||||
|
||||
Field getDefault() const override { return Null(); }
|
||||
|
||||
size_t getSizeOfField() const override { return nested_data_type->getSizeOfField(); } /// TODO Absolutely wrong.
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return true; }
|
||||
bool cannotBeStoredInTables() const override { return nested_data_type->cannotBeStoredInTables(); }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return nested_data_type->shouldAlignRightInPrettyFormats(); }
|
||||
bool textCanContainOnlyValidUTF8() const override { return nested_data_type->textCanContainOnlyValidUTF8(); }
|
||||
bool canBeComparedWithCollation() const override { return nested_data_type->canBeComparedWithCollation(); }
|
||||
bool canBeUsedAsVersion() const override { return false; }
|
||||
bool isSummable() const override { return nested_data_type->isSummable(); }
|
||||
bool canBeUsedInBooleanContext() const override { return nested_data_type->canBeUsedInBooleanContext(); }
|
||||
bool haveMaximumSizeOfValue() const override { return nested_data_type->haveMaximumSizeOfValue(); }
|
||||
size_t getMaximumSizeOfValueInMemory() const override { return 1 + nested_data_type->getMaximumSizeOfValueInMemory(); }
|
||||
bool isNullable() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override;
|
||||
bool onlyNull() const override;
|
||||
|
||||
DataTypePtr & getNestedType() { return nested_data_type; }
|
||||
const DataTypePtr & getNestedType() const { return nested_data_type; }
|
||||
@ -82,4 +87,8 @@ private:
|
||||
DataTypePtr nested_data_type;
|
||||
};
|
||||
|
||||
|
||||
DataTypePtr makeNullable(const DataTypePtr & type);
|
||||
DataTypePtr removeNullable(const DataTypePtr & type);
|
||||
|
||||
}
|
||||
|
@ -1,3 +1,4 @@
|
||||
#include <type_traits>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
#include <Columns/ColumnVector.h>
|
||||
#include <Columns/ColumnConst.h>
|
||||
@ -238,6 +239,12 @@ ColumnPtr DataTypeNumberBase<T>::createColumn() const
|
||||
return std::make_shared<ColumnVector<T>>();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool DataTypeNumberBase<T>::isValueRepresentedByInteger() const
|
||||
{
|
||||
return std::is_integral_v<T>;
|
||||
}
|
||||
|
||||
|
||||
/// Explicit template instantiations - to avoid code bloat in headers.
|
||||
template class DataTypeNumberBase<UInt8>;
|
||||
|
@ -17,9 +17,6 @@ public:
|
||||
|
||||
const char * getFamilyName() const override { return TypeName<T>::get(); }
|
||||
|
||||
bool isNumeric() const override { return true; }
|
||||
bool behavesAsNumber() const override { return true; }
|
||||
|
||||
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const override;
|
||||
@ -29,7 +26,6 @@ public:
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override;
|
||||
size_t getSizeOfField() const override { return sizeof(FieldType); }
|
||||
Field getDefault() const override;
|
||||
|
||||
/** Format is platform-dependent. */
|
||||
@ -42,6 +38,17 @@ public:
|
||||
void deserializeBinaryBulk(IColumn & column, ReadBuffer & istr, size_t limit, double avg_value_size_hint) const override;
|
||||
|
||||
ColumnPtr createColumn() const override;
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool shouldAlignRightInPrettyFormats() const override { return true; }
|
||||
bool textCanContainOnlyValidUTF8() const override { return true; }
|
||||
bool isValueRepresentedByNumber() const override { return true; }
|
||||
bool isValueRepresentedByInteger() const override;
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return sizeof(T); }
|
||||
bool isCategorial() const override { return isValueRepresentedByInteger(); }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -15,6 +15,7 @@ public:
|
||||
static constexpr bool is_parametric = true;
|
||||
const char * getFamilyName() const override { return "Set"; }
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeSet>(); }
|
||||
bool isParametric() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -54,6 +54,14 @@ public:
|
||||
{
|
||||
return String();
|
||||
}
|
||||
|
||||
bool isParametric() const override { return false; }
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool canBeComparedWithCollation() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isString() const override { return true; };
|
||||
bool isCategorial() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
}}
|
@ -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);
|
||||
|
||||
}
|
||||
|
||||
}
|
@ -297,6 +297,33 @@ void DataTypeTuple::insertDefaultInto(IColumn & column) const
|
||||
}
|
||||
|
||||
|
||||
bool DataTypeTuple::textCanContainOnlyValidUTF8() const
|
||||
{
|
||||
return std::all_of(elems.begin(), elems.end(), [](auto && elem) { return elem->textCanContainOnlyValidUTF8(); });
|
||||
}
|
||||
|
||||
bool DataTypeTuple::haveMaximumSizeOfValue() const
|
||||
{
|
||||
return std::all_of(elems.begin(), elems.end(), [](auto && elem) { return elem->haveMaximumSizeOfValue(); });
|
||||
}
|
||||
|
||||
size_t DataTypeTuple::getMaximumSizeOfValueInMemory() const
|
||||
{
|
||||
size_t res = 0;
|
||||
for (const auto & elem : elems)
|
||||
res += elem->getMaximumSizeOfValueInMemory();
|
||||
return res;
|
||||
}
|
||||
|
||||
size_t DataTypeTuple::getSizeOfValueInMemory() const
|
||||
{
|
||||
size_t res = 0;
|
||||
for (const auto & elem : elems)
|
||||
res += elem->getSizeOfValueInMemory();
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
static DataTypePtr create(const ASTPtr & arguments)
|
||||
{
|
||||
if (arguments->children.empty())
|
||||
|
@ -68,6 +68,13 @@ public:
|
||||
Field getDefault() const override;
|
||||
void insertDefaultInto(IColumn & column) const override;
|
||||
|
||||
bool isParametric() const override { return true; }
|
||||
bool haveSubtypes() const override { return !elems.empty(); }
|
||||
bool textCanContainOnlyValidUTF8() const override;
|
||||
bool haveMaximumSizeOfValue() const override;
|
||||
size_t getMaximumSizeOfValueInMemory() const override;
|
||||
size_t getSizeOfValueInMemory() const override;
|
||||
|
||||
const DataTypes & getElements() const { return elems; }
|
||||
};
|
||||
|
||||
|
@ -13,8 +13,6 @@ class DataTypeUUID final : public DataTypeNumberBase<UInt128>
|
||||
{
|
||||
|
||||
public:
|
||||
bool behavesAsNumber() const override { return false; }
|
||||
|
||||
const char * getFamilyName() const override { return "UUID"; }
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeUUID>(); }
|
||||
|
||||
@ -27,5 +25,9 @@ public:
|
||||
void deserializeTextJSON(IColumn & column, ReadBuffer & istr) const override;
|
||||
void serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const override;
|
||||
|
||||
bool canBeUsedInBitOperations() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -11,7 +11,6 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
|
||||
factory.registerSimpleDataType("UInt16", [] { return DataTypePtr(std::make_shared<DataTypeUInt16>()); });
|
||||
factory.registerSimpleDataType("UInt32", [] { return DataTypePtr(std::make_shared<DataTypeUInt32>()); });
|
||||
factory.registerSimpleDataType("UInt64", [] { return DataTypePtr(std::make_shared<DataTypeUInt64>()); });
|
||||
factory.registerSimpleDataType("UInt128", [] { return DataTypePtr(std::make_shared<DataTypeUInt128>()); });
|
||||
|
||||
factory.registerSimpleDataType("Int8", [] { return DataTypePtr(std::make_shared<DataTypeInt8>()); });
|
||||
factory.registerSimpleDataType("Int16", [] { return DataTypePtr(std::make_shared<DataTypeInt16>()); });
|
||||
|
@ -1,5 +1,6 @@
|
||||
#pragma once
|
||||
|
||||
#include <type_traits>
|
||||
#include <DataTypes/DataTypeNumberBase.h>
|
||||
|
||||
|
||||
@ -10,13 +11,21 @@ template <typename T>
|
||||
class DataTypeNumber final : public DataTypeNumberBase<T>
|
||||
{
|
||||
DataTypePtr clone() const override { return std::make_shared<DataTypeNumber<T>>(); }
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool isSummable() const override { return true; }
|
||||
bool canBeUsedInBitOperations() const override { return true; }
|
||||
bool canBeUsedAsNonNegativeArrayIndex() const override { return isInteger() && std::is_unsigned_v<T>; }
|
||||
bool canBeUsedInBooleanContext() const override { return true; }
|
||||
bool isNumber() const override { return true; }
|
||||
bool isInteger() const override { return std::is_integral_v<T>; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
};
|
||||
|
||||
using DataTypeUInt8 = DataTypeNumber<UInt8>;
|
||||
using DataTypeUInt16 = DataTypeNumber<UInt16>;
|
||||
using DataTypeUInt32 = DataTypeNumber<UInt32>;
|
||||
using DataTypeUInt64 = DataTypeNumber<UInt64>;
|
||||
using DataTypeUInt128 = DataTypeNumber<UInt128>;
|
||||
using DataTypeInt8 = DataTypeNumber<Int8>;
|
||||
using DataTypeInt16 = DataTypeNumber<Int16>;
|
||||
using DataTypeInt32 = DataTypeNumber<Int32>;
|
||||
|
@ -55,9 +55,6 @@ DataTypePtr FieldToDataType::operator() (String &) const
|
||||
|
||||
DataTypePtr FieldToDataType::operator() (Array & x) const
|
||||
{
|
||||
if (x.empty())
|
||||
throw Exception("Cannot infer type of empty array", ErrorCodes::EMPTY_DATA_PASSED);
|
||||
|
||||
DataTypes element_types;
|
||||
element_types.reserve(x.size());
|
||||
|
||||
|
@ -18,6 +18,7 @@ namespace DB
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int MULTIPLE_STREAMS_REQUIRED;
|
||||
extern const int LOGICAL_ERROR;
|
||||
}
|
||||
|
||||
|
||||
@ -37,7 +38,7 @@ void IDataType::updateAvgValueSizeHint(const IColumn & column, double & avg_valu
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr IDataType::createConstColumn(size_t size, const Field & field) const
|
||||
ColumnPtr IDataType::createColumnConst(size_t size, const Field & field) const
|
||||
{
|
||||
ColumnPtr column = createColumn();
|
||||
column->insert(field);
|
||||
@ -55,6 +56,11 @@ void IDataType::deserializeBinaryBulk(IColumn &, ReadBuffer &, size_t, double) c
|
||||
throw Exception("Data type " + getName() + " must be deserialized with multiple streams", ErrorCodes::MULTIPLE_STREAMS_REQUIRED);
|
||||
}
|
||||
|
||||
size_t IDataType::getSizeOfValueInMemory() const
|
||||
{
|
||||
throw Exception("Value of type " + getName() + " in memory is not of fixed size.", ErrorCodes::LOGICAL_ERROR);
|
||||
}
|
||||
|
||||
|
||||
String IDataType::getFileNameForStream(const String & column_name, const IDataType::SubstreamPath & path)
|
||||
{
|
||||
|
@ -41,28 +41,6 @@ public:
|
||||
/// Name of data type family (example: FixedString, Array).
|
||||
virtual const char * getFamilyName() const = 0;
|
||||
|
||||
/// Is this type nullable?
|
||||
virtual bool isNullable() const { return false; }
|
||||
|
||||
/// Is this type can represent only NULL value? (It also implies isNullable)
|
||||
virtual bool isNull() const { return false; }
|
||||
|
||||
/// Is this type numeric? Date and DateTime types are considered as such.
|
||||
virtual bool isNumeric() const { return false; }
|
||||
|
||||
/// Is this type numeric and not nullable?
|
||||
virtual bool isNumericNotNullable() const { return isNumeric(); }
|
||||
|
||||
/// If this type is numeric, are all the arithmetic operations and type casting
|
||||
/// relevant for it? True for numbers. False for Date and DateTime types.
|
||||
virtual bool behavesAsNumber() const { return false; }
|
||||
|
||||
/// If this data type cannot appear in table declaration - only for intermediate values of calculations.
|
||||
virtual bool notForTables() const { return false; }
|
||||
|
||||
/// If this data type cannot be wrapped in Nullable data type.
|
||||
virtual bool canBeInsideNullable() const { return true; }
|
||||
|
||||
virtual DataTypePtr clone() const = 0;
|
||||
|
||||
/** Binary serialization for range of values in column - for writing to disk/network, etc.
|
||||
@ -220,13 +198,13 @@ public:
|
||||
serializeText(column, row_num, ostr);
|
||||
}
|
||||
|
||||
/** Create empty (non-constant) column for corresponding type.
|
||||
/** Create empty column for corresponding type.
|
||||
*/
|
||||
virtual ColumnPtr createColumn() const = 0;
|
||||
|
||||
/** Create constant column for corresponding type, with specified size and value.
|
||||
/** Create ColumnConst for corresponding type, with specified size and value.
|
||||
*/
|
||||
virtual ColumnPtr createConstColumn(size_t size, const Field & field) const;
|
||||
ColumnPtr createColumnConst(size_t size, const Field & field) const;
|
||||
|
||||
/** Get default value of data type.
|
||||
* It is the "default" default, regardless the fact that a table could contain different user-specified default.
|
||||
@ -238,12 +216,6 @@ public:
|
||||
*/
|
||||
virtual void insertDefaultInto(IColumn & column) const;
|
||||
|
||||
/// For fixed-size types, return size of value in bytes. For other data types, return some approximate size just for estimation.
|
||||
virtual size_t getSizeOfField() const
|
||||
{
|
||||
throw Exception("getSizeOfField() method is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
/// Checks that two instances belong to the same type
|
||||
inline bool equals(const IDataType & rhs) const
|
||||
{
|
||||
@ -252,6 +224,133 @@ public:
|
||||
|
||||
virtual ~IDataType() {}
|
||||
|
||||
|
||||
/// Various properties on behaviour of data type.
|
||||
|
||||
/** The data type is dependent on parameters and types with different parameters are different.
|
||||
* Examples: FixedString(N), Tuple(T1, T2), Nullable(T).
|
||||
* Otherwise all instances of the same class are the same types.
|
||||
*/
|
||||
virtual bool isParametric() const = 0;
|
||||
|
||||
/** The data type is dependent on parameters and at least one of them is another type.
|
||||
* Examples: Tuple(T1, T2), Nullable(T). But FixedString(N) is not.
|
||||
*/
|
||||
virtual bool haveSubtypes() const = 0;
|
||||
|
||||
/** Can appear in table definition.
|
||||
* Counterexamples: Interval, Nothing.
|
||||
*/
|
||||
virtual bool cannotBeStoredInTables() const { return false; };
|
||||
|
||||
/** In text formats that render "pretty" tables,
|
||||
* is it better to align value right in table cell.
|
||||
* Examples: numbers, even nullable.
|
||||
*/
|
||||
virtual bool shouldAlignRightInPrettyFormats() const { return false; };
|
||||
|
||||
/** Does formatted value in any text format can contain anything but valid UTF8 sequences.
|
||||
* Example: String (because it can contain arbitary bytes).
|
||||
* Counterexamples: numbers, Date, DateTime.
|
||||
* For Enum, it depends.
|
||||
*/
|
||||
virtual bool textCanContainOnlyValidUTF8() const { return false; };
|
||||
|
||||
/** Does it make sense to use this type with COLLATE modifier in ORDER BY.
|
||||
* Example: String, but not FixedString.
|
||||
*/
|
||||
virtual bool canBeComparedWithCollation() const { return false; };
|
||||
|
||||
/** If the type is totally comparable (Ints, Date, DateTime, not nullable, not floats)
|
||||
* and "simple" enough (not String, FixedString) to be used as version number
|
||||
* (to select rows with maximum version).
|
||||
*/
|
||||
virtual bool canBeUsedAsVersion() const { return false; };
|
||||
|
||||
/** Values of data type can be summed. Example: numbers, even nullable. Not Date/DateTime.
|
||||
*/
|
||||
virtual bool isSummable() const { return false; };
|
||||
|
||||
/** Can be used in operations like bit and, bit shift, bit not, etc.
|
||||
*/
|
||||
virtual bool canBeUsedInBitOperations() const { return false; };
|
||||
|
||||
/** Unsigned integer.
|
||||
*/
|
||||
virtual bool canBeUsedAsNonNegativeArrayIndex() const { return false; };
|
||||
|
||||
/** Can be used in boolean context (WHERE, HAVING).
|
||||
* UInt8, maybe nullable.
|
||||
*/
|
||||
virtual bool canBeUsedInBooleanContext() const { return false; };
|
||||
|
||||
/** Integers, floats, not Nullable. Not Enums. Not Date/DateTime.
|
||||
*/
|
||||
virtual bool isNumber() const { return false; };
|
||||
|
||||
/** Integers. Not Nullable. Not Enums. Not Date/DateTime.
|
||||
*/
|
||||
virtual bool isInteger() const { return false; };
|
||||
|
||||
virtual bool isDateOrDateTime() const { return false; };
|
||||
|
||||
/** Numbers, Enums, Date, DateTime. Not nullable.
|
||||
*/
|
||||
virtual bool isValueRepresentedByNumber() const { return false; };
|
||||
|
||||
/** Integers, Enums, Date, DateTime. Not nullable.
|
||||
*/
|
||||
virtual bool isValueRepresentedByInteger() const { return false; };
|
||||
|
||||
/** Values are unambiguously identified by contents of contiguous memory region,
|
||||
* that can be obtained by IColumn::getDataAt method.
|
||||
* Examples: numbers, Date, DateTime, String, FixedString,
|
||||
* and Arrays of numbers, Date, DateTime, FixedString, Enum, but not String.
|
||||
* (because Array(String) values became ambiguous if you concatenate Strings).
|
||||
* Counterexamples: Nullable, Tuple.
|
||||
*/
|
||||
virtual bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const { return false; };
|
||||
|
||||
virtual bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const
|
||||
{
|
||||
return isValueRepresentedByNumber() || isFixedString();
|
||||
};
|
||||
|
||||
virtual bool isString() const { return false; };
|
||||
virtual bool isFixedString() const { return false; };
|
||||
virtual bool isStringOrFixedString() const { return isString() || isFixedString(); };
|
||||
|
||||
/** Example: numbers, Date, DateTime, FixedString, Enum... Nullable and Tuple of such types.
|
||||
* Counterexamples: String, Array.
|
||||
* It's Ok to return false for AggregateFunction despite the fact that some of them have fixed size state.
|
||||
*/
|
||||
virtual bool haveMaximumSizeOfValue() const { return false; };
|
||||
|
||||
/** Size in amount of bytes in memory. Throws an exception if not haveMaximumSizeOfValue.
|
||||
*/
|
||||
virtual size_t getMaximumSizeOfValueInMemory() const { return getSizeOfValueInMemory(); }
|
||||
|
||||
/** Throws an exception if value is not of fixed size.
|
||||
*/
|
||||
virtual size_t getSizeOfValueInMemory() const;
|
||||
|
||||
/** Integers (not floats), Enum, String, FixedString.
|
||||
*/
|
||||
virtual bool isCategorial() const { return false; };
|
||||
|
||||
virtual bool isEnum() const { return false; };
|
||||
|
||||
virtual bool isNullable() const { return false; }
|
||||
|
||||
/** Is this type can represent only NULL value? (It also implies isNullable)
|
||||
*/
|
||||
virtual bool onlyNull() const { return false; }
|
||||
|
||||
/** If this data type cannot be wrapped in Nullable data type.
|
||||
*/
|
||||
virtual bool canBeInsideNullable() const { return false; };
|
||||
|
||||
|
||||
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
|
||||
static void updateAvgValueSizeHint(const IColumn & column, double & avg_value_size_hint);
|
||||
|
||||
|
@ -14,16 +14,6 @@ namespace DB
|
||||
class IDataTypeDummy : public IDataType
|
||||
{
|
||||
private:
|
||||
bool notForTables() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
bool canBeInsideNullable() const override
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
void throwNoSerialization() const
|
||||
{
|
||||
throw Exception("Serialization is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
@ -60,6 +50,9 @@ public:
|
||||
{
|
||||
throw Exception("Method insertDefaultInto() is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);
|
||||
}
|
||||
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool cannotBeStoredInTables() const override { return true; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -9,8 +9,6 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeNothing.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeFixedString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
|
||||
@ -74,6 +72,19 @@ DataTypePtr getLeastCommonType(const DataTypes & types)
|
||||
|
||||
/// Recursive rules
|
||||
|
||||
/// If there are Nothing types, skip them
|
||||
{
|
||||
DataTypes non_nothing_types;
|
||||
non_nothing_types.reserve(types.size());
|
||||
|
||||
for (const auto & type : types)
|
||||
if (!typeid_cast<const DataTypeNothing *>(type.get()))
|
||||
non_nothing_types.emplace_back(type);
|
||||
|
||||
if (non_nothing_types.size() < types.size())
|
||||
return getLeastCommonType(non_nothing_types);
|
||||
}
|
||||
|
||||
/// For Arrays
|
||||
{
|
||||
bool have_array = false;
|
||||
@ -159,7 +170,7 @@ DataTypePtr getLeastCommonType(const DataTypes & types)
|
||||
{
|
||||
have_nullable = true;
|
||||
|
||||
if (!type_nullable->isNull())
|
||||
if (!type_nullable->onlyNull())
|
||||
nested_types.emplace_back(type_nullable->getNestedType());
|
||||
}
|
||||
else
|
||||
@ -175,15 +186,14 @@ DataTypePtr getLeastCommonType(const DataTypes & types)
|
||||
/// Non-recursive rules
|
||||
|
||||
/// For String and FixedString, or for different FixedStrings, the common type is String.
|
||||
/// No other types are compatible with Strings.
|
||||
/// No other types are compatible with Strings. TODO Enums?
|
||||
{
|
||||
bool have_string = false;
|
||||
bool all_strings = true;
|
||||
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (typeid_cast<const DataTypeString *>(type.get())
|
||||
|| typeid_cast<const DataTypeFixedString *>(type.get()))
|
||||
if (type->isStringOrFixedString())
|
||||
have_string = true;
|
||||
else
|
||||
all_strings = false;
|
||||
@ -205,8 +215,7 @@ DataTypePtr getLeastCommonType(const DataTypes & types)
|
||||
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (typeid_cast<const DataTypeDate *>(type.get())
|
||||
|| typeid_cast<const DataTypeDateTime *>(type.get()))
|
||||
if (type->isDateOrDateTime())
|
||||
have_date_or_datetime = true;
|
||||
else
|
||||
all_date_or_datetime = false;
|
||||
|
@ -12,7 +12,6 @@
|
||||
#include <Common/ProfileEvents.h>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <Dictionaries/CacheDictionary.h>
|
||||
#include <Dictionaries/DictionaryBlockInputStream.h>
|
||||
#include <ext/size.h>
|
||||
|
@ -5,6 +5,7 @@
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -1,8 +1,14 @@
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <Columns/IColumn.h>
|
||||
#include <Common/StringUtils.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
|
||||
#include <ext/range.h>
|
||||
#include <numeric>
|
||||
#include <unordered_set>
|
||||
#include <unordered_map>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -222,7 +228,7 @@ bool DictionaryStructure::isKeySizeFixed() const
|
||||
size_t DictionaryStructure::getKeySize() const
|
||||
{
|
||||
return std::accumulate(std::begin(*key), std::end(*key), size_t{},
|
||||
[] (const auto running_size, const auto & key_i) {return running_size + key_i.type->getSizeOfField(); });
|
||||
[] (const auto running_size, const auto & key_i) {return running_size + key_i.type->getSizeOfValueInMemory(); });
|
||||
}
|
||||
|
||||
|
||||
|
@ -1,13 +1,10 @@
|
||||
#pragma once
|
||||
|
||||
#include <DataTypes/DataTypeFactory.h>
|
||||
#include <DataTypes/IDataType.h>
|
||||
#include <IO/ReadBufferFromString.h>
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteHelpers.h>
|
||||
#include <Interpreters/IExternalLoadable.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <ext/range.h>
|
||||
#include <numeric>
|
||||
|
||||
#include <vector>
|
||||
#include <string>
|
||||
#include <map>
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <DataStreams/OwningBlockInputStream.h>
|
||||
#include <Dictionaries/DictionarySourceHelpers.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <DataStreams/OwningBlockInputStream.h>
|
||||
#include <IO/ReadWriteBufferFromHTTP.h>
|
||||
#include <DataStreams/IBlockOutputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <IO/WriteBufferFromOStream.h>
|
||||
#include <Dictionaries/DictionarySourceHelpers.h>
|
||||
#include <common/logger_useful.h>
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <Dictionaries/IDictionarySource.h>
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <common/LocalDateTime.h>
|
||||
|
||||
|
||||
namespace Poco { class Logger; }
|
||||
|
@ -1,10 +1,12 @@
|
||||
#include <DataStreams/OneBlockInputStream.h>
|
||||
#include <Dictionaries/LibraryDictionarySource.h>
|
||||
#include <Dictionaries/LibraryDictionarySourceExternal.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Poco/File.h>
|
||||
#include "LibraryDictionarySourceExternal.h"
|
||||
#include <common/logger_useful.h>
|
||||
#include <ext/bit_cast.h>
|
||||
#include <ext/range.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -4,13 +4,17 @@
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Dictionaries/ExternalResultDescription.h>
|
||||
#include <Dictionaries/IDictionarySource.h>
|
||||
|
||||
#include <iostream>
|
||||
#include <common/LocalDateTime.h>
|
||||
|
||||
|
||||
namespace Poco
|
||||
{
|
||||
class Logger;
|
||||
class Logger;
|
||||
|
||||
namespace Util
|
||||
{
|
||||
class AbstractConfiguration;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -61,4 +65,5 @@ private:
|
||||
ExternalResultDescription description;
|
||||
std::shared_ptr<CStringsHolder> settings;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -14,10 +14,6 @@
|
||||
|
||||
#include <Dictionaries/DictionaryStructure.h>
|
||||
#include <Dictionaries/MongoDBBlockInputStream.h>
|
||||
#include <DataTypes/DataTypesNumber.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <DataTypes/DataTypeDate.h>
|
||||
#include <DataTypes/DataTypeDateTime.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Columns/ColumnsNumber.h>
|
||||
#include <ext/range.h>
|
||||
|
@ -1,9 +1,10 @@
|
||||
#include <Common/config.h>
|
||||
#if USE_MYSQL
|
||||
|
||||
#include <IO/WriteBufferFromString.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
#include <Columns/ColumnString.h>
|
||||
#include <Common/config.h>
|
||||
|
||||
#if USE_MYSQL
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
|
||||
#include <common/logger_useful.h>
|
||||
|
||||
@ -187,4 +188,3 @@ std::string MySQLDictionarySource::doInvalidateQuery(const std::string & request
|
||||
}
|
||||
|
||||
#endif
|
||||
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user