merged with master

This commit is contained in:
Nikolai Kochetov 2017-07-17 22:48:22 +03:00
commit 7e293df9af
178 changed files with 1308 additions and 1065 deletions

View File

@ -1,3 +1,6 @@
include (CMakePushCheckState)
cmake_push_check_state ()
set (READLINE_PATHS "/usr/local/opt/readline/lib")
# First try find custom lib for macos users (default lib without history support)
find_library (READLINE_LIB NAMES readline PATHS ${READLINE_PATHS} NO_DEFAULT_PATH)
@ -50,3 +53,5 @@ if (LINE_EDITING_LIBS AND READLINE_INCLUDE_DIR)
else ()
message (STATUS "Not using any library for line editing.")
endif ()
cmake_pop_check_state ()

View File

@ -1,4 +1,7 @@
include (CheckCXXSourceCompiles)
include (CMakePushCheckState)
cmake_push_check_state ()
set (TEST_FLAG "-no-pie")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
@ -14,3 +17,5 @@ set (CMAKE_REQUIRED_FLAGS "")
if (HAVE_NO_PIE)
set (FLAG_NO_PIE ${TEST_FLAG})
endif ()
cmake_pop_check_state ()

View File

@ -1,6 +1,9 @@
# https://software.intel.com/sites/landingpage/IntrinsicsGuide/
include (CheckCXXSourceCompiles)
include (CMakePushCheckState)
cmake_push_check_state ()
# gcc -dM -E -mno-sse2 - < /dev/null | sort > gcc-dump-nosse2
# gcc -dM -E -msse2 - < /dev/null | sort > gcc-dump-sse2
@ -58,6 +61,6 @@ if (HAVE_POPCNT AND NOT AARCH64)
set (COMPILER_FLAGS "${COMPILER_FLAGS} ${TEST_FLAG}")
endif ()
set (CMAKE_REQUIRED_FLAGS "")
cmake_pop_check_state ()
# TODO: add here sse3 test if you want use it

View File

@ -3,8 +3,12 @@
#
include (CheckCXXSourceRuns)
include (CMakePushCheckState)
find_package (Threads)
cmake_push_check_state ()
if (USE_STATIC_LIBRARIES)
set (ANL_LIB_NAME "libanl.a")
else ()
@ -32,3 +36,5 @@ check_cxx_source_runs("
if (HAVE_GETADDRINFO_A)
add_definitions (-DHAVE_GETADDRINFO_A=1)
endif ()
cmake_pop_check_state ()

View File

@ -41,11 +41,6 @@ AggregateFunctionPtr createAggregateFunctionNullVariadic(AggregateFunctionPtr &
AggregateFunctionPtr createAggregateFunctionCountNotNull(const DataTypes & argument_types);
AggregateFunctionFactory::AggregateFunctionFactory()
{
}
void AggregateFunctionFactory::registerFunction(const String & name, Creator creator, CaseSensitiveness case_sensitiveness)
{
if (creator == nullptr)

View File

@ -25,7 +25,6 @@ private:
using AggregateFunctions = std::unordered_map<String, Creator>;
public:
AggregateFunctionFactory();
AggregateFunctionPtr get(const String & name, const DataTypes & argument_types, int recursion_level = 0) const;
AggregateFunctionPtr tryGet(const String & name, const DataTypes & argument_types) const;
bool isAggregateFunctionName(const String & name, int recursion_level = 0) const;
@ -40,9 +39,6 @@ public:
/// Register an aggregate function by its name.
void registerFunction(const String & name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
AggregateFunctionFactory(const AggregateFunctionFactory &) = delete;
AggregateFunctionFactory & operator=(const AggregateFunctionFactory &) = delete;
private:
AggregateFunctionPtr getImpl(const String & name, const DataTypes & argument_types, int recursion_level) const;

View File

@ -96,7 +96,7 @@ public:
using Type = T;
using FieldType = typename NearestFieldType<T>::Type;
std::string getName() const override { return "ColumnConst<" + TypeName<T>::get() + ">"; }
std::string getName() const override { return "ColumnConst<" + String(TypeName<T>::get()) + ">"; }
bool isNumeric() const override { return IsNumber<T>::value; }
bool isFixed() const override { return IsNumber<T>::value; }
size_t sizeOfField() const override { return sizeof(T); }

View File

@ -99,7 +99,7 @@ void ColumnVector<T>::getPermutation(bool reverse, size_t limit, int nan_directi
template <typename T>
std::string ColumnVector<T>::getName() const
{
return "ColumnVector<" + TypeName<T>::get() + ">";
return "ColumnVector<" + String(TypeName<T>::get()) + ">";
}
template <typename T>

View File

@ -21,30 +21,32 @@ struct UInt128
#pragma GCC diagnostic ignored "-Wmaybe-uninitialized"
#endif
/// This naming assumes little endian.
UInt64 low;
UInt64 high;
UInt128() = default;
explicit UInt128(const UInt64 rhs) : low(0), high(rhs) { }
explicit UInt128(const UInt64 low, const UInt64 high) : low(low), high(high) { }
explicit UInt128(const UInt64 rhs) : low(rhs), high() {}
explicit UInt128(const UInt64 low, const UInt64 high) : low(low), high(high) {}
bool inline operator== (const UInt128 rhs) const { return std::tie(low, high) == std::tie(rhs.low, rhs.high); }
bool inline operator!= (const UInt128 rhs) const { return !operator==(rhs); }
auto tuple() const { return std::tie(high, low); }
bool inline operator< (const UInt128 rhs) const { return std::tie(low, high) < std::tie(rhs.low, rhs.high); }
bool inline operator<= (const UInt128 rhs) const { return !rhs.operator<(*this); }
bool inline operator> (const UInt128 rhs) const { return rhs.operator<(*this); }
bool inline operator>= (const UInt128 rhs) const { return !operator<(rhs); }
bool inline operator== (const UInt128 rhs) const { return tuple() == rhs.tuple(); }
bool inline operator!= (const UInt128 rhs) const { return tuple() != rhs.tuple(); }
bool inline operator< (const UInt128 rhs) const { return tuple() < rhs.tuple(); }
bool inline operator<= (const UInt128 rhs) const { return tuple() <= rhs.tuple(); }
bool inline operator> (const UInt128 rhs) const { return tuple() > rhs.tuple(); }
bool inline operator>= (const UInt128 rhs) const { return tuple() >= rhs.tuple(); }
/** Types who are stored at the moment in the database have no more than 64bits and can be handle
* inside an unique UInt64.
*/
template<typename T> bool inline operator== (const T rhs) const { return *this == UInt128(0, rhs); }
template<typename T> bool inline operator!= (const T rhs) const { return *this != UInt128(0, rhs); }
template<typename T> bool inline operator>= (const T rhs) const { return *this >= UInt128(0, rhs); }
template<typename T> bool inline operator> (const T rhs) const { return *this > UInt128(0, rhs); }
template<typename T> bool inline operator<= (const T rhs) const { return *this <= UInt128(0, rhs); }
template<typename T> bool inline operator< (const T rhs) const { return *this < UInt128(0, rhs); }
template <typename T> bool inline operator== (const T rhs) const { return *this == UInt128(rhs); }
template <typename T> bool inline operator!= (const T rhs) const { return *this != UInt128(rhs); }
template <typename T> bool inline operator>= (const T rhs) const { return *this >= UInt128(rhs); }
template <typename T> bool inline operator> (const T rhs) const { return *this > UInt128(rhs); }
template <typename T> bool inline operator<= (const T rhs) const { return *this <= UInt128(rhs); }
template <typename T> bool inline operator< (const T rhs) const { return *this < UInt128(rhs); }
template<typename T> explicit operator T() const { return static_cast<T>(high); }
@ -55,15 +57,15 @@ struct UInt128
UInt128 & operator= (const UInt64 rhs) { low = 0; high = rhs; return *this; }
};
template<typename T> bool inline operator== (T a, const UInt128 b) { return b == a; }
template<typename T> bool inline operator!= (T a, const UInt128 b) { return b != a; }
template<typename T> bool inline operator>= (T a, const UInt128 b) { return b.low == 0 && a >= static_cast<T>(b.high); }
template<typename T> bool inline operator> (T a, const UInt128 b) { return b.low == 0 && a > static_cast<T>(b.high); }
template<typename T> bool inline operator<= (T a, const UInt128 b) { return b.low != 0 || a <= static_cast<T>(b.high); }
template<typename T> bool inline operator< (T a, const UInt128 b) { return b.low != 0 || a < static_cast<T>(b.high); }
template <typename T> bool inline operator== (T a, const UInt128 b) { return UInt128(a) == b; }
template <typename T> bool inline operator!= (T a, const UInt128 b) { return UInt128(a) != b; }
template <typename T> bool inline operator>= (T a, const UInt128 b) { return UInt128(a) >= b; }
template <typename T> bool inline operator> (T a, const UInt128 b) { return UInt128(a) > b; }
template <typename T> bool inline operator<= (T a, const UInt128 b) { return UInt128(a) <= b; }
template <typename T> bool inline operator< (T a, const UInt128 b) { return UInt128(a) < b; }
template <> struct IsNumber<UInt128> { static constexpr bool value = true; };
template <> struct TypeName<UInt128> { static std::string get() { return "UInt128"; } };
template <> struct TypeName<UInt128> { static const char * get() { return "UInt128"; } };
struct UInt128Hash
{

View File

@ -158,7 +158,7 @@ bool filterBlockWithQuery(const ASTPtr & query, Block & block, const Context & c
if (!expression_ast)
return false;
/// Let's parse and calculate the expression.
/// Let's analyze and calculate the expression.
ExpressionAnalyzer analyzer(expression_ast, context, {}, block.getColumnsList());
ExpressionActionsPtr actions = analyzer.getActions(false);
actions->execute(block);

View File

@ -379,6 +379,8 @@ namespace ErrorCodes
extern const int INVALID_SESSION_TIMEOUT = 374;
extern const int CANNOT_DLOPEN = 375;
extern const int CANNOT_PARSE_UUID = 376;
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE = 377;
extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS = 378;
extern const int KEEPER_EXCEPTION = 999;
extern const int POCO_EXCEPTION = 1000;

View File

@ -134,22 +134,22 @@ class FieldVisitorConvertToNumber : public StaticVisitor<T>
public:
T operator() (const Null & x) const
{
throw Exception("Cannot convert NULL to " + TypeName<T>::get(), ErrorCodes::CANNOT_CONVERT_TYPE);
throw Exception("Cannot convert NULL to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const String & x) const
{
throw Exception("Cannot convert String to " + TypeName<T>::get(), ErrorCodes::CANNOT_CONVERT_TYPE);
throw Exception("Cannot convert String to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const Array & x) const
{
throw Exception("Cannot convert Array to " + TypeName<T>::get(), ErrorCodes::CANNOT_CONVERT_TYPE);
throw Exception("Cannot convert Array to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const Tuple & x) const
{
throw Exception("Cannot convert Tuple to " + TypeName<T>::get(), ErrorCodes::CANNOT_CONVERT_TYPE);
throw Exception("Cannot convert Tuple to " + String(TypeName<T>::get()), ErrorCodes::CANNOT_CONVERT_TYPE);
}
T operator() (const UInt64 & x) const { return x; }

View File

@ -41,40 +41,37 @@ template <typename T> struct RemoveNullable<Nullable<T>> { using Type = T; };
template <typename T> struct IsNullable { static constexpr bool value = false; };
template <typename T> struct IsNullable<Nullable<T>> { static constexpr bool value = true; };
template <typename T> struct IsNumber { static constexpr bool value = false; };
template <typename T> struct IsNumber { static constexpr bool value = false; };
template <typename T> struct IsNumber<Nullable<T> > { static constexpr bool value = IsNumber<T>::value; };
template <> struct IsNumber<UInt8> { static constexpr bool value = true; };
template <> struct IsNumber<UInt16> { static constexpr bool value = true; };
template <> struct IsNumber<UInt32> { static constexpr bool value = true; };
template <> struct IsNumber<UInt64> { static constexpr bool value = true; };
template <> struct IsNumber<Int8> { static constexpr bool value = true; };
template <> struct IsNumber<Int16> { static constexpr bool value = true; };
template <> struct IsNumber<Int32> { static constexpr bool value = true; };
template <> struct IsNumber<Int64> { static constexpr bool value = true; };
template <> struct IsNumber<Float32> { static constexpr bool value = true; };
template <> struct IsNumber<Float64> { static constexpr bool value = true; };
template <> struct IsNumber<UInt8> { static constexpr bool value = true; };
template <> struct IsNumber<UInt16> { static constexpr bool value = true; };
template <> struct IsNumber<UInt32> { static constexpr bool value = true; };
template <> struct IsNumber<UInt64> { static constexpr bool value = true; };
template <> struct IsNumber<Int8> { static constexpr bool value = true; };
template <> struct IsNumber<Int16> { static constexpr bool value = true; };
template <> struct IsNumber<Int32> { static constexpr bool value = true; };
template <> struct IsNumber<Int64> { static constexpr bool value = true; };
template <> struct IsNumber<Float32> { static constexpr bool value = true; };
template <> struct IsNumber<Float64> { static constexpr bool value = true; };
template <typename T> struct TypeName;
template <typename T> struct TypeName<Nullable<T>> { static std::string get() { return "Nullable(" + TypeName<T>::get() + ")"; } };
template <typename T> struct TypeName<Nullable<T>> { static const char * get() { return "Nullable"; } };
template <> struct TypeName<Null> { static std::string get() { return "Null"; } };
template <> struct TypeName<Null> { static const char * get() { return "Null"; } };
template <> struct TypeName<Nullable<void>> : TypeName<Null> {};
template <> struct TypeName<UInt8> { static std::string get() { return "UInt8"; } };
template <> struct TypeName<UInt16> { static std::string get() { return "UInt16"; } };
template <> struct TypeName<UInt32> { static std::string get() { return "UInt32"; } };
template <> struct TypeName<UInt64> { static std::string get() { return "UInt64"; } };
template <> struct TypeName<Int8> { static std::string get() { return "Int8"; } };
template <> struct TypeName<Int16> { static std::string get() { return "Int16"; } };
template <> struct TypeName<Int32> { static std::string get() { return "Int32"; } };
template <> struct TypeName<Int64> { static std::string get() { return "Int64"; } };
template <> struct TypeName<Float32> { static std::string get() { return "Float32"; } };
template <> struct TypeName<Float64> { static std::string get() { return "Float64"; } };
template <> struct TypeName<String> { static std::string get() { return "String"; } };
/// This type is not supported by the DBMS, but is used in some internal transformations.
template <> struct TypeName<long double>{ static std::string get() { return "long double"; } };
template <> struct TypeName<UInt8> { static const char * get() { return "UInt8"; } };
template <> struct TypeName<UInt16> { static const char * get() { return "UInt16"; } };
template <> struct TypeName<UInt32> { static const char * get() { return "UInt32"; } };
template <> struct TypeName<UInt64> { static const char * get() { return "UInt64"; } };
template <> struct TypeName<Int8> { static const char * get() { return "Int8"; } };
template <> struct TypeName<Int16> { static const char * get() { return "Int16"; } };
template <> struct TypeName<Int32> { static const char * get() { return "Int32"; } };
template <> struct TypeName<Int64> { static const char * get() { return "Int64"; } };
template <> struct TypeName<Float32> { static const char * get() { return "Float32"; } };
template <> struct TypeName<Float64> { static const char * get() { return "Float64"; } };
template <> struct TypeName<String> { static const char * get() { return "String"; } };
}

View File

@ -85,7 +85,7 @@ Block ColumnGathererStream::readImpl()
if (sources.empty())
init();
if (row_sources_buf.eof() && !source_to_fully_copy)
if (!source_to_fully_copy && row_sources_buf.eof())
return Block();
output_block = Block{column.cloneEmpty()};

View File

@ -1,7 +1,6 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Columns/ColumnConst.h>
#include <iostream>
namespace DB

View File

@ -1,7 +1,6 @@
#include <iomanip>
#include <random>
#include <Columns/ColumnConst.h>
#include <Interpreters/Quota.h>
#include <Interpreters/ProcessList.h>
#include <DataStreams/IProfilingBlockInputStream.h>

View File

@ -1,5 +1,4 @@
#include <DataStreams/MaterializingBlockInputStream.h>
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>

View File

@ -1,4 +1,3 @@
#include <Columns/ColumnConst.h>
#include <DataStreams/MaterializingBlockOutputStream.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>

View File

@ -5,7 +5,6 @@
#include <IO/VarInt.h>
#include <IO/CompressedWriteBuffer.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnsNumber.h>

View File

@ -4,7 +4,6 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataStreams/PrettyBlockOutputStream.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnsNumber.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>

View File

@ -102,7 +102,7 @@ void RemoteBlockInputStream::sendExternalTables()
{
StoragePtr cur = table.second;
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
DB::BlockInputStreams input = cur->read(cur->getColumnNamesList(), ASTPtr(), context,
BlockInputStreams input = cur->read(cur->getColumnNamesList(), {}, context,
stage, DEFAULT_BLOCK_SIZE, 1);
if (input.size() == 0)
res.push_back(std::make_pair(std::make_shared<OneBlockInputStream>(cur->getSampleBlock()), table.first));

View File

@ -1,7 +1,6 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Columns/ColumnConst.h>

View File

@ -117,25 +117,7 @@ bool ValuesRowInputStream::read(Block & block)
if (value.isNull())
{
/// Check that we are indeed allowed to insert a NULL.
bool is_null_allowed = false;
if (type.isNullable())
is_null_allowed = true;
else
{
/// NOTE: For now we support only one level of null values, i.e.
/// there are not yet such things as Array(Nullable(Array(Nullable(T))).
/// Therefore the code below is valid within the current limitations.
const auto array_type = typeid_cast<const DataTypeArray *>(&type);
if (array_type != nullptr)
{
const auto & nested_type = array_type->getMostNestedType();
if (nested_type->isNullable())
is_null_allowed = true;
}
}
if (!is_null_allowed)
if (!type.isNullable())
throw Exception{"Expression returns value " + applyVisitor(FieldVisitorToString(), value)
+ ", that is out of range of type " + type.getName()
+ ", at: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),

View File

@ -1,4 +1,5 @@
#include <Columns/ColumnConst.h>
#include <cmath>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <IO/WriteHelpers.h>

View File

@ -50,7 +50,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in;
in = table->read(column_names, 0, context, stage, 8192, 1)[0];
in = table->read(column_names, {}, context, stage, 8192, 1)[0];
in = std::make_shared<ExpressionBlockInputStream>(in, expression);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));

View File

@ -55,7 +55,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, stage, 8192, 1)[0];
BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 1);
in = std::make_shared<LimitBlockInputStream>(in, 10, std::max(static_cast<Int64>(0), static_cast<Int64>(n) - 10));

View File

@ -128,7 +128,7 @@ int main(int argc, char ** argv)
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, stage, 8192, 1)[0];
BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0];
in = std::make_shared<FilterBlockInputStream>(in, expression, 4);
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0);

View File

@ -72,7 +72,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, context, stage, 8192, 1)[0];
BlockInputStreamPtr in = table->read(column_names, {}, context, stage, 8192, 1)[0];
ForkBlockInputStreams fork(in);

View File

@ -103,7 +103,7 @@ try
if (argc == 2 && 0 == strcmp(argv[1], "read"))
{
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, Context::createGlobal(), stage, 8192, 1)[0];
BlockInputStreamPtr in = table->read(column_names, {}, Context::createGlobal(), stage, 8192, 1)[0];
WriteBufferFromFileDescriptor out1(STDOUT_FILENO);
CompressedWriteBuffer out2(out1);
NativeBlockOutputStream out3(out2, ClickHouseRevision::get());

View File

@ -145,7 +145,7 @@ try
QueryProcessingStage::Enum stage;
BlockInputStreamPtr in = table->read(column_names, 0, Context::createGlobal(), stage, argc == 2 ? atoi(argv[1]) : 65536, 1)[0];
BlockInputStreamPtr in = table->read(column_names, {}, Context::createGlobal(), stage, argc == 2 ? atoi(argv[1]) : 65536, 1)[0];
in = std::make_shared<PartialSortingBlockInputStream>(in, sort_columns);
in = std::make_shared<MergeSortingBlockInputStream>(in, sort_columns, DEFAULT_BLOCK_SIZE, 0, 0, "");
//in = std::make_shared<LimitBlockInputStream>(in, 10, 0);

View File

@ -35,9 +35,9 @@ void test1()
QueryProcessingStage::Enum stage3;
BlockInputStreams streams;
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage1, 1, 1)[0], 30, 30000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage2, 1, 1)[0], 30, 2000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage3, 1, 1)[0], 30, 100));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, {}, context, stage1, 1, 1)[0], 30, 30000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, {}, context, stage2, 1, 1)[0], 30, 2000));
streams.emplace_back(std::make_shared<LimitBlockInputStream>(table->read(column_names, {}, context, stage3, 1, 1)[0], 30, 100));
UnionBlockInputStream<> union_stream(streams, nullptr, 2);
@ -84,15 +84,15 @@ void test2()
BlockInputStreams streams;
BlockInputStreamPtr stream1 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage1, 1, 1)[0], 30, 30000);
BlockInputStreamPtr stream1 = std::make_shared<LimitBlockInputStream>(table->read(column_names, {}, context, stage1, 1, 1)[0], 30, 30000);
stream1 = std::make_shared<BlockExtraInfoInputStream>(stream1, extra_info1);
streams.emplace_back(stream1);
BlockInputStreamPtr stream2 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage2, 1, 1)[0], 30, 2000);
BlockInputStreamPtr stream2 = std::make_shared<LimitBlockInputStream>(table->read(column_names, {}, context, stage2, 1, 1)[0], 30, 2000);
stream2 = std::make_shared<BlockExtraInfoInputStream>(stream2, extra_info2);
streams.emplace_back(stream2);
BlockInputStreamPtr stream3 = std::make_shared<LimitBlockInputStream>(table->read(column_names, 0, context, stage3, 1, 1)[0], 30, 100);
BlockInputStreamPtr stream3 = std::make_shared<LimitBlockInputStream>(table->read(column_names, {}, context, stage3, 1, 1)[0], 30, 100);
stream3 = std::make_shared<BlockExtraInfoInputStream>(stream3, extra_info3);
streams.emplace_back(stream3);

View File

@ -35,7 +35,7 @@ try
StoragePtr table = context.getTable("default", "hits6");
QueryProcessingStage::Enum stage;
BlockInputStreams streams = table->read(column_names, nullptr, context, stage, settings.max_block_size, settings.max_threads);
BlockInputStreams streams = table->read(column_names, {}, context, stage, settings.max_block_size, settings.max_threads);
for (size_t i = 0, size = streams.size(); i < size; ++i)
streams[i] = std::make_shared<AsynchronousBlockInputStream>(streams[i]);

View File

@ -9,11 +9,24 @@
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeFactory.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTIdentifier.h>
namespace DB
{
namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
extern const int BAD_ARGUMENTS;
extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS;
}
std::string DataTypeAggregateFunction::getName() const
{
std::stringstream stream;
@ -268,5 +281,67 @@ Field DataTypeAggregateFunction::getDefault() const
}
static DataTypePtr create(const ASTPtr & arguments)
{
String function_name;
AggregateFunctionPtr function;
DataTypes argument_types;
Array params_row;
if (arguments->children.empty())
throw Exception("Data type AggregateFunction requires parameters: "
"name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (const ASTFunction * parametric = typeid_cast<const ASTFunction *>(arguments->children[0].get()))
{
if (parametric->parameters)
throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR);
function_name = parametric->name;
const ASTs & parameters = typeid_cast<const ASTExpressionList &>(*parametric->arguments).children;
params_row.resize(parameters.size());
for (size_t i = 0; i < parameters.size(); ++i)
{
const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(parameters[i].get());
if (!lit)
throw Exception("Parameters to aggregate functions must be literals",
ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS);
params_row[i] = lit->value;
}
}
else if (const ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(arguments->children[0].get()))
{
function_name = identifier->name;
}
else if (typeid_cast<ASTLiteral *>(arguments->children[0].get()))
{
throw Exception("Aggregate function name for data type AggregateFunction must be passed as identifier (without quotes) or function",
ErrorCodes::BAD_ARGUMENTS);
}
else
throw Exception("Unexpected AST element passed as aggregate function name for data type AggregateFunction. Must be identifier or function.",
ErrorCodes::BAD_ARGUMENTS);
for (size_t i = 1; i < arguments->children.size(); ++i)
argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i]));
if (function_name.empty())
throw Exception("Logical error: empty name of aggregate function passed", ErrorCodes::LOGICAL_ERROR);
function = AggregateFunctionFactory::instance().get(function_name, argument_types);
if (!params_row.empty())
function->setParameters(params_row);
function->setArguments(argument_types);
return std::make_shared<DataTypeAggregateFunction>(function, argument_types, params_row);
}
void registerDataTypeAggregateFunction(DataTypeFactory & factory)
{
factory.registerDataType("AggregateFunction", create);
}
}

View File

@ -36,6 +36,8 @@ public:
std::string getName() const override;
const char * getFamilyName() const override { return "AggregateFunction"; }
DataTypePtr getReturnType() const { return function->getReturnType(); };
DataTypes getArgumentsDataTypes() const { return argument_types; }

View File

@ -8,15 +8,22 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNull.h>
#include <DataTypes/DataTypeNullable.h>
#include <Parsers/IAST.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_READ_ARRAY_FROM_TEXT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
@ -396,24 +403,28 @@ ColumnPtr DataTypeArray::createConstColumn(size_t size, const Field & field) con
}
const DataTypePtr & DataTypeArray::getMostNestedType() const
static DataTypePtr create(const ASTPtr & arguments)
{
const DataTypeArray * array = this;
const IDataType * array_nested_type = array->getNestedType().get();
if (arguments->children.size() != 1)
throw Exception("Array data type family must have exactly one argument - type of elements", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
while (true)
DataTypePtr nested_type = DataTypeFactory::instance().get(arguments->children[0]);
if (typeid_cast<const DataTypeNull *>(nested_type.get()))
{
const DataTypeArray * type = typeid_cast<const DataTypeArray *>(array_nested_type);
if (type == nullptr)
break;
else
{
array = type;
array_nested_type = array->getNestedType().get();
}
/// Special case: Array(Null) is actually Array(Nullable(UInt8)).
return std::make_shared<DataTypeArray>(
std::make_shared<DataTypeNullable>(
std::make_shared<DataTypeUInt8>()));
}
return array->getNestedType();
return std::make_shared<DataTypeArray>(nested_type);
}
void registerDataTypeArray(DataTypeFactory & factory)
{
factory.registerDataType("Array", create);
}
}

View File

@ -27,6 +27,16 @@ public:
return "Array(" + nested->getName() + ")";
}
const char * getFamilyName() const override
{
return "Array";
}
bool canBeInsideNullable() const override
{
return false;
}
DataTypePtr clone() const override
{
return std::make_shared<DataTypeArray>(enriched_nested);
@ -86,9 +96,6 @@ public:
const DataTypePtr & getNestedType() const { return nested; }
const DataTypeTraits::EnrichedDataTypePtr & getEnrichedNestedType() const { return enriched_nested; }
const DataTypePtr & getOffsetsType() const { return offsets; }
/// Returns the data type found at the most nested level.
const DataTypePtr & getMostNestedType() const;
};
}

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB
@ -76,4 +77,10 @@ void DataTypeDate::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const
static_cast<ColumnUInt16 &>(column).getData().push_back(value.getDayNum());
}
void registerDataTypeDate(DataTypeFactory & factory)
{
factory.registerSimpleDataType("Date", [] { return DataTypePtr(std::make_shared<DataTypeDate>()); }, DataTypeFactory::CaseInsensitive);
}
}

View File

@ -12,6 +12,7 @@ public:
bool behavesAsNumber() const override { return false; }
std::string getName() const override { return "Date"; }
const char * getFamilyName() const override { return "Date"; }
DataTypePtr clone() const override { return std::make_shared<DataTypeDate>(); }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB
@ -76,4 +77,10 @@ void DataTypeDateTime::deserializeTextCSV(IColumn & column, ReadBuffer & istr, c
static_cast<ColumnUInt32 &>(column).getData().push_back(static_cast<time_t>(value));
}
void registerDataTypeDateTime(DataTypeFactory & factory)
{
factory.registerSimpleDataType("DateTime", [] { return DataTypePtr(std::make_shared<DataTypeDateTime>()); }, DataTypeFactory::CaseInsensitive);
}
}

View File

@ -12,6 +12,7 @@ public:
bool behavesAsNumber() const override { return false; }
std::string getName() const override { return "DateTime"; }
const char * getFamilyName() const override { return "DateTime"; }
DataTypePtr clone() const override { return std::make_shared<DataTypeDateTime>(); }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;

View File

@ -1,9 +1,14 @@
#include <IO/WriteBufferFromString.h>
#include <DataTypes/DataTypeEnum.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Common/typeid_cast.h>
#include <limits>
namespace DB
{
@ -11,6 +16,8 @@ namespace ErrorCodes
{
extern const int SYNTAX_ERROR;
extern const int EMPTY_DATA_PASSED;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int ARGUMENT_OUT_OF_BOUND;
}
@ -19,6 +26,13 @@ template <> struct EnumName<Int8> { static constexpr auto value = "Enum8"; };
template <> struct EnumName<Int16> { static constexpr auto value = "Enum16"; };
template <typename Type>
const char * DataTypeEnum<Type>::getFamilyName() const
{
return EnumName<FieldType>::value;
}
template <typename Type>
std::string DataTypeEnum<Type>::generateName(const Values & values)
{
@ -78,12 +92,12 @@ DataTypeEnum<Type>::DataTypeEnum(const Values & values_) : values{values_}
if (values.empty())
throw Exception{
"DataTypeEnum enumeration cannot be empty",
ErrorCodes::EMPTY_DATA_PASSED
};
ErrorCodes::EMPTY_DATA_PASSED};
fillMaps();
std::sort(std::begin(values), std::end(values), [] (auto & left, auto & right) {
std::sort(std::begin(values), std::end(values), [] (auto & left, auto & right)
{
return left.second < right.second;
});
@ -286,4 +300,59 @@ Field DataTypeEnum<Type>::castToValue(const Field & value_or_name) const
template class DataTypeEnum<Int8>;
template class DataTypeEnum<Int16>;
template <typename DataTypeEnum>
static DataTypePtr create(const ASTPtr & arguments)
{
if (arguments->children.empty())
throw Exception("Enum data type cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
typename DataTypeEnum::Values values;
values.reserve(arguments->children.size());
using FieldType = typename DataTypeEnum::FieldType;
/// Children must be functions 'equals' with string literal as left argument and numeric literal as right argument.
for (const ASTPtr & child : arguments->children)
{
const ASTFunction * func = typeid_cast<const ASTFunction *>(child.get());
if (!func
|| func->name != "equals"
|| func->parameters
|| !func->arguments
|| func->arguments->children.size() != 2)
throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer",
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
const ASTLiteral * name_literal = typeid_cast<const ASTLiteral *>(func->arguments->children[0].get());
const ASTLiteral * value_literal = typeid_cast<const ASTLiteral *>(func->arguments->children[1].get());
if (!name_literal
|| !value_literal
|| name_literal->value.getType() != Field::Types::String
|| (value_literal->value.getType() != Field::Types::UInt64 && value_literal->value.getType() != Field::Types::Int64))
throw Exception("Elements of Enum data type must be of form: 'name' = number, where name is string literal and number is an integer",
ErrorCodes::UNEXPECTED_AST_STRUCTURE);
const String & name = name_literal->value.get<String>();
const auto value = value_literal->value.get<typename NearestFieldType<FieldType>::Type>();
if (value > std::numeric_limits<FieldType>::max() || value < std::numeric_limits<FieldType>::min())
throw Exception{
"Value " + toString(value) + " for element '" + name + "' exceeds range of " + EnumName<FieldType>::value,
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
values.emplace_back(name, value);
}
return std::make_shared<DataTypeEnum>(values);
}
void registerDataTypeEnum(DataTypeFactory & factory)
{
factory.registerDataType("Enum8", create<DataTypeEnum<Int8>>);
factory.registerDataType("Enum16", create<DataTypeEnum<Int16>>);
}
}

View File

@ -52,6 +52,7 @@ public:
const Values & getValues() const { return values; }
std::string getName() const override { return name; }
const char * getFamilyName() const override;
bool isNumeric() const override { return true; }
bool behavesAsNumber() const override { return true; }
@ -61,8 +62,7 @@ public:
if (it == std::end(value_to_name_map))
throw Exception{
"Unexpected value " + toString(value) + " for type " + getName(),
ErrorCodes::LOGICAL_ERROR
};
ErrorCodes::LOGICAL_ERROR};
return it->second;
}

View File

@ -20,6 +20,7 @@ public:
: argument_types(argument_types_), return_type(return_type_) {}
std::string getName() const override;
const char * getFamilyName() const override { return "Expression"; }
DataTypePtr clone() const override
{

View File

@ -1,33 +1,10 @@
#include <IO/ReadHelpers.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeAggregateFunction.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/DataTypeNull.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeFactory.h>
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTNameTypePair.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTEnumElement.h>
#include <Parsers/ParserEnumElement.h>
#include <Parsers/parseQuery.h>
#include <DataTypes/DataTypeEnum.h>
#include <ext/map.h>
#include <Parsers/ParserCreateQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
#include <Common/typeid_cast.h>
#include <Poco/String.h>
namespace DB
@ -35,218 +12,121 @@ namespace DB
namespace ErrorCodes
{
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_TYPE;
extern const int NESTED_TYPE_TOO_DEEP;
extern const int PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS;
extern const int SYNTAX_ERROR;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int ILLEGAL_SYNTAX_FOR_DATA_TYPE;
extern const int UNEXPECTED_AST_STRUCTURE;
extern const int DATA_TYPE_CANNOT_HAVE_ARGUMENTS;
}
DataTypePtr DataTypeFactory::get(const String & full_name) const
{
ParserIdentifierWithOptionalParameters parser;
ASTPtr ast = parseQuery(parser, full_name.data(), full_name.data() + full_name.size(), "data type");
return get(ast);
}
DataTypePtr DataTypeFactory::get(const ASTPtr & ast) const
{
if (const ASTFunction * func = typeid_cast<const ASTFunction *>(ast.get()))
{
if (func->parameters)
throw Exception("Data type cannot have multiple parenthesed parameters.", ErrorCodes::ILLEGAL_SYNTAX_FOR_DATA_TYPE);
return get(func->name, func->arguments);
}
if (const ASTIdentifier * ident = typeid_cast<const ASTIdentifier *>(ast.get()))
{
return get(ident->name, {});
}
throw Exception("Unexpected AST element for data type.", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
}
DataTypePtr DataTypeFactory::get(const String & family_name, const ASTPtr & parameters) const
{
{
DataTypesDictionary::const_iterator it = data_types.find(family_name);
if (data_types.end() != it)
return it->second(parameters);
}
{
String family_name_lowercase = Poco::toLower(family_name);
DataTypesDictionary::const_iterator it = case_insensitive_data_types.find(family_name_lowercase);
if (case_insensitive_data_types.end() != it)
return it->second(parameters);
}
throw Exception("Unknown data type family: " + family_name, ErrorCodes::UNKNOWN_TYPE);
}
void DataTypeFactory::registerDataType(const String & family_name, Creator creator, CaseSensitiveness case_sensitiveness)
{
if (creator == nullptr)
throw Exception("DataTypeFactory: the data type family " + family_name + " has been provided "
" a null constructor", ErrorCodes::LOGICAL_ERROR);
if (!data_types.emplace(family_name, creator).second)
throw Exception("DataTypeFactory: the data type family name '" + family_name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
String family_name_lowercase = Poco::toLower(family_name);
if (case_sensitiveness == CaseInsensitive
&& !case_insensitive_data_types.emplace(family_name_lowercase, creator).second)
throw Exception("DataTypeFactory: the case insensitive data type family name '" + family_name + "' is not unique",
ErrorCodes::LOGICAL_ERROR);
}
void DataTypeFactory::registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness)
{
if (creator == nullptr)
throw Exception("DataTypeFactory: the data type " + name + " has been provided "
" a null constructor", ErrorCodes::LOGICAL_ERROR);
registerDataType(name, [name, creator](const ASTPtr & ast)
{
if (ast)
throw Exception("Data type " + name + " cannot have arguments", ErrorCodes::DATA_TYPE_CANNOT_HAVE_ARGUMENTS);
return creator();
}, case_sensitiveness);
}
void registerDataTypeNumbers(DataTypeFactory & factory);
void registerDataTypeDate(DataTypeFactory & factory);
void registerDataTypeDateTime(DataTypeFactory & factory);
void registerDataTypeString(DataTypeFactory & factory);
void registerDataTypeFixedString(DataTypeFactory & factory);
void registerDataTypeEnum(DataTypeFactory & factory);
void registerDataTypeArray(DataTypeFactory & factory);
void registerDataTypeTuple(DataTypeFactory & factory);
void registerDataTypeNullable(DataTypeFactory & factory);
void registerDataTypeNull(DataTypeFactory & factory);
void registerDataTypeUUID(DataTypeFactory & factory);
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
void registerDataTypeNested(DataTypeFactory & factory);
DataTypeFactory::DataTypeFactory()
: non_parametric_data_types
{
{"UInt8", std::make_shared<DataTypeUInt8>()},
{"UInt16", std::make_shared<DataTypeUInt16>()},
{"UInt32", std::make_shared<DataTypeUInt32>()},
{"UInt64", std::make_shared<DataTypeUInt64>()},
{"Int8", std::make_shared<DataTypeInt8>()},
{"Int16", std::make_shared<DataTypeInt16>()},
{"Int32", std::make_shared<DataTypeInt32>()},
{"Int64", std::make_shared<DataTypeInt64>()},
{"Float32", std::make_shared<DataTypeFloat32>()},
{"Float64", std::make_shared<DataTypeFloat64>()},
{"Date", std::make_shared<DataTypeDate>()},
{"DateTime", std::make_shared<DataTypeDateTime>()},
{"UUID", std::make_shared<DataTypeUUID>()},
{"String", std::make_shared<DataTypeString>()},
{"Null", std::make_shared<DataTypeNull>()}
}
{
registerDataTypeNumbers(*this);
registerDataTypeDate(*this);
registerDataTypeDateTime(*this);
registerDataTypeString(*this);
registerDataTypeFixedString(*this);
registerDataTypeEnum(*this);
registerDataTypeArray(*this);
registerDataTypeTuple(*this);
registerDataTypeNullable(*this);
registerDataTypeNull(*this);
registerDataTypeUUID(*this);
registerDataTypeAggregateFunction(*this);
registerDataTypeNested(*this);
}
template <typename DataTypeEnum>
inline DataTypePtr parseEnum(const String & name, const String & base_name, const String & parameters)
{
ParserList parser{std::make_unique<ParserEnumElement>(), std::make_unique<ParserToken>(TokenType::Comma), false};
ASTPtr elements = parseQuery(parser, parameters.data(), parameters.data() + parameters.size(), "parameters for enum type " + name);
typename DataTypeEnum::Values values;
values.reserve(elements->children.size());
using FieldType = typename DataTypeEnum::FieldType;
for (const auto & element : typeid_cast<const ASTExpressionList &>(*elements).children)
{
const auto & e = static_cast<const ASTEnumElement &>(*element);
const auto value = e.value.get<typename NearestFieldType<FieldType>::Type>();
if (value > std::numeric_limits<FieldType>::max() || value < std::numeric_limits<FieldType>::min())
throw Exception{
"Value " + applyVisitor(FieldVisitorToString{}, e.value) + " for element '" + e.name + "' exceeds range of " + base_name,
ErrorCodes::ARGUMENT_OUT_OF_BOUND};
values.emplace_back(e.name, value);
}
return std::make_shared<DataTypeEnum>(values);
}
DataTypePtr DataTypeFactory::get(const String & name) const
{
return getImpl(name, true);
}
DataTypePtr DataTypeFactory::getImpl(const String & name, bool allow_nullable) const
{
NonParametricDataTypes::const_iterator it = non_parametric_data_types.find(name);
if (it != non_parametric_data_types.end())
return it->second;
Poco::RegularExpression::MatchVec matches;
if (fixed_string_regexp.match(name, 0, matches) && matches.size() == 2)
return std::make_shared<DataTypeFixedString>(parse<size_t>(name.data() + matches[1].offset, matches[1].length));
if (nested_regexp.match(name, 0, matches) && matches.size() == 3)
{
String base_name(name.data() + matches[1].offset, matches[1].length);
String parameters(name.data() + matches[2].offset, matches[2].length);
if (base_name == "Nullable")
{
if (!allow_nullable)
throw Exception{"A Nullable type cannot contain another Nullable type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeNullable>(getImpl(parameters, false));
}
if (base_name == "Array")
{
if (parameters == "Null")
{
/// Special case: Array(Null) is actually Array(Nullable(UInt8)).
return std::make_shared<DataTypeArray>(
std::make_shared<DataTypeNullable>(
std::make_shared<DataTypeUInt8>()));
}
else
return std::make_shared<DataTypeArray>(getImpl(parameters, allow_nullable));
}
if (base_name == "AggregateFunction")
{
String function_name;
AggregateFunctionPtr function;
DataTypes argument_types;
Array params_row;
ParserExpressionList args_parser(false);
ASTPtr args_ast = parseQuery(args_parser, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
ASTExpressionList & args_list = typeid_cast<ASTExpressionList &>(*args_ast);
if (args_list.children.empty())
throw Exception("Data type AggregateFunction requires parameters: "
"name of aggregate function and list of data types for arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (ASTFunction * parametric = typeid_cast<ASTFunction *>(args_list.children[0].get()))
{
if (parametric->parameters)
throw Exception("Unexpected level of parameters to aggregate function", ErrorCodes::SYNTAX_ERROR);
function_name = parametric->name;
ASTs & parameters = typeid_cast<ASTExpressionList &>(*parametric->arguments).children;
params_row.resize(parameters.size());
for (size_t i = 0; i < parameters.size(); ++i)
{
ASTLiteral * lit = typeid_cast<ASTLiteral *>(parameters[i].get());
if (!lit)
throw Exception("Parameters to aggregate functions must be literals",
ErrorCodes::PARAMETERS_TO_AGGREGATE_FUNCTIONS_MUST_BE_LITERALS);
params_row[i] = lit->value;
}
}
else if (ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(args_list.children[0].get()))
{
function_name = identifier->name;
}
else if (typeid_cast<ASTLiteral *>(args_list.children[0].get()))
{
throw Exception("Aggregate function name for data type AggregateFunction must be passed as identifier (without quotes) or function",
ErrorCodes::BAD_ARGUMENTS);
}
else
throw Exception("Unexpected AST element passed as aggregate function name for data type AggregateFunction. Must be identifier or function.",
ErrorCodes::BAD_ARGUMENTS);
for (size_t i = 1; i < args_list.children.size(); ++i)
argument_types.push_back(getImpl(
std::string{args_list.children[i]->range.first, args_list.children[i]->range.second}, allow_nullable));
if (function_name.empty())
throw Exception("Logical error: empty name of aggregate function passed", ErrorCodes::LOGICAL_ERROR);
function = AggregateFunctionFactory::instance().get(function_name, argument_types);
if (!params_row.empty())
function->setParameters(params_row);
function->setArguments(argument_types);
return std::make_shared<DataTypeAggregateFunction>(function, argument_types, params_row);
}
if (base_name == "Nested")
{
ParserNameTypePairList columns_p;
ASTPtr columns_ast = parseQuery(columns_p, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>();
ASTExpressionList & columns_list = typeid_cast<ASTExpressionList &>(*columns_ast);
for (ASTs::iterator it = columns_list.children.begin(); it != columns_list.children.end(); ++it)
{
ASTNameTypePair & name_and_type_pair = typeid_cast<ASTNameTypePair &>(**it);
StringRange type_range = name_and_type_pair.type->range;
DataTypePtr type = getImpl(String(type_range.first, type_range.second - type_range.first), allow_nullable);
if (typeid_cast<const DataTypeNested *>(type.get()))
throw Exception("Nested inside Nested is not allowed", ErrorCodes::NESTED_TYPE_TOO_DEEP);
columns->push_back(NameAndTypePair(
name_and_type_pair.name,
type));
}
return std::make_shared<DataTypeNested>(columns);
}
if (base_name == "Tuple")
{
ParserExpressionList columns_p(false);
ASTPtr columns_ast = parseQuery(columns_p, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
auto & columns_list = typeid_cast<ASTExpressionList &>(*columns_ast);
const auto elems = ext::map<DataTypes>(columns_list.children, [&] (const ASTPtr & elem_ast) {
return getImpl(String(elem_ast->range.first, elem_ast->range.second), allow_nullable);
});
return std::make_shared<DataTypeTuple>(elems);
}
if (base_name == "Enum8")
return parseEnum<DataTypeEnum8>(name, base_name, parameters);
if (base_name == "Enum16")
return parseEnum<DataTypeEnum16>(name, base_name, parameters);
throw Exception("Unknown type " + base_name, ErrorCodes::UNKNOWN_TYPE);
}
throw Exception("Unknown type " + name, ErrorCodes::UNKNOWN_TYPE);
}
}

View File

@ -1,32 +1,57 @@
#pragma once
#include <map>
#include <ext/singleton.h>
#include <Poco/RegularExpression.h>
#include <memory>
#include <functional>
#include <unordered_map>
#include <DataTypes/IDataType.h>
#include <ext/singleton.h>
namespace DB
{
/** Creates data type by its name (possibly name contains parameters in parens).
class IDataType;
using DataTypePtr = std::shared_ptr<IDataType>;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
/** Creates a data type by name of data type family and parameters.
*/
class DataTypeFactory : public ext::singleton<DataTypeFactory>
class DataTypeFactory final : public ext::singleton<DataTypeFactory>
{
private:
using Creator = std::function<DataTypePtr(const ASTPtr & parameters)>;
using SimpleCreator = std::function<DataTypePtr()>;
using DataTypesDictionary = std::unordered_map<String, Creator>;
public:
DataTypeFactory();
DataTypePtr get(const String & name) const;
DataTypePtr get(const String & full_name) const;
DataTypePtr get(const String & family_name, const ASTPtr & parameters) const;
DataTypePtr get(const ASTPtr & ast) const;
/// For compatibility with SQL, it's possible to specify that certain data type name is case insensitive.
enum CaseSensitiveness
{
CaseSensitive,
CaseInsensitive
};
/// Register a type family by its name.
void registerDataType(const String & family_name, Creator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
/// Register a simple data type, that have no parameters.
void registerSimpleDataType(const String & name, SimpleCreator creator, CaseSensitiveness case_sensitiveness = CaseSensitive);
private:
DataTypePtr getImpl(const String & name, bool allow_nullable) const;
DataTypesDictionary data_types;
using NonParametricDataTypes = std::map<String, DataTypePtr>;
NonParametricDataTypes non_parametric_data_types;
/// Case insensitive data types will be additionally added here with lowercased name.
DataTypesDictionary case_insensitive_data_types;
Poco::RegularExpression fixed_string_regexp {R"--(^FixedString\s*\(\s*(\d+)\s*\)$)--"};
Poco::RegularExpression nested_regexp {R"--(^(\w+)\s*\(\s*(.+)\s*\)$)--",
Poco::RegularExpression::RE_MULTILINE | Poco::RegularExpression::RE_DOTALL};
DataTypeFactory();
friend class ext::singleton<DataTypeFactory>;
};
}

View File

@ -6,13 +6,18 @@
#include <Columns/ColumnConst.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/VarInt.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTLiteral.h>
#include <Common/typeid_cast.h>
namespace DB
{
@ -20,6 +25,8 @@ namespace ErrorCodes
{
extern const int CANNOT_READ_ALL_DATA;
extern const int TOO_LARGE_STRING_SIZE;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int UNEXPECTED_AST_STRUCTURE;
}
@ -204,4 +211,23 @@ ColumnPtr DataTypeFixedString::createConstColumn(size_t size, const Field & fiel
return std::make_shared<ColumnConstString>(size, get<const String &>(field), clone());
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (arguments->children.size() != 1)
throw Exception("FixedString data type family must have exactly one argument - size in bytes", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
const ASTLiteral * argument = typeid_cast<const ASTLiteral *>(arguments->children[0].get());
if (!argument || argument->value.getType() != Field::Types::UInt64 || argument->value.get<UInt64>() == 0)
throw Exception("FixedString data type family must have a number (positive integer) as its argument", ErrorCodes::UNEXPECTED_AST_STRUCTURE);
return std::make_shared<DataTypeFixedString>(argument->value.get<UInt64>());
}
void registerDataTypeFixedString(DataTypeFactory & factory)
{
factory.registerDataType("FixedString", create);
}
}

View File

@ -26,6 +26,8 @@ public:
std::string getName() const override;
const char * getFamilyName() const override { return "FixedString"; }
DataTypePtr clone() const override
{
return std::make_shared<DataTypeFixedString>(n);

View File

@ -5,6 +5,10 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeNested.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
#include <Parsers/ASTNameTypePair.h>
namespace DB
@ -13,6 +17,8 @@ namespace DB
namespace ErrorCodes
{
extern const int INVALID_NESTED_NAME;
extern const int EMPTY_DATA_PASSED;
extern const int NESTED_TYPE_TOO_DEEP;
}
@ -90,4 +96,30 @@ NamesAndTypesListPtr DataTypeNested::expandNestedColumns(const NamesAndTypesList
return columns;
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (arguments->children.empty())
throw Exception("Nested structure cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
NamesAndTypesListPtr columns = std::make_shared<NamesAndTypesList>();
for (const auto & child : arguments->children)
{
const ASTNameTypePair & name_and_type_pair = typeid_cast<const ASTNameTypePair &>(*child);
DataTypePtr type = DataTypeFactory::instance().get(name_and_type_pair.type);
if (typeid_cast<const DataTypeNested *>(type.get()))
throw Exception("Nested inside Nested is not allowed", ErrorCodes::NESTED_TYPE_TOO_DEEP);
columns->emplace_back(name_and_type_pair.name, type);
}
return std::make_shared<DataTypeNested>(columns);
}
void registerDataTypeNested(DataTypeFactory & factory)
{
factory.registerDataType("Nested", create);
}
}

View File

@ -20,6 +20,7 @@ public:
DataTypeNested(NamesAndTypesListPtr nested_);
std::string getName() const override;
const char * getFamilyName() const override { return "Nested"; }
DataTypePtr clone() const override
{

View File

@ -1,11 +1,13 @@
#include <Columns/ColumnConst.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <DataTypes/DataTypeNull.h>
#include <DataTypes/DataTypeFactory.h>
#include <Columns/IColumn.h>
#include <Columns/ColumnConst.h>
namespace DB
@ -118,4 +120,10 @@ void DataTypeNull::deserializeTextJSON(IColumn & column, ReadBuffer & istr) cons
assertString("null", istr);
}
void registerDataTypeNull(DataTypeFactory & factory)
{
factory.registerSimpleDataType("Null", [] { return DataTypePtr(std::make_shared<DataTypeNull>()); });
}
}

View File

@ -21,6 +21,11 @@ public:
return "Null";
}
const char * getFamilyName() const override
{
return "Null";
}
bool isNull() const override
{
return true;
@ -31,6 +36,11 @@ public:
return true;
}
bool canBeInsideNullable() const override
{
return false;
}
DataTypePtr clone() const override
{
return std::make_shared<DataTypeNull>();

View File

@ -1,4 +1,5 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeFactory.h>
#include <Columns/ColumnNullable.h>
#include <IO/ReadBuffer.h>
#include <IO/ReadBufferFromMemory.h>
@ -6,11 +7,19 @@
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>
#include <IO/ConcatReadBuffer.h>
#include <Parsers/IAST.h>
#include <Common/typeid_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
}
DataTypeNullable::DataTypeNullable(DataTypePtr nested_data_type_)
: nested_data_type{nested_data_type_}
@ -236,4 +245,24 @@ ColumnPtr DataTypeNullable::createConstColumn(size_t size, const Field & field)
std::make_shared<ColumnUInt8>(size, 0));
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (arguments->children.size() != 1)
throw Exception("Nullable data type family must have exactly one argument - nested type", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
DataTypePtr nested_type = DataTypeFactory::instance().get(arguments->children[0]);
if (!nested_type->canBeInsideNullable())
throw Exception("Nested type " + nested_type->getName() + " cannot be inside Nullable type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeNullable>(nested_type);
}
void registerDataTypeNullable(DataTypeFactory & factory)
{
factory.registerDataType("Nullable", create);
}
}

View File

@ -13,11 +13,13 @@ class DataTypeNullable final : public IDataType
public:
DataTypeNullable(DataTypePtr nested_data_type_);
std::string getName() const override { return "Nullable(" + nested_data_type->getName() + ")"; }
const char * getFamilyName() const override { return "Nullable"; }
bool isNullable() const override { return true; }
bool isNumeric() const override { return nested_data_type->isNumeric(); } /// TODO Absolutely wrong.
bool isNumericNotNullable() const override { return false; }
bool behavesAsNumber() const override { return nested_data_type->behavesAsNumber(); } /// TODO Absolutely wrong.
bool canBeInsideNullable() const override { return false; }
DataTypePtr clone() const override { return std::make_shared<DataTypeNullable>(nested_data_type->clone()); }

View File

@ -259,4 +259,5 @@ template class DataTypeNumberBase<Int32>;
template class DataTypeNumberBase<Int64>;
template class DataTypeNumberBase<Float32>;
template class DataTypeNumberBase<Float64>;
}

View File

@ -6,8 +6,7 @@
namespace DB
{
/** Implements part of the IDataType interface, common to all numbers
* - input and output in text form.
/** Implements part of the IDataType interface, common to all numbers and for Date and DateTime.
*/
template <typename T>
class DataTypeNumberBase : public IDataType
@ -16,6 +15,7 @@ public:
using FieldType = T;
std::string getName() const override { return TypeName<T>::get(); }
const char * getFamilyName() const override { return TypeName<T>::get(); }
bool isNumeric() const override { return true; }
bool behavesAsNumber() const override { return true; }

View File

@ -7,12 +7,13 @@ namespace DB
{
/** The data type corresponding to the set of values in the IN section.
* Used only as an intermediate option when evaluating expressions.
* Used only as an intermediate when evaluating expressions.
*/
class DataTypeSet final : public IDataTypeDummy
{
public:
std::string getName() const override { return "Set"; }
const char * getFamilyName() const override { return "Set"; }
DataTypePtr clone() const override { return std::make_shared<DataTypeSet>(); }
};

View File

@ -7,6 +7,7 @@
#include <Common/typeid_cast.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
@ -301,4 +302,25 @@ ColumnPtr DataTypeString::createConstColumn(size_t size, const Field & field) co
return std::make_shared<ColumnConstString>(size, get<const String &>(field));
}
void registerDataTypeString(DataTypeFactory & factory)
{
auto creator = static_cast<DataTypePtr(*)()>([] { return DataTypePtr(std::make_shared<DataTypeString>()); });
factory.registerSimpleDataType("String", creator);
/// These synonims are added for compatibility.
factory.registerSimpleDataType("CHAR", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("VARCHAR", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("TEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("TINYTEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("MEDIUMTEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("LONGTEXT", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("BLOB", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("TINYBLOB", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("MEDIUMBLOB", creator, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("LONGBLOB", creator, DataTypeFactory::CaseInsensitive);
}
}

View File

@ -18,6 +18,11 @@ public:
return "String";
}
const char * getFamilyName() const override
{
return "String";
}
DataTypePtr clone() const override
{
return std::make_shared<DataTypeString>();

View File

@ -3,6 +3,8 @@
#include <DataStreams/NativeBlockInputStream.h>
#include <DataStreams/NativeBlockOutputStream.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeFactory.h>
#include <Parsers/IAST.h>
#include <ext/map.h>
#include <ext/enumerate.h>
@ -12,6 +14,12 @@
namespace DB
{
namespace ErrorCodes
{
extern const int EMPTY_DATA_PASSED;
}
std::string DataTypeTuple::getName() const
{
std::stringstream s;
@ -253,4 +261,25 @@ Field DataTypeTuple::getDefault() const
return Tuple(ext::map<TupleBackend>(elems, [] (const DataTypePtr & elem) { return elem->getDefault(); }));
}
static DataTypePtr create(const ASTPtr & arguments)
{
if (arguments->children.empty())
throw Exception("Tuple cannot be empty", ErrorCodes::EMPTY_DATA_PASSED);
DataTypes nested_types;
nested_types.reserve(arguments->children.size());
for (const ASTPtr & child : arguments->children)
nested_types.emplace_back(DataTypeFactory::instance().get(child));
return std::make_shared<DataTypeTuple>(nested_types);
}
void registerDataTypeTuple(DataTypeFactory & factory)
{
factory.registerDataType("Tuple", create);
}
}

View File

@ -19,8 +19,11 @@ public:
DataTypeTuple(DataTypes elems_) : elems(elems_) {}
std::string getName() const override;
const char * getFamilyName() const override { return "Tuple"; }
DataTypePtr clone() const override { return std::make_shared<DataTypeTuple>(elems); }
bool canBeInsideNullable() const override { return false; }
void serializeBinary(const Field & field, WriteBuffer & ostr) const override;
void deserializeBinary(Field & field, ReadBuffer & istr) const override;
void serializeBinary(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;

View File

@ -1,72 +1,82 @@
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB
{
void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
writeText(UUID(static_cast<const ColumnUInt128 &>(column).getData()[row_num]), ostr);
}
static void deserializeText(IColumn & column, ReadBuffer & istr)
{
UUID x;
readText(x, istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(x);
}
void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
serializeText(column, row_num, ostr);
}
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const
{
deserializeText(column, istr);
}
void DataTypeUUID::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
writeChar('\'', ostr);
serializeText(column, row_num, ostr);
writeChar('\'', ostr);
}
void DataTypeUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const
{
UUID x;
assertChar('\'', istr);
readText(x, istr);
assertChar('\'', istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
}
void DataTypeUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettingsJSON &) const
{
writeChar('"', ostr);
serializeText(column, row_num, ostr);
writeChar('"', ostr);
}
void DataTypeUUID::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{
UUID x;
assertChar('"', istr);
readText(x, istr);
assertChar('"', istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(x);
}
void DataTypeUUID::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
writeChar('"', ostr);
serializeText(column, row_num, ostr);
writeChar('"', ostr);
}
void DataTypeUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
UUID value;
readCSV(value, istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(value);
}
void DataTypeUUID::serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
writeText(UUID(static_cast<const ColumnUInt128 &>(column).getData()[row_num]), ostr);
}
static void deserializeText(IColumn & column, ReadBuffer & istr)
{
UUID x;
readText(x, istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(x);
}
void DataTypeUUID::serializeTextEscaped(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
serializeText(column, row_num, ostr);
}
void DataTypeUUID::deserializeTextEscaped(IColumn & column, ReadBuffer & istr) const
{
deserializeText(column, istr);
}
void DataTypeUUID::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
writeChar('\'', ostr);
serializeText(column, row_num, ostr);
writeChar('\'', ostr);
}
void DataTypeUUID::deserializeTextQuoted(IColumn & column, ReadBuffer & istr) const
{
UUID x;
assertChar('\'', istr);
readText(x, istr);
assertChar('\'', istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(x); /// It's important to do this at the end - for exception safety.
}
void DataTypeUUID::serializeTextJSON(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettingsJSON &) const
{
writeChar('"', ostr);
serializeText(column, row_num, ostr);
writeChar('"', ostr);
}
void DataTypeUUID::deserializeTextJSON(IColumn & column, ReadBuffer & istr) const
{
UUID x;
assertChar('"', istr);
readText(x, istr);
assertChar('"', istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(x);
}
void DataTypeUUID::serializeTextCSV(const IColumn & column, size_t row_num, WriteBuffer & ostr) const
{
writeChar('"', ostr);
serializeText(column, row_num, ostr);
writeChar('"', ostr);
}
void DataTypeUUID::deserializeTextCSV(IColumn & column, ReadBuffer & istr, const char delimiter) const
{
UUID value;
readCSV(value, istr);
static_cast<ColumnUInt128 &>(column).getData().push_back(value);
}
void registerDataTypeUUID(DataTypeFactory & factory)
{
factory.registerSimpleDataType("UUID", [] { return DataTypePtr(std::make_shared<DataTypeUUID>()); });
}
}

View File

@ -16,6 +16,7 @@ public:
bool behavesAsNumber() const override { return false; }
std::string getName() const override { return "UUID"; }
const char * getFamilyName() const override { return "UUID"; }
DataTypePtr clone() const override { return std::make_shared<DataTypeUUID>(); }
void serializeText(const IColumn & column, size_t row_num, WriteBuffer & ostr) const override;

View File

@ -0,0 +1,32 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB
{
void registerDataTypeNumbers(DataTypeFactory & factory)
{
factory.registerSimpleDataType("UInt8", [] { return DataTypePtr(std::make_shared<DataTypeUInt8>()); });
factory.registerSimpleDataType("UInt16", [] { return DataTypePtr(std::make_shared<DataTypeUInt16>()); });
factory.registerSimpleDataType("UInt32", [] { return DataTypePtr(std::make_shared<DataTypeUInt32>()); });
factory.registerSimpleDataType("UInt64", [] { return DataTypePtr(std::make_shared<DataTypeUInt64>()); });
factory.registerSimpleDataType("UInt128", [] { return DataTypePtr(std::make_shared<DataTypeUInt128>()); });
factory.registerSimpleDataType("Int8", [] { return DataTypePtr(std::make_shared<DataTypeInt8>()); });
factory.registerSimpleDataType("Int16", [] { return DataTypePtr(std::make_shared<DataTypeInt16>()); });
factory.registerSimpleDataType("Int32", [] { return DataTypePtr(std::make_shared<DataTypeInt32>()); });
factory.registerSimpleDataType("Int64", [] { return DataTypePtr(std::make_shared<DataTypeInt64>()); });
factory.registerSimpleDataType("Float32", [] { return DataTypePtr(std::make_shared<DataTypeFloat32>()); });
factory.registerSimpleDataType("Float64", [] { return DataTypePtr(std::make_shared<DataTypeFloat64>()); });
/// These synonims are added for compatibility.
factory.registerSimpleDataType("FLOAT", [] { return DataTypePtr(std::make_shared<DataTypeFloat32>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("DOUBLE", [] { return DataTypePtr(std::make_shared<DataTypeFloat64>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("INT", [] { return DataTypePtr(std::make_shared<DataTypeInt32>()); }, DataTypeFactory::CaseInsensitive);
factory.registerSimpleDataType("INTEGER", [] { return DataTypePtr(std::make_shared<DataTypeInt32>()); }, DataTypeFactory::CaseInsensitive);
}
}

View File

@ -35,6 +35,7 @@ public:
using FieldType = void;
std::string getName() const override { return "Void"; }
const char * getFamilyName() const override { return "Void"; }
DataTypePtr clone() const override { return std::make_shared<DataTypeNumber<void>>(); }
};
@ -47,6 +48,7 @@ public:
using FieldType = Null;
std::string getName() const override { return "Null"; }
const char * getFamilyName() const override { return "Null"; }
DataTypePtr clone() const override { return std::make_shared<DataTypeNumber<Null>>(); }
};

View File

@ -23,12 +23,17 @@ using DataTypes = std::vector<DataTypePtr>;
/** Properties of data type.
* Contains methods for serialization/deserialization.
* Implementations of this interface represent a data type (example: UInt8)
* or parapetric family of data types (example: Array(...)).
*/
class IDataType
{
public:
/// Name of data type (examples: UInt64, Array(String)).
virtual std::string getName() const = 0;
virtual String getName() const = 0;
/// Name of data type family (example: FixedString, Array).
virtual const char * getFamilyName() const = 0;
/// Is this type the null type? TODO Move this method to separate "traits" classes.
virtual bool isNull() const { return false; }
@ -49,6 +54,9 @@ public:
/// If this data type cannot appear in table declaration - only for intermediate values of calculations.
virtual bool notForTables() const { return false; }
/// If this data type cannot be wrapped in Nullable data type.
virtual bool canBeInsideNullable() const { return true; }
virtual DataTypePtr clone() const = 0;
/** Binary serialization for range of values in column - for writing to disk/network, etc.

View File

@ -19,6 +19,11 @@ private:
return true;
}
bool canBeInsideNullable() const override
{
return false;
}
void throwNoSerialization() const
{
throw Exception("Serialization is not implemented for data type " + getName(), ErrorCodes::NOT_IMPLEMENTED);

View File

@ -616,8 +616,8 @@ private:
bool executeRightTypeDispatch(Block & block, const ColumnNumbers & arguments, const size_t result,
const ColumnType * col_left)
{
throw Exception("Types " + TypeName<typename LeftDataType::FieldType>::get()
+ " and " + TypeName<typename LeftDataType::FieldType>::get()
throw Exception("Types " + String(TypeName<typename LeftDataType::FieldType>::get())
+ " and " + String(TypeName<typename LeftDataType::FieldType>::get())
+ " are incompatible for function " + getName() + " or not upscaleable to common type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}

View File

@ -1303,7 +1303,7 @@ private:
};
template <typename DataType>
const String FunctionEmptyArray<DataType>::name = FunctionEmptyArray::base_name + DataTypeToName<DataType>::get();
const String FunctionEmptyArray<DataType>::name = FunctionEmptyArray::base_name + String(DataTypeToName<DataType>::get());
class FunctionRange : public IFunction
{

View File

@ -202,12 +202,12 @@ public:
if (arguments[0]->getName() != TypeName<T>::get())
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName()
+ " (must be " + TypeName<T>::get() + ")",
+ " (must be " + String(TypeName<T>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName()
+ " (must be " + TypeName<String>::get() + ")",
+ " (must be " + String(TypeName<T>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return arguments[0];
@ -295,17 +295,17 @@ public:
if (arguments[0]->getName() != TypeName<T>::get())
throw Exception("Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ " (must be " + TypeName<T>::get() + ")",
+ " (must be " + String(TypeName<T>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments[1]->getName() != TypeName<T>::get())
throw Exception("Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ " (must be " + TypeName<T>::get() + ")",
+ " (must be " + String(TypeName<T>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 3 && arguments[2]->getName() != TypeName<String>::get())
throw Exception("Illegal type " + arguments[2]->getName() + " of the third ('point of view') argument of function " + getName()
+ " (must be " + TypeName<String>::get() + ")",
+ " (must be " + String(TypeName<String>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt8>();
@ -428,12 +428,12 @@ public:
if (arguments[0]->getName() != TypeName<T>::get())
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName()
+ " (must be " + TypeName<T>::get() + ")",
+ " (must be " + String(TypeName<T>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
throw Exception("Illegal type " + arguments[1]->getName() + " of the second ('point of view') argument of function " + getName()
+ " (must be " + TypeName<String>::get() + ")",
+ " (must be " + String(TypeName<String>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeArray>(arguments[0]);
@ -714,12 +714,12 @@ public:
if (arguments[0]->getName() != TypeName<UInt32>::get())
throw Exception("Illegal type " + arguments[0]->getName() + " of the first argument of function " + getName()
+ " (must be " + TypeName<UInt32>::get() + ")",
+ " (must be " + String(TypeName<UInt32>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
if (arguments.size() == 2 && arguments[1]->getName() != TypeName<String>::get())
throw Exception("Illegal type " + arguments[0]->getName() + " of the second argument of function " + getName()
+ " (must be " + TypeName<String>::get() + ")",
+ " (must be " + String(TypeName<String>::get()) + ")",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeString>();

View File

@ -19,7 +19,7 @@ struct TypeProcessorImpl
auto type_res = DataTypeTraits::ToEnrichedDataTypeObject<TCombined, true>::execute();
if ((type_res.first == DataTypePtr()) && (type_res.second == DataTypePtr()))
throw Exception("Types " + TypeName<TLeft>::get() + " and " + TypeName<TType>::get()
throw Exception("Types " + String(TypeName<TLeft>::get()) + " and " + String(TypeName<TType>::get())
+ " are not upscalable to a common type without loss of precision", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type_res;
@ -31,16 +31,16 @@ struct RightTypeProcessor
{
static DataTypeTraits::EnrichedDataTypePtr execute(const IDataType & type2)
{
if (typeid_cast<const DataTypeUInt8 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt8>::execute();
if (typeid_cast<const DataTypeUInt16 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt16>::execute();
if (typeid_cast<const DataTypeUInt32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt32>::execute();
if (typeid_cast<const DataTypeUInt64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt64>::execute();
if (typeid_cast<const DataTypeInt8 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int8>::execute();
if (typeid_cast<const DataTypeInt16 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int16>::execute();
if (typeid_cast<const DataTypeInt32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int32>::execute();
if (typeid_cast<const DataTypeInt64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int64>::execute();
if (typeid_cast<const DataTypeFloat32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Float32>::execute();
if (typeid_cast<const DataTypeFloat64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Float64>::execute();
if (typeid_cast<const DataTypeUInt8 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt8>::execute();
if (typeid_cast<const DataTypeUInt16 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt16>::execute();
if (typeid_cast<const DataTypeUInt32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt32>::execute();
if (typeid_cast<const DataTypeUInt64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, UInt64>::execute();
if (typeid_cast<const DataTypeInt8 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int8>::execute();
if (typeid_cast<const DataTypeInt16 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int16>::execute();
if (typeid_cast<const DataTypeInt32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int32>::execute();
if (typeid_cast<const DataTypeInt64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Int64>::execute();
if (typeid_cast<const DataTypeFloat32 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Float32>::execute();
if (typeid_cast<const DataTypeFloat64 *>(&type2)) return TypeProcessorImpl<TLeft, TRight, Float64>::execute();
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
@ -51,17 +51,17 @@ struct LeftTypeProcessor
{
static DataTypeTraits::EnrichedDataTypePtr execute(const DataTypePtr & right, const IDataType & type2)
{
if (typeid_cast<const DataTypeVoid *>(&*right)) return RightTypeProcessor<TLeft, void>::execute(type2);
if (typeid_cast<const DataTypeUInt8 *>(&*right)) return RightTypeProcessor<TLeft, UInt8>::execute(type2);
if (typeid_cast<const DataTypeUInt16 *>(&*right)) return RightTypeProcessor<TLeft, UInt16>::execute(type2);
if (typeid_cast<const DataTypeUInt32 *>(&*right)) return RightTypeProcessor<TLeft, UInt32>::execute(type2);
if (typeid_cast<const DataTypeUInt64 *>(&*right)) return RightTypeProcessor<TLeft, UInt64>::execute(type2);
if (typeid_cast<const DataTypeInt8 *>(&*right)) return RightTypeProcessor<TLeft, Int8>::execute(type2);
if (typeid_cast<const DataTypeInt16 *>(&*right)) return RightTypeProcessor<TLeft, Int16>::execute(type2);
if (typeid_cast<const DataTypeInt32 *>(&*right)) return RightTypeProcessor<TLeft, Int32>::execute(type2);
if (typeid_cast<const DataTypeInt64 *>(&*right)) return RightTypeProcessor<TLeft, Int64>::execute(type2);
if (typeid_cast<const DataTypeFloat32 *>(&*right)) return RightTypeProcessor<TLeft, Float32>::execute(type2);
if (typeid_cast<const DataTypeFloat64 *>(&*right)) return RightTypeProcessor<TLeft, Float64>::execute(type2);
if (typeid_cast<const DataTypeVoid *>(&*right)) return RightTypeProcessor<TLeft, void>::execute(type2);
if (typeid_cast<const DataTypeUInt8 *>(&*right)) return RightTypeProcessor<TLeft, UInt8>::execute(type2);
if (typeid_cast<const DataTypeUInt16 *>(&*right)) return RightTypeProcessor<TLeft, UInt16>::execute(type2);
if (typeid_cast<const DataTypeUInt32 *>(&*right)) return RightTypeProcessor<TLeft, UInt32>::execute(type2);
if (typeid_cast<const DataTypeUInt64 *>(&*right)) return RightTypeProcessor<TLeft, UInt64>::execute(type2);
if (typeid_cast<const DataTypeInt8 *>(&*right)) return RightTypeProcessor<TLeft, Int8>::execute(type2);
if (typeid_cast<const DataTypeInt16 *>(&*right)) return RightTypeProcessor<TLeft, Int16>::execute(type2);
if (typeid_cast<const DataTypeInt32 *>(&*right)) return RightTypeProcessor<TLeft, Int32>::execute(type2);
if (typeid_cast<const DataTypeInt64 *>(&*right)) return RightTypeProcessor<TLeft, Int64>::execute(type2);
if (typeid_cast<const DataTypeFloat32 *>(&*right)) return RightTypeProcessor<TLeft, Float32>::execute(type2);
if (typeid_cast<const DataTypeFloat64 *>(&*right)) return RightTypeProcessor<TLeft, Float64>::execute(type2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}
@ -74,16 +74,16 @@ DataTypeTraits::EnrichedDataTypePtr getSmallestCommonNumericType(const DataTypeT
const DataTypePtr & left = type1.first;
const DataTypePtr & right = type1.second;
if (typeid_cast<const DataTypeUInt8 *>(&*left)) return LeftTypeProcessor<UInt8>::execute(right, type2);
if (typeid_cast<const DataTypeUInt16 *>(&*left)) return LeftTypeProcessor<UInt16>::execute(right, type2);
if (typeid_cast<const DataTypeUInt32 *>(&*left)) return LeftTypeProcessor<UInt32>::execute(right, type2);
if (typeid_cast<const DataTypeUInt64 *>(&*left)) return LeftTypeProcessor<UInt64>::execute(right, type2);
if (typeid_cast<const DataTypeInt8 *>(&*left)) return LeftTypeProcessor<Int8>::execute(right, type2);
if (typeid_cast<const DataTypeInt16 *>(&*left)) return LeftTypeProcessor<Int16>::execute(right, type2);
if (typeid_cast<const DataTypeInt32 *>(&*left)) return LeftTypeProcessor<Int32>::execute(right, type2);
if (typeid_cast<const DataTypeInt64 *>(&*left)) return LeftTypeProcessor<Int64>::execute(right, type2);
if (typeid_cast<const DataTypeFloat32 *>(&*left)) return LeftTypeProcessor<Float32>::execute(right, type2);
if (typeid_cast<const DataTypeFloat64 *>(&*left)) return LeftTypeProcessor<Float64>::execute(right, type2);
if (typeid_cast<const DataTypeUInt8 *>(&*left)) return LeftTypeProcessor<UInt8>::execute(right, type2);
if (typeid_cast<const DataTypeUInt16 *>(&*left)) return LeftTypeProcessor<UInt16>::execute(right, type2);
if (typeid_cast<const DataTypeUInt32 *>(&*left)) return LeftTypeProcessor<UInt32>::execute(right, type2);
if (typeid_cast<const DataTypeUInt64 *>(&*left)) return LeftTypeProcessor<UInt64>::execute(right, type2);
if (typeid_cast<const DataTypeInt8 *>(&*left)) return LeftTypeProcessor<Int8>::execute(right, type2);
if (typeid_cast<const DataTypeInt16 *>(&*left)) return LeftTypeProcessor<Int16>::execute(right, type2);
if (typeid_cast<const DataTypeInt32 *>(&*left)) return LeftTypeProcessor<Int32>::execute(right, type2);
if (typeid_cast<const DataTypeInt64 *>(&*left)) return LeftTypeProcessor<Int64>::execute(right, type2);
if (typeid_cast<const DataTypeFloat32 *>(&*left)) return LeftTypeProcessor<Float32>::execute(right, type2);
if (typeid_cast<const DataTypeFloat64 *>(&*left)) return LeftTypeProcessor<Float64>::execute(right, type2);
throw Exception("Logical error: not a numeric type passed to function getSmallestCommonNumericType", ErrorCodes::LOGICAL_ERROR);
}

View File

@ -2,7 +2,6 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnString.h>
#include <Columns/ColumnConst.h>
#include <Common/StringUtils.h>
#include <Common/StringView.h>
#include <Common/typeid_cast.h>

View File

@ -9,7 +9,6 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnConst.h>
#include <Common/hex.h>
#include <Common/Volnitsky.h>
#include <Functions/IFunction.h>

View File

@ -1,5 +1,4 @@
#include <Functions/IFunction.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <DataTypes/DataTypeNull.h>
#include <DataTypes/DataTypeNullable.h>

View File

@ -2,6 +2,7 @@
#include <Columns/ColumnsNumber.h>
#include <Functions/IFunction.h>
#include <Common/Stopwatch.h>
#include <Common/typeid_cast.h>
#include <IO/WriteHelpers.h>
#include <iomanip>

View File

@ -11,7 +11,6 @@
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSet.h>
#include <Parsers/ASTOrderByElement.h>
#include <DataTypes/DataTypeSet.h>
@ -192,7 +191,7 @@ void ExpressionAnalyzer::init()
/// Executing scalar subqueries - replacing them with constant values.
executeScalarSubqueries();
/// Optimize if with constant condition after constats are substituted instead of sclalar subqueries
/// Optimize if with constant condition after constants was substituted instead of sclalar subqueries.
optimizeIfWithConstantCondition();
/// GROUP BY injective function elimination.
@ -624,8 +623,118 @@ void ExpressionAnalyzer::findExternalTables(ASTPtr & ast)
}
static std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier)
{
std::pair<String, String> res;
res.second = identifier.name;
if (!identifier.children.empty())
{
if (identifier.children.size() != 2)
throw Exception("Qualified table name could have only two components", ErrorCodes::LOGICAL_ERROR);
res.first = typeid_cast<const ASTIdentifier &>(*identifier.children[0]).name;
res.second = typeid_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
return res;
}
static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
ASTPtr & subquery_or_table_name, const Context & context, size_t subquery_depth, const Names & required_columns);
const ASTPtr & subquery_or_table_name, const Context & context, size_t subquery_depth, const Names & required_columns)
{
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
const ASTSubquery * subquery = typeid_cast<const ASTSubquery *>(subquery_or_table_name.get());
const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(subquery_or_table_name.get());
if (!subquery && !table)
throw Exception("IN/JOIN supports only SELECT subqueries.", ErrorCodes::BAD_ARGUMENTS);
/** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result.
* Because the result of this query is not the result of the entire query.
* Constraints work instead
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
* max_rows_in_join, max_bytes_in_join, join_overflow_mode,
* which are checked separately (in the Set, Join objects).
*/
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.limits.max_result_rows = 0;
subquery_settings.limits.max_result_bytes = 0;
/// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query).
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
ASTPtr query;
if (table)
{
/// create ASTSelectQuery for "SELECT * FROM table" as if written by hand
const auto select_query = std::make_shared<ASTSelectQuery>();
query = select_query;
const auto select_expression_list = std::make_shared<ASTExpressionList>();
select_query->select_expression_list = select_expression_list;
select_query->children.emplace_back(select_query->select_expression_list);
/// get columns list for target table
auto database_table = getDatabaseAndTableNameFromIdentifier(*table);
const auto & storage = context.getTable(database_table.first, database_table.second);
const auto & columns = storage->getColumnsListNonMaterialized();
select_expression_list->children.reserve(columns.size());
/// manually substitute column names in place of asterisk
for (const auto & column : columns)
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(
StringRange{}, column.name));
select_query->replaceDatabaseAndTable(database_table.first, database_table.second);
}
else
{
query = subquery->children.at(0);
/** Columns with the same name can be specified in a subquery. For example, SELECT x, x FROM t
* This is bad, because the result of such a query can not be saved to the table, because the table can not have the same name columns.
* Saving to the table is required for GLOBAL subqueries.
*
* To avoid this situation, we will rename the same columns.
*/
std::set<std::string> all_column_names;
std::set<std::string> assigned_column_names;
if (ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(query.get()))
{
for (auto & expr : select->select_expression_list->children)
all_column_names.insert(expr->getAliasOrColumnName());
for (auto & expr : select->select_expression_list->children)
{
auto name = expr->getAliasOrColumnName();
if (!assigned_column_names.insert(name).second)
{
size_t i = 1;
while (all_column_names.end() != all_column_names.find(name + "_" + toString(i)))
++i;
name = name + "_" + toString(i);
expr = expr->clone(); /// Cancels fuse of the same expressions in the tree.
expr->setAlias(name);
all_column_names.insert(name);
assigned_column_names.insert(name);
}
}
}
}
if (required_columns.empty())
return std::make_shared<InterpreterSelectQuery>(
query, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1);
else
return std::make_shared<InterpreterSelectQuery>(
query, subquery_context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1);
}
void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_table_expression)
@ -1381,18 +1490,19 @@ void ExpressionAnalyzer::makeSetsForIndex()
makeSetsForIndexImpl(ast, storage->getSampleBlock());
}
void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sample_block)
void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block)
{
for (auto & child : node->children)
makeSetsForIndexImpl(child, sample_block);
ASTFunction * func = typeid_cast<ASTFunction *>(node.get());
const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get());
if (func && func->kind == ASTFunction::FUNCTION && functionIsInOperator(func->name))
{
IAST & args = *func->arguments;
ASTPtr & arg = args.children.at(1);
const IAST & args = *func->arguments;
const ASTPtr & arg = args.children.at(1);
if (!typeid_cast<ASTSet *>(arg.get()) && !typeid_cast<ASTSubquery *>(arg.get()) && !typeid_cast<ASTIdentifier *>(arg.get()))
if (!prepared_sets.count(arg.get()) /// Not already prepared.
&& !typeid_cast<ASTSubquery *>(arg.get()) && !typeid_cast<ASTIdentifier *>(arg.get()))
{
try
{
@ -1409,141 +1519,25 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(ASTPtr & node, const Block & sampl
}
static std::pair<String, String> getDatabaseAndTableNameFromIdentifier(const ASTIdentifier & identifier)
{
std::pair<String, String> res;
res.second = identifier.name;
if (!identifier.children.empty())
{
if (identifier.children.size() != 2)
throw Exception("Qualified table name could have only two components", ErrorCodes::LOGICAL_ERROR);
res.first = typeid_cast<const ASTIdentifier &>(*identifier.children[0]).name;
res.second = typeid_cast<const ASTIdentifier &>(*identifier.children[1]).name;
}
return res;
}
static std::shared_ptr<InterpreterSelectQuery> interpretSubquery(
ASTPtr & subquery_or_table_name, const Context & context, size_t subquery_depth, const Names & required_columns)
{
/// Subquery or table name. The name of the table is similar to the subquery `SELECT * FROM t`.
const ASTSubquery * subquery = typeid_cast<const ASTSubquery *>(subquery_or_table_name.get());
const ASTIdentifier * table = typeid_cast<const ASTIdentifier *>(subquery_or_table_name.get());
if (!subquery && !table)
throw Exception("IN/JOIN supports only SELECT subqueries.", ErrorCodes::BAD_ARGUMENTS);
/** The subquery in the IN / JOIN section does not have any restrictions on the maximum size of the result.
* Because the result of this query is not the result of the entire query.
* Constraints work instead
* max_rows_in_set, max_bytes_in_set, set_overflow_mode,
* max_rows_in_join, max_bytes_in_join, join_overflow_mode,
* which are checked separately (in the Set, Join objects).
*/
Context subquery_context = context;
Settings subquery_settings = context.getSettings();
subquery_settings.limits.max_result_rows = 0;
subquery_settings.limits.max_result_bytes = 0;
/// The calculation of `extremes` does not make sense and is not necessary (if you do it, then the `extremes` of the subquery can be taken instead of the whole query).
subquery_settings.extremes = 0;
subquery_context.setSettings(subquery_settings);
ASTPtr query;
if (table)
{
/// create ASTSelectQuery for "SELECT * FROM table" as if written by hand
const auto select_query = std::make_shared<ASTSelectQuery>();
query = select_query;
const auto select_expression_list = std::make_shared<ASTExpressionList>();
select_query->select_expression_list = select_expression_list;
select_query->children.emplace_back(select_query->select_expression_list);
/// get columns list for target table
auto database_table = getDatabaseAndTableNameFromIdentifier(*table);
const auto & storage = context.getTable(database_table.first, database_table.second);
const auto & columns = storage->getColumnsListNonMaterialized();
select_expression_list->children.reserve(columns.size());
/// manually substitute column names in place of asterisk
for (const auto & column : columns)
select_expression_list->children.emplace_back(std::make_shared<ASTIdentifier>(
StringRange{}, column.name));
select_query->replaceDatabaseAndTable(database_table.first, database_table.second);
}
else
{
query = subquery->children.at(0);
/** Columns with the same name can be specified in a subquery. For example, SELECT x, x FROM t
* This is bad, because the result of such a query can not be saved to the table, because the table can not have the same name columns.
* Saving to the table is required for GLOBAL subqueries.
*
* To avoid this situation, we will rename the same columns.
*/
std::set<std::string> all_column_names;
std::set<std::string> assigned_column_names;
if (ASTSelectQuery * select = typeid_cast<ASTSelectQuery *>(query.get()))
{
for (auto & expr : select->select_expression_list->children)
all_column_names.insert(expr->getAliasOrColumnName());
for (auto & expr : select->select_expression_list->children)
{
auto name = expr->getAliasOrColumnName();
if (!assigned_column_names.insert(name).second)
{
size_t i = 1;
while (all_column_names.end() != all_column_names.find(name + "_" + toString(i)))
++i;
name = name + "_" + toString(i);
expr = expr->clone(); /// Cancels fuse of the same expressions in the tree.
expr->setAlias(name);
all_column_names.insert(name);
assigned_column_names.insert(name);
}
}
}
}
if (required_columns.empty())
return std::make_shared<InterpreterSelectQuery>(
query, subquery_context, QueryProcessingStage::Complete, subquery_depth + 1);
else
return std::make_shared<InterpreterSelectQuery>(
query, subquery_context, required_columns, QueryProcessingStage::Complete, subquery_depth + 1);
}
void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_block)
{
/** You need to convert the right argument to a set.
* This can be a table name, a value, a value enumeration, or a subquery.
* The enumeration of values is parsed as a function `tuple`.
*/
IAST & args = *node->arguments;
ASTPtr & arg = args.children.at(1);
const IAST & args = *node->arguments;
const ASTPtr & arg = args.children.at(1);
/// Already converted.
if (typeid_cast<ASTSet *>(arg.get()))
if (prepared_sets.count(arg.get()))
return;
/// If the subquery or table name for SELECT.
ASTIdentifier * identifier = typeid_cast<ASTIdentifier *>(arg.get());
if (typeid_cast<ASTSubquery *>(arg.get()) || identifier)
const ASTIdentifier * identifier = typeid_cast<const ASTIdentifier *>(arg.get());
if (typeid_cast<const ASTSubquery *>(arg.get()) || identifier)
{
/// We get the stream of blocks for the subquery. Create Set and put it in place of the subquery.
String set_id = arg->getColumnName();
auto ast_set = std::make_shared<ASTSet>(set_id);
ASTPtr ast_set_ptr = ast_set;
/// A special case is if the name of the table is specified on the right side of the IN statement,
/// and the table has the type Set (a previously prepared set).
@ -1558,9 +1552,7 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
if (storage_set)
{
SetPtr & set = storage_set->getSet();
ast_set->set = set;
arg = ast_set_ptr;
prepared_sets[arg.get()] = storage_set->getSet();
return;
}
}
@ -1571,12 +1563,11 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
/// If you already created a Set with the same subquery / table.
if (subquery_for_set.set)
{
ast_set->set = subquery_for_set.set;
arg = ast_set_ptr;
prepared_sets[arg.get()] = subquery_for_set.set;
return;
}
ast_set->set = std::make_shared<Set>(settings.limits);
SetPtr set = std::make_shared<Set>(settings.limits);
/** The following happens for GLOBAL INs:
* - in the addExternalStorage function, the IN (SELECT ...) subquery is replaced with IN _data1,
@ -1618,8 +1609,8 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
*/
}
subquery_for_set.set = ast_set->set;
arg = ast_set_ptr;
subquery_for_set.set = set;
prepared_sets[arg.get()] = set;
}
else
{
@ -1629,23 +1620,23 @@ void ExpressionAnalyzer::makeSet(ASTFunction * node, const Block & sample_block)
}
/// The case of an explicit enumeration of values.
void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sample_block, bool create_ordered_set)
void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set)
{
IAST & args = *node->arguments;
const IAST & args = *node->arguments;
if (args.children.size() != 2)
throw Exception("Wrong number of arguments passed to function in", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTPtr & arg = args.children.at(1);
const ASTPtr & arg = args.children.at(1);
DataTypes set_element_types;
ASTPtr & left_arg = args.children.at(0);
const ASTPtr & left_arg = args.children.at(0);
ASTFunction * left_arg_tuple = typeid_cast<ASTFunction *>(left_arg.get());
const ASTFunction * left_arg_tuple = typeid_cast<const ASTFunction *>(left_arg.get());
/** NOTE If tuple in left hand side specified non-explicitly
* Example: identity((a, b)) IN ((1, 2), (3, 4))
* instead of (a, b)) IN ((1, 2), (3, 4))
* instead of (a, b)) IN ((1, 2), (3, 4))
* then set creation of set doesn't work correctly.
*/
if (left_arg_tuple && left_arg_tuple->name == "tuple")
@ -1654,7 +1645,7 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl
{
const auto & data_type = sample_block.getByName(arg->getColumnName()).type;
/// @note prevent crash in query: SELECT (1, [1]) in (1, 1)
/// NOTE prevent crash in query: SELECT (1, [1]) in (1, 1)
if (const auto array = typeid_cast<const DataTypeArray * >(data_type.get()))
throw Exception("Incorrect element of tuple: " + array->getName(), ErrorCodes::INCORRECT_ELEMENT_OF_SET);
@ -1720,11 +1711,9 @@ void ExpressionAnalyzer::makeExplicitSet(ASTFunction * node, const Block & sampl
elements_ast = exp_list;
}
auto ast_set = std::make_shared<ASTSet>(arg->getColumnName());
ast_set->set = std::make_shared<Set>(settings.limits);
ast_set->is_explicit = true;
ast_set->set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
arg = ast_set;
SetPtr set = std::make_shared<Set>(settings.limits);
set->createFromAST(set_element_types, elements_ast, context, create_ordered_set);
prepared_sets[arg.get()] = std::move(set);
}
@ -2057,7 +2046,6 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
for (auto & child : node->arguments->children)
{
ASTFunction * lambda = typeid_cast<ASTFunction *>(child.get());
ASTSet * set = typeid_cast<ASTSet *>(child.get());
if (lambda && lambda->name == "lambda")
{
/// If the argument is a lambda expression, just remember its approximate type.
@ -2074,21 +2062,23 @@ void ExpressionAnalyzer::getActionsImpl(ASTPtr ast, bool no_subqueries, bool onl
/// Select the name in the next cycle.
argument_names.emplace_back();
}
else if (set)
else if (prepared_sets.count(child.get()))
{
ColumnWithTypeAndName column;
column.type = std::make_shared<DataTypeSet>();
/// If the argument is a set given by an enumeration of values, give it a unique name,
const SetPtr & set = prepared_sets[child.get()];
/// If the argument is a set given by an enumeration of values (so, the set was already built), give it a unique name,
/// so that sets with the same record do not fuse together (they can have different types).
if (set->is_explicit)
if (!set->empty())
column.name = getUniqueName(actions_stack.getSampleBlock(), "__set");
else
column.name = set->getColumnName();
column.name = child->getColumnName();
if (!actions_stack.getSampleBlock().has(column.name))
{
column.column = std::make_shared<ColumnSet>(1, set->set);
column.column = std::make_shared<ColumnSet>(1, set);
actions_stack.addAction(ExpressionAction::addColumn(column));
}

View File

@ -13,15 +13,16 @@ class Context;
class ExpressionActions;
struct ExpressionActionsChain;
class Set;
using SetPtr = std::shared_ptr<Set>;
class Join;
using JoinPtr = std::shared_ptr<Join>;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class Set;
using SetPtr = std::shared_ptr<Set>;
using PreparedSets = std::unordered_map<IAST*, SetPtr>;
class IBlockInputStream;
using BlockInputStreamPtr = std::shared_ptr<IBlockInputStream>;
@ -42,10 +43,8 @@ struct SubqueryForSet
BlockInputStreamPtr source;
Block source_sample;
/// If set, create from Set result.
/// If set, build it from result.
SetPtr set;
/// If set, create from Join result.
JoinPtr join;
/// If set, put the result into the table.
@ -130,6 +129,8 @@ public:
*/
SubqueriesForSets getSubqueriesForSets() { return subqueries_for_sets; }
PreparedSets getPreparedSets() { return prepared_sets; }
/** Tables that will need to be sent to remote servers for distributed query processing.
*/
const Tables & getExternalTables() const { return external_tables; }
@ -167,6 +168,8 @@ private:
SubqueriesForSets subqueries_for_sets;
PreparedSets prepared_sets;
/// NOTE: So far, only one JOIN per query is supported.
/** Query of the form `SELECT expr(x) AS FROM t1 ANY LEFT JOIN (SELECT expr(x) AS k FROM t2) USING k`
@ -244,8 +247,7 @@ private:
void optimizeIfWithConstantConditionImpl(ASTPtr & current_ast, Aliases & aliases) const;
bool tryExtractConstValueFromCondition(const ASTPtr & condition, bool & value) const;
/// Transform the value enumeration or subquery into ASTSet. `node` - `in` or `notIn` function.
void makeSet(ASTFunction * node, const Block & sample_block);
void makeSet(const ASTFunction * node, const Block & sample_block);
/// Adds a list of ALIAS columns from the table
void addAliasColumns();
@ -309,8 +311,8 @@ private:
/** Create Set from an explicit enumeration of values in the query.
* If create_ordered_set = true - create a data structure suitable for using the index.
*/
void makeExplicitSet(ASTFunction * node, const Block & sample_block, bool create_ordered_set);
void makeSetsForIndexImpl(ASTPtr & node, const Block & sample_block);
void makeExplicitSet(const ASTFunction * node, const Block & sample_block, bool create_ordered_set);
void makeSetsForIndexImpl(const ASTPtr & node, const Block & sample_block);
/** Translate qualified names such as db.table.column, table.column, table_alias.column
* to unqualified names. This is done in a poor transitional way:

View File

@ -30,6 +30,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int ARGUMENT_OUT_OF_BOUND;
extern const int BAD_ARGUMENTS;
extern const int ILLEGAL_COLUMN;
}
@ -53,6 +54,7 @@ BlockIO InterpreterAlterQuery::execute()
PartitionCommands partition_commands;
parseAlter(alter.parameters, alter_commands, partition_commands);
partition_commands.validate(table.get());
for (const PartitionCommand & command : partition_commands)
{
switch (command.type)
@ -89,7 +91,6 @@ BlockIO InterpreterAlterQuery::execute()
return {};
alter_commands.validate(table.get(), context);
table->alter(alter_commands, database_name, table_name, context);
return {};
@ -230,4 +231,23 @@ void InterpreterAlterQuery::parseAlter(
}
}
void InterpreterAlterQuery::PartitionCommands::validate(const IStorage * table)
{
for (const PartitionCommand & command : *this)
{
if (command.type == PartitionCommand::CLEAR_COLUMN)
{
String column_name = command.column_name.safeGet<String>();
if (!table->hasRealColumn(column_name))
{
throw Exception("Wrong column name. Cannot find column " + column_name + " to clear it from partition",
DB::ErrorCodes::ILLEGAL_COLUMN);
}
}
}
}
}

View File

@ -112,7 +112,11 @@ private:
}
};
using PartitionCommands = std::vector<PartitionCommand>;
class PartitionCommands : public std::vector<PartitionCommand>
{
public:
void validate(const IStorage * table);
};
ASTPtr query_ptr;

View File

@ -178,17 +178,13 @@ static ColumnsAndDefaults parseColumns(
ASTPtr default_expr_list = std::make_shared<ASTExpressionList>();
default_expr_list->children.reserve(column_list_ast.children.size());
const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
for (auto & ast : column_list_ast.children)
{
auto & col_decl = typeid_cast<ASTColumnDeclaration &>(*ast);
if (col_decl.type)
{
const auto & type_range = col_decl.type->range;
columns.emplace_back(col_decl.name,
data_type_factory.get({ type_range.first, type_range.second }));
columns.emplace_back(col_decl.name, DataTypeFactory::instance().get(col_decl.type));
}
else
/// we're creating dummy DataTypeUInt8 in order to prevent the NullPointerException in ExpressionActions

View File

@ -836,13 +836,17 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
if (max_streams > 1 && !is_remote)
max_streams *= settings.max_streams_to_max_threads_ratio;
SelectQueryInfo query_info;
query_info.query = query_ptr;
query_info.sets = query_analyzer->getPreparedSets();
/// PREWHERE optimization
{
auto optimize_prewhere = [&](auto & merge_tree)
{
/// Try transferring some condition from WHERE to PREWHERE if enabled and viable
if (settings.optimize_move_to_prewhere && query.where_expression && !query.prewhere_expression && !query.final())
MergeTreeWhereOptimizer{query_ptr, context, merge_tree.getData(), required_columns, log};
MergeTreeWhereOptimizer{query_info, context, merge_tree.getData(), required_columns, log};
};
if (const StorageMergeTree * merge_tree = typeid_cast<const StorageMergeTree *>(storage.get()))
@ -851,7 +855,7 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns()
optimize_prewhere(*merge_tree);
}
streams = storage->read(required_columns, query_ptr, context, from_stage, max_block_size, max_streams);
streams = storage->read(required_columns, query_info, context, from_stage, max_block_size, max_streams);
if (alias_actions)
{

View File

@ -2,7 +2,6 @@
#include <Poco/RWLock.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnNullable.h>
#include <DataStreams/IBlockInputStream.h>
#include <Interpreters/Limits.h>

View File

@ -10,6 +10,7 @@
#include <Storages/IStorage.h>
#include <Databases/IDatabase.h>
#include <Databases/DatabaseOrdinary.h>
#include <Common/typeid_cast.h>
#include <iostream>
#include <vector>

View File

@ -1,40 +0,0 @@
#pragma once
#include <Parsers/IAST.h>
namespace DB
{
class Set;
/** The set. During the calculation, the expression in the IN section is replaced by the set
* - a subquery or an explicit enumeration of values.
* TODO: This is bad practice, lead to bugs. Remove it.
*/
class ASTSet : public IAST
{
public:
std::shared_ptr<Set> set;
String column_name;
bool is_explicit = false;
ASTSet(const String & column_name_) : column_name(column_name_) {}
ASTSet(const StringRange range_, const String & column_name_) : IAST(range_), column_name(column_name_) {}
String getID() const override { return "Set_" + getColumnName(); }
ASTPtr clone() const override { return std::make_shared<ASTSet>(*this); }
String getColumnName() const override { return column_name; }
protected:
void formatImpl(const FormatSettings & settings, FormatState & state, FormatStateStacked frame) const override
{
/** Prepared set. In user requests, this does not happen, but this happens after the intermediate query transformation.
* Output it for not real (this will not be a valid query, but it will show that there was a set).
*/
settings.ostr << (settings.hilite ? hilite_keyword : "")
<< "(...)"
<< (settings.hilite ? hilite_none : "");
}
};
}

View File

@ -1,35 +0,0 @@
#include <Parsers/ParserEnumElement.h>
#include <Parsers/ASTEnumElement.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
namespace DB
{
bool ParserEnumElement::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ParserStringLiteral name_parser;
ParserNumber value_parser;
ParserToken equality_sign_parser(TokenType::Equals);
const auto begin = pos;
ASTPtr name;
if (!name_parser.parse(pos, name, expected))
return false;
if (!equality_sign_parser.ignore(pos, expected))
return false;
ASTPtr value;
if (!value_parser.parse(pos, value, expected))
return false;
node = std::make_shared<ASTEnumElement>(
StringRange{begin, pos}, static_cast<const ASTLiteral &>(*name).value.get<String>(), static_cast<const ASTLiteral &>(*value).value);
return true;
}
}

View File

@ -1,16 +0,0 @@
#pragma once
#include <Parsers/IParserBase.h>
namespace DB
{
class ParserEnumElement : public IParserBase
{
protected:
const char * getName() const override { return "Enum element"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
}

View File

@ -2,19 +2,28 @@
#include <stdlib.h>
#include <fcntl.h>
#include <signal.h>
#include <iostream>
#include <fstream>
#include <iomanip>
#include <experimental/optional>
#include <unordered_set>
#include <algorithm>
#include <experimental/optional>
#include <boost/program_options.hpp>
#include <Poco/File.h>
#include <Poco/Util/Application.h>
#include <Common/ClickHouseRevision.h>
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
#include <Common/ShellCommand.h>
#include <Common/ExternalTable.h>
#include <Common/UnicodeBar.h>
#include <Common/formatReadable.h>
#include <Common/NetException.h>
#include <common/readline_use.h>
#include <Common/typeid_cast.h>
#include <Core/Types.h>
#include <Core/QueryProcessingStage.h>
#include <IO/ReadBufferFromFileDescriptor.h>
@ -25,7 +34,6 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <DataStreams/AsynchronousBlockInputStream.h>
#include <DataStreams/TabSeparatedRowInputStream.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Parsers/ASTUseQuery.h>
@ -39,13 +47,6 @@
#include <Interpreters/Context.h>
#include <Client/Connection.h>
#include "InterruptListener.h"
#include <Common/ExternalTable.h>
#include <Common/UnicodeBar.h>
#include <Common/formatReadable.h>
#include <Columns/ColumnString.h>
#include <Common/NetException.h>
#include <common/readline_use.h>
#include <Common/typeid_cast.h>
#include <Functions/registerFunctions.h>
#include <AggregateFunctions/registerAggregateFunctions.h>

View File

@ -4,6 +4,7 @@
#include <Common/Exception.h>
#include <Core/QueryProcessingStage.h>
#include <Storages/ITableDeclaration.h>
#include <Storages/SelectQueryInfo.h>
#include <Poco/RWLock.h>
#include <memory>
#include <experimental/optional>
@ -30,10 +31,6 @@ class IStorage;
using StoragePtr = std::shared_ptr<IStorage>;
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
struct Settings;
class AlterCommands;
@ -156,7 +153,7 @@ public:
* (indexes, locks, etc.)
* Returns a stream with which you can read data sequentially
* or multiple streams for parallel data reading.
* The into `processed_stage` info is also written to what stage the request was processed.
* The `processed_stage` info is also written to what stage the request was processed.
* (Normally, the function only reads the columns from the list, but in other cases,
* for example, the request can be partially processed on a remote server.)
*
@ -171,7 +168,7 @@ public:
*/
virtual BlockInputStreams read(
const Names & column_names,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,

View File

@ -129,7 +129,7 @@ static RelativeSize convertAbsoluteSampleSizeToRelative(const ASTPtr & node, siz
BlockInputStreams MergeTreeDataSelectExecutor::read(
const Names & column_names_to_return,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
@ -188,7 +188,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
/// If `_part` virtual column is requested, we try to use it as an index.
Block virtual_columns_block = getBlockWithPartColumn(parts);
if (part_column_queried)
VirtualColumnUtils::filterBlockWithQuery(query, virtual_columns_block, context);
VirtualColumnUtils::filterBlockWithQuery(query_info.query, virtual_columns_block, context);
std::multiset<String> part_values = VirtualColumnUtils::extractSingleValueFromBlock<String>(virtual_columns_block, "_part");
@ -199,9 +199,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
SortDescription sort_descr = data.getSortDescription();
ColumnsWithTypeAndName date_columns = {{DataTypeDate{}.createColumn(), std::make_shared<DataTypeDate>(), data.date_column_name}};
PKCondition key_condition(query, context, available_real_and_virtual_columns, sort_descr,
PKCondition key_condition(query_info, context, available_real_and_virtual_columns, sort_descr,
data.getPrimaryExpression());
PKCondition date_condition(query, context, available_real_and_virtual_columns,
PKCondition date_condition(query_info, context, available_real_and_virtual_columns,
SortDescription(1, SortColumnDescription(data.date_column_name, 1, 1)),
std::make_shared<ExpressionActions>(date_columns, settings));
@ -254,7 +254,7 @@ BlockInputStreams MergeTreeDataSelectExecutor::read(
RelativeSize relative_sample_size = 0;
RelativeSize relative_sample_offset = 0;
ASTSelectQuery & select = *typeid_cast<ASTSelectQuery*>(&*query);
ASTSelectQuery & select = typeid_cast<ASTSelectQuery &>(*query_info.query);
auto select_sample_size = select.sample_size();
auto select_sample_offset = select.sample_offset();

View File

@ -22,7 +22,7 @@ public:
*/
BlockInputStreams read(
const Names & column_names,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,

View File

@ -7,7 +7,6 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSubquery.h>
#include <Parsers/ASTSet.h>
#include <Parsers/formatAST.h>
#include <Common/escapeForFileName.h>
#include <Common/typeid_cast.h>
@ -35,18 +34,21 @@ static constexpr auto global_not_in_function_name = "globalNotIn";
MergeTreeWhereOptimizer::MergeTreeWhereOptimizer(
ASTPtr & query, const Context & context, const MergeTreeData & data, const Names & column_names,
SelectQueryInfo & query_info,
const Context & context,
const MergeTreeData & data,
const Names & column_names,
Logger * log)
: primary_key_columns{ext::map<std::unordered_set>(data.getSortDescription(),
[] (const SortColumnDescription & col) { return col.column_name; })
},
: primary_key_columns{ext::map<std::unordered_set>(data.getSortDescription(),
[] (const SortColumnDescription & col) { return col.column_name; })},
table_columns{ext::map<std::unordered_set>(data.getColumnsList(),
[] (const NameAndTypePair & col) { return col.name; })
}, block_with_constants{PKCondition::getBlockWithConstants(query, context, data.getColumnsList())},
[] (const NameAndTypePair & col) { return col.name; })},
block_with_constants{PKCondition::getBlockWithConstants(query_info.query, context, data.getColumnsList())},
prepared_sets(query_info.sets),
log{log}
{
calculateColumnSizes(data, column_names);
auto & select = typeid_cast<ASTSelectQuery &>(*query);
auto & select = typeid_cast<ASTSelectQuery &>(*query_info.query);
determineArrayJoinedNames(select);
optimize(select);
}
@ -330,7 +332,7 @@ bool MergeTreeWhereOptimizer::isPrimaryKeyAtom(const IAST * const ast) const
if ((primary_key_columns.count(first_arg_name) && isConstant(args[1])) ||
(primary_key_columns.count(second_arg_name) && isConstant(args[0])) ||
(primary_key_columns.count(first_arg_name)
&& (typeid_cast<const ASTSet *>(args[1].get()) || typeid_cast<const ASTSubquery *>(args[1].get()))))
&& (prepared_sets.count(args[1].get()) || typeid_cast<const ASTSubquery *>(args[1].get()))))
return true;
}

View File

@ -5,6 +5,7 @@
#include <set>
#include <boost/noncopyable.hpp>
#include <Core/Block.h>
#include <Storages/SelectQueryInfo.h>
namespace Poco { class Logger; }
@ -12,9 +13,6 @@ namespace Poco { class Logger; }
namespace DB
{
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class ASTSelectQuery;
class ASTFunction;
class MergeTreeData;
@ -35,7 +33,10 @@ class MergeTreeWhereOptimizer : private boost::noncopyable
{
public:
MergeTreeWhereOptimizer(
ASTPtr & query, const Context & context, const MergeTreeData & data, const Names & column_names,
SelectQueryInfo & query_info,
const Context & context,
const MergeTreeData & data,
const Names & column_names,
Poco::Logger * log);
private:
@ -76,6 +77,7 @@ private:
const string_set_t primary_key_columns;
const string_set_t table_columns;
const Block block_with_constants;
const PreparedSets & prepared_sets;
Poco::Logger * log;
std::unordered_map<std::string, std::size_t> column_sizes{};
std::size_t total_column_size{};

View File

@ -9,7 +9,6 @@
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnSet.h>
#include <Columns/ColumnTuple.h>
#include <Parsers/ASTSet.h>
#include <Functions/FunctionFactory.h>
#include <Functions/IFunction.h>
#include <Core/FieldVisitors.h>
@ -219,9 +218,13 @@ Block PKCondition::getBlockWithConstants(
}
PKCondition::PKCondition(const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns,
const SortDescription & sort_descr_, ExpressionActionsPtr pk_expr_)
: sort_descr(sort_descr_), pk_expr(pk_expr_)
PKCondition::PKCondition(
const SelectQueryInfo & query_info,
const Context & context,
const NamesAndTypesList & all_columns,
const SortDescription & sort_descr_,
ExpressionActionsPtr pk_expr_)
: sort_descr(sort_descr_), pk_expr(pk_expr_), prepared_sets(query_info.sets)
{
for (size_t i = 0; i < sort_descr.size(); ++i)
{
@ -233,10 +236,10 @@ PKCondition::PKCondition(const ASTPtr & query, const Context & context, const Na
/** Evaluation of expressions that depend only on constants.
* For the index to be used, if it is written, for example `WHERE Date = toDate(now())`.
*/
Block block_with_constants = getBlockWithConstants(query, context, all_columns);
Block block_with_constants = getBlockWithConstants(query_info.query, context, all_columns);
/// Trasform WHERE section to Reverse Polish notation
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query);
const ASTSelectQuery & select = typeid_cast<const ASTSelectQuery &>(*query_info.query);
if (select.where_expression)
{
traverseAST(select.where_expression, context, block_with_constants);
@ -545,7 +548,7 @@ bool PKCondition::atomFromAST(const ASTPtr & node, const Context & context, Bloc
key_arg_pos = 1;
is_constant_transformed = true;
}
else if (typeid_cast<const ASTSet *>(args[1].get())
else if (prepared_sets.count(args[1].get())
&& isPrimaryKeyPossiblyWrappedByMonotonicFunctions(args[0], context, key_column_num, key_expr_type, chain))
{
key_arg_pos = 0;
@ -899,10 +902,10 @@ bool PKCondition::mayBeTrueInRangeImpl(const std::vector<Range> & key_ranges, co
{
auto in_func = typeid_cast<const ASTFunction *>(element.in_function.get());
const ASTs & args = typeid_cast<const ASTExpressionList &>(*in_func->arguments).children;
auto ast_set = typeid_cast<const ASTSet *>(args[1].get());
if (in_func && ast_set)
PreparedSets::const_iterator it = prepared_sets.find(args[1].get());
if (in_func && it != prepared_sets.end())
{
rpn_stack.push_back(ast_set->set->mayBeTrueInRange(*key_range));
rpn_stack.push_back(it->second->mayBeTrueInRange(*key_range));
if (element.function == RPNElement::FUNCTION_NOT_IN_SET)
rpn_stack.back() = !rpn_stack.back();
}
@ -962,14 +965,6 @@ bool PKCondition::mayBeTrueAfter(
}
static const ASTSet & inFunctionToSet(const ASTPtr & in_function)
{
const auto & in_func = typeid_cast<const ASTFunction &>(*in_function);
const auto & args = typeid_cast<const ASTExpressionList &>(*in_func.arguments).children;
const auto & ast_set = typeid_cast<const ASTSet &>(*args[1]);
return ast_set;
}
String PKCondition::RPNElement::toString() const
{
auto print_wrapped_column = [this](std::ostringstream & ss)
@ -999,7 +994,7 @@ String PKCondition::RPNElement::toString() const
{
ss << "(";
print_wrapped_column(ss);
ss << (function == FUNCTION_IN_SET ? " in " : " notIn ") << inFunctionToSet(in_function).set->describe();
ss << (function == FUNCTION_IN_SET ? " in set" : " notIn set");
ss << ")";
return ss.str();
}

View File

@ -4,11 +4,13 @@
#include <Interpreters/Context.h>
#include <Interpreters/ExpressionActions.h>
#include <Interpreters/Set.h>
#include <Core/SortDescription.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Storages/SelectQueryInfo.h>
namespace DB
@ -200,7 +202,11 @@ class PKCondition
{
public:
/// Does not include the SAMPLE section. all_columns - the set of all columns of the table.
PKCondition(const ASTPtr & query, const Context & context, const NamesAndTypesList & all_columns, const SortDescription & sort_descr,
PKCondition(
const SelectQueryInfo & query_info,
const Context & context,
const NamesAndTypesList & all_columns,
const SortDescription & sort_descr,
ExpressionActionsPtr pk_expr);
/// Whether the condition is feasible in the key range.
@ -324,6 +330,7 @@ private:
SortDescription sort_descr;
ColumnIndices pk_columns;
ExpressionActionsPtr pk_expr;
PreparedSets prepared_sets;
};
}

View File

@ -0,0 +1,32 @@
#pragma once
#include <memory>
namespace DB
{
class IAST;
using ASTPtr = std::shared_ptr<IAST>;
class Set;
using SetPtr = std::shared_ptr<Set>;
/// Information about calculated sets in right hand side of IN.
using PreparedSets = std::unordered_map<IAST*, SetPtr>;
/** Query along with some additional data,
* that can be used during query processing
* inside storage engines.
*/
struct SelectQueryInfo
{
ASTPtr query;
/// Prepared sets are used for indices by storage engine.
/// Example: x IN (1, 2, 3)
PreparedSets sets;
};
}

View File

@ -114,7 +114,7 @@ private:
BlockInputStreams StorageBuffer::read(
const Names & column_names,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,
@ -131,7 +131,7 @@ BlockInputStreams StorageBuffer::read(
if (destination.get() == this)
throw Exception("Destination table is myself. Read will cause infinite loop.", ErrorCodes::INFINITE_LOOP);
streams_from_dst = destination->read(column_names, query, context, processed_stage, max_block_size, num_streams);
streams_from_dst = destination->read(column_names, query_info, context, processed_stage, max_block_size, num_streams);
}
BlockInputStreams streams_from_buffers;
@ -144,7 +144,7 @@ BlockInputStreams StorageBuffer::read(
*/
if (processed_stage > QueryProcessingStage::FetchColumns)
for (auto & stream : streams_from_buffers)
stream = InterpreterSelectQuery(query, context, processed_stage, 0, stream).execute().in;
stream = InterpreterSelectQuery(query_info.query, context, processed_stage, 0, stream).execute().in;
streams_from_dst.insert(streams_from_dst.end(), streams_from_buffers.begin(), streams_from_buffers.end());
return streams_from_dst;

View File

@ -58,7 +58,7 @@ public:
BlockInputStreams read(
const Names & column_names,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,

View File

@ -35,7 +35,7 @@ public:
BlockInputStreams read(
const Names & column_names,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size,

View File

@ -68,7 +68,7 @@ StorageDictionary::StorageDictionary(
BlockInputStreams StorageDictionary::read(
const Names & column_names,
const ASTPtr& query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,

View File

@ -41,7 +41,7 @@ public:
std::string getTableName() const override { return table_name; }
const NamesAndTypesList & getColumnsListImpl() const override { return *columns; }
BlockInputStreams read(const Names & column_names,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size = DEFAULT_BLOCK_SIZE,

View File

@ -189,7 +189,7 @@ StoragePtr StorageDistributed::createWithOwnCluster(
BlockInputStreams StorageDistributed::read(
const Names & column_names,
const ASTPtr & query,
const SelectQueryInfo & query_info,
const Context & context,
QueryProcessingStage::Enum & processed_stage,
const size_t max_block_size,
@ -206,7 +206,7 @@ BlockInputStreams StorageDistributed::read(
: QueryProcessingStage::WithMergeableState;
const auto & modified_query_ast = rewriteSelectQuery(
query, remote_database, remote_table);
query_info.query, remote_database, remote_table);
Tables external_tables;
@ -218,7 +218,6 @@ BlockInputStreams StorageDistributed::read(
/** The functionality of shard_multiplexing is not completed - turn it off.
* (Because connecting to different shards within a single thread is not done in parallel.)
* For more information, see https: //███████████.yandex-team.ru/METR-18300
*/
//bool enable_shard_multiplexing = !(ast.order_expression_list && !ast.group_expression_list);
bool enable_shard_multiplexing = false;
@ -354,7 +353,6 @@ void StorageDistributed::reshardPartitions(
/** The functionality of shard_multiplexing is not completed - turn it off.
* (Because connecting to different shards within a single thread is not done in parallel.)
* For more information, see https: //███████████.yandex-team.ru/METR-18300
*/
bool enable_shard_multiplexing = false;
@ -433,7 +431,6 @@ BlockInputStreams StorageDistributed::describe(const Context & context, const Se
/** The functionality of shard_multiplexing is not completed - turn it off.
* (Because connecting connections to different shards within a single thread is not done in parallel.)
* For more information, see https://███████████.yandex-team.ru/METR-18300
*/
bool enable_shard_multiplexing = false;

Some files were not shown because too many files have changed in this diff Show More