提交 ce5fe046 编写于 作者: B BayoNet

Merge remote-tracking branch 'upstream/master'

......@@ -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,43 +21,15 @@ 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())
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);
if (!nested_column->canBeInsideNullable())
throw Exception{getName() + " cannot be inside Nullable column", ErrorCodes::ILLEGAL_COLUMN};
return new_col_holder;
if (null_map->isColumnConst())
throw Exception{"ColumnNullable cannot have constant null map", ErrorCodes::ILLEGAL_COLUMN};
}
......@@ -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)
{
......
#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())
{
// Leave the same column
res.insert(src_column);
}
else
if (cast_description.end() != it)
{
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);
}
}
......
#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 = init_observed_column();
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();
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();
/// Access the null values byte map content.
ColumnPtr & null_map = nullable_col.getNullMapColumn();
ColumnUInt8 & content = static_cast<ColumnUInt8 &>(*null_map);
auto & data = content.getData();
const NullMap & null_map = static_cast<ColumnNullable &>(*column).getNullMap();
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();
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;
}
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
{
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);
for (size_t j = 0; j < rows; ++j)
{
{
WriteBufferFromString out(serialized_value);
elem.type->serializeTextEscaped(*elem.column, j, out);
}
widths[i].resize(rows);
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>
......
#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();
......
#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);
}
}
#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);
}
#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);
}
#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; }
};
}
#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;
}
}}
#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>()); });
......
#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; }
};
}
......
#include <sstream>
#include <IO/WriteBufferFromString.h>
#include <IO/Operators.h>
#include <Common/typeid_cast.h>
#include <DataTypes/getLeastCommonType.h>
......@@ -8,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>
......@@ -27,7 +26,7 @@ namespace
{
String getExceptionMessagePrefix(const DataTypes & types)
{
std::stringstream res;
WriteBufferFromOwnString res;
res << "There is no common type for types ";
bool first = true;
......@@ -73,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;
......@@ -130,18 +142,18 @@ DataTypePtr getLeastCommonType(const DataTypes & types)
}
else
all_tuples = false;
}
if (have_tuple)
{
if (!all_tuples)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Tuple and some of them are not", ErrorCodes::NO_COMMON_TYPE);
if (have_tuple)
{
if (!all_tuples)
throw Exception(getExceptionMessagePrefix(types) + " because some of them are Tuple and some of them are not", ErrorCodes::NO_COMMON_TYPE);
DataTypes common_tuple_types(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
common_tuple_types[elem_idx] = getLeastCommonType(nested_types[elem_idx]);
DataTypes common_tuple_types(tuple_size);
for (size_t elem_idx = 0; elem_idx < tuple_size; ++elem_idx)
common_tuple_types[elem_idx] = getLeastCommonType(nested_types[elem_idx]);
return std::make_shared<DataTypeTuple>(common_tuple_types);
}
return std::make_shared<DataTypeTuple>(common_tuple_types);
}
}
......@@ -158,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
......@@ -174,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;
......@@ -204,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
......
#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(); });
}
......
#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; }
......
#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>
......
#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
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
此差异已折叠。
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册