mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into CLICKHOUSE-3723.3
This commit is contained in:
commit
603b31bdb6
@ -8,7 +8,12 @@
|
||||
# sudo apt-get install ninja-build
|
||||
|
||||
# CLion does not support Ninja
|
||||
if (NOT ${CMAKE_COMMAND} MATCHES "clion")
|
||||
# You can add your vote on CLion task tracker:
|
||||
# https://youtrack.jetbrains.com/issue/CPP-2659
|
||||
# https://youtrack.jetbrains.com/issue/CPP-870
|
||||
|
||||
string(TOLOWER "${CMAKE_COMMAND}" CMAKE_COMMAND_LOWER)
|
||||
if (NOT ${CMAKE_COMMAND_LOWER} MATCHES "clion")
|
||||
find_program(NINJA_PATH ninja)
|
||||
if (NINJA_PATH)
|
||||
set(CMAKE_GENERATOR "Ninja" CACHE INTERNAL "" FORCE)
|
||||
|
@ -29,7 +29,7 @@ if (ENABLE_CAPNP)
|
||||
find_library (CAPNP capnp PATHS ${CAPNP_PATHS})
|
||||
find_library (CAPNPC capnpc PATHS ${CAPNP_PATHS})
|
||||
find_library (KJ kj PATHS ${CAPNP_PATHS})
|
||||
set (CAPNP_LIBRARY ${CAPNP} ${CAPNPC} ${KJ})
|
||||
set (CAPNP_LIBRARY ${CAPNPC} ${CAPNP} ${KJ})
|
||||
find_path (CAPNP_INCLUDE_DIR NAMES capnp/schema-parser.h PATHS ${CAPNP_INCLUDE_PATHS})
|
||||
endif ()
|
||||
|
||||
|
2
contrib/poco
vendored
2
contrib/poco
vendored
@ -1 +1 @@
|
||||
Subproject commit 3df947389e6d9654919002797bdd86ed190b3963
|
||||
Subproject commit d7a4383c4d85b51938b62ed5812bc0935245edb3
|
@ -862,9 +862,9 @@ class ModelFactory
|
||||
public:
|
||||
ModelPtr get(const IDataType & data_type, UInt64 seed, MarkovModelParameters markov_model_params) const
|
||||
{
|
||||
if (data_type.isInteger())
|
||||
if (isInteger(data_type))
|
||||
{
|
||||
if (data_type.isUnsignedInteger())
|
||||
if (isUnsignedInteger(data_type))
|
||||
return std::make_unique<UnsignedIntegerModel>(seed);
|
||||
else
|
||||
return std::make_unique<SignedIntegerModel>(seed);
|
||||
|
@ -79,7 +79,7 @@ public:
|
||||
if (arguments.size() != 2)
|
||||
throw Exception("Aggregate function " + getName() + " requires two arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[1]->isUnsignedInteger())
|
||||
if (!isUnsignedInteger(arguments[1]))
|
||||
throw Exception("Second argument of aggregate function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
type = arguments.front();
|
||||
|
@ -61,10 +61,10 @@ public:
|
||||
AggregateFunctionIntersectionsMax(AggregateFunctionIntersectionsKind kind_, const DataTypes & arguments)
|
||||
: kind(kind_)
|
||||
{
|
||||
if (!arguments[0]->isNumber())
|
||||
if (!isNumber(arguments[0]))
|
||||
throw Exception{getName() + ": first argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!arguments[1]->isNumber())
|
||||
if (!isNumber(arguments[1]))
|
||||
throw Exception{getName() + ": second argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!arguments[0]->equals(*arguments[1]))
|
||||
|
@ -20,7 +20,7 @@ namespace ErrorCodes
|
||||
}
|
||||
|
||||
///
|
||||
inline bool allowDecimalComparison(const IDataType * left_type, const IDataType * right_type)
|
||||
inline bool allowDecimalComparison(const DataTypePtr & left_type, const DataTypePtr & right_type)
|
||||
{
|
||||
if (isDecimal(left_type))
|
||||
{
|
||||
|
@ -59,7 +59,7 @@ template <> struct TypeName<String> { static const char * get() { return "Strin
|
||||
|
||||
enum class TypeIndex
|
||||
{
|
||||
None = 0,
|
||||
Nothing = 0,
|
||||
UInt8,
|
||||
UInt16,
|
||||
UInt32,
|
||||
@ -84,6 +84,12 @@ enum class TypeIndex
|
||||
UUID,
|
||||
Array,
|
||||
Tuple,
|
||||
Set,
|
||||
Interval,
|
||||
Nullable,
|
||||
Function,
|
||||
AggregateFunction,
|
||||
LowCardinality,
|
||||
};
|
||||
|
||||
template <typename T> struct TypeId;
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <DataStreams/RollupBlockInputStream.h>
|
||||
#include <DataStreams/finalizeBlock.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Columns/FilterDescription.h>
|
||||
@ -6,23 +7,7 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
static void finalize(Block & block)
|
||||
{
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
ColumnWithTypeAndName & current = block.getByPosition(i);
|
||||
const DataTypeAggregateFunction * unfinalized_type = typeid_cast<const DataTypeAggregateFunction *>(current.type.get());
|
||||
|
||||
if (unfinalized_type)
|
||||
{
|
||||
current.type = unfinalized_type->getReturnType();
|
||||
if (current.column)
|
||||
current.column = typeid_cast<const ColumnAggregateFunction &>(*current.column).convertToValues();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
RollupBlockInputStream::RollupBlockInputStream(
|
||||
const BlockInputStreamPtr & input_, const Aggregator::Params & params_) : aggregator(params_),
|
||||
keys(params_.keys)
|
||||
@ -36,7 +21,7 @@ RollupBlockInputStream::RollupBlockInputStream(
|
||||
Block RollupBlockInputStream::getHeader() const
|
||||
{
|
||||
Block res = children.at(0)->getHeader();
|
||||
finalize(res);
|
||||
finalizeBlock(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
@ -58,7 +43,7 @@ Block RollupBlockInputStream::readImpl()
|
||||
rollup_block = aggregator.mergeBlocks(rollup_blocks, false);
|
||||
|
||||
Block finalized = rollup_block;
|
||||
finalize(finalized);
|
||||
finalizeBlock(finalized);
|
||||
return finalized;
|
||||
}
|
||||
|
||||
@ -66,7 +51,7 @@ Block RollupBlockInputStream::readImpl()
|
||||
current_key = keys.size() - 1;
|
||||
|
||||
rollup_block = block;
|
||||
finalize(block);
|
||||
finalizeBlock(block);
|
||||
|
||||
return block;
|
||||
}
|
||||
|
@ -76,7 +76,7 @@ SummingSortedBlockInputStream::SummingSortedBlockInputStream(
|
||||
}
|
||||
else
|
||||
{
|
||||
bool is_agg_func = checkDataType<DataTypeAggregateFunction>(column.type.get());
|
||||
bool is_agg_func = WhichDataType(column.type).isAggregateFunction();
|
||||
if (!column.type->isSummable() && !is_agg_func)
|
||||
{
|
||||
column_numbers_not_to_aggregate.push_back(i);
|
||||
@ -273,7 +273,7 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
{
|
||||
// Wrap aggregated columns in a tuple to match function signature
|
||||
if (!desc.is_agg_func_type && checkDataType<DataTypeTuple>(desc.function->getReturnType().get()))
|
||||
if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType()))
|
||||
{
|
||||
size_t tuple_size = desc.column_numbers.size();
|
||||
MutableColumns tuple_columns(tuple_size);
|
||||
@ -292,7 +292,7 @@ Block SummingSortedBlockInputStream::readImpl()
|
||||
/// Place aggregation results into block.
|
||||
for (auto & desc : columns_to_aggregate)
|
||||
{
|
||||
if (!desc.is_agg_func_type && checkDataType<DataTypeTuple>(desc.function->getReturnType().get()))
|
||||
if (!desc.is_agg_func_type && isTuple(desc.function->getReturnType()))
|
||||
{
|
||||
/// Unpack tuple into block.
|
||||
size_t tuple_size = desc.column_numbers.size();
|
||||
|
@ -1,4 +1,5 @@
|
||||
#include <DataStreams/TotalsHavingBlockInputStream.h>
|
||||
#include <DataStreams/finalizeBlock.h>
|
||||
#include <Interpreters/ExpressionActions.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
@ -53,23 +54,6 @@ TotalsHavingBlockInputStream::TotalsHavingBlockInputStream(
|
||||
}
|
||||
|
||||
|
||||
static void finalize(Block & block)
|
||||
{
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
ColumnWithTypeAndName & current = block.getByPosition(i);
|
||||
const DataTypeAggregateFunction * unfinalized_type = typeid_cast<const DataTypeAggregateFunction *>(current.type.get());
|
||||
|
||||
if (unfinalized_type)
|
||||
{
|
||||
current.type = unfinalized_type->getReturnType();
|
||||
if (current.column)
|
||||
current.column = typeid_cast<const ColumnAggregateFunction &>(*current.column).convertToValues();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Block TotalsHavingBlockInputStream::getTotals()
|
||||
{
|
||||
if (!totals)
|
||||
@ -87,7 +71,7 @@ Block TotalsHavingBlockInputStream::getTotals()
|
||||
}
|
||||
|
||||
totals = children.at(0)->getHeader().cloneWithColumns(std::move(current_totals));
|
||||
finalize(totals);
|
||||
finalizeBlock(totals);
|
||||
}
|
||||
|
||||
if (totals && expression)
|
||||
@ -101,7 +85,7 @@ Block TotalsHavingBlockInputStream::getHeader() const
|
||||
{
|
||||
Block res = children.at(0)->getHeader();
|
||||
if (final)
|
||||
finalize(res);
|
||||
finalizeBlock(res);
|
||||
if (expression)
|
||||
expression->execute(res);
|
||||
return res;
|
||||
@ -129,7 +113,7 @@ Block TotalsHavingBlockInputStream::readImpl()
|
||||
|
||||
finalized = block;
|
||||
if (final)
|
||||
finalize(finalized);
|
||||
finalizeBlock(finalized);
|
||||
|
||||
total_keys += finalized.rows();
|
||||
|
||||
|
24
dbms/src/DataStreams/finalizeBlock.cpp
Normal file
24
dbms/src/DataStreams/finalizeBlock.cpp
Normal file
@ -0,0 +1,24 @@
|
||||
#include <DataStreams/finalizeBlock.h>
|
||||
#include <DataTypes/DataTypeAggregateFunction.h>
|
||||
#include <Columns/ColumnAggregateFunction.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
void finalizeBlock(Block & block)
|
||||
{
|
||||
for (size_t i = 0; i < block.columns(); ++i)
|
||||
{
|
||||
ColumnWithTypeAndName & current = block.getByPosition(i);
|
||||
const DataTypeAggregateFunction * unfinalized_type = typeid_cast<const DataTypeAggregateFunction *>(current.type.get());
|
||||
|
||||
if (unfinalized_type)
|
||||
{
|
||||
current.type = unfinalized_type->getReturnType();
|
||||
if (current.column)
|
||||
current.column = typeid_cast<const ColumnAggregateFunction &>(*current.column).convertToValues();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
9
dbms/src/DataStreams/finalizeBlock.h
Normal file
9
dbms/src/DataStreams/finalizeBlock.h
Normal file
@ -0,0 +1,9 @@
|
||||
#pragma once
|
||||
|
||||
#include <Core/Block.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
/// Converts aggregate function columns with non-finalized states to final values
|
||||
void finalizeBlock(Block & block);
|
||||
}
|
@ -30,8 +30,8 @@ public:
|
||||
AggregateFunctionPtr getFunction() const { return function; }
|
||||
|
||||
std::string getName() const override;
|
||||
|
||||
const char * getFamilyName() const override { return "AggregateFunction"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::AggregateFunction; }
|
||||
|
||||
bool canBeInsideNullable() const override { return false; }
|
||||
|
||||
|
@ -23,7 +23,6 @@ public:
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool isDateOrDateTime() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
@ -48,7 +48,6 @@ public:
|
||||
void deserializeTextCSV(IColumn & column, ReadBuffer & istr, const FormatSettings & settings) const override;
|
||||
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool isDateOrDateTime() const override { return true; }
|
||||
bool canBeInsideNullable() const override { return true; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
@ -30,7 +30,6 @@ public:
|
||||
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; }
|
||||
bool isComparable() const override { return true; }
|
||||
};
|
||||
|
@ -77,7 +77,7 @@ public:
|
||||
bool haveSubtypes() const override { return false; }
|
||||
bool isComparable() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isFixedString() const override { return true; }
|
||||
bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const override { return true; }
|
||||
bool haveMaximumSizeOfValue() const override { return true; }
|
||||
size_t getSizeOfValueInMemory() const override { return n; }
|
||||
bool isCategorial() const override { return true; }
|
||||
|
@ -24,6 +24,7 @@ public:
|
||||
|
||||
std::string getName() const override;
|
||||
const char * getFamilyName() const override { return "Function"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Function; }
|
||||
|
||||
const DataTypes & getArgumentTypes() const
|
||||
{
|
||||
|
@ -55,6 +55,7 @@ public:
|
||||
|
||||
std::string getName() const override { return std::string("Interval") + kindToString(); }
|
||||
const char * getFamilyName() const override { return "Interval"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Interval; }
|
||||
|
||||
bool equals(const IDataType & rhs) const override;
|
||||
|
||||
|
@ -16,6 +16,7 @@ public:
|
||||
static constexpr bool is_parametric = false;
|
||||
|
||||
const char * getFamilyName() const override { return "Nothing"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Nothing; }
|
||||
|
||||
MutableColumnPtr createColumn() const override;
|
||||
|
||||
|
@ -16,6 +16,7 @@ public:
|
||||
explicit DataTypeNullable(const DataTypePtr & nested_data_type_);
|
||||
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
|
||||
const char * getFamilyName() const override { return "Nullable"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Nullable; }
|
||||
|
||||
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||
|
||||
|
@ -14,6 +14,7 @@ class DataTypeSet final : public IDataTypeDummy
|
||||
public:
|
||||
static constexpr bool is_parametric = true;
|
||||
const char * getFamilyName() const override { return "Set"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::Set; }
|
||||
bool equals(const IDataType & rhs) const override { return typeid(rhs) == typeid(*this); }
|
||||
bool isParametric() const override { return true; }
|
||||
};
|
||||
|
@ -59,7 +59,6 @@ public:
|
||||
bool isComparable() const override { return true; }
|
||||
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; }
|
||||
};
|
||||
|
@ -42,9 +42,9 @@ DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_)
|
||||
if (dictionary_type->isNullable())
|
||||
inner_type = static_cast<const DataTypeNullable &>(*dictionary_type).getNestedType();
|
||||
|
||||
if (!inner_type->isStringOrFixedString()
|
||||
&& !inner_type->isDateOrDateTime()
|
||||
&& !inner_type->isNumber())
|
||||
if (!isStringOrFixedString(inner_type)
|
||||
&& !isDateOrDateTime(inner_type)
|
||||
&& !isNumber(inner_type))
|
||||
throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got "
|
||||
+ dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -769,15 +769,15 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDat
|
||||
if (auto * nullable_type = typeid_cast<const DataTypeNullable *>(&keys_type))
|
||||
type = nullable_type->getNestedType().get();
|
||||
|
||||
if (type->isString())
|
||||
if (isString(type))
|
||||
return creator((ColumnString *)(nullptr));
|
||||
if (type->isFixedString())
|
||||
if (isFixedString(type))
|
||||
return creator((ColumnFixedString *)(nullptr));
|
||||
if (typeid_cast<const DataTypeDate *>(type))
|
||||
return creator((ColumnVector<UInt16> *)(nullptr));
|
||||
if (typeid_cast<const DataTypeDateTime *>(type))
|
||||
return creator((ColumnVector<UInt32> *)(nullptr));
|
||||
if (type->isNumber())
|
||||
if (isNumber(type))
|
||||
{
|
||||
MutableColumnUniquePtr column;
|
||||
TypeListNumbers::forEach(CreateColumnVector(column, *type, creator));
|
||||
|
@ -20,6 +20,7 @@ public:
|
||||
return "LowCardinality(" + dictionary_type->getName() + ")";
|
||||
}
|
||||
const char * getFamilyName() const override { return "LowCardinality"; }
|
||||
TypeIndex getTypeId() const override { return TypeIndex::LowCardinality; }
|
||||
|
||||
void enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const override;
|
||||
|
||||
@ -126,20 +127,13 @@ public:
|
||||
bool isSummable() const override { return dictionary_type->isSummable(); }
|
||||
bool canBeUsedInBitOperations() const override { return dictionary_type->canBeUsedInBitOperations(); }
|
||||
bool canBeUsedInBooleanContext() const override { return dictionary_type->canBeUsedInBooleanContext(); }
|
||||
bool isNumber() const override { return false; }
|
||||
bool isInteger() const override { return false; }
|
||||
bool isUnsignedInteger() const override { return false; }
|
||||
bool isDateOrDateTime() const override { return false; }
|
||||
bool isValueRepresentedByNumber() const override { return dictionary_type->isValueRepresentedByNumber(); }
|
||||
bool isValueRepresentedByInteger() const override { return dictionary_type->isValueRepresentedByInteger(); }
|
||||
bool isValueUnambiguouslyRepresentedInContiguousMemoryRegion() const override { return true; }
|
||||
bool isString() const override { return false; }
|
||||
bool isFixedString() const override { return false; }
|
||||
bool haveMaximumSizeOfValue() const override { return dictionary_type->haveMaximumSizeOfValue(); }
|
||||
size_t getMaximumSizeOfValueInMemory() const override { return dictionary_type->getMaximumSizeOfValueInMemory(); }
|
||||
size_t getSizeOfValueInMemory() const override { return dictionary_type->getSizeOfValueInMemory(); }
|
||||
bool isCategorial() const override { return false; }
|
||||
bool isEnum() const override { return false; }
|
||||
bool isNullable() const override { return false; }
|
||||
bool onlyNull() const override { return false; }
|
||||
bool withDictionary() const override { return true; }
|
||||
|
@ -15,10 +15,7 @@ class DataTypeNumber final : public DataTypeNumberBase<T>
|
||||
bool canBeUsedAsVersion() const override { return true; }
|
||||
bool isSummable() const override { return true; }
|
||||
bool canBeUsedInBitOperations() const override { return true; }
|
||||
bool isUnsignedInteger() 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; }
|
||||
};
|
||||
|
||||
|
@ -46,7 +46,7 @@ public:
|
||||
virtual const char * getFamilyName() const = 0;
|
||||
|
||||
/// Unique type number or zero
|
||||
virtual TypeIndex getTypeId() const { return TypeIndex::None; }
|
||||
virtual TypeIndex getTypeId() const = 0;
|
||||
|
||||
/** Binary serialization for range of values in column - for writing to disk/network, etc.
|
||||
*
|
||||
@ -342,17 +342,6 @@ public:
|
||||
*/
|
||||
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 isUnsignedInteger() const { return false; }
|
||||
|
||||
virtual bool isDateOrDateTime() const { return false; }
|
||||
|
||||
/** Numbers, Enums, Date, DateTime. Not nullable.
|
||||
*/
|
||||
virtual bool isValueRepresentedByNumber() const { return false; }
|
||||
@ -376,13 +365,9 @@ public:
|
||||
|
||||
virtual bool isValueUnambiguouslyRepresentedInFixedSizeContiguousMemoryRegion() const
|
||||
{
|
||||
return isValueRepresentedByNumber() || isFixedString();
|
||||
return isValueRepresentedByNumber();
|
||||
}
|
||||
|
||||
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.
|
||||
@ -401,8 +386,6 @@ public:
|
||||
*/
|
||||
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)
|
||||
@ -423,11 +406,20 @@ public:
|
||||
};
|
||||
|
||||
|
||||
struct DataTypeExtractor
|
||||
/// Some sugar to check data type of IDataType
|
||||
struct WhichDataType
|
||||
{
|
||||
TypeIndex idx;
|
||||
|
||||
DataTypeExtractor(const IDataType * data_type)
|
||||
WhichDataType(const IDataType & data_type)
|
||||
: idx(data_type.getTypeId())
|
||||
{}
|
||||
|
||||
WhichDataType(const IDataType * data_type)
|
||||
: idx(data_type->getTypeId())
|
||||
{}
|
||||
|
||||
WhichDataType(const DataTypePtr & data_type)
|
||||
: idx(data_type->getTypeId())
|
||||
{}
|
||||
|
||||
@ -437,6 +429,7 @@ struct DataTypeExtractor
|
||||
bool isUInt64() const { return idx == TypeIndex::UInt64; }
|
||||
bool isUInt128() const { return idx == TypeIndex::UInt128; }
|
||||
bool isUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64() || isUInt128(); }
|
||||
bool isNativeUInt() const { return isUInt8() || isUInt16() || isUInt32() || isUInt64(); }
|
||||
|
||||
bool isInt8() const { return idx == TypeIndex::Int8; }
|
||||
bool isInt16() const { return idx == TypeIndex::Int16; }
|
||||
@ -444,6 +437,7 @@ struct DataTypeExtractor
|
||||
bool isInt64() const { return idx == TypeIndex::Int64; }
|
||||
bool isInt128() const { return idx == TypeIndex::Int128; }
|
||||
bool isInt() const { return isInt8() || isInt16() || isInt32() || isInt64() || isInt128(); }
|
||||
bool isNativeInt() const { return isInt8() || isInt16() || isInt32() || isInt64(); }
|
||||
|
||||
bool isDecimal32() const { return idx == TypeIndex::Decimal32; }
|
||||
bool isDecimal64() const { return idx == TypeIndex::Decimal64; }
|
||||
@ -469,27 +463,69 @@ struct DataTypeExtractor
|
||||
bool isUUID() const { return idx == TypeIndex::UUID; }
|
||||
bool isArray() const { return idx == TypeIndex::Array; }
|
||||
bool isTuple() const { return idx == TypeIndex::Tuple; }
|
||||
bool isSet() const { return idx == TypeIndex::Set; }
|
||||
bool isInterval() const { return idx == TypeIndex::Interval; }
|
||||
|
||||
bool isNothing() const { return idx == TypeIndex::Nothing; }
|
||||
bool isNullable() const { return idx == TypeIndex::Nullable; }
|
||||
bool isFunction() const { return idx == TypeIndex::Function; }
|
||||
bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; }
|
||||
};
|
||||
|
||||
/// IDataType helpers (alternative for IDataType virtual methods)
|
||||
/// IDataType helpers (alternative for IDataType virtual methods with single point of truth)
|
||||
|
||||
inline bool isEnum(const IDataType * data_type)
|
||||
inline bool isDateOrDateTime(const DataTypePtr & data_type) { return WhichDataType(data_type).isDateOrDateTime(); }
|
||||
inline bool isEnum(const DataTypePtr & data_type) { return WhichDataType(data_type).isEnum(); }
|
||||
inline bool isDecimal(const DataTypePtr & data_type) { return WhichDataType(data_type).isDecimal(); }
|
||||
inline bool isTuple(const DataTypePtr & data_type) { return WhichDataType(data_type).isTuple(); }
|
||||
inline bool isArray(const DataTypePtr & data_type) { return WhichDataType(data_type).isArray(); }
|
||||
|
||||
template <typename T>
|
||||
inline bool isUnsignedInteger(const T & data_type)
|
||||
{
|
||||
return DataTypeExtractor(data_type).isEnum();
|
||||
return WhichDataType(data_type).isUInt();
|
||||
}
|
||||
|
||||
inline bool isDecimal(const IDataType * data_type)
|
||||
template <typename T>
|
||||
inline bool isInteger(const T & data_type)
|
||||
{
|
||||
return DataTypeExtractor(data_type).isDecimal();
|
||||
}
|
||||
|
||||
inline bool isNotDecimalButComparableToDecimal(const IDataType * data_type)
|
||||
{
|
||||
DataTypeExtractor which(data_type);
|
||||
WhichDataType which(data_type);
|
||||
return which.isInt() || which.isUInt();
|
||||
}
|
||||
|
||||
inline bool isCompilableType(const IDataType * data_type)
|
||||
template <typename T>
|
||||
inline bool isNumber(const T & data_type)
|
||||
{
|
||||
WhichDataType which(data_type);
|
||||
return which.isInt() || which.isUInt() || which.isFloat();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline bool isString(const T & data_type)
|
||||
{
|
||||
return WhichDataType(data_type).isString();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline bool isFixedString(const T & data_type)
|
||||
{
|
||||
return WhichDataType(data_type).isFixedString();
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
inline bool isStringOrFixedString(const T & data_type)
|
||||
{
|
||||
return WhichDataType(data_type).isStringOrFixedString();
|
||||
}
|
||||
|
||||
|
||||
inline bool isNotDecimalButComparableToDecimal(const DataTypePtr & data_type)
|
||||
{
|
||||
WhichDataType which(data_type);
|
||||
return which.isInt() || which.isUInt();
|
||||
}
|
||||
|
||||
inline bool isCompilableType(const DataTypePtr & data_type)
|
||||
{
|
||||
return data_type->isValueRepresentedByNumber() && !isDecimal(data_type);
|
||||
}
|
||||
|
@ -213,7 +213,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
|
||||
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (type->isFixedString())
|
||||
if (isFixedString(type))
|
||||
{
|
||||
have_string = true;
|
||||
if (!fixed_string_type)
|
||||
@ -221,7 +221,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
|
||||
else if (!type->equals(*fixed_string_type))
|
||||
return getNothingOrThrow(" because some of them are FixedStrings with different length");
|
||||
}
|
||||
else if (type->isString())
|
||||
else if (isString(type))
|
||||
have_string = true;
|
||||
else
|
||||
all_strings = false;
|
||||
@ -243,7 +243,7 @@ DataTypePtr getMostSubtype(const DataTypes & types, bool throw_if_result_is_noth
|
||||
|
||||
for (const auto & type : types)
|
||||
{
|
||||
if (type->isDateOrDateTime())
|
||||
if (isDateOrDateTime(type))
|
||||
have_date_or_datetime = true;
|
||||
else
|
||||
all_date_or_datetime = false;
|
||||
|
@ -227,7 +227,7 @@ bool CSVRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns & columns,
|
||||
if (curr_position < prev_position)
|
||||
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (data_types[i]->isNumber() || data_types[i]->isDateOrDateTime())
|
||||
if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i]))
|
||||
{
|
||||
/// An empty string instead of a value.
|
||||
if (curr_position == prev_position)
|
||||
|
@ -195,7 +195,7 @@ bool TabSeparatedRowInputStream::parseRowAndPrintDiagnosticInfo(MutableColumns &
|
||||
if (curr_position < prev_position)
|
||||
throw Exception("Logical error: parsing is non-deterministic.", ErrorCodes::LOGICAL_ERROR);
|
||||
|
||||
if (data_types[i]->isNumber() || data_types[i]->isDateOrDateTime())
|
||||
if (isNumber(data_types[i]) || isDateOrDateTime(data_types[i]))
|
||||
{
|
||||
/// An empty string instead of a value.
|
||||
if (curr_position == prev_position)
|
||||
|
@ -20,13 +20,6 @@ const Type * checkAndGetDataType(const IDataType * data_type)
|
||||
return typeid_cast<const Type *>(data_type);
|
||||
}
|
||||
|
||||
template <typename Type>
|
||||
bool checkDataType(const IDataType * data_type)
|
||||
{
|
||||
return checkAndGetDataType<Type>(data_type);
|
||||
}
|
||||
|
||||
|
||||
template <typename Type>
|
||||
const Type * checkAndGetColumn(const IColumn * column)
|
||||
{
|
||||
|
@ -1156,7 +1156,7 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
void executeAggregateMultiply(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) const
|
||||
{
|
||||
ColumnNumbers new_arguments = arguments;
|
||||
if (checkDataType<DataTypeAggregateFunction>(block.getByPosition(new_arguments[1]).type.get()))
|
||||
if (WhichDataType(block.getByPosition(new_arguments[1]).type).isAggregateFunction())
|
||||
std::swap(new_arguments[0], new_arguments[1]);
|
||||
|
||||
if (!block.getByPosition(new_arguments[1]).column->isColumnConst())
|
||||
@ -1231,7 +1231,7 @@ class FunctionBinaryArithmetic : public IFunction
|
||||
ColumnNumbers new_arguments = arguments;
|
||||
|
||||
/// Interval argument must be second.
|
||||
if (checkDataType<DataTypeInterval>(block.getByPosition(arguments[0]).type.get()))
|
||||
if (WhichDataType(block.getByPosition(arguments[0]).type).isInterval())
|
||||
std::swap(new_arguments[0], new_arguments[1]);
|
||||
|
||||
/// Change interval argument type to its representation
|
||||
@ -1267,7 +1267,7 @@ public:
|
||||
/// Special case when multiply aggregate function state
|
||||
if (isAggregateMultiply(arguments[0], arguments[1]))
|
||||
{
|
||||
if (checkDataType<DataTypeAggregateFunction>(arguments[0].get()))
|
||||
if (WhichDataType(arguments[0]).isAggregateFunction())
|
||||
return arguments[0];
|
||||
return arguments[1];
|
||||
}
|
||||
@ -1291,7 +1291,7 @@ public:
|
||||
new_arguments[i].type = arguments[i];
|
||||
|
||||
/// Interval argument must be second.
|
||||
if (checkDataType<DataTypeInterval>(new_arguments[0].type.get()))
|
||||
if (WhichDataType(new_arguments[0].type).isInterval())
|
||||
std::swap(new_arguments[0], new_arguments[1]);
|
||||
|
||||
/// Change interval argument to its representation
|
||||
@ -1332,6 +1332,18 @@ public:
|
||||
return type_res;
|
||||
}
|
||||
|
||||
bool isAggregateMultiply(const DataTypePtr & type0, const DataTypePtr & type1) const
|
||||
{
|
||||
if constexpr (!std::is_same_v<Op<UInt8, UInt8>, MultiplyImpl<UInt8, UInt8>>)
|
||||
return false;
|
||||
|
||||
WhichDataType which0(type0);
|
||||
WhichDataType which1(type1);
|
||||
|
||||
return (which0.isAggregateFunction() && which1.isNativeUInt())
|
||||
|| (which0.isNativeUInt() && which1.isAggregateFunction());
|
||||
}
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
/// Special case when multiply aggregate function state
|
||||
@ -1966,17 +1978,17 @@ public:
|
||||
throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be at least 2.", ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
|
||||
|
||||
const auto first_arg = arguments.front().get();
|
||||
const auto & first_arg = arguments.front();
|
||||
|
||||
if (!first_arg->isInteger())
|
||||
if (!isInteger(first_arg))
|
||||
throw Exception{"Illegal type " + first_arg->getName() + " of first argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
|
||||
for (const auto i : ext::range(1, arguments.size()))
|
||||
{
|
||||
const auto pos_arg = arguments[i].get();
|
||||
const auto & pos_arg = arguments[i];
|
||||
|
||||
if (!pos_arg->isUnsignedInteger())
|
||||
if (!isUnsignedInteger(pos_arg))
|
||||
throw Exception{"Illegal type " + pos_arg->getName() + " of " + toString(i) + " argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
|
@ -770,7 +770,7 @@ DataTypePtr FunctionArrayElement::getReturnTypeImpl(const DataTypes & arguments)
|
||||
if (!array_type)
|
||||
throw Exception("First argument for function " + getName() + " must be array.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isInteger())
|
||||
if (!isInteger(arguments[1]))
|
||||
throw Exception("Second argument for function " + getName() + " must be integer.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return array_type->getNestedType();
|
||||
@ -2342,7 +2342,7 @@ DataTypePtr FunctionRange::getReturnTypeImpl(const DataTypes & arguments) const
|
||||
{
|
||||
const DataTypePtr & arg = arguments.front();
|
||||
|
||||
if (!arg->isUnsignedInteger())
|
||||
if (!isUnsignedInteger(arg))
|
||||
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -2973,7 +2973,7 @@ DataTypePtr FunctionArraySlice::getReturnTypeImpl(const DataTypes & arguments) c
|
||||
|
||||
for (size_t i = 1; i < number_of_arguments; ++i)
|
||||
{
|
||||
if (!removeNullable(arguments[i])->isInteger() && !arguments[i]->onlyNull())
|
||||
if (!isInteger(removeNullable(arguments[i])) && !arguments[i]->onlyNull())
|
||||
throw Exception(
|
||||
"Argument " + toString(i) + " for function " + getName() + " must be integer but it has type "
|
||||
+ arguments[i]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -3412,9 +3412,9 @@ Columns FunctionArrayIntersect::castColumns(
|
||||
auto & type_nested = type_array->getNestedType();
|
||||
auto type_not_nullable_nested = removeNullable(type_nested);
|
||||
|
||||
const bool is_numeric_or_string = type_not_nullable_nested->isNumber()
|
||||
|| type_not_nullable_nested->isDateOrDateTime()
|
||||
|| type_not_nullable_nested->isStringOrFixedString();
|
||||
const bool is_numeric_or_string = isNumber(type_not_nullable_nested)
|
||||
|| isDateOrDateTime(type_not_nullable_nested)
|
||||
|| isStringOrFixedString(type_not_nullable_nested);
|
||||
|
||||
DataTypePtr nullable_return_type;
|
||||
|
||||
@ -3548,14 +3548,15 @@ void FunctionArrayIntersect::executeImpl(Block & block, const ColumnNumbers & ar
|
||||
if (!result_column)
|
||||
{
|
||||
auto column = not_nullable_nested_return_type->createColumn();
|
||||
WhichDataType which(not_nullable_nested_return_type);
|
||||
|
||||
if (checkDataType<DataTypeDate>(not_nullable_nested_return_type.get()))
|
||||
if (which.isDate())
|
||||
result_column = execute<DateMap, ColumnVector<DataTypeDate::FieldType>, true>(arrays, std::move(column));
|
||||
else if (checkDataType<DataTypeDateTime>(not_nullable_nested_return_type.get()))
|
||||
else if (which.isDateTime())
|
||||
result_column = execute<DateTimeMap, ColumnVector<DataTypeDateTime::FieldType>, true>(arrays, std::move(column));
|
||||
else if(not_nullable_nested_return_type->isString())
|
||||
else if(which.isString())
|
||||
result_column = execute<StringMap, ColumnString, false>(arrays, std::move(column));
|
||||
else if(not_nullable_nested_return_type->isFixedString())
|
||||
else if(which.isFixedString())
|
||||
result_column = execute<StringMap, ColumnFixedString, false>(arrays, std::move(column));
|
||||
else
|
||||
{
|
||||
@ -3706,10 +3707,10 @@ DataTypePtr FunctionArrayResize::getReturnTypeImpl(const DataTypes & arguments)
|
||||
throw Exception("First argument for function " + getName() + " must be an array but it has type "
|
||||
+ arguments[0]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (checkDataType<DataTypeNothing>(array_type->getNestedType().get()))
|
||||
if (WhichDataType(array_type->getNestedType()).isNothing())
|
||||
throw Exception("Function " + getName() + " cannot resize " + array_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!removeNullable(arguments[1])->isInteger() && !arguments[1]->onlyNull())
|
||||
if (!isInteger(removeNullable(arguments[1])) && !arguments[1]->onlyNull())
|
||||
throw Exception(
|
||||
"Argument " + toString(1) + " for function " + getName() + " must be integer but it has type "
|
||||
+ arguments[1]->getName() + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
@ -734,9 +734,12 @@ struct ArrayIndexGenericNullImpl
|
||||
};
|
||||
|
||||
|
||||
inline bool allowArrayIndex(const IDataType * data_type0, const IDataType * data_type1)
|
||||
inline bool allowArrayIndex(const DataTypePtr & type0, const DataTypePtr & type1)
|
||||
{
|
||||
return ((data_type0->isNumber() || isEnum(data_type0)) && data_type1->isNumber())
|
||||
DataTypePtr data_type0 = removeNullable(type0);
|
||||
DataTypePtr data_type1 = removeNullable(type1);
|
||||
|
||||
return ((isNumber(data_type0) || isEnum(data_type0)) && isNumber(data_type1))
|
||||
|| data_type0->equals(*data_type1);
|
||||
}
|
||||
|
||||
@ -1015,10 +1018,7 @@ public:
|
||||
|
||||
if (!arguments[1]->onlyNull())
|
||||
{
|
||||
DataTypePtr observed_type0 = removeNullable(array_type->getNestedType());
|
||||
DataTypePtr observed_type1 = removeNullable(arguments[1]);
|
||||
|
||||
if (!allowArrayIndex(observed_type0.get(), observed_type1.get()))
|
||||
if (!allowArrayIndex(array_type->getNestedType(), arguments[1]))
|
||||
throw Exception("Types of array and 2nd argument of function "
|
||||
+ getName() + " must be identical up to nullability or numeric types or Enum and numeric type. Passed: "
|
||||
+ arguments[0]->getName() + " and " + arguments[1]->getName() + ".",
|
||||
|
@ -171,7 +171,7 @@ public:
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
for (size_t i : ext::range(0, 3))
|
||||
if (!arguments[i]->isString())
|
||||
if (!isString(arguments[i]))
|
||||
throw Exception("Illegal type " + arguments[i]->getName() + " of argument of function " + getName()
|
||||
+ ", must be String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -150,12 +150,12 @@ public:
|
||||
", expected FixedString(" + toString(ipv6_bytes_length) + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!checkDataType<DataTypeUInt8>(arguments[1].get()))
|
||||
if (!WhichDataType(arguments[1]).isUInt8())
|
||||
throw Exception("Illegal type " + arguments[1]->getName() +
|
||||
" of argument 2 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!checkDataType<DataTypeUInt8>(arguments[2].get()))
|
||||
if (!WhichDataType(arguments[2]).isUInt8())
|
||||
throw Exception("Illegal type " + arguments[2]->getName() +
|
||||
" of argument 3 of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -266,7 +266,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -519,7 +519,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeUInt32>(arguments[0].get()))
|
||||
if (!WhichDataType(arguments[0]).isUInt32())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt32",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -579,7 +579,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -714,7 +714,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[0].get()))
|
||||
if (!WhichDataType(arguments[0]).isUInt64())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected UInt64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -843,7 +843,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1006,7 +1006,7 @@ public:
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
/// String or FixedString(36)
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
{
|
||||
const auto ptr = checkAndGetDataType<DataTypeFixedString>(arguments[0].get());
|
||||
if (!ptr || ptr->getN() != uuid_text_length)
|
||||
@ -1151,13 +1151,11 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString()
|
||||
&& !arguments[0]->isFixedString()
|
||||
&& !arguments[0]->isDateOrDateTime()
|
||||
&& !checkDataType<DataTypeUInt8>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeUInt16>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeUInt32>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeUInt64>(arguments[0].get()))
|
||||
WhichDataType which(arguments[0]);
|
||||
|
||||
if (!which.isStringOrFixedString()
|
||||
&& !which.isDateOrDateTime()
|
||||
&& !which.isUInt())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1370,7 +1368,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1460,7 +1458,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isInteger())
|
||||
if (!isInteger(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1543,7 +1541,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -817,7 +817,7 @@ private:
|
||||
const IColumn * column_number = left_is_num ? col_left_untyped : col_right_untyped;
|
||||
const IDataType * number_type = left_is_num ? left_type.get() : right_type.get();
|
||||
|
||||
DataTypeExtractor which(number_type);
|
||||
WhichDataType which(number_type);
|
||||
|
||||
const bool legal_types = which.isDateOrDateTime() || which.isEnum() || which.isUUID();
|
||||
|
||||
@ -1077,8 +1077,8 @@ public:
|
||||
/// Get result types by argument types. If the function does not apply to these arguments, throw an exception.
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
DataTypeExtractor left(arguments[0].get());
|
||||
DataTypeExtractor right(arguments[1].get());
|
||||
WhichDataType left(arguments[0].get());
|
||||
WhichDataType right(arguments[1].get());
|
||||
|
||||
const DataTypeTuple * left_tuple = checkAndGetDataType<DataTypeTuple>(arguments[0].get());
|
||||
const DataTypeTuple * right_tuple = checkAndGetDataType<DataTypeTuple>(arguments[1].get());
|
||||
@ -1159,9 +1159,9 @@ public:
|
||||
{
|
||||
executeTuple(block, result, col_with_type_and_name_left, col_with_type_and_name_right, input_rows_count);
|
||||
}
|
||||
else if (isDecimal(left_type.get()) || isDecimal(right_type.get()))
|
||||
else if (isDecimal(left_type) || isDecimal(right_type))
|
||||
{
|
||||
if (!allowDecimalComparison(left_type.get(), right_type.get()))
|
||||
if (!allowDecimalComparison(left_type, right_type))
|
||||
throw Exception("No operation " + getName() + " between " + left_type->getName() + " and " + right_type->getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1193,7 +1193,7 @@ public:
|
||||
auto isFloatingPoint = &typeIsEither<DataTypeFloat32, DataTypeFloat64>;
|
||||
if ((isBigInteger(*types[0]) && isFloatingPoint(*types[1])) || (isBigInteger(*types[1]) && isFloatingPoint(*types[0])))
|
||||
return false; /// TODO: implement (double, int_N where N > double's mantissa width)
|
||||
return isCompilableType(types[0].get()) && isCompilableType(types[1].get());
|
||||
return isCompilableType(types[0]) && isCompilableType(types[1]);
|
||||
}
|
||||
|
||||
llvm::Value * compileImpl(llvm::IRBuilderBase & builder, const DataTypes & types, ValuePlaceholders values) const override
|
||||
|
@ -205,7 +205,7 @@ DataTypePtr FunctionMultiIf::getReturnTypeImpl(const DataTypes & args) const
|
||||
nested_type = arg.get();
|
||||
}
|
||||
|
||||
if (!checkDataType<DataTypeUInt8>(nested_type))
|
||||
if (!WhichDataType(nested_type).isUInt8())
|
||||
throw Exception{"Illegal type " + arg->getName() + " of argument (condition) "
|
||||
"of function " + getName() + ". Must be UInt8.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
@ -122,7 +122,7 @@ public:
|
||||
bool isCompilableImpl(const DataTypes & types) const override
|
||||
{
|
||||
for (const auto & type : types)
|
||||
if (!isCompilableType(removeNullable(type).get()))
|
||||
if (!isCompilableType(removeNullable(type)))
|
||||
return false;
|
||||
return true;
|
||||
}
|
||||
@ -895,7 +895,7 @@ public:
|
||||
return makeNullable(getReturnTypeImpl({
|
||||
removeNullable(arguments[0]), arguments[1], arguments[2]}));
|
||||
|
||||
if (!checkDataType<DataTypeUInt8>(arguments[0].get()))
|
||||
if (!WhichDataType(arguments[0]).isUInt8())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument (condition) of function if. Must be UInt8.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -106,7 +106,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isInteger())
|
||||
if (!isInteger(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of the first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -115,7 +115,7 @@ public:
|
||||
+ ", got " + arguments[0]->getName(),
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
if (!arguments[1]->isInteger())
|
||||
if (!isInteger(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of the second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -178,21 +178,23 @@ private:
|
||||
const IDataType * hash_type = block.getByPosition(arguments[0]).type.get();
|
||||
auto res_col = ColumnVector<ResultType>::create();
|
||||
|
||||
if (checkDataType<DataTypeUInt8>(hash_type))
|
||||
WhichDataType which(hash_type);
|
||||
|
||||
if (which.isUInt8())
|
||||
executeType<UInt8>(hash_col, num_buckets, res_col.get());
|
||||
else if (checkDataType<DataTypeUInt16>(hash_type))
|
||||
else if (which.isUInt16())
|
||||
executeType<UInt16>(hash_col, num_buckets, res_col.get());
|
||||
else if (checkDataType<DataTypeUInt32>(hash_type))
|
||||
else if (which.isUInt32())
|
||||
executeType<UInt32>(hash_col, num_buckets, res_col.get());
|
||||
else if (checkDataType<DataTypeUInt64>(hash_type))
|
||||
else if (which.isUInt64())
|
||||
executeType<UInt64>(hash_col, num_buckets, res_col.get());
|
||||
else if (checkDataType<DataTypeInt8>(hash_type))
|
||||
else if (which.isInt8())
|
||||
executeType<Int8>(hash_col, num_buckets, res_col.get());
|
||||
else if (checkDataType<DataTypeInt16>(hash_type))
|
||||
else if (which.isInt16())
|
||||
executeType<Int16>(hash_col, num_buckets, res_col.get());
|
||||
else if (checkDataType<DataTypeInt32>(hash_type))
|
||||
else if (which.isInt32())
|
||||
executeType<Int32>(hash_col, num_buckets, res_col.get());
|
||||
else if (checkDataType<DataTypeInt64>(hash_type))
|
||||
else if (which.isInt64())
|
||||
executeType<Int64>(hash_col, num_buckets, res_col.get());
|
||||
else
|
||||
throw Exception("Illegal type " + hash_type->getName() + " of the first argument of function " + getName(),
|
||||
|
@ -20,7 +20,7 @@ void throwExceptionForIncompletelyParsedValue(
|
||||
else
|
||||
message_buf << " at begin of string";
|
||||
|
||||
if (to_type.isNumber())
|
||||
if (isNumber(to_type))
|
||||
message_buf << ". Note: there are to" << to_type.getName() << "OrZero and to" << to_type.getName() << "OrNull functions, which returns zero/NULL instead of throwing exception.";
|
||||
|
||||
throw Exception(message_buf.str(), ErrorCodes::CANNOT_PARSE_TEXT);
|
||||
|
@ -802,7 +802,7 @@ public:
|
||||
|| std::is_same_v<Name, NameToUnixTimestamp>;
|
||||
|
||||
if (!(to_date_or_time
|
||||
|| (std::is_same_v<Name, NameToString> && checkDataType<DataTypeDateTime>(arguments[0].type.get()))))
|
||||
|| (std::is_same_v<Name, NameToString> && WhichDataType(arguments[0].type).isDateTime())))
|
||||
{
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 1.",
|
||||
@ -950,7 +950,7 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 1 or 2. Second argument (time zone) is optional only make sense for DateTime.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[0].type->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[0].type))
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -963,7 +963,7 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 1. Second argument makes sense only when converting to DateTime.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[1].type->isString())
|
||||
if (!isString(arguments[1].type))
|
||||
throw Exception("Illegal type " + arguments[1].type->getName() + " of 2nd argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -1020,11 +1020,11 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override
|
||||
{
|
||||
if (!arguments[1].type->isUnsignedInteger())
|
||||
if (!isUnsignedInteger(arguments[1].type))
|
||||
throw Exception("Second argument for function " + getName() + " must be unsigned integer", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!arguments[1].column)
|
||||
throw Exception("Second argument for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (!arguments[0].type->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[0].type))
|
||||
throw Exception(getName() + " is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED);
|
||||
|
||||
const size_t n = arguments[1].column->getUInt(0);
|
||||
@ -1140,8 +1140,8 @@ struct ToIntMonotonicity
|
||||
}
|
||||
|
||||
/// If type is same, too. (Enum has separate case, because it is different data type)
|
||||
if (checkDataType<DataTypeNumber<T>>(&type) ||
|
||||
checkDataType<DataTypeEnum<T>>(&type))
|
||||
if (checkAndGetDataType<DataTypeNumber<T>>(&type) ||
|
||||
checkAndGetDataType<DataTypeEnum<T>>(&type))
|
||||
return { true, true, true };
|
||||
|
||||
/// In other cases, if range is unbounded, we don't know, whether function is monotonic or not.
|
||||
@ -1149,8 +1149,7 @@ struct ToIntMonotonicity
|
||||
return {};
|
||||
|
||||
/// If converting from float, for monotonicity, arguments must fit in range of result type.
|
||||
if (checkDataType<DataTypeFloat32>(&type)
|
||||
|| checkDataType<DataTypeFloat64>(&type))
|
||||
if (WhichDataType(type).isFloat())
|
||||
{
|
||||
Float64 left_float = left.get<Float64>();
|
||||
Float64 right_float = right.get<Float64>();
|
||||
@ -1460,7 +1459,7 @@ private:
|
||||
|
||||
static WrapperType createFixedStringWrapper(const DataTypePtr & from_type, const size_t N)
|
||||
{
|
||||
if (!from_type->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(from_type))
|
||||
throw Exception{"CAST AS FixedString is only implemented for types String and FixedString", ErrorCodes::NOT_IMPLEMENTED};
|
||||
|
||||
return [N] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t /*input_rows_count*/)
|
||||
@ -1469,6 +1468,24 @@ private:
|
||||
};
|
||||
}
|
||||
|
||||
WrapperType createUUIDWrapper(const DataTypePtr & from_type, const DataTypeUUID * const, bool requested_result_is_nullable) const
|
||||
{
|
||||
if (requested_result_is_nullable)
|
||||
throw Exception{"CAST AS Nullable(UUID) is not implemented", ErrorCodes::NOT_IMPLEMENTED};
|
||||
|
||||
FunctionPtr function = FunctionTo<DataTypeUUID>::Type::create(context);
|
||||
|
||||
/// Check conversion using underlying function
|
||||
{
|
||||
function->getReturnType(ColumnsWithTypeAndName(1, { nullptr, from_type, "" }));
|
||||
}
|
||||
|
||||
return [function] (Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
|
||||
{
|
||||
function->execute(block, arguments, result, input_rows_count);
|
||||
};
|
||||
}
|
||||
|
||||
template <typename FieldType>
|
||||
WrapperType createDecimalWrapper(const DataTypePtr & from_type, const DataTypeDecimal<FieldType> * to_type) const
|
||||
{
|
||||
@ -1628,7 +1645,7 @@ private:
|
||||
return createStringToEnumWrapper<ColumnString, EnumType>();
|
||||
else if (checkAndGetDataType<DataTypeFixedString>(from_type.get()))
|
||||
return createStringToEnumWrapper<ColumnFixedString, EnumType>();
|
||||
else if (from_type->isNumber() || from_type->isEnum())
|
||||
else if (isNumber(from_type) || isEnum(from_type))
|
||||
{
|
||||
auto function = Function::create(context);
|
||||
|
||||
@ -1878,7 +1895,7 @@ private:
|
||||
{
|
||||
if (from_type->equals(*to_type))
|
||||
return createIdentityWrapper(from_type);
|
||||
else if (checkDataType<DataTypeNothing>(from_type.get()))
|
||||
else if (WhichDataType(from_type).isNothing())
|
||||
return createNothingWrapper(to_type.get());
|
||||
|
||||
WrapperType ret;
|
||||
@ -1920,6 +1937,14 @@ private:
|
||||
ret = createDecimalWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()));
|
||||
return true;
|
||||
}
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeUUID>)
|
||||
{
|
||||
if (isStringOrFixedString(from_type))
|
||||
{
|
||||
ret = createUUIDWrapper(from_type, checkAndGetDataType<ToDataType>(to_type.get()), requested_result_is_nullable);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
||||
return false;
|
||||
};
|
||||
@ -2027,7 +2052,7 @@ private:
|
||||
return monotonicityForType(type);
|
||||
else if (const auto type = checkAndGetDataType<DataTypeString>(to_type))
|
||||
return monotonicityForType(type);
|
||||
else if (from_type->isEnum())
|
||||
else if (isEnum(from_type))
|
||||
{
|
||||
if (const auto type = checkAndGetDataType<DataTypeEnum8>(to_type))
|
||||
return monotonicityForType(type);
|
||||
|
@ -637,14 +637,14 @@ public:
|
||||
{
|
||||
if (arguments.size() == 1)
|
||||
{
|
||||
if (!arguments[0].type->isDateOrDateTime())
|
||||
if (!isDateOrDateTime(arguments[0].type))
|
||||
throw Exception("Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() +
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
else if (arguments.size() == 2)
|
||||
{
|
||||
if (!checkDataType<DataTypeDateTime>(arguments[0].type.get())
|
||||
|| !checkDataType<DataTypeString>(arguments[1].type.get()))
|
||||
if (!WhichDataType(arguments[0].type).isDateTime()
|
||||
|| !WhichDataType(arguments[1].type).isString())
|
||||
throw Exception(
|
||||
"Function " + getName() + " supports 1 or 2 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument (optional) must be "
|
||||
@ -670,10 +670,11 @@ public:
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (checkDataType<DataTypeDate>(from_type))
|
||||
if (which.isDate())
|
||||
DateTimeTransformImpl<DataTypeDate::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
|
||||
else if (checkDataType<DataTypeDateTime>(from_type))
|
||||
else if (which.isDateTime())
|
||||
DateTimeTransformImpl<DataTypeDateTime::FieldType, typename ToDataType::FieldType, Transform>::execute(block, arguments, result, input_rows_count);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
@ -945,20 +946,20 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[1].type->isNumber())
|
||||
if (!isNumber(arguments[1].type))
|
||||
throw Exception("Second argument for function " + getName() + " (delta) must be number",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2)
|
||||
{
|
||||
if (!arguments[0].type->isDateOrDateTime())
|
||||
if (!isDateOrDateTime(arguments[0].type))
|
||||
throw Exception{"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() +
|
||||
". Should be a date or a date with time", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!checkDataType<DataTypeDateTime>(arguments[0].type.get())
|
||||
|| !checkDataType<DataTypeString>(arguments[2].type.get()))
|
||||
if (!WhichDataType(arguments[0].type).isDateTime()
|
||||
|| !WhichDataType(arguments[2].type).isString())
|
||||
throw Exception(
|
||||
"Function " + getName() + " supports 2 or 3 arguments. The 1st argument "
|
||||
"must be of type Date or DateTime. The 2nd argument must be number. "
|
||||
@ -968,7 +969,7 @@ public:
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
||||
if (checkDataType<DataTypeDate>(arguments[0].type.get()))
|
||||
if (WhichDataType(arguments[0].type).isDate())
|
||||
{
|
||||
if (std::is_same_v<decltype(Transform::execute(DataTypeDate::FieldType(), 0, std::declval<DateLUTImpl>())), UInt16>)
|
||||
return std::make_shared<DataTypeDate>();
|
||||
@ -990,10 +991,11 @@ public:
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (checkDataType<DataTypeDate>(from_type))
|
||||
if (which.isDate())
|
||||
DateTimeAddIntervalImpl<DataTypeDate::FieldType, Transform>::execute(block, arguments, result);
|
||||
else if (checkDataType<DataTypeDateTime>(from_type))
|
||||
else if (which.isDateTime())
|
||||
DateTimeAddIntervalImpl<DataTypeDateTime::FieldType, Transform>::execute(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
@ -1032,19 +1034,19 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 3 or 4",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("First argument for function " + getName() + " (unit) must be String",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isDateOrDateTime())
|
||||
if (!isDateOrDateTime(arguments[1]))
|
||||
throw Exception("Second argument for function " + getName() + " must be Date or DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[2]->isDateOrDateTime())
|
||||
if (!isDateOrDateTime(arguments[2]))
|
||||
throw Exception("Third argument for function " + getName() + " must be Date or DateTime",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 4 && !arguments[3]->isString())
|
||||
if (arguments.size() == 4 && !isString(arguments[3]))
|
||||
throw Exception("Fourth argument for function " + getName() + " (timezone) must be String",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1296,7 +1298,7 @@ public:
|
||||
+ toString(arguments.size()) + ", should be 2",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!checkDataType<DataTypeDateTime>(arguments[0].type.get()))
|
||||
if (!WhichDataType(arguments[0].type).isDateTime())
|
||||
throw Exception{"Illegal type " + arguments[0].type->getName() + " of argument of function " + getName() +
|
||||
". Should be DateTime", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -1326,7 +1328,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeDateTime>(arguments[0].get()))
|
||||
if (!WhichDataType(arguments[0]).isDateTime())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1453,11 +1455,11 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeDateTime>(arguments[0].get()))
|
||||
if (!WhichDataType(arguments[0]).isDateTime())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be DateTime.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!checkDataType<DataTypeUInt32>(arguments[1].get()))
|
||||
if (!WhichDataType(arguments[1]).isUInt32())
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be UInt32.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -82,12 +82,12 @@ private:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[1].get()) &&
|
||||
!checkDataType<DataTypeTuple>(arguments[1].get()))
|
||||
if (!WhichDataType(arguments[1]).isUInt64() &&
|
||||
!isTuple(arguments[1]))
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -230,27 +230,27 @@ private:
|
||||
throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ toString(arguments.size()) + ", should be 3 or 4.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
{
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
{
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
|
||||
!checkDataType<DataTypeTuple>(arguments[2].get()))
|
||||
if (!WhichDataType(arguments[2]).isUInt64() &&
|
||||
!isTuple(arguments[2]))
|
||||
{
|
||||
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
|
||||
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
/// This is for the case of range dictionaries.
|
||||
if (arguments.size() == 4 && !checkDataType<DataTypeDate>(arguments[3].get()))
|
||||
if (arguments.size() == 4 && !WhichDataType(arguments[3]).isDate())
|
||||
{
|
||||
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
|
||||
+ ", must be Date.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
@ -467,22 +467,22 @@ private:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() +
|
||||
", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() +
|
||||
", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
|
||||
!checkDataType<DataTypeTuple>(arguments[2].get()))
|
||||
if (!WhichDataType(arguments[2]).isUInt64() &&
|
||||
!isTuple(arguments[2]))
|
||||
{
|
||||
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
|
||||
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
if (!arguments[3]->isString())
|
||||
if (!isString(arguments[3]))
|
||||
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() +
|
||||
", must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -735,20 +735,20 @@ private:
|
||||
if (arguments.size() != 3 && arguments.size() != 4)
|
||||
throw Exception{"Function " + getName() + " takes 3 or 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
|
||||
!checkDataType<DataTypeTuple>(arguments[2].get()))
|
||||
if (!WhichDataType(arguments[2]).isUInt64() &&
|
||||
!isTuple(arguments[2]))
|
||||
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
|
||||
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (arguments.size() == 4 && !checkDataType<DataTypeDate>(arguments[3].get()))
|
||||
if (arguments.size() == 4 && !WhichDataType(arguments[3]).isDate())
|
||||
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
|
||||
+ ", must be Date.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -1010,20 +1010,20 @@ private:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
|
||||
!checkDataType<DataTypeTuple>(arguments[2].get()))
|
||||
if (!WhichDataType(arguments[2]).isUInt64() &&
|
||||
!isTuple(arguments[2]))
|
||||
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
|
||||
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataType>(arguments[3].get()))
|
||||
if (!checkAndGetDataType<DataType>(arguments[3].get()))
|
||||
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
|
||||
+ ", must be " + String(DataType{}.getFamilyName()) + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -1252,11 +1252,11 @@ private:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[1].get()))
|
||||
if (!WhichDataType(arguments[1]).isUInt64())
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
@ -1408,15 +1408,15 @@ private:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[1].get()))
|
||||
if (!WhichDataType(arguments[1]).isUInt64())
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
|
||||
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!checkDataType<DataTypeUInt64>(arguments[2].get()))
|
||||
if (!WhichDataType(arguments[2]).isUInt64())
|
||||
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
|
||||
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
|
@ -32,7 +32,7 @@ DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const DataTypes & arguments
|
||||
throw Exception("Function " + getName() + " expects at least 2 arguments",
|
||||
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION);
|
||||
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
|
||||
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -116,7 +116,7 @@ public:
|
||||
|
||||
const auto type_x = arguments[0];
|
||||
|
||||
if (!type_x->isNumber())
|
||||
if (!isNumber(type_x))
|
||||
throw Exception{"Unsupported type " + type_x->getName() + " of first argument of function " + getName() + " must be a numeric type",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
|
@ -44,9 +44,9 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const IDataType * type = arguments[0].get();
|
||||
const DataTypePtr & type = arguments[0];
|
||||
|
||||
if (!type->isInteger())
|
||||
if (!isInteger(type))
|
||||
throw Exception("Cannot format " + type->getName() + " as bitmask string", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
@ -139,7 +139,7 @@ public:
|
||||
{
|
||||
const IDataType & type = *arguments[0];
|
||||
|
||||
if (!type.isNumber())
|
||||
if (!isNumber(type))
|
||||
throw Exception("Cannot format " + type.getName() + " as size in bytes", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
@ -133,7 +133,7 @@ public:
|
||||
|
||||
for (auto j : ext::range(0, elements.size()))
|
||||
{
|
||||
if (!elements[j]->isNumber())
|
||||
if (!isNumber(elements[j]))
|
||||
{
|
||||
throw Exception(getMsgPrefix(i) + " must contains numeric tuple at position " + toString(j + 1),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -162,7 +162,7 @@ public:
|
||||
const Columns & tuple_columns = tuple_col->getColumns();
|
||||
const DataTypes & tuple_types = typeid_cast<const DataTypeTuple &>(*block.getByPosition(arguments[0]).type).getElements();
|
||||
|
||||
bool use_float64 = checkDataType<DataTypeFloat64>(tuple_types[0].get()) || checkDataType<DataTypeFloat64>(tuple_types[1].get());
|
||||
bool use_float64 = WhichDataType(tuple_types[0]).isFloat64() || WhichDataType(tuple_types[1]).isFloat64();
|
||||
|
||||
auto & result_column = block.safeGetByPosition(result).column;
|
||||
|
||||
|
@ -61,7 +61,7 @@ private:
|
||||
for (const auto arg_idx : ext::range(0, arguments.size()))
|
||||
{
|
||||
const auto arg = arguments[arg_idx].get();
|
||||
if (!checkDataType<DataTypeFloat64>(arg))
|
||||
if (!WhichDataType(arg).isFloat64())
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName() + ". Must be Float64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -213,7 +213,7 @@ private:
|
||||
for (const auto arg_idx : ext::range(0, arguments.size()))
|
||||
{
|
||||
const auto arg = arguments[arg_idx].get();
|
||||
if (!checkDataType<DataTypeFloat64>(arg))
|
||||
if (!WhichDataType(arg).isFloat64())
|
||||
{
|
||||
throw Exception(
|
||||
"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName() + ". Must be Float64",
|
||||
|
@ -200,7 +200,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -303,17 +303,18 @@ public:
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
|
||||
{
|
||||
const IDataType * from_type = block.getByPosition(arguments[0]).type.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (checkDataType<DataTypeUInt8>(from_type)) executeType<UInt8>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeUInt16>(from_type)) executeType<UInt16>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeUInt32>(from_type)) executeType<UInt32>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeUInt64>(from_type)) executeType<UInt64>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeInt8>(from_type)) executeType<Int8>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeInt16>(from_type)) executeType<Int16>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeInt32>(from_type)) executeType<Int32>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeInt64>(from_type)) executeType<Int64>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeDate>(from_type)) executeType<UInt16>(block, arguments, result);
|
||||
else if (checkDataType<DataTypeDateTime>(from_type)) executeType<UInt32>(block, arguments, result);
|
||||
if (which.isUInt8()) executeType<UInt8>(block, arguments, result);
|
||||
else if (which.isUInt16()) executeType<UInt16>(block, arguments, result);
|
||||
else if (which.isUInt32()) executeType<UInt32>(block, arguments, result);
|
||||
else if (which.isUInt64()) executeType<UInt64>(block, arguments, result);
|
||||
else if (which.isInt8()) executeType<Int8>(block, arguments, result);
|
||||
else if (which.isInt16()) executeType<Int16>(block, arguments, result);
|
||||
else if (which.isInt32()) executeType<Int32>(block, arguments, result);
|
||||
else if (which.isInt64()) executeType<Int64>(block, arguments, result);
|
||||
else if (which.isDate()) executeType<UInt16>(block, arguments, result);
|
||||
else if (which.isDateTime()) executeType<UInt32>(block, arguments, result);
|
||||
else
|
||||
throw Exception("Illegal type " + block.getByPosition(arguments[0]).type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -479,23 +480,25 @@ private:
|
||||
template <bool first>
|
||||
void executeAny(const IDataType * from_type, const IColumn * icolumn, ColumnUInt64::Container & vec_to)
|
||||
{
|
||||
if (checkDataType<DataTypeUInt8>(from_type)) executeIntType<UInt8, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeUInt16>(from_type)) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeUInt32>(from_type)) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeUInt64>(from_type)) executeIntType<UInt64, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt8>(from_type)) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt16>(from_type)) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt32>(from_type)) executeIntType<Int32, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt64>(from_type)) executeIntType<Int64, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeEnum8>(from_type)) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeEnum16>(from_type)) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeDate>(from_type)) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeDateTime>(from_type)) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeFloat32>(from_type)) executeIntType<Float32, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeFloat64>(from_type)) executeIntType<Float64, first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeString>(from_type)) executeString<first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeFixedString>(from_type)) executeString<first>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeArray>(from_type)) executeArray<first>(from_type, icolumn, vec_to);
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (which.isUInt8()) executeIntType<UInt8, first>(icolumn, vec_to);
|
||||
else if (which.isUInt16()) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (which.isUInt32()) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
else if (which.isUInt64()) executeIntType<UInt64, first>(icolumn, vec_to);
|
||||
else if (which.isInt8()) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (which.isInt16()) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (which.isInt32()) executeIntType<Int32, first>(icolumn, vec_to);
|
||||
else if (which.isInt64()) executeIntType<Int64, first>(icolumn, vec_to);
|
||||
else if (which.isEnum8()) executeIntType<Int8, first>(icolumn, vec_to);
|
||||
else if (which.isEnum16()) executeIntType<Int16, first>(icolumn, vec_to);
|
||||
else if (which.isDate()) executeIntType<UInt16, first>(icolumn, vec_to);
|
||||
else if (which.isDateTime()) executeIntType<UInt32, first>(icolumn, vec_to);
|
||||
else if (which.isFloat32()) executeIntType<Float32, first>(icolumn, vec_to);
|
||||
else if (which.isFloat64()) executeIntType<Float64, first>(icolumn, vec_to);
|
||||
else if (which.isString()) executeString<first>(icolumn, vec_to);
|
||||
else if (which.isFixedString()) executeString<first>(icolumn, vec_to);
|
||||
else if (which.isArray()) executeArray<first>(from_type, icolumn, vec_to);
|
||||
else
|
||||
throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -602,23 +605,23 @@ public:
|
||||
const ColumnWithTypeAndName & col = block.getByPosition(arguments[0]);
|
||||
const IDataType * from_type = col.type.get();
|
||||
const IColumn * icolumn = col.column.get();
|
||||
WhichDataType which(from_type);
|
||||
|
||||
if (checkDataType<DataTypeUInt8>(from_type)) executeIntType<UInt8>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeUInt16>(from_type)) executeIntType<UInt16>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeUInt32>(from_type)) executeIntType<UInt32>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeUInt64>(from_type)) executeIntType<UInt64>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt8>(from_type)) executeIntType<Int8>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt16>(from_type)) executeIntType<Int16>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt32>(from_type)) executeIntType<Int32>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeInt64>(from_type)) executeIntType<Int64>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeEnum8>(from_type)) executeIntType<Int8>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeEnum16>(from_type)) executeIntType<Int16>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeDate>(from_type)) executeIntType<UInt16>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeDateTime>(from_type)) executeIntType<UInt32>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeFloat32>(from_type)) executeIntType<Float32>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeFloat64>(from_type)) executeIntType<Float64>(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeString>(from_type)) executeString(icolumn, vec_to);
|
||||
else if (checkDataType<DataTypeFixedString>(from_type)) executeString(icolumn, vec_to);
|
||||
if (which.isUInt8()) executeIntType<UInt8>(icolumn, vec_to);
|
||||
else if (which.isUInt16()) executeIntType<UInt16>(icolumn, vec_to);
|
||||
else if (which.isUInt32()) executeIntType<UInt32>(icolumn, vec_to);
|
||||
else if (which.isUInt64()) executeIntType<UInt64>(icolumn, vec_to);
|
||||
else if (which.isInt8()) executeIntType<Int8>(icolumn, vec_to);
|
||||
else if (which.isInt16()) executeIntType<Int16>(icolumn, vec_to);
|
||||
else if (which.isInt32()) executeIntType<Int32>(icolumn, vec_to);
|
||||
else if (which.isInt64()) executeIntType<Int64>(icolumn, vec_to);
|
||||
else if (which.isEnum8()) executeIntType<Int8>(icolumn, vec_to);
|
||||
else if (which.isEnum16()) executeIntType<Int16>(icolumn, vec_to);
|
||||
else if (which.isDate()) executeIntType<UInt16>(icolumn, vec_to);
|
||||
else if (which.isDateTime()) executeIntType<UInt32>(icolumn, vec_to);
|
||||
else if (which.isFloat32()) executeIntType<Float32>(icolumn, vec_to);
|
||||
else if (which.isFloat64()) executeIntType<Float64>(icolumn, vec_to);
|
||||
else if (which.isStringOrFixedString()) executeString(icolumn, vec_to);
|
||||
else
|
||||
throw Exception("Unexpected type " + from_type->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -843,13 +846,13 @@ public:
|
||||
toString(arg_count) + ", should be 1 or 2.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
|
||||
|
||||
const auto first_arg = arguments.front().get();
|
||||
if (!checkDataType<DataTypeString>(first_arg))
|
||||
if (!WhichDataType(first_arg).isString())
|
||||
throw Exception{"Illegal type " + first_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (arg_count == 2)
|
||||
{
|
||||
const auto second_arg = arguments.back().get();
|
||||
if (!second_arg->isInteger())
|
||||
const auto & second_arg = arguments.back();
|
||||
if (!isInteger(second_arg))
|
||||
throw Exception{"Illegal type " + second_arg->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
|
||||
|
@ -310,20 +310,15 @@ struct ArraySumImpl
|
||||
|
||||
static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypePtr & /*array_element*/)
|
||||
{
|
||||
if (checkDataType<DataTypeUInt8>(&*expression_return) ||
|
||||
checkDataType<DataTypeUInt16>(&*expression_return) ||
|
||||
checkDataType<DataTypeUInt32>(&*expression_return) ||
|
||||
checkDataType<DataTypeUInt64>(&*expression_return))
|
||||
WhichDataType which(expression_return);
|
||||
|
||||
if (which.isNativeUInt())
|
||||
return std::make_shared<DataTypeUInt64>();
|
||||
|
||||
if (checkDataType<DataTypeInt8>(&*expression_return) ||
|
||||
checkDataType<DataTypeInt16>(&*expression_return) ||
|
||||
checkDataType<DataTypeInt32>(&*expression_return) ||
|
||||
checkDataType<DataTypeInt64>(&*expression_return))
|
||||
if (which.isNativeInt())
|
||||
return std::make_shared<DataTypeInt64>();
|
||||
|
||||
if (checkDataType<DataTypeFloat32>(&*expression_return) ||
|
||||
checkDataType<DataTypeFloat64>(&*expression_return))
|
||||
if (which.isFloat())
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
|
||||
throw Exception("arraySum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -602,20 +597,15 @@ struct ArrayCumSumImpl
|
||||
|
||||
static DataTypePtr getReturnType(const DataTypePtr & expression_return, const DataTypePtr & /*array_element*/)
|
||||
{
|
||||
if (checkDataType<DataTypeUInt8>(&*expression_return) ||
|
||||
checkDataType<DataTypeUInt16>(&*expression_return) ||
|
||||
checkDataType<DataTypeUInt32>(&*expression_return) ||
|
||||
checkDataType<DataTypeUInt64>(&*expression_return))
|
||||
WhichDataType which(expression_return);
|
||||
|
||||
if (which.isNativeUInt())
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
|
||||
|
||||
if (checkDataType<DataTypeInt8>(&*expression_return) ||
|
||||
checkDataType<DataTypeInt16>(&*expression_return) ||
|
||||
checkDataType<DataTypeInt32>(&*expression_return) ||
|
||||
checkDataType<DataTypeInt64>(&*expression_return))
|
||||
if (which.isNativeInt())
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeInt64>());
|
||||
|
||||
if (checkDataType<DataTypeFloat32>(&*expression_return) ||
|
||||
checkDataType<DataTypeFloat64>(&*expression_return))
|
||||
if (which.isFloat())
|
||||
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeFloat64>());
|
||||
|
||||
throw Exception("arrayCumSum cannot add values of type " + expression_return->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -824,7 +814,7 @@ public:
|
||||
|
||||
DataTypePtr nested_type = array_type->getNestedType();
|
||||
|
||||
if (Impl::needBoolean() && !checkDataType<DataTypeUInt8>(&*nested_type))
|
||||
if (Impl::needBoolean() && !WhichDataType(nested_type).isUInt8())
|
||||
throw Exception("The only argument for function " + getName() + " must be array of UInt8. Found "
|
||||
+ arguments[0].type->getName() + " instead.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -845,7 +835,7 @@ public:
|
||||
/// The types of the remaining arguments are already checked in getLambdaArgumentTypes.
|
||||
|
||||
DataTypePtr return_type = data_type_function->getReturnType();
|
||||
if (Impl::needBoolean() && !checkDataType<DataTypeUInt8>(&*return_type))
|
||||
if (Impl::needBoolean() && !WhichDataType(return_type).isUInt8())
|
||||
throw Exception("Expression for function " + getName() + " must return UInt8, found "
|
||||
+ return_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -309,8 +309,8 @@ public:
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (size_t i = 0; i < arguments.size(); ++i)
|
||||
if (!(arguments[i]->isNumber()
|
||||
|| (Impl::specialImplementationForNulls() && (arguments[i]->onlyNull() || removeNullable(arguments[i])->isNumber()))))
|
||||
if (!(isNumber(arguments[i])
|
||||
|| (Impl::specialImplementationForNulls() && (arguments[i]->onlyNull() || isNumber(removeNullable(arguments[i]))))))
|
||||
throw Exception("Illegal type ("
|
||||
+ arguments[i]->getName()
|
||||
+ ") of " + toString(i + 1) + " argument of function " + getName(),
|
||||
@ -488,7 +488,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isNumber())
|
||||
if (!isNumber(arguments[0]))
|
||||
throw Exception("Illegal type ("
|
||||
+ arguments[0]->getName()
|
||||
+ ") of argument of function " + getName(),
|
||||
|
@ -78,7 +78,7 @@ private:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments.front()->isNumber())
|
||||
if (!isNumber(arguments.front()))
|
||||
throw Exception{"Illegal type " + arguments.front()->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
@ -199,7 +199,7 @@ private:
|
||||
{
|
||||
const auto check_argument_type = [this] (const IDataType * arg)
|
||||
{
|
||||
if (!arg->isNumber())
|
||||
if (!isNumber(arg))
|
||||
throw Exception{"Illegal type " + arg->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
};
|
||||
|
@ -262,9 +262,11 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (checkDataType<DataTypeEnum8>(arguments[0].get()))
|
||||
WhichDataType which(arguments[0]);
|
||||
|
||||
if (which.isEnum8())
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
else if (checkDataType<DataTypeEnum16>(arguments[0].get()))
|
||||
else if (which.isEnum16())
|
||||
return std::make_shared<DataTypeUInt16>();
|
||||
|
||||
throw Exception("The argument for function " + getName() + " must be Enum", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -608,11 +610,10 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!checkDataType<DataTypeFloat64>(arguments[0].get()) && !checkDataType<DataTypeFloat32>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeUInt64>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeUInt32>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeUInt16>(arguments[0].get())
|
||||
&& !checkDataType<DataTypeUInt8>(arguments[0].get()))
|
||||
WhichDataType which(arguments[0]);
|
||||
|
||||
if (!which.isFloat()
|
||||
&& !which.isNativeUInt())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName() + ", expected Float64",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -1016,8 +1017,8 @@ public:
|
||||
+ ".",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[0]->isNumber() || !arguments[1]->isNumber() || !arguments[2]->isNumber()
|
||||
|| (arguments.size() == 4 && !arguments[3]->isNumber()))
|
||||
if (!isNumber(arguments[0]) || !isNumber(arguments[1]) || !isNumber(arguments[2])
|
||||
|| (arguments.size() == 4 && !isNumber(arguments[3])))
|
||||
throw Exception("All arguments for function " + getName() + " must be numeric.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
@ -1144,7 +1145,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments.front()->isNumber())
|
||||
if (!isNumber(arguments.front()))
|
||||
throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
@ -1511,29 +1512,31 @@ private:
|
||||
template <typename F>
|
||||
void dispatchForSourceType(const IDataType & src_type, F && f) const
|
||||
{
|
||||
if (checkDataType<DataTypeUInt8>(&src_type))
|
||||
WhichDataType which(src_type);
|
||||
|
||||
if (which.isUInt8())
|
||||
f(UInt8());
|
||||
else if (checkDataType<DataTypeUInt16>(&src_type))
|
||||
else if (which.isUInt16())
|
||||
f(UInt16());
|
||||
else if (checkDataType<DataTypeUInt32>(&src_type))
|
||||
else if (which.isUInt32())
|
||||
f(UInt32());
|
||||
else if (checkDataType<DataTypeUInt64>(&src_type))
|
||||
else if (which.isUInt64())
|
||||
f(UInt64());
|
||||
else if (checkDataType<DataTypeInt8>(&src_type))
|
||||
else if (which.isInt8())
|
||||
f(Int8());
|
||||
else if (checkDataType<DataTypeInt16>(&src_type))
|
||||
else if (which.isInt16())
|
||||
f(Int16());
|
||||
else if (checkDataType<DataTypeInt32>(&src_type))
|
||||
else if (which.isInt32())
|
||||
f(Int32());
|
||||
else if (checkDataType<DataTypeInt64>(&src_type))
|
||||
else if (which.isInt64())
|
||||
f(Int64());
|
||||
else if (checkDataType<DataTypeFloat32>(&src_type))
|
||||
else if (which.isFloat32())
|
||||
f(Float32());
|
||||
else if (checkDataType<DataTypeFloat64>(&src_type))
|
||||
else if (which.isFloat64())
|
||||
f(Float64());
|
||||
else if (checkDataType<DataTypeDate>(&src_type))
|
||||
else if (which.isDate())
|
||||
f(DataTypeDate::FieldType());
|
||||
else if (checkDataType<DataTypeDateTime>(&src_type))
|
||||
else if (which.isDateTime())
|
||||
f(DataTypeDateTime::FieldType());
|
||||
else
|
||||
throw Exception("Argument for function " + getName() + " must have numeric type.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -1822,7 +1825,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments.front()->isNumber())
|
||||
if (!isNumber(arguments.front()))
|
||||
throw Exception{"Argument for function " + getName() + " must be number", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
return std::make_shared<DataTypeUInt8>();
|
||||
|
@ -167,7 +167,7 @@ public:
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
const IDataType & type = *arguments[0];
|
||||
if (!type.isStringOrFixedString())
|
||||
if (!isStringOrFixedString(type))
|
||||
throw Exception("Cannot reinterpret " + type.getName() + " as " + ToDataType().getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<ToDataType>();
|
||||
|
@ -578,7 +578,7 @@ public:
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
for (const auto & type : arguments)
|
||||
if (!type->isNumber())
|
||||
if (!isNumber(type))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -550,8 +550,8 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isStringOrFixedString()
|
||||
&& !checkDataType<DataTypeArray>(arguments[0].get()))
|
||||
if (!isStringOrFixedString(arguments[0])
|
||||
&& !isArray(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -638,8 +638,8 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isStringOrFixedString()
|
||||
&& !checkDataType<DataTypeArray>(arguments[0].get()))
|
||||
if (!isStringOrFixedString(arguments[0])
|
||||
&& !isArray(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -710,7 +710,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!is_injective && !arguments.empty() && checkDataType<DataTypeArray>(arguments[0].get()))
|
||||
if (!is_injective && !arguments.empty() && isArray(arguments[0]))
|
||||
return FunctionArrayConcat(context).getReturnTypeImpl(arguments);
|
||||
|
||||
if (arguments.size() < 2)
|
||||
@ -721,7 +721,7 @@ public:
|
||||
for (const auto arg_idx : ext::range(0, arguments.size()))
|
||||
{
|
||||
const auto arg = arguments[arg_idx].get();
|
||||
if (!arg->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arg))
|
||||
throw Exception{"Illegal type " + arg->getName() + " of argument " + std::to_string(arg_idx + 1) + " of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
}
|
||||
@ -731,7 +731,7 @@ public:
|
||||
|
||||
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override
|
||||
{
|
||||
if (!is_injective && !arguments.empty() && checkDataType<DataTypeArray>(block.getByPosition(arguments[0]).type.get()))
|
||||
if (!is_injective && !arguments.empty() && isArray(block.getByPosition(arguments[0]).type))
|
||||
return FunctionArrayConcat(context).executeImpl(block, arguments, result, input_rows_count);
|
||||
|
||||
if (arguments.size() == 2)
|
||||
@ -814,16 +814,16 @@ public:
|
||||
+ toString(number_of_arguments) + ", should be 2 or 3",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
if (!arguments[0]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isNumber())
|
||||
if (!isNumber(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName()
|
||||
+ " of second argument of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (number_of_arguments == 3 && !arguments[2]->isNumber())
|
||||
if (number_of_arguments == 3 && !isNumber(arguments[2]))
|
||||
throw Exception("Illegal type " + arguments[2]->getName()
|
||||
+ " of second argument of function "
|
||||
+ getName(),
|
||||
@ -947,12 +947,12 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isNumber() || !arguments[2]->isNumber())
|
||||
throw Exception("Illegal type " + (arguments[1]->isNumber() ? arguments[2]->getName() : arguments[1]->getName())
|
||||
if (!isNumber(arguments[1]) || !isNumber(arguments[2]))
|
||||
throw Exception("Illegal type " + (isNumber(arguments[1]) ? arguments[2]->getName() : arguments[1]->getName())
|
||||
+ " of argument of function "
|
||||
+ getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
@ -1022,10 +1022,10 @@ private:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception{"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception{"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
@ -1127,10 +1127,10 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeNumber<UInt8>>();
|
||||
|
@ -157,7 +157,7 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
/// Check the type of the function's arguments.
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -124,11 +124,11 @@ public:
|
||||
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -504,11 +504,11 @@ public:
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
const DataTypeArray * array_type = checkAndGetDataType<DataTypeArray>(arguments[0].get());
|
||||
if (!array_type || !array_type->getNestedType()->isString())
|
||||
if (!array_type || !isString(array_type->getNestedType()))
|
||||
throw Exception("First argument for function " + getName() + " must be array of strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (arguments.size() == 2
|
||||
&& !arguments[1]->isString())
|
||||
&& !isString(arguments[1]))
|
||||
throw Exception("Second argument for function " + getName() + " must be constant string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return std::make_shared<DataTypeString>();
|
||||
|
@ -954,15 +954,15 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[1]))
|
||||
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[2]->isStringOrFixedString())
|
||||
if (!isStringOrFixedString(arguments[2]))
|
||||
throw Exception("Illegal type " + arguments[2]->getName() + " of third argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -65,11 +65,11 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
@ -149,11 +149,11 @@ public:
|
||||
|
||||
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[0]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
if (!arguments[1]->isString())
|
||||
if (!isString(arguments[1]))
|
||||
throw Exception(
|
||||
"Illegal type " + arguments[1]->getName() + " of argument of function " + getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
|
@ -77,7 +77,7 @@ public:
|
||||
|
||||
const DataTypePtr & type_x = arguments[0];
|
||||
|
||||
if (!type_x->isValueRepresentedByNumber() && !type_x->isString())
|
||||
if (!type_x->isValueRepresentedByNumber() && !isString(type_x))
|
||||
throw Exception{"Unsupported type " + type_x->getName()
|
||||
+ " of first argument of function " + getName()
|
||||
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
@ -91,7 +91,7 @@ public:
|
||||
const auto type_arr_from_nested = type_arr_from->getNestedType();
|
||||
|
||||
if ((type_x->isValueRepresentedByNumber() != type_arr_from_nested->isValueRepresentedByNumber())
|
||||
|| (!!type_x->isString() != !!type_arr_from_nested->isString()))
|
||||
|| (isString(type_x) != isString(type_arr_from_nested)))
|
||||
{
|
||||
throw Exception{"First argument and elements of array of second argument of function " + getName()
|
||||
+ " must have compatible types: both numeric or both strings.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
@ -108,7 +108,7 @@ public:
|
||||
if (args_size == 3)
|
||||
{
|
||||
if ((type_x->isValueRepresentedByNumber() != type_arr_to_nested->isValueRepresentedByNumber())
|
||||
|| (!!type_x->isString() != !!checkDataType<DataTypeString>(type_arr_to_nested.get())))
|
||||
|| (isString(type_x) != isString(type_arr_to_nested)))
|
||||
throw Exception{"Function " + getName()
|
||||
+ " has signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
@ -119,13 +119,16 @@ public:
|
||||
{
|
||||
const DataTypePtr & type_default = arguments[3];
|
||||
|
||||
if (!type_default->isValueRepresentedByNumber() && !type_default->isString())
|
||||
if (!type_default->isValueRepresentedByNumber() && !isString(type_default))
|
||||
throw Exception{"Unsupported type " + type_default->getName()
|
||||
+ " of fourth argument (default value) of function " + getName()
|
||||
+ ", must be numeric type or Date/DateTime or String", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
||||
bool default_is_string = WhichDataType(type_default).isString();
|
||||
bool nested_is_string = WhichDataType(type_arr_to_nested).isString();
|
||||
|
||||
if ((type_default->isValueRepresentedByNumber() != type_arr_to_nested->isValueRepresentedByNumber())
|
||||
|| (!!checkDataType<DataTypeString>(type_default.get()) != !!checkDataType<DataTypeString>(type_arr_to_nested.get())))
|
||||
|| (default_is_string != nested_is_string))
|
||||
throw Exception{"Function " + getName()
|
||||
+ " have signature: transform(T, Array(T), Array(U), U) -> U; or transform(T, Array(T), Array(T)) -> T; where T and U are types.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
|
||||
|
@ -584,7 +584,7 @@ public:
|
||||
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -669,7 +669,7 @@ public:
|
||||
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -746,7 +746,7 @@ public:
|
||||
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
@ -841,7 +841,7 @@ public:
|
||||
|
||||
static void checkArguments(const DataTypes & arguments)
|
||||
{
|
||||
if (!arguments[0]->isString())
|
||||
if (!isString(arguments[0]))
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() + ". Must be String.",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
}
|
||||
|
@ -473,10 +473,10 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
|
||||
}
|
||||
|
||||
/// If single string key - will use hash table with references to it. Strings itself are stored separately in Arena.
|
||||
if (params.keys_size == 1 && types_removed_nullable[0]->isString())
|
||||
if (params.keys_size == 1 && isString(types_removed_nullable[0]))
|
||||
return AggregatedDataVariants::Type::key_string;
|
||||
|
||||
if (params.keys_size == 1 && types_removed_nullable[0]->isFixedString())
|
||||
if (params.keys_size == 1 && isFixedString(types_removed_nullable[0]))
|
||||
return AggregatedDataVariants::Type::key_fixed_string;
|
||||
|
||||
return AggregatedDataVariants::Type::serialized;
|
||||
|
@ -2668,13 +2668,17 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
|
||||
return true;
|
||||
}
|
||||
|
||||
bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types)
|
||||
bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types, const ASTPtr & sampling_expression)
|
||||
{
|
||||
assertSelect();
|
||||
|
||||
if (!select_query->prewhere_expression)
|
||||
return false;
|
||||
|
||||
Names required_sample_columns;
|
||||
if (sampling_expression)
|
||||
required_sample_columns = ExpressionAnalyzer(sampling_expression, context, nullptr, source_columns).getRequiredSourceColumns();
|
||||
|
||||
initChain(chain, source_columns);
|
||||
auto & step = chain.getLastStep();
|
||||
getRootActions(select_query->prewhere_expression, only_types, false, step.actions);
|
||||
@ -2682,6 +2686,15 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl
|
||||
step.required_output.push_back(prewhere_column_name);
|
||||
step.can_remove_required_output.push_back(true);
|
||||
|
||||
/// Add required columns for sample expression to required output in order not to remove them after
|
||||
/// prewhere execution because sampling is executed after prewhere.
|
||||
/// TODO: add sampling execution to common chain.
|
||||
for (const auto & column : required_sample_columns)
|
||||
{
|
||||
step.required_output.push_back(column);
|
||||
step.can_remove_required_output.push_back(true);
|
||||
}
|
||||
|
||||
{
|
||||
/// Remove unused source_columns from prewhere actions.
|
||||
auto tmp_actions = std::make_shared<ExpressionActions>(source_columns, context);
|
||||
|
@ -142,7 +142,8 @@ public:
|
||||
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
|
||||
/// remove_filter is set in ExpressionActionsChain::finalize();
|
||||
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types);
|
||||
/// sampling_expression is needed if sampling is used in order to not remove columns are used in it.
|
||||
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const ASTPtr & sampling_expression);
|
||||
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
|
||||
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
|
||||
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);
|
||||
|
@ -370,7 +370,7 @@ void InterpreterCreateQuery::checkSupportedTypes(const ColumnsDescription & colu
|
||||
+ "Set setting allow_experimental_low_cardinality_type = 1 in order to allow it.";
|
||||
throw Exception(message, ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
if (!allow_decimal && column.type && isDecimal(column.type.get()))
|
||||
if (!allow_decimal && column.type && isDecimal(column.type))
|
||||
{
|
||||
String message = "Cannot create table with column '" + column.name + "' which type is '" + column.type->getName()
|
||||
+ "'. Set setting allow_experimental_decimal_type = 1 in order to allow it.";
|
||||
|
@ -47,6 +47,7 @@
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Parsers/queryToString.h>
|
||||
#include <ext/map.h>
|
||||
#include <memory>
|
||||
|
||||
|
||||
namespace DB
|
||||
@ -63,6 +64,7 @@ namespace ErrorCodes
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
extern const int PARAMETER_OUT_OF_BOUND;
|
||||
extern const int ARGUMENT_OUT_OF_BOUND;
|
||||
}
|
||||
|
||||
InterpreterSelectQuery::InterpreterSelectQuery(
|
||||
@ -279,7 +281,6 @@ BlockInputStreams InterpreterSelectQuery::executeWithMultipleStreams()
|
||||
return pipeline.streams;
|
||||
}
|
||||
|
||||
|
||||
InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpressions(QueryProcessingStage::Enum from_stage, bool dry_run)
|
||||
{
|
||||
AnalysisResult res;
|
||||
@ -305,7 +306,27 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
|
||||
chain.finalize();
|
||||
|
||||
if (has_prewhere)
|
||||
res.prewhere_info->remove_prewhere_column = chain.steps.at(0).can_remove_required_output.at(0);
|
||||
{
|
||||
const ExpressionActionsChain::Step & step = chain.steps.at(0);
|
||||
res.prewhere_info->remove_prewhere_column = step.can_remove_required_output.at(0);
|
||||
|
||||
Names columns_to_remove_after_sampling;
|
||||
for (size_t i = 1; i < step.required_output.size(); ++i)
|
||||
{
|
||||
if (step.can_remove_required_output[i])
|
||||
columns_to_remove_after_sampling.push_back(step.required_output[i]);
|
||||
}
|
||||
|
||||
if (!columns_to_remove_after_sampling.empty())
|
||||
{
|
||||
auto columns = res.prewhere_info->prewhere_actions->getSampleBlock().getNamesAndTypesList();
|
||||
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(columns, context);
|
||||
for (const auto & column : columns_to_remove_after_sampling)
|
||||
actions->add(ExpressionAction::removeColumn(column));
|
||||
|
||||
res.prewhere_info->after_sampling_actions = std::move(actions);
|
||||
}
|
||||
}
|
||||
if (has_where)
|
||||
res.remove_where_filter = chain.steps.at(where_step_num).can_remove_required_output.at(0);
|
||||
|
||||
@ -317,7 +338,8 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
|
||||
{
|
||||
ExpressionActionsChain chain(context);
|
||||
|
||||
if (query_analyzer->appendPrewhere(chain, !res.first_stage))
|
||||
ASTPtr sampling_expression = storage && query.sample_size() ? storage->getSamplingExpression() : nullptr;
|
||||
if (query_analyzer->appendPrewhere(chain, !res.first_stage, sampling_expression))
|
||||
{
|
||||
has_prewhere = true;
|
||||
|
||||
|
@ -194,7 +194,7 @@ Field convertFieldToTypeImpl(const Field & src, const IDataType & type)
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (type.isStringOrFixedString())
|
||||
else if (isStringOrFixedString(type))
|
||||
{
|
||||
if (src.getType() == Field::Types::String)
|
||||
return src;
|
||||
|
@ -343,6 +343,9 @@ public:
|
||||
/// Returns data path if storage supports it, empty string otherwise.
|
||||
virtual String getDataPath() const { return {}; }
|
||||
|
||||
/// Returns sampling expression for storage or nullptr if there is no.
|
||||
virtual ASTPtr getSamplingExpression() const { return nullptr; }
|
||||
|
||||
protected:
|
||||
using ITableDeclaration::ITableDeclaration;
|
||||
using std::enable_shared_from_this<IStorage>::shared_from_this;
|
||||
|
@ -708,7 +708,7 @@ bool KeyCondition::atomFromAST(const ASTPtr & node, const Context & context, Blo
|
||||
|
||||
bool cast_not_needed =
|
||||
is_set_const /// Set args are already casted inside Set::createFromAST
|
||||
|| (key_expr_type->isNumber() && const_type->isNumber()); /// Numbers are accurately compared without cast.
|
||||
|| (isNumber(key_expr_type) && isNumber(const_type)); /// Numbers are accurately compared without cast.
|
||||
|
||||
if (!cast_not_needed)
|
||||
castValueToType(key_expr_type, const_value, const_type, node);
|
||||
|
@ -77,8 +77,8 @@ MergeTreeBlockInputStream::MergeTreeBlockInputStream(
|
||||
}
|
||||
}
|
||||
|
||||
injectVirtualColumns(header);
|
||||
executePrewhereActions(header, prewhere_info);
|
||||
injectVirtualColumns(header);
|
||||
|
||||
ordered_names = getHeader().getNames();
|
||||
}
|
||||
|
@ -593,6 +593,10 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
|
||||
stream = std::make_shared<AddingConstColumnBlockInputStream<Float64>>(
|
||||
stream, std::make_shared<DataTypeFloat64>(), used_sample_factor, "_sample_factor");
|
||||
|
||||
if (query_info.prewhere_info && query_info.prewhere_info->after_sampling_actions)
|
||||
for (auto & stream : res)
|
||||
stream = std::make_shared<ExpressionBlockInputStream>(stream, query_info.prewhere_info->after_sampling_actions);
|
||||
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -42,8 +42,8 @@ MergeTreeThreadBlockInputStream::MergeTreeThreadBlockInputStream(
|
||||
Block MergeTreeThreadBlockInputStream::getHeader() const
|
||||
{
|
||||
auto res = pool->getHeader();
|
||||
injectVirtualColumns(res);
|
||||
executePrewhereActions(res, prewhere_info);
|
||||
injectVirtualColumns(res);
|
||||
return res;
|
||||
}
|
||||
|
||||
|
@ -21,10 +21,12 @@ using PreparedSets = std::unordered_map<StringRange, SetPtr, StringRangePointers
|
||||
|
||||
struct PrewhereInfo
|
||||
{
|
||||
/// Ections which are executed in order to alias columns are used for prewhere actions.
|
||||
/// Actions which are executed in order to alias columns are used for prewhere actions.
|
||||
ExpressionActionsPtr alias_actions;
|
||||
/// Actions which are executed on block in order to get filter column for prewhere step.
|
||||
ExpressionActionsPtr prewhere_actions;
|
||||
/// Actions which are executed after sampling in order to remove unused columns.
|
||||
ExpressionActionsPtr after_sampling_actions;
|
||||
String prewhere_column_name;
|
||||
bool remove_prewhere_column = false;
|
||||
|
||||
|
@ -94,6 +94,8 @@ public:
|
||||
|
||||
String getDataPath() const override { return full_path; }
|
||||
|
||||
ASTPtr getSamplingExpression() const override { return data.sampling_expression; }
|
||||
|
||||
private:
|
||||
String path;
|
||||
String database_name;
|
||||
|
@ -193,6 +193,8 @@ public:
|
||||
|
||||
String getDataPath() const override { return full_path; }
|
||||
|
||||
ASTPtr getSamplingExpression() const override { return data.sampling_expression; }
|
||||
|
||||
private:
|
||||
/// Delete old parts from disk and from ZooKeeper.
|
||||
void clearOldPartsAndRemoveFromZK();
|
||||
|
@ -76,6 +76,12 @@ class ClickHouseCluster:
|
||||
self.is_up = False
|
||||
|
||||
|
||||
def get_client_cmd(self):
|
||||
cmd = self.client_bin_path
|
||||
if p.basename(cmd) == 'clickhouse':
|
||||
cmd += " client"
|
||||
return cmd
|
||||
|
||||
def add_instance(self, name, config_dir=None, main_configs=[], user_configs=[], macros={}, with_zookeeper=False, with_mysql=False, with_kafka=False, clickhouse_path_dir=None, with_odbc_drivers=False, hostname=None, env_variables={}, image="ubuntu:14.04"):
|
||||
"""Add an instance to the cluster.
|
||||
|
||||
|
@ -67,20 +67,20 @@ def test_deduplication_works_in_case_of_intensive_inserts(started_cluster):
|
||||
inserters.append(CommandRequest(['/bin/bash'], timeout=10, stdin="""
|
||||
set -e
|
||||
for i in `seq 1000`; do
|
||||
clickhouse-client --host {} -q "INSERT INTO simple VALUES (0, 0)"
|
||||
{} --host {} -q "INSERT INTO simple VALUES (0, 0)"
|
||||
done
|
||||
""".format(host)))
|
||||
""".format(cluster.get_client_cmd(), host)))
|
||||
|
||||
fetchers.append(CommandRequest(['/bin/bash'], timeout=10, stdin="""
|
||||
set -e
|
||||
for i in `seq 1000`; do
|
||||
res=`clickhouse-client --host {} -q "SELECT count() FROM simple"`
|
||||
res=`{} --host {} -q "SELECT count() FROM simple"`
|
||||
if [[ $? -ne 0 || $res -ne 1 ]]; then
|
||||
echo "Selected $res elements! Host: {}" 1>&2
|
||||
exit -1
|
||||
fi;
|
||||
done
|
||||
""".format(host, node.name)))
|
||||
""".format(cluster.get_client_cmd(), host, node.name)))
|
||||
|
||||
# There were not errors during INSERTs
|
||||
for inserter in inserters:
|
||||
|
@ -50,7 +50,7 @@ def test_random_inserts(started_cluster):
|
||||
bash_script = os.path.join(os.path.dirname(__file__), "test.sh")
|
||||
inserters = []
|
||||
for node in nodes:
|
||||
cmd = ['/bin/bash', bash_script, node.ip_address, str(min_timestamp), str(max_timestamp)]
|
||||
cmd = ['/bin/bash', bash_script, node.ip_address, str(min_timestamp), str(max_timestamp), str(cluster.get_client_cmd())]
|
||||
inserters.append(CommandRequest(cmd, timeout=DURATION_SECONDS * 2, stdin=''))
|
||||
print node.name, node.ip_address
|
||||
|
||||
@ -60,7 +60,7 @@ def test_random_inserts(started_cluster):
|
||||
answer="{}\t{}\t{}\t{}\n".format(num_timestamps, num_timestamps, min_timestamp, max_timestamp)
|
||||
|
||||
for node in nodes:
|
||||
res = node.query("SELECT count(), uniqExact(i), min(i), max(i) FROM simple")
|
||||
res = node.query_with_retry("SELECT count(), uniqExact(i), min(i), max(i) FROM simple", check_callback=lambda res: TSV(res) == TSV(answer))
|
||||
assert TSV(res) == TSV(answer), node.name + " : " + node.query("SELECT groupArray(_part), i, count() AS c FROM simple GROUP BY i ORDER BY c DESC LIMIT 1")
|
||||
|
||||
node1.query("""DROP TABLE simple ON CLUSTER test_cluster""")
|
||||
|
@ -4,7 +4,7 @@
|
||||
[[ -n "$1" ]] && host="$1" || host="localhost"
|
||||
[[ -n "$2" ]] && min_timestamp="$2" || min_timestamp=$(( $(date +%s) - 60 ))
|
||||
[[ -n "$3" ]] && max_timestamp="$3" || max_timestamp=$(( $(date +%s) + 60 ))
|
||||
[[ -n "$4" ]] && iters_per_timestamp="$4" || iters_per_timestamp=5
|
||||
[[ -n "$4" ]] && client="$4" || client="clickhouse-client"
|
||||
|
||||
timestamps=`seq $min_timestamp $max_timestamp`
|
||||
|
||||
@ -18,7 +18,7 @@ function reliable_insert {
|
||||
fi
|
||||
|
||||
#echo clickhouse-client --host $host -q "INSERT INTO simple VALUES (0, $ts, '$ts')"
|
||||
res=`clickhouse-client --host $host -q "INSERT INTO simple VALUES (0, $ts, '$ts')" 2>&1`
|
||||
res=`$client --host $host -q "INSERT INTO simple VALUES (0, $ts, '$ts')" 2>&1`
|
||||
rt=$?
|
||||
num_tries=$(($num_tries+1))
|
||||
|
||||
|
@ -130,7 +130,7 @@ class Runner:
|
||||
|
||||
|
||||
def test_mutations(started_cluster):
|
||||
DURATION_SECONDS = 50
|
||||
DURATION_SECONDS = 30
|
||||
|
||||
runner = Runner()
|
||||
|
||||
@ -155,8 +155,8 @@ def test_mutations(started_cluster):
|
||||
assert runner.total_mutations > 0
|
||||
|
||||
all_done = False
|
||||
for i in range(100): # wait for replication 50 seconds max
|
||||
time.sleep(0.5)
|
||||
for i in range(100): # wait for replication 80 seconds max
|
||||
time.sleep(0.8)
|
||||
|
||||
def get_done_mutations(node):
|
||||
return int(node.query("SELECT sum(is_done) FROM system.mutations WHERE table = 'test_mutations'").rstrip())
|
||||
|
@ -0,0 +1 @@
|
||||
1
|
@ -0,0 +1,6 @@
|
||||
drop table if exists test.tab;
|
||||
create table test.tab (a UInt32, b UInt32) engine = MergeTree order by b % 2 sample by b % 2;
|
||||
insert into test.tab values (1, 2), (1, 4);
|
||||
select a from test.tab sample 1 / 2 prewhere b = 2;
|
||||
drop table if exists test.tab;
|
||||
|
@ -0,0 +1,6 @@
|
||||
00000000-0000-01f8-9cb8-cb1b82fb3900 00000000-0000-01f8-9cb8-cb1b82fb3900
|
||||
00000000-0000-02f8-9cb8-cb1b82fb3900 00000000-0000-02f8-9cb8-cb1b82fb3900
|
||||
00000000-0000-03f8-9cb8-cb1b82fb3900 00000000-0000-03f8-9cb8-cb1b82fb3900
|
||||
00000000-0000-04f8-9cb8-cb1b82fb3900 00000000-0000-04f8-9cb8-cb1b82fb3900
|
||||
00000000-0000-05f8-9cb8-cb1b82fb3900 00000000-0000-06f8-9cb8-cb1b82fb3900
|
||||
UUID UUID
|
47
dbms/tests/queries/0_stateless/00714_alter_uuid.sql
Normal file
47
dbms/tests/queries/0_stateless/00714_alter_uuid.sql
Normal file
@ -0,0 +1,47 @@
|
||||
USE test;
|
||||
|
||||
SELECT '00000000-0000-01f8-9cb8-cb1b82fb3900' AS str, toUUID(str);
|
||||
SELECT toFixedString('00000000-0000-02f8-9cb8-cb1b82fb3900', 36) AS str, toUUID(str);
|
||||
|
||||
SELECT '00000000-0000-03f8-9cb8-cb1b82fb3900' AS str, CAST(str, 'UUID');
|
||||
SELECT toFixedString('00000000-0000-04f8-9cb8-cb1b82fb3900', 36) AS str, CAST(str, 'UUID');
|
||||
|
||||
DROP TABLE IF EXISTS uuid;
|
||||
CREATE TABLE IF NOT EXISTS uuid
|
||||
(
|
||||
created_at DateTime,
|
||||
id0 String,
|
||||
id1 FixedString(36)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
PARTITION BY toDate(created_at)
|
||||
ORDER BY (created_at);
|
||||
|
||||
INSERT INTO uuid VALUES ('2018-01-01 01:02:03', '00000000-0000-05f8-9cb8-cb1b82fb3900', '00000000-0000-06f8-9cb8-cb1b82fb3900');
|
||||
|
||||
ALTER TABLE uuid MODIFY COLUMN id0 UUID;
|
||||
ALTER TABLE uuid MODIFY COLUMN id1 UUID;
|
||||
|
||||
SELECT id0, id1 FROM uuid;
|
||||
SELECT toTypeName(id0), toTypeName(id1) FROM uuid;
|
||||
|
||||
DROP TABLE uuid;
|
||||
|
||||
-- with UUID in key
|
||||
|
||||
CREATE TABLE IF NOT EXISTS uuid
|
||||
(
|
||||
created_at DateTime,
|
||||
id0 String,
|
||||
id1 FixedString(36)
|
||||
)
|
||||
ENGINE = MergeTree
|
||||
PARTITION BY toDate(created_at)
|
||||
ORDER BY (created_at, id0, id1);
|
||||
|
||||
SET send_logs_level = 'none';
|
||||
|
||||
ALTER TABLE uuid MODIFY COLUMN id0 UUID; -- { serverError 44 }
|
||||
ALTER TABLE uuid MODIFY COLUMN id1 UUID; -- { serverError 44 }
|
||||
|
||||
DROP TABLE uuid;
|
9
debian/pbuilder-hooks/B90test-server
vendored
9
debian/pbuilder-hooks/B90test-server
vendored
@ -44,14 +44,15 @@ if [ "${TEST_CONNECT}" ]; then
|
||||
[ "${TEST_PORT_RANDOM}" ] && echo "<yandex><http_port>${CLICKHOUSE_PORT_HTTP}</http_port><tcp_port>${CLICKHOUSE_PORT_TCP}</tcp_port><interserver_http_port>${CLICKHOUSE_PORT_INTERSERVER}</interserver_http_port></yandex>" > /etc/clickhouse-server/config.d/port.xml
|
||||
|
||||
if [ "${TEST_SSL}" ]; then
|
||||
echo "<yandex><https_port>${CLICKHOUSE_PORT_HTTPS}</https_port><tcp_port_secure>${CLICKHOUSE_PORT_TCP_SECURE}</tcp_port_secure></yandex>" > /etc/clickhouse-server/config.d/ssl.xml
|
||||
echo "<yandex><tcp_port_secure>${CLICKHOUSE_PORT_TCP_SECURE}</tcp_port_secure><tcp_port>${CLICKHOUSE_PORT_TCP}</tcp_port><openSSL><client><verificationMode>none</verificationMode><invalidCertificateHandler><name>AcceptCertificateHandler</name></invalidCertificateHandler></client></openSSL></yandex>" > /etc/clickhouse-client/config.xml
|
||||
CLICKHOUSE_SSL_CONFIG="<openSSL><client><verificationMode>none</verificationMode><invalidCertificateHandler><name>AcceptCertificateHandler</name></invalidCertificateHandler></client></openSSL>"
|
||||
echo "<yandex><https_port>${CLICKHOUSE_PORT_HTTPS}</https_port><tcp_port_secure>${CLICKHOUSE_PORT_TCP_SECURE}</tcp_port_secure>${CLICKHOUSE_SSL_CONFIG}</yandex>" > /etc/clickhouse-server/config.d/ssl.xml
|
||||
echo "<yandex><tcp_port>${CLICKHOUSE_PORT_TCP}</tcp_port><tcp_port_secure>${CLICKHOUSE_PORT_TCP_SECURE}</tcp_port_secure>${CLICKHOUSE_SSL_CONFIG}</yandex>" > /etc/clickhouse-client/config.xml
|
||||
openssl dhparam -out /etc/clickhouse-server/dhparam.pem 256
|
||||
openssl req -subj "/CN=localhost" -new -newkey rsa:2048 -days 365 -nodes -x509 -keyout /etc/clickhouse-server/server.key -out /etc/clickhouse-server/server.crt
|
||||
chmod a+r /etc/clickhouse-server/* /etc/clickhouse-client/*
|
||||
CLIENT_ADD+="--secure --port $CLICKHOUSE_PORT_TCP_SECURE"
|
||||
CLIENT_ADD+="--secure --port ${CLICKHOUSE_PORT_TCP_SECURE}"
|
||||
else
|
||||
CLIENT_ADD+="--port $CLICKHOUSE_PORT_TCP"
|
||||
CLIENT_ADD+="--port ${CLICKHOUSE_PORT_TCP}"
|
||||
fi
|
||||
|
||||
# For debug
|
||||
|
@ -52,4 +52,4 @@ FROM t_null
|
||||
└────────────┘
|
||||
|
||||
2 rows in set. Elapsed: 0.144 sec.
|
||||
``
|
||||
```
|
||||
|
@ -20,7 +20,7 @@ In a column-oriented DBMS, data is stored like this:
|
||||
|
||||
| Row: | #0 | #1 | #2 | #N |
|
||||
| ----------- | ------------------- | ------------------- | ------------------- | ------------------- |
|
||||
| WatchID: | 5385521489354350662 | 5385521490329509958 | 5385521489953706054 | ... |
|
||||
| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... |
|
||||
| JavaEnable: | 1 | 0 | 1 | ... |
|
||||
| Title: | Investor Relations | Contact us | Mission | ... |
|
||||
| GoodEvent: | 1 | 1 | 1 | ... |
|
||||
@ -61,11 +61,11 @@ Column-oriented databases are better suited to OLAP scenarios: they are at least
|
||||
|
||||
**Row-oriented DBMS**
|
||||
|
||||
![Row-oriented ]( images/row_oriented.gif#)
|
||||
![Row-oriented](images/row_oriented.gif#)
|
||||
|
||||
**Column-oriented DBMS**
|
||||
|
||||
![Column-oriented](images / column_oriented.gif#)
|
||||
![Column-oriented](images/column_oriented.gif#)
|
||||
|
||||
See the difference?
|
||||
|
||||
|
@ -4,12 +4,12 @@ ClickHouse - столбцовая система управления базам
|
||||
|
||||
В обычной, "строковой" СУБД, данные хранятся в таком порядке:
|
||||
|
||||
| Строка | WatchID | JavaEnable | Title | GoodEvent | EventTime |
|
||||
| ------ | ------------------- | ---------- | ------------------ | --------- | ------------------- |
|
||||
| #0 | 5385521489354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 |
|
||||
| #1 | 5385521490329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 |
|
||||
| #2 | 5385521489953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 |
|
||||
| #N | ... | ... | ... | ... | ... |
|
||||
| Строка | WatchID | JavaEnable | Title | GoodEvent | EventTime |
|
||||
| ------ | ----------- | ---------- | ------------------ | --------- | ------------------- |
|
||||
| #0 | 89354350662 | 1 | Investor Relations | 1 | 2016-05-18 05:19:20 |
|
||||
| #1 | 90329509958 | 0 | Contact us | 1 | 2016-05-18 08:10:20 |
|
||||
| #2 | 89953706054 | 1 | Mission | 1 | 2016-05-18 07:38:00 |
|
||||
| #N | ... | ... | ... | ... | ... |
|
||||
|
||||
То есть, значения, относящиеся к одной строке, физически хранятся рядом.
|
||||
|
||||
@ -20,7 +20,7 @@ ClickHouse - столбцовая система управления базам
|
||||
|
||||
| Строка: | #0 | #1 | #2 | #N |
|
||||
| ----------- | ------------------- | ------------------- | ------------------- | ------------------- |
|
||||
| WatchID: | 5385521489354350662 | 5385521490329509958 | 5385521489953706054 | ... |
|
||||
| WatchID: | 89354350662 | 90329509958 | 89953706054 | ... |
|
||||
| JavaEnable: | 1 | 0 | 1 | ... |
|
||||
| Title: | Investor Relations | Contact us | Mission | ... |
|
||||
| GoodEvent: | 1 | 1 | 1 | ... |
|
||||
|
3
release
3
release
@ -38,10 +38,9 @@ DEB_CC=${DEB_CC:=gcc-7}
|
||||
DEB_CXX=${DEB_CXX:=g++-7}
|
||||
PBUILDER_AUTOUPDATE=${PBUILDER_AUTOUPDATE=4320}
|
||||
|
||||
CONTROL=debian/control
|
||||
DEBUILD_NOSIGN_OPTIONS="-us -uc"
|
||||
DEBUILD_NODEPS_OPTIONS="-d"
|
||||
USE_PBUILDER=1
|
||||
USE_PBUILDER=${USE_PBUILDER=1}
|
||||
|
||||
if [ -z "$VERSION_STRING" ] ; then
|
||||
get_revision_author
|
||||
|
@ -26,7 +26,7 @@ var paths = {
|
||||
docstxt: ['docs/**/*.txt', 'docs/redirects.conf'],
|
||||
docsjson: ['docs/**/*.json'],
|
||||
docsxml: ['docs/**/*.xml'],
|
||||
docssitemap: ['sitemap.xml'],
|
||||
docssitemap: ['sitemap.xml', 'sitemap_static.xml'],
|
||||
scripts: [
|
||||
'**/*.js',
|
||||
'!gulpfile.js',
|
||||
|
@ -1,8 +1,10 @@
|
||||
User-agent: *
|
||||
Disallow: /docs/en/single/
|
||||
Disallow: /docs/ru/single/
|
||||
Disallow: /docs/fa/single/
|
||||
Disallow: /docs/en/search.html
|
||||
Disallow: /docs/ru/search.html
|
||||
Disallow: /docs/fa/search.html
|
||||
Disallow: /deprecated/reference_en.html
|
||||
Disallow: /deprecated/reference_ru.html
|
||||
Allow: /
|
||||
|
@ -1,10 +1,13 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<sitemapindex xmlns="http://www.sitemaps.org/schemas/sitemap/0.9">
|
||||
<sitemap>
|
||||
<loc>https://clickhouse.yandex/docs/en/sitemap.xml</loc>
|
||||
</sitemap>
|
||||
<sitemap>
|
||||
<loc>https://clickhouse.yandex/docs/ru/sitemap.xml</loc>
|
||||
</sitemap>
|
||||
<sitemap>
|
||||
<loc>https://clickhouse.yandex/docs/en/sitemap.xml</loc>
|
||||
<loc>https://clickhouse.yandex/docs/fa/sitemap.xml</loc>
|
||||
</sitemap>
|
||||
<sitemap>
|
||||
<loc>https://clickhouse.yandex/docs/sitemap_static.xml</loc>
|
||||
|
Loading…
Reference in New Issue
Block a user