Merge remote-tracking branch 'upstream/master'

This commit is contained in:
mf5137 2018-03-26 16:03:51 +02:00
commit 07135bccba
329 changed files with 4056 additions and 2637 deletions

View File

@ -13,7 +13,7 @@ matrix:
# apt:
# sources:
# - ubuntu-toolchain-r-test
# packages: [ g++-7, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libzookeeper-mt-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo ]
# packages: [ g++-7, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libzookeeper-mt-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo, openssl ]
#
# env:
# - MATRIX_EVAL="export CC=gcc-7 && export CXX=g++-7"
@ -36,7 +36,7 @@ matrix:
sources:
- ubuntu-toolchain-r-test
- llvm-toolchain-trusty-5.0
packages: [ g++-7, clang-5.0, lld-5.0, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libzookeeper-mt-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo ]
packages: [ g++-7, clang-5.0, lld-5.0, libicu-dev, libreadline-dev, libmysqlclient-dev, unixodbc-dev, libltdl-dev, libssl-dev, libboost-dev, zlib1g-dev, libdouble-conversion-dev, libzookeeper-mt-dev, libsparsehash-dev, librdkafka-dev, libcapnp-dev, libsparsehash-dev, libgoogle-perftools-dev, bash, expect, python, python-lxml, python-termcolor, curl, perl, sudo, openssl]
env:
- MATRIX_EVAL="export CC=clang-5.0 && export CXX=clang++-5.0"

View File

@ -1,4 +1,31 @@
# ClickHouse 1.1.54356 Release Candidate, 2018-03-06
# ClickHouse 1.1.54370 Release Candidate, 2018-03-16
## New features:
* Added the `system.macros` table and auto updating of macros when the config file is changed.
* Added the `SYSTEM RELOAD CONFIG` query.
* Added the `maxIntersections(left_col, right_col)` aggregate function, which returns the maximum number of simultaneously intersecting intervals `[left; right]`. The `maxIntersectionsPosition(left, right)` function returns the beginning of the "maximum" interval. ([Michael Furmur](https://github.com/yandex/ClickHouse/pull/2012)).
## Improvements:
* When inserting data in a `Replicated` table, fewer requests are made to `ZooKeeper` (and most of the user-level errors have disappeared from the `ZooKeeper` log).
* Added the ability to create aliases for sets. Example: `WITH (1, 2, 3) AS set SELECT number IN set FROM system.numbers LIMIT 10`.
## Bug fixes:
* Fixed the `Illegal PREWHERE` error when reading from `Merge` tables over `Distributed` tables.
* Added fixes that allow you to run `clickhouse-server` in IPv4-only Docker containers.
* Fixed a race condition when reading from system `system.parts_columns` tables.
* Removed double buffering during a synchronous insert to a `Distributed` table, which could have caused the connection to timeout.
* Fixed a bug that caused excessively long waits for an unavailable replica before beginning a `SELECT` query.
* Fixed incorrect dates in the `system.parts` table.
* Fixed a bug that made it impossible to insert data in a `Replicated` table if `chroot` was non-empty in the configuration of the `ZooKeeper` cluster.
* Fixed the vertical merging algorithm for an empty `ORDER BY` table.
* Restored the ability to use dictionaries in queries to remote tables, even if these dictionaries are not present on the requestor server. This functionality was lost in release 1.1.54362.
* Restored the behavior for queries like `SELECT * FROM remote('server2', default.table) WHERE col IN (SELECT col2 FROM default.table)` when the right side argument of the `IN` should use a remote `default.table` instead of a local one. This behavior was broken in version 1.1.54358.
* Removed extraneous error-level logging of `Not found column ... in block`.
# ClickHouse release 1.1.54356, 2018-03-06
## New features:

View File

@ -1,4 +1,32 @@
# ClickHouse 1.1.54362 Release Candidate, 2018-03-11
# ClickHouse 1.1.54370 Release Candidate, 2018-03-16
## Новые возможности:
* Добавлена системная таблица `system.macros` и автоматическое обновление макросов при изменении конфигурационного файла.
* Добавлен запрос `SYSTEM RELOAD CONFIG`.
* Добавлена агрегатная функция `maxIntersections(left_col, right_col)`, возвращающая максимальное количество одновременно пересекающихся интервалов `[left; right]`. Функция `maxIntersectionsPosition(left, right)` возвращает начало такого "максимального" интервала. ([Michael Furmur](https://github.com/yandex/ClickHouse/pull/2012)).
## Улучшения:
* При вставке данных в `Replicated`-таблицу делается меньше обращений к `ZooKeeper` (также из лога `ZooKeeper` исчезло большинство user-level ошибок).
* Добавлена возможность создавать алиасы для множеств. Пример: `WITH (1, 2, 3) AS set SELECT number IN set FROM system.numbers LIMIT 10`.
## Исправление ошибок:
* Исправлена ошибка `Illegal PREWHERE` при чтении из Merge-таблицы над `Distributed`-таблицами.
* Добавлены исправления, позволяющие запускать clickhouse-server в IPv4-only docker-контейнерах.
* Исправлен race condition при чтении из системной таблицы `system.parts_columns`
* Убрана двойная буферизация при синхронной вставке в `Distributed`-таблицу, которая могла приводить к timeout-ам соединений.
* Исправлена ошибка, вызывающая чрезмерно долгое ожидание недоступной реплики перед началом выполнения `SELECT`.
* Исправлено некорректное отображение дат в таблице `system.parts`.
* Исправлена ошибка, приводящая к невозможности вставить данные в `Replicated`-таблицу, если в конфигурации кластера `ZooKeeper` задан непустой `chroot`.
* Исправлен алгоритм вертикального мержа при пустом ключе `ORDER BY` таблицы.
* Возвращена возможность использовать словари в запросах к удаленным таблицам, даже если этих словарей нет на сервере-инициаторе. Данная функциональность была потеряна в версии 1.1.54362.
* Восстановлено поведение, при котором в запросах типа `SELECT * FROM remote('server2', default.table) WHERE col IN (SELECT col2 FROM default.table)` в правой части `IN` должна использоваться удаленная таблица `default.table`, а не локальная. данное поведение было нарушено в версии 1.1.54358.
* Устранено ненужное Error-level логирование `Not found column ... in block`.
# Релиз ClickHouse 1.1.54362, 2018-03-11
## Новые возможности:

View File

@ -49,7 +49,8 @@ elseif (NOT MISSING_INTERNAL_POCO_LIBRARY)
set (Poco_DataODBC_INCLUDE_DIRS "${ClickHouse_SOURCE_DIR}/contrib/poco/Data/ODBC/include/")
endif ()
if (OPENSSL_FOUND)
# TODO! fix internal ssl
if (OPENSSL_FOUND AND NOT USE_INTERNAL_SSL_LIBRARY)
set (Poco_NetSSL_FOUND 1)
set (Poco_NetSSL_LIBRARY PocoNetSSL)
set (Poco_Crypto_LIBRARY PocoCrypto)

View File

@ -100,13 +100,17 @@ if (USE_INTERNAL_RDKAFKA_LIBRARY)
mark_as_advanced (ZLIB_INCLUDE_DIR)
if (USE_INTERNAL_SSL_LIBRARY)
add_library(bundled-ssl ALIAS ${OPENSSL_SSL_LIBRARY})
set (WITH_BUNDLED_SSL 1)
if (MAKE_STATIC_LIBRARIES)
add_library(bundled-ssl ALIAS ${OPENSSL_SSL_LIBRARY})
set (WITH_BUNDLED_SSL 1 CACHE INTERNAL "")
else ()
set (WITH_SSL 0 CACHE INTERNAL "")
endif ()
endif ()
add_subdirectory (librdkafka)
if (USE_INTERNAL_SSL_LIBRARY)
if (USE_INTERNAL_SSL_LIBRARY AND MAKE_STATIC_LIBRARIES)
target_include_directories(rdkafka PRIVATE BEFORE ${OPENSSL_INCLUDE_DIR})
endif ()
target_include_directories(rdkafka PRIVATE BEFORE ${ZLIB_INCLUDE_DIR})
@ -127,16 +131,18 @@ endif ()
if (USE_INTERNAL_POCO_LIBRARY)
set (ALLOW_DUPLICATE_CUSTOM_TARGETS 1)
set (save_CMAKE_CXX_FLAGS ${CMAKE_CXX_FLAGS})
set (save_CMAKE_C_FLAGS ${CMAKE_C_FLAGS})
set (_save ${ENABLE_TESTS})
set (ENABLE_TESTS 0)
set (CMAKE_DISABLE_FIND_PACKAGE_ZLIB 1)
if (USE_INTERNAL_SSL_LIBRARY)
set (DISABLE_INTERNAL_OPENSSL 1)
set (ENABLE_NETSSL 0) # TODO!
set (ENABLE_CRYPTO 0) # TODO!
set (DISABLE_INTERNAL_OPENSSL 1 CACHE INTERNAL "")
set (ENABLE_NETSSL 0 CACHE INTERNAL "") # TODO!
set (ENABLE_CRYPTO 0 CACHE INTERNAL "") # TODO!
endif ()
if (MSVC)
set (ENABLE_DATA_ODBC 0 CACHE INTERNAL "") # TODO (build fail)
endif ()
add_subdirectory (poco)
unset (CMAKE_DISABLE_FIND_PACKAGE_ZLIB)

2
contrib/boost vendored

@ -1 +1 @@
Subproject commit eb5943711e88d1008583e6ae3720a5489313d02e
Subproject commit 5121cc9d0375c7b81b24b6087a51684e6cd62ded

View File

@ -5,7 +5,11 @@
/* #undef AC_APPLE_UNIVERSAL_BUILD */
/* Define to 1 if the compiler supports __builtin_expect. */
#if _MSC_VER
#define HAVE_BUILTIN_EXPECT 0
#else
#define HAVE_BUILTIN_EXPECT 1
#endif
/* Define to 1 if you have the <dlfcn.h> header file. */
#define HAVE_DLFCN_H 1

View File

@ -2,4 +2,8 @@ add_library(farmhash
farmhash.cc
farmhash.h)
if (MSVC)
target_compile_definitions (farmhash PRIVATE FARMHASH_NO_BUILTIN_EXPECT=1)
endif ()
target_include_directories (farmhash PUBLIC ${CMAKE_CURRENT_BINARY_DIR})

View File

@ -13,7 +13,11 @@
/* #undef ENABLE_SIZED_DELETE */
/* Define to 1 if compiler supports __builtin_expect */
#if _MSC_VER
#define HAVE_BUILTIN_EXPECT 0
#else
#define HAVE_BUILTIN_EXPECT 1
#endif
/* Define to 1 if compiler supports __builtin_stack_pointer */
/* #undef HAVE_BUILTIN_STACK_POINTER */

View File

@ -131,7 +131,6 @@ target_link_libraries (clickhouse_common_io
${LINK_LIBRARIES_ONLY_ON_X86_64}
${LZ4_LIBRARY}
${ZSTD_LIBRARY}
${ZOOKEEPER_LIBRARY}
${DOUBLE_CONVERSION_LIBRARIES}
${Poco_Net_LIBRARY}
${Poco_Data_LIBRARY}
@ -158,10 +157,6 @@ if (NOT USE_INTERNAL_RE2_LIBRARY)
target_include_directories (dbms BEFORE PRIVATE ${RE2_INCLUDE_DIR})
endif ()
if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY)
target_include_directories (clickhouse_common_io BEFORE PUBLIC ${ZOOKEEPER_INCLUDE_DIR})
endif ()
if (NOT USE_INTERNAL_BOOST_LIBRARY)
target_include_directories (clickhouse_common_io BEFORE PUBLIC ${Boost_INCLUDE_DIRS})
endif ()

View File

@ -1,6 +1,6 @@
# This strings autochanged from release_lib.sh:
set(VERSION_DESCRIBE v1.1.54363-testing)
set(VERSION_REVISION 54363)
set(VERSION_DESCRIBE v1.1.54371-testing)
set(VERSION_REVISION 54371)
# end of autochange
set (VERSION_MAJOR 1)

View File

@ -70,7 +70,10 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
return combinator->transformAggregateFunction(nested_function, argument_types, parameters);
}
return getImpl(name, argument_types, parameters, recursion_level);
auto res = getImpl(name, argument_types, parameters, recursion_level);
if (!res)
throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR);
return res;
}

View File

@ -0,0 +1,37 @@
#include <AggregateFunctions/AggregateFunctionFactory.h>
#include <AggregateFunctions/AggregateFunctionMaxIntersections.h>
#include <AggregateFunctions/FactoryHelpers.h>
#include <AggregateFunctions/Helpers.h>
namespace DB
{
namespace
{
AggregateFunctionPtr createAggregateFunctionMaxIntersections(
AggregateFunctionIntersectionsKind kind,
const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
assertBinary(name, argument_types);
assertNoParameters(name, parameters);
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionIntersectionsMax>(*argument_types[0], kind, argument_types));
if (!res)
throw Exception("Illegal types " + argument_types[0]->getName() + " and " + argument_types[1]->getName()
+ " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory & factory)
{
factory.registerFunction("maxIntersections", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
{ return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Count, name, argument_types, parameters); });
factory.registerFunction("maxIntersectionsPosition", [](const std::string & name, const DataTypes & argument_types, const Array & parameters)
{ return createAggregateFunctionMaxIntersections(AggregateFunctionIntersectionsKind::Position, name, argument_types, parameters); });
}
}

View File

@ -0,0 +1,170 @@
#pragma once
#include <common/logger_useful.h>
#include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/ArenaAllocator.h>
#include <Common/NaNUtils.h>
#include <AggregateFunctions/IAggregateFunction.h>
#define AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE 0xFFFFFF
namespace DB
{
namespace ErrorCodes
{
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int TOO_LARGE_ARRAY_SIZE;
}
/** maxIntersections: returns maximum count of the intersected intervals defined by start_column and end_column values,
* maxIntersectionsPosition: returns leftmost position of maximum intersection of intervals.
*/
/// Similar to GroupArrayNumericData.
template <typename T>
struct MaxIntersectionsData
{
/// Left or right end of the interval and signed weight; with positive sign for begin of interval and negative sign for end of interval.
using Value = std::pair<T, Int64>;
// Switch to ordinary Allocator after 4096 bytes to avoid fragmentation and trash in Arena
using Allocator = MixedArenaAllocator<4096>;
using Array = PODArray<Value, 32, Allocator>;
Array value;
};
enum class AggregateFunctionIntersectionsKind
{
Count,
Position
};
template <typename PointType>
class AggregateFunctionIntersectionsMax final
: public IAggregateFunctionDataHelper<MaxIntersectionsData<PointType>, AggregateFunctionIntersectionsMax<PointType>>
{
private:
AggregateFunctionIntersectionsKind kind;
public:
AggregateFunctionIntersectionsMax(AggregateFunctionIntersectionsKind kind_, const DataTypes & arguments)
: kind(kind_)
{
if (!arguments[0]->isNumber())
throw Exception{getName() + ": first argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!arguments[1]->isNumber())
throw Exception{getName() + ": second argument must be represented by integer", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!arguments[0]->equals(*arguments[1]))
throw Exception{getName() + ": arguments must have the same type", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
String getName() const override
{
return kind == AggregateFunctionIntersectionsKind::Count
? "maxIntersections"
: "maxIntersectionsPosition";
}
DataTypePtr getReturnType() const override
{
if (kind == AggregateFunctionIntersectionsKind::Count)
return std::make_shared<DataTypeUInt64>();
else
return std::make_shared<DataTypeNumber<PointType>>();
}
void add(AggregateDataPtr place, const IColumn ** columns, size_t row_num, Arena * arena) const override
{
PointType left = static_cast<const ColumnVector<PointType> &>(*columns[0]).getData()[row_num];
PointType right = static_cast<const ColumnVector<PointType> &>(*columns[1]).getData()[row_num];
if (!isNaN(left))
this->data(place).value.push_back(std::make_pair(left, Int64(1)), arena);
if (!isNaN(right))
this->data(place).value.push_back(std::make_pair(right, Int64(-1)), arena);
}
void merge(AggregateDataPtr place, ConstAggregateDataPtr rhs, Arena * arena) const override
{
auto & cur_elems = this->data(place);
auto & rhs_elems = this->data(rhs);
cur_elems.value.insert(rhs_elems.value.begin(), rhs_elems.value.end(), arena);
}
void serialize(ConstAggregateDataPtr place, WriteBuffer & buf) const override
{
const auto & value = this->data(place).value;
size_t size = value.size();
writeVarUInt(size, buf);
buf.write(reinterpret_cast<const char *>(&value[0]), size * sizeof(value[0]));
}
void deserialize(AggregateDataPtr place, ReadBuffer & buf, Arena * arena) const override
{
size_t size = 0;
readVarUInt(size, buf);
if (unlikely(size > AGGREGATE_FUNCTION_MAX_INTERSECTIONS_MAX_ARRAY_SIZE))
throw Exception("Too large array size", ErrorCodes::TOO_LARGE_ARRAY_SIZE);
auto & value = this->data(place).value;
value.resize(size, arena);
buf.read(reinterpret_cast<char *>(&value[0]), size * sizeof(value[0]));
}
void insertResultInto(ConstAggregateDataPtr place, IColumn & to) const override
{
Int64 current_intersections = 0;
Int64 max_intersections = 0;
PointType position_of_max_intersections = 0;
/// const_cast because we will sort the array
auto & array = const_cast<typename MaxIntersectionsData<PointType>::Array &>(this->data(place).value);
std::sort(array.begin(), array.end(), [](const auto & a, const auto & b) { return a.first < b.first; });
for (const auto & point_weight : array)
{
current_intersections += point_weight.second;
if (current_intersections > max_intersections)
{
max_intersections = current_intersections;
position_of_max_intersections = point_weight.first;
}
}
if (kind == AggregateFunctionIntersectionsKind::Count)
{
auto & result_column = static_cast<ColumnUInt64 &>(to).getData();
result_column.push_back(max_intersections);
}
else
{
auto & result_column = static_cast<ColumnVector<PointType> &>(to).getData();
result_column.push_back(position_of_max_intersections);
}
}
const char * getHeaderFilePath() const override
{
return __FILE__;
}
};
}

View File

@ -37,7 +37,11 @@ AggregateFunctionPtr createAggregateFunctionSumMap(const std::string & name, con
values_types.push_back(array_type->getNestedType());
}
return AggregateFunctionPtr(createWithNumericType<AggregateFunctionSumMap>(*keys_type, keys_type, std::move(values_types)));
AggregateFunctionPtr res(createWithNumericType<AggregateFunctionSumMap>(*keys_type, keys_type, std::move(values_types)));
if (!res)
throw Exception("Illegal type of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return res;
}
}

View File

@ -33,6 +33,10 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
{
assertNoParameters(name, params);
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (argument_types.size() == 1)
{
const IDataType & argument_type = *argument_types[0];
@ -51,23 +55,18 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, true>>(argument_types);
else if (typeid_cast<const DataTypeUUID *>(&argument_type))
return std::make_shared<AggregateFunctionUniq<DataTypeUUID::FieldType, Data>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
else if (argument_types.size() > 1)
else
{
/// If there are several arguments, then no tuples allowed among them.
for (const auto & type : argument_types)
if (typeid_cast<const DataTypeTuple *>(type.get()))
throw Exception("Tuple argument of function " + name + " must be the only argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, false>>(argument_types);
}
else
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// "Variadic" method also works as a fallback generic case for single argument.
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, false>>(argument_types);
}
template <template <typename> class Data, typename DataForVariadic>
@ -75,6 +74,10 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
{
assertNoParameters(name, params);
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (argument_types.size() == 1)
{
const IDataType & argument_type = *argument_types[0];
@ -93,23 +96,18 @@ AggregateFunctionPtr createAggregateFunctionUniq(const std::string & name, const
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, true>>(argument_types);
else if (typeid_cast<const DataTypeUUID *>(&argument_type))
return std::make_shared<AggregateFunctionUniq<DataTypeUUID::FieldType, Data<DataTypeUUID::FieldType>>>();
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
else if (argument_types.size() > 1)
else
{
/// If there are several arguments, then no tuples allowed among them.
for (const auto & type : argument_types)
if (typeid_cast<const DataTypeTuple *>(type.get()))
throw Exception("Tuple argument of function " + name + " must be the only argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, false>>(argument_types);
}
else
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// "Variadic" method also works as a fallback generic case for single argument.
return std::make_shared<AggregateFunctionUniqVariadic<DataForVariadic, false>>(argument_types);
}
}

View File

@ -122,8 +122,8 @@ struct AggregateFunctionUniqExactData<String>
static String getName() { return "uniqExact"; }
};
template <typename T, HyperLogLogMode mode>
struct BaseUniqCombinedData
template <typename T>
struct AggregateFunctionUniqCombinedData
{
using Key = UInt32;
using Set = CombinedCardinalityEstimator<
@ -135,14 +135,15 @@ struct BaseUniqCombinedData
TrivialHash,
UInt32,
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
mode
>;
HyperLogLogMode::FullFeatured>;
Set set;
static String getName() { return "uniqCombined"; }
};
template <HyperLogLogMode mode>
struct BaseUniqCombinedData<String, mode>
template <>
struct AggregateFunctionUniqCombinedData<String>
{
using Key = UInt64;
using Set = CombinedCardinalityEstimator<
@ -154,44 +155,14 @@ struct BaseUniqCombinedData<String, mode>
TrivialHash,
UInt64,
HyperLogLogBiasEstimator<UniqCombinedBiasData>,
mode
>;
HyperLogLogMode::FullFeatured>;
Set set;
};
/// Aggregate functions uniqCombinedRaw, uniqCombinedLinearCounting, and uniqCombinedBiasCorrected
/// are intended for development of new versions of the uniqCombined function.
/// Users should only use uniqCombined.
template <typename T>
struct AggregateFunctionUniqCombinedRawData
: public BaseUniqCombinedData<T, HyperLogLogMode::Raw>
{
static String getName() { return "uniqCombinedRaw"; }
};
template <typename T>
struct AggregateFunctionUniqCombinedLinearCountingData
: public BaseUniqCombinedData<T, HyperLogLogMode::LinearCounting>
{
static String getName() { return "uniqCombinedLinearCounting"; }
};
template <typename T>
struct AggregateFunctionUniqCombinedBiasCorrectedData
: public BaseUniqCombinedData<T, HyperLogLogMode::BiasCorrected>
{
static String getName() { return "uniqCombinedBiasCorrected"; }
};
template <typename T>
struct AggregateFunctionUniqCombinedData
: public BaseUniqCombinedData<T, HyperLogLogMode::FullFeatured>
{
static String getName() { return "uniqCombined"; }
};
namespace detail
{
@ -288,10 +259,7 @@ struct OneAdder
data.set.insert(CityHash_v1_0_2::CityHash64(value.data, value.size));
}
}
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqCombinedRawData<T>>
|| std::is_same_v<Data, AggregateFunctionUniqCombinedLinearCountingData<T>>
|| std::is_same_v<Data, AggregateFunctionUniqCombinedBiasCorrectedData<T>>
|| std::is_same_v<Data, AggregateFunctionUniqCombinedData<T>>)
else if constexpr (std::is_same_v<Data, AggregateFunctionUniqCombinedData<T>>)
{
if constexpr (!std::is_same_v<T, String>)
{
@ -371,7 +339,7 @@ public:
/** For multiple arguments. To compute, hashes them.
* You can pass multiple arguments as is; You can also pass one argument - a tuple.
* But (for the possibility of effective implementation), you can not pass several arguments, among which there are tuples.
* But (for the possibility of efficient implementation), you can not pass several arguments, among which there are tuples.
*/
template <typename Data, bool argument_is_tuple>
class AggregateFunctionUniqVariadic final : public IAggregateFunctionDataHelper<Data, AggregateFunctionUniqVariadic<Data, argument_is_tuple>>

View File

@ -42,6 +42,10 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
threshold = threshold_param;
}
if (argument_types.empty())
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (argument_types.size() == 1)
{
const IDataType & argument_type = *argument_types[0];
@ -60,22 +64,18 @@ AggregateFunctionPtr createAggregateFunctionUniqUpTo(const std::string & name, c
return std::make_shared<AggregateFunctionUniqUpToVariadic<true>>(argument_types, threshold);
else if (typeid_cast<const DataTypeUUID *>(&argument_type))
return std::make_shared<AggregateFunctionUniqUpTo<DataTypeUUID::FieldType>>(threshold);
else
throw Exception("Illegal type " + argument_types[0]->getName() + " of argument for aggregate function " + name, ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
else if (argument_types.size() > 1)
else
{
/// If there are several arguments, then no tuples allowed among them.
for (const auto & type : argument_types)
if (typeid_cast<const DataTypeTuple *>(type.get()))
throw Exception("Tuple argument of function " + name + " must be the only argument",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<AggregateFunctionUniqUpToVariadic<false>>(argument_types, threshold);
}
else
throw Exception("Incorrect number of arguments for aggregate function " + name,
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
/// "Variadic" method also works as a fallback generic case for single argument.
return std::make_shared<AggregateFunctionUniqUpToVariadic<false>>(argument_types, threshold);
}
}

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/PODArray.h>
#include <Common/NaNUtils.h>
#include <IO/WriteBuffer.h>
#include <IO/ReadBuffer.h>
#include <Core/Types.h>
@ -32,7 +33,9 @@ struct QuantileExact
void add(const Value & x)
{
array.push_back(x);
/// We must skip NaNs as they are not compatible with comparison sorting.
if (!isNaN(x))
array.push_back(x);
}
template <typename Weight>

View File

@ -1,6 +1,7 @@
#pragma once
#include <Common/HashTable/HashMap.h>
#include <Common/NaNUtils.h>
namespace DB
@ -33,12 +34,15 @@ struct QuantileExactWeighted
void add(const Value & x)
{
++map[x];
/// We must skip NaNs as they are not compatible with comparison sorting.
if (!isNaN(x))
++map[x];
}
void add(const Value & x, const Weight & weight)
{
map[x] += weight;
if (!isNaN(x))
map[x] += weight;
}
void merge(const QuantileExactWeighted & rhs)

View File

@ -4,6 +4,7 @@
#include <Core/Field.h>
#include <Common/FieldVisitors.h>
#include <Common/NaNUtils.h>
namespace DB
@ -12,6 +13,7 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int PARAMETER_OUT_OF_BOUND;
}
@ -55,6 +57,10 @@ struct QuantileLevels
for (size_t i = 0; i < size; ++i)
{
levels[i] = applyVisitor(FieldVisitorConvertToNumber<Float64>(), params[i]);
if (isNaN(levels[i]) || levels[i] < 0 || levels[i] > 1)
throw Exception("Quantile level is out of range [0..1]", ErrorCodes::PARAMETER_OUT_OF_BOUND);
permutation[i] = i;
}

View File

@ -9,6 +9,7 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/PODArray.h>
#include <Common/NaNUtils.h>
#include <Poco/Exception.h>
#include <pcg_random.hpp>
@ -67,6 +68,9 @@ public:
void insert(const T & v)
{
if (isNaN(v))
return;
sorted = false;
++total_values;
if (samples.size() < sample_count)

View File

@ -11,6 +11,7 @@
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Common/PODArray.h>
#include <Common/NaNUtils.h>
#include <Poco/Exception.h>
@ -66,6 +67,9 @@ public:
void insert(const T & v, const UInt64 determinator)
{
if (isNaN(v))
return;
const UInt32 hash = intHash64(determinator);
if (!good(hash))
return;

View File

@ -23,6 +23,7 @@ void registerAggregateFunctionsUniq(AggregateFunctionFactory &);
void registerAggregateFunctionUniqUpTo(AggregateFunctionFactory &);
void registerAggregateFunctionTopK(AggregateFunctionFactory &);
void registerAggregateFunctionsBitwise(AggregateFunctionFactory &);
void registerAggregateFunctionsMaxIntersections(AggregateFunctionFactory &);
void registerAggregateFunctionCombinatorIf(AggregateFunctionCombinatorFactory &);
void registerAggregateFunctionCombinatorArray(AggregateFunctionCombinatorFactory &);
@ -53,6 +54,7 @@ void registerAggregateFunctions()
registerAggregateFunctionUniqUpTo(factory);
registerAggregateFunctionTopK(factory);
registerAggregateFunctionsBitwise(factory);
registerAggregateFunctionsMaxIntersections(factory);
}
{

View File

@ -173,7 +173,7 @@ ASTPtr createASTIdentifierForColumnInTable(const String & column, const CollectT
void createASTsForAllColumnsInTable(const CollectTables::TableInfo & table, ASTs & res)
{
if (table.storage)
for (const auto & name : table.storage->getColumnNamesList())
for (const auto & name : table.storage->getColumns().getNamesOfPhysical())
res.emplace_back(createASTIdentifierForColumnInTable(name, table));
else
for (size_t i = 0, size = table.structure_of_subquery.columns(); i < size; ++i)
@ -315,7 +315,7 @@ void processIdentifier(
}
else if (table->storage)
{
info.data_type = table->storage->getDataTypeByName(column_name);
info.data_type = table->storage->getColumn(column_name).type;
}
else
throw Exception("Logical error: no storage and no structure of subquery is specified for table", ErrorCodes::LOGICAL_ERROR);

View File

@ -72,7 +72,7 @@ void ExecuteTableFunctions::dump(WriteBuffer & out) const
{
writeString(table.second->getName(), out);
writeCString("\n\n", out);
writeString(table.second->getColumnsList().toString(), out);
writeString(table.second->getColumns().getAllPhysical().toString(), out);
writeCString("\n", out);
}
}

View File

@ -148,6 +148,10 @@ void Connection::receiveHello()
{
readStringBinary(server_timezone, *in);
}
if (server_revision >= DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME)
{
readStringBinary(server_display_name, *in);
}
}
else if (packet_type == Protocol::Server::Exception)
receiveException()->rethrow();
@ -203,6 +207,14 @@ const String & Connection::getServerTimezone()
return server_timezone;
}
const String & Connection::getServerDisplayName()
{
if (!connected)
connect();
return server_display_name;
}
void Connection::forceConnected()
{
if (!connected)

View File

@ -134,6 +134,7 @@ public:
void getServerVersion(String & name, UInt64 & version_major, UInt64 & version_minor, UInt64 & revision);
const String & getServerTimezone();
const String & getServerDisplayName();
/// For log and exception messages.
const String & getDescription() const;
@ -213,6 +214,7 @@ private:
UInt64 server_version_minor = 0;
UInt64 server_revision = 0;
String server_timezone;
String server_display_name;
std::unique_ptr<Poco::Net::StreamSocket> socket;
std::shared_ptr<ReadBuffer> in;

View File

@ -1,6 +1,6 @@
#pragma once
#if !(defined(__FreeBSD__) || defined(__APPLE__))
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#include <Common/Exception.h>
#include <common/logger_useful.h>

View File

@ -73,6 +73,8 @@ void ConfigReloader::run()
void ConfigReloader::reloadIfNewer(bool force, bool throw_on_error, bool fallback_to_preprocessed)
{
std::lock_guard<std::mutex> lock(reload_mutex);
FilesChangesTracker new_files = getNewFileList();
if (force || new_files.isDifferOrNewerThan(files))
{

View File

@ -42,6 +42,9 @@ public:
/// Call this method to run the backround thread.
void start();
/// Reload immediately. For SYSTEM RELOAD CONFIG query.
void reload() { reloadIfNewer(/* force */ true, /* throw_on_error */ true, /* fallback_to_preprocessed */ false); }
private:
void run();
@ -74,6 +77,9 @@ private:
std::atomic<bool> quit{false};
std::thread thread;
/// Locked inside reloadIfNewer.
std::mutex reload_mutex;
};
}

View File

@ -372,6 +372,7 @@ namespace ErrorCodes
extern const int QUERY_WAS_CANCELLED = 394;
extern const int FUNCTION_THROW_IF_VALUE_IS_NON_ZERO = 395;
extern const int TOO_MANY_ROWS_OR_BYTES = 396;
extern const int QUERY_IS_NOT_SUPPORTED_IN_MATERIALIZED_VIEW = 397;
extern const int KEEPER_EXCEPTION = 999;

View File

@ -198,7 +198,7 @@ public:
/// Create table
NamesAndTypesList columns = sample_block.getNamesAndTypesList();
StoragePtr storage = StorageMemory::create(data.second, columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{});
StoragePtr storage = StorageMemory::create(data.second, ColumnsDescription{columns});
storage->startup();
context.addExternalTable(data.second, storage);
BlockOutputStreamPtr output = storage->write(ASTPtr(), context.getSettingsRef());

View File

@ -1,6 +1,8 @@
#include <Poco/Util/AbstractConfiguration.h>
#include <Common/Macros.h>
#include <Common/Exception.h>
namespace DB
{

View File

@ -1,10 +1,18 @@
#pragma once
#include <Core/Types.h>
#include <Poco/Util/AbstractConfiguration.h>
#include <map>
namespace Poco
{
namespace Util
{
class AbstractConfiguration;
}
}
namespace DB
{
@ -21,10 +29,12 @@ public:
*/
String expand(const String & s, size_t level = 0) const;
private:
using MacroMap = std::map<String, String>;
const MacroMap getMacroMap() const { return macros; }
private:
MacroMap macros;
};
}

View File

@ -1,13 +1,12 @@
#include <sys/types.h>
#include <sys/wait.h>
#include <unistd.h>
#include <fcntl.h>
#include <dlfcn.h>
#include <Common/Exception.h>
#include <Common/ShellCommand.h>
#include <IO/WriteBufferFromVector.h>
#include <IO/WriteHelpers.h>
#include <port/unistd.h>
namespace DB

View File

@ -5,7 +5,11 @@ add_headers_and_sources(clickhouse_common_zookeeper .)
add_library(clickhouse_common_zookeeper ${SPLIT_SHARED} ${clickhouse_common_zookeeper_headers} ${clickhouse_common_zookeeper_sources})
target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io)
if (NOT USE_INTERNAL_ZOOKEEPER_LIBRARY)
target_include_directories (clickhouse_common_zookeeper BEFORE PUBLIC ${ZOOKEEPER_INCLUDE_DIR})
endif ()
target_link_libraries (clickhouse_common_zookeeper clickhouse_common_io ${ZOOKEEPER_LIBRARY})
if (ENABLE_TESTS)
add_subdirectory (tests)

View File

@ -102,4 +102,19 @@ private:
};
class KeeperMultiException : public KeeperException
{
public:
MultiTransactionInfo info;
/// If it is user error throws KeeperMultiException else throws ordinary KeeperException
/// If it is ZOK does nothing
static void check(const MultiTransactionInfo & info);
static void check(int code, const Ops & ops, const OpResultsPtr & op_results);
protected:
KeeperMultiException(const MultiTransactionInfo & info, size_t failed_op_index);
};
};

View File

@ -12,6 +12,8 @@ namespace zkutil
using ACLPtr = const ACL_vector *;
using Stat = ::Stat;
class ZooKeeper;
struct Op
{
@ -19,7 +21,7 @@ public:
Op() : data(new zoo_op_t) {}
virtual ~Op() {}
virtual std::unique_ptr<Op> clone() const = 0;
virtual std::shared_ptr<Op> clone() const = 0;
virtual std::string getPath() const = 0;
@ -33,6 +35,9 @@ public:
struct Check;
};
using OpPtr = std::shared_ptr<Op>;
struct Op::Remove : public Op
{
Remove(const std::string & path_, int32_t version_) :
@ -41,9 +46,9 @@ struct Op::Remove : public Op
zoo_delete_op_init(data.get(), path.c_str(), version);
}
std::unique_ptr<Op> clone() const override
OpPtr clone() const override
{
return std::unique_ptr<zkutil::Op>(new Remove(path, version));
return std::make_shared<Remove>(path, version);
}
std::string getPath() const override { return path; }
@ -59,9 +64,9 @@ struct Op::Create : public Op
{
Create(const std::string & path_pattern_, const std::string & value_, ACLPtr acl_, int32_t flags_);
std::unique_ptr<Op> clone() const override
OpPtr clone() const override
{
return std::unique_ptr<zkutil::Op>(new Create(path_pattern, value, acl, flags));
return std::make_shared<Create>(path_pattern, value, acl, flags);
}
std::string getPathCreated() { return created_path.data(); }
@ -91,9 +96,9 @@ struct Op::SetData : public Op
zoo_set_op_init(data.get(), path.c_str(), value.c_str(), value.size(), version, &stat);
}
std::unique_ptr<Op> clone() const override
OpPtr clone() const override
{
return std::unique_ptr<zkutil::Op>(new SetData(path, value, version));
return std::make_shared<SetData>(path, value, version);
}
std::string getPath() const override { return path; }
@ -122,9 +127,9 @@ struct Op::Check : public Op
zoo_check_op_init(data.get(), path.c_str(), version);
}
std::unique_ptr<Op> clone() const override
OpPtr clone() const override
{
return std::unique_ptr<zkutil::Op>(new Check(path, version));
return std::make_shared<Check>(path, version);
}
std::string getPath() const override { return path; }
@ -136,18 +141,46 @@ private:
int32_t version;
};
struct OpResult : public zoo_op_result_t
{
/// Pointers in this class point to fields of class Op.
/// Op instances have the same (or longer lifetime), therefore destructor is not required.
};
using OpPtr = std::unique_ptr<Op>;
using Ops = std::vector<OpPtr>;
/// C++ version of zoo_op_result_t
struct OpResult
{
int err;
std::string value;
std::unique_ptr<Stat> stat;
/// ZooKeeper is required for correct chroot path prefixes handling
explicit OpResult(const zoo_op_result_t & op_result, const ZooKeeper * zookeeper = nullptr);
};
using OpResults = std::vector<OpResult>;
using OpResultsPtr = std::shared_ptr<OpResults>;
using Strings = std::vector<std::string>;
/// Simple structure to handle transaction execution results
struct MultiTransactionInfo
{
Ops ops;
int32_t code = ZOK;
OpResultsPtr op_results;
MultiTransactionInfo() = default;
MultiTransactionInfo(int32_t code_, const Ops & ops_, const OpResultsPtr & op_results_)
: ops(ops_), code(code_), op_results(op_results_) {}
bool empty() const
{
return ops.empty();
}
/// Returns failed op if zkutil::isUserError(code) is true
const Op & getFailedOp() const;
};
namespace CreateMode
{
extern const int Persistent;

View File

@ -82,19 +82,21 @@ void ZooKeeper::processCallback(zhandle_t *, int type, int state, const char * p
}
void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
int32_t session_timeout_ms_, bool check_root_exists)
int32_t session_timeout_ms_, const std::string & chroot_)
{
log = &Logger::get("ZooKeeper");
zoo_set_debug_level(ZOO_LOG_LEVEL_ERROR);
hosts = hosts_;
identity = identity_;
session_timeout_ms = session_timeout_ms_;
chroot = chroot_;
impl = zookeeper_init(hosts.c_str(), nullptr, session_timeout_ms, nullptr, nullptr, 0);
std::string hosts_for_lib = hosts + chroot;
impl = zookeeper_init(hosts_for_lib.c_str(), nullptr, session_timeout_ms, nullptr, nullptr, 0);
ProfileEvents::increment(ProfileEvents::ZooKeeperInit);
if (!impl)
throw KeeperException("Fail to initialize zookeeper. Hosts are " + hosts);
throw KeeperException("Fail to initialize zookeeper. Hosts are " + hosts_for_lib);
if (!identity.empty())
{
@ -107,16 +109,16 @@ void ZooKeeper::init(const std::string & hosts_, const std::string & identity_,
else
default_acl = &ZOO_OPEN_ACL_UNSAFE;
LOG_TRACE(log, "initialized, hosts: " << hosts);
LOG_TRACE(log, "initialized, hosts: " << hosts << (chroot.empty() ? "" : ", chroot: " + chroot));
if (check_root_exists && !exists("/"))
throw KeeperException("Zookeeper root doesn't exist. You should create root node before start.");
if (!chroot.empty() && !exists("/"))
throw KeeperException("Zookeeper root doesn't exist. You should create root node " + chroot + " before start.");
}
ZooKeeper::ZooKeeper(const std::string & hosts, const std::string & identity,
int32_t session_timeout_ms, bool check_root_exists)
int32_t session_timeout_ms, const std::string & chroot)
{
init(hosts, identity, session_timeout_ms, check_root_exists);
init(hosts, identity, session_timeout_ms, chroot);
}
struct ZooKeeperArgs
@ -127,10 +129,8 @@ struct ZooKeeperArgs
config.keys(config_name, keys);
std::vector<std::string> hosts_strings;
std::string root;
session_timeout_ms = DEFAULT_SESSION_TIMEOUT;
has_chroot = false;
for (const auto & key : keys)
{
if (startsWith(key, "node"))
@ -150,7 +150,7 @@ struct ZooKeeperArgs
}
else if (key == "root")
{
root = config.getString(config_name + "." + key);
chroot = config.getString(config_name + "." + key);
}
else throw KeeperException(std::string("Unknown key ") + key + " in config file");
}
@ -166,28 +166,25 @@ struct ZooKeeperArgs
hosts += host;
}
if (!root.empty())
if (!chroot.empty())
{
if (root.front() != '/')
throw KeeperException(std::string("Root path in config file should start with '/', but got ") + root);
if (root.back() == '/')
root.pop_back();
hosts += root;
has_chroot = true;
if (chroot.front() != '/')
throw KeeperException(std::string("Root path in config file should start with '/', but got ") + chroot);
if (chroot.back() == '/')
chroot.pop_back();
}
}
std::string hosts;
std::string identity;
int session_timeout_ms;
bool has_chroot;
std::string chroot;
};
ZooKeeper::ZooKeeper(const Poco::Util::AbstractConfiguration & config, const std::string & config_name)
{
ZooKeeperArgs args(config, config_name);
init(args.hosts, args.identity, args.session_timeout_ms, args.has_chroot);
init(args.hosts, args.identity, args.session_timeout_ms, args.chroot);
}
WatchCallback ZooKeeper::callbackForEvent(const EventPtr & event)
@ -290,19 +287,18 @@ int32_t ZooKeeper::createImpl(const std::string & path, const std::string & data
int code;
/// The name of the created node can be longer than path if the sequential node is created.
size_t name_buffer_size = path.size() + SEQUENTIAL_SUFFIX_SIZE;
char * name_buffer = new char[name_buffer_size];
std::string name_buffer(name_buffer_size, '\0');
code = zoo_create(impl, path.c_str(), data.c_str(), data.size(), getDefaultACL(), mode, name_buffer, name_buffer_size);
code = zoo_create(impl, path.c_str(), data.c_str(), data.size(), getDefaultACL(), mode, name_buffer.data(), name_buffer_size);
ProfileEvents::increment(ProfileEvents::ZooKeeperCreate);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
if (code == ZOK)
{
path_created = std::string(name_buffer);
name_buffer.resize(strlen(name_buffer.data()));
path_created = std::move(name_buffer);
}
delete[] name_buffer;
return code;
}
@ -571,7 +567,19 @@ int32_t ZooKeeper::trySet(const std::string & path, const std::string & data,
return code;
}
int32_t ZooKeeper::multiImpl(const Ops & ops_, OpResultsPtr * out_results_)
/// Makes deep copy of zoo_op_result_t and removes chroot prefix from paths
static void convertOpResults(const std::vector<zoo_op_result_t> & op_results_native, OpResultsPtr & out_op_results,
const ZooKeeper * zookeeper = nullptr)
{
if (!out_op_results)
out_op_results = std::make_shared<OpResults>();
out_op_results->reserve(op_results_native.size());
for (const zoo_op_result_t & res_native : op_results_native)
out_op_results->emplace_back(res_native, zookeeper);
}
int32_t ZooKeeper::multiImpl(const Ops & ops_, OpResultsPtr * out_op_results, MultiTransactionInfo * out_info)
{
if (ops_.empty())
return ZOK;
@ -585,7 +593,7 @@ int32_t ZooKeeper::multiImpl(const Ops & ops_, OpResultsPtr * out_results_)
return ZINVALIDSTATE;
size_t count = ops_.size();
OpResultsPtr out_results(new OpResults(count));
std::vector<zoo_op_result_t> out_results_native(count);
/// Copy the struct containing pointers with default copy-constructor.
/// It is safe because it hasn't got a destructor.
@ -594,34 +602,31 @@ int32_t ZooKeeper::multiImpl(const Ops & ops_, OpResultsPtr * out_results_)
for (const auto & op : ops_)
ops.push_back(*(op->data));
int32_t code = zoo_multi(impl, static_cast<int>(ops.size()), ops.data(), out_results->data());
int32_t code = zoo_multi(impl, static_cast<int>(ops.size()), ops.data(), out_results_native.data());
ProfileEvents::increment(ProfileEvents::ZooKeeperMulti);
ProfileEvents::increment(ProfileEvents::ZooKeeperTransactions);
if (out_results_)
*out_results_ = out_results;
if (out_op_results || out_info)
{
OpResultsPtr op_results;
convertOpResults(out_results_native, op_results, this);
if (out_op_results)
*out_op_results = op_results;
if (out_info)
*out_info = MultiTransactionInfo(code, ops_, op_results);
}
return code;
}
OpResultsPtr ZooKeeper::multi(const Ops & ops)
{
OpResultsPtr results;
int code = tryMulti(ops, &results);
if (code != ZOK)
{
if (results && results->size() == ops.size())
{
for (size_t i = 0; i < ops.size(); ++i)
{
if (results->at(i).err == code)
throw KeeperException("Transaction failed at op #" + std::to_string(i) + ": " + ops[i]->describe(), code);
}
}
throw KeeperException(code);
}
return results;
OpResultsPtr op_results;
int code = multiImpl(ops, &op_results);
KeeperMultiException::check(code, ops, op_results);
return op_results;
}
int32_t ZooKeeper::tryMulti(const Ops & ops_, OpResultsPtr * out_results_)
@ -638,17 +643,9 @@ int32_t ZooKeeper::tryMulti(const Ops & ops_, OpResultsPtr * out_results_)
return code;
}
int32_t ZooKeeper::tryMultiUnsafe(const Ops & ops, MultiTransactionInfo & info)
{
info.code = multiImpl(ops, &info.op_results);
for (const OpPtr & op : ops)
info.ops.emplace_back(op->clone());
return info.code;
}
int32_t ZooKeeper::tryMultiWithRetries(const Ops & ops, OpResultsPtr * out_results, size_t * attempt)
{
int32_t code = retry(std::bind(&ZooKeeper::multiImpl, this, std::ref(ops), out_results), attempt);
int32_t code = retry(std::bind(&ZooKeeper::multiImpl, this, std::ref(ops), out_results, nullptr), attempt);
if (!(code == ZOK ||
code == ZNONODE ||
code == ZNODEEXISTS ||
@ -754,7 +751,7 @@ ZooKeeper::~ZooKeeper()
ZooKeeperPtr ZooKeeper::startNewSession() const
{
return std::make_shared<ZooKeeper>(hosts, identity, session_timeout_ms);
return std::make_shared<ZooKeeper>(hosts, identity, session_timeout_ms, chroot);
}
Op::Create::Create(const std::string & path_pattern_, const std::string & value_, ACLPtr acl_, int32_t flags_)
@ -981,26 +978,27 @@ ZooKeeper::TryRemoveFuture ZooKeeper::asyncTryRemove(const std::string & path, i
ZooKeeper::MultiFuture ZooKeeper::asyncMultiImpl(const zkutil::Ops & ops_, bool throw_exception)
{
size_t count = ops_.size();
OpResultsPtr results(new OpResults(count));
/// We need to hold all references to ops data until the end of multi callback
struct OpsHolder
{
std::shared_ptr<zkutil::Ops> ops_ptr = std::make_shared<zkutil::Ops>();
std::shared_ptr<std::vector<zoo_op_t>> ops_raw_ptr = std::make_shared<std::vector<zoo_op_t>>();
std::shared_ptr<zkutil::Ops> ops_ptr;
std::shared_ptr<std::vector<zoo_op_t>> ops_native;
std::shared_ptr<std::vector<zoo_op_result_t>> op_results_native;
} holder;
for (const auto & op : ops_)
{
holder.ops_ptr->emplace_back(op->clone());
holder.ops_raw_ptr->push_back(*holder.ops_ptr->back()->data);
}
/// Copy ops (swallow copy)
holder.ops_ptr = std::make_shared<zkutil::Ops>(ops_);
/// Copy native ops to contiguous vector
holder.ops_native = std::make_shared<std::vector<zoo_op_t>>();
for (const OpPtr & op : *holder.ops_ptr)
holder.ops_native->push_back(*op->data);
/// Allocate native result holders
holder.op_results_native = std::make_shared<std::vector<zoo_op_result_t>>(holder.ops_ptr->size());
MultiFuture future{ [throw_exception, results, holder] (int rc) {
MultiFuture future{ [throw_exception, holder, zookeeper=this] (int rc) {
OpResultsAndCode res;
res.code = rc;
res.results = results;
convertOpResults(*holder.op_results_native, res.results, zookeeper);
res.ops_ptr = holder.ops_ptr;
if (throw_exception && rc != ZOK)
throw zkutil::KeeperException(rc);
@ -1018,9 +1016,9 @@ ZooKeeper::MultiFuture ZooKeeper::asyncMultiImpl(const zkutil::Ops & ops_, bool
if (expired())
throw KeeperException(ZINVALIDSTATE);
auto & ops = *holder.ops_raw_ptr;
int32_t code = zoo_amulti(impl, static_cast<int>(ops.size()), ops.data(), results->data(),
int32_t code = zoo_amulti(impl, static_cast<int>(holder.ops_native->size()),
holder.ops_native->data(),
holder.op_results_native->data(),
[] (int rc, const void * data)
{
MultiFuture::TaskPtr owned_task =
@ -1069,4 +1067,58 @@ size_t getFailedOpIndex(const OpResultsPtr & op_results, int32_t transaction_ret
}
OpResult::OpResult(const zoo_op_result_t & op_result, const ZooKeeper * zookeeper)
: err(op_result.err)
{
if (op_result.value)
{
value = std::string(op_result.value, op_result.value + op_result.valuelen);
/// Current version of libzookeeper does not cut chroot path prefixes
/// We do it here manually
if (zookeeper && !zookeeper->chroot.empty())
{
if (startsWith(value, zookeeper->chroot))
value = value.substr(zookeeper->chroot.length());
else
throw DB::Exception("Expected ZooKeeper path with chroot " + zookeeper->chroot + ", got " + value,
DB::ErrorCodes::LOGICAL_ERROR);
}
}
if (op_result.stat)
stat = std::make_unique<Stat>(*op_result.stat);
}
KeeperMultiException::KeeperMultiException(const MultiTransactionInfo & info_, size_t failed_op_index_)
:KeeperException(
"Transaction failed at op #" + std::to_string(failed_op_index_) + ": " + info_.ops.at(failed_op_index_)->describe(),
info_.code),
info(info_) {}
void KeeperMultiException::check(int code, const Ops & ops, const OpResultsPtr & op_results)
{
if (code == ZOK) {}
else if (zkutil::isUserError(code))
throw KeeperMultiException(MultiTransactionInfo(code, ops, op_results), getFailedOpIndex(op_results, code));
else
throw KeeperException(code);
}
void KeeperMultiException::check(const MultiTransactionInfo & info)
{
if (info.code == ZOK) {}
else if (zkutil::isUserError(info.code))
throw KeeperMultiException(info, getFailedOpIndex(info.op_results, info.code));
else
throw KeeperException(info.code);
}
const Op & MultiTransactionInfo::getFailedOp() const
{
return *ops.at(getFailedOpIndex(op_results, code));
}
}

View File

@ -8,10 +8,10 @@
#include <memory>
#include <mutex>
#include <string>
#include <unistd.h>
#include <common/logger_useful.h>
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <port/unistd.h>
namespace ProfileEvents
@ -56,7 +56,7 @@ public:
using Ptr = std::shared_ptr<ZooKeeper>;
ZooKeeper(const std::string & hosts, const std::string & identity = "",
int32_t session_timeout_ms = DEFAULT_SESSION_TIMEOUT, bool check_root_exists = false);
int32_t session_timeout_ms = DEFAULT_SESSION_TIMEOUT, const std::string & chroot = "");
/** Config of the form:
<zookeeper>
@ -196,14 +196,16 @@ public:
/// Performs several operations in a transaction.
/// Throws on every error.
OpResultsPtr multi(const Ops & ops);
/// Throws only if some operation has returned an "unexpected" error
/// - an error that would cause the corresponding try- method to throw.
int32_t tryMulti(const Ops & ops, OpResultsPtr * out_results = nullptr);
/// Like previous one, but does not throw any ZooKeeper exceptions
int32_t tryMultiUnsafe(const Ops & ops, MultiTransactionInfo & info);
/// Use only with read-only operations.
int32_t tryMultiWithRetries(const Ops & ops, OpResultsPtr * out_results = nullptr, size_t * attempt = nullptr);
/// Throws nothing, just alias of multiImpl
int32_t tryMultiNoThrow(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr)
{
return multiImpl(ops, out_op_results, out_info);
}
Int64 getClientID();
@ -366,9 +368,10 @@ public:
private:
friend struct WatchContext;
friend class EphemeralNodeHolder;
friend struct OpResult;
void init(const std::string & hosts, const std::string & identity, int32_t session_timeout_ms, const std::string & chroot);
void init(const std::string & hosts, const std::string & identity,
int32_t session_timeout_ms, bool check_root_exists);
void removeChildrenRecursive(const std::string & path);
void tryRemoveChildrenRecursive(const std::string & path);
@ -391,7 +394,7 @@ private:
/// If the connection has been lost, wait timeout/3 hoping for connection re-establishment.
static const int MAX_SLEEP_TIME = 10;
if (code == ZCONNECTIONLOSS)
usleep(std::min(session_timeout_ms * 1000 / 3, MAX_SLEEP_TIME * 1000 * 1000));
usleep(std::min(session_timeout_ms * 1000u / 3, MAX_SLEEP_TIME * 1000u * 1000u));
LOG_WARNING(log, "Error on attempt " << i << ": " << error2string(code) << ". Retry");
code = operation();
@ -406,7 +409,7 @@ private:
int32_t getImpl(const std::string & path, std::string & res, Stat * stat, WatchCallback watch_callback);
int32_t setImpl(const std::string & path, const std::string & data, int32_t version = -1, Stat * stat = nullptr);
int32_t getChildrenImpl(const std::string & path, Strings & res, Stat * stat, WatchCallback watch_callback);
int32_t multiImpl(const Ops & ops, OpResultsPtr * out_results = nullptr);
int32_t multiImpl(const Ops & ops, OpResultsPtr * out_op_results = nullptr, MultiTransactionInfo * out_info = nullptr);
int32_t existsImpl(const std::string & path, Stat * stat_, WatchCallback watch_callback);
MultiFuture asyncMultiImpl(const zkutil::Ops & ops_, bool throw_exception);
@ -414,6 +417,7 @@ private:
std::string hosts;
std::string identity;
int32_t session_timeout_ms;
std::string chroot;
std::mutex mutex;
ACLPtr default_acl;
@ -493,41 +497,4 @@ private:
using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr;
/// Simple structure to handle transaction execution results
struct MultiTransactionInfo
{
MultiTransactionInfo() = default;
Ops ops;
int32_t code = ZOK;
OpResultsPtr op_results;
bool empty() const
{
return ops.empty();
}
bool hasFailedOp() const
{
return zkutil::isUserError(code);
}
const Op & getFailedOp() const
{
return *ops.at(getFailedOpIndex(op_results, code));
}
KeeperException getException() const
{
if (hasFailedOp())
{
size_t i = getFailedOpIndex(op_results, code);
return KeeperException("Transaction failed at op #" + std::to_string(i) + ": " + ops.at(i)->describe(), code);
}
else
return KeeperException(code);
}
};
}

View File

@ -6,6 +6,9 @@
#pragma GCC diagnostic push
#pragma GCC diagnostic ignored "-Wsign-compare"
#include <gtest/gtest.h>
#include <Common/ShellCommand.h>
#pragma GCC diagnostic pop
using namespace DB;
@ -32,32 +35,32 @@ TEST(zkutil, multi_nice_exception_msg)
zkutil::Ops ops;
ASSERT_NO_THROW(
zookeeper->tryRemoveRecursive("/clickhouse_test_zkutil_multi");
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
);
try
{
ops.clear();
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/c", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test_zkutil_multi/c", -1));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/a", "BadBoy", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/b", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/c", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi/c", -1));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "BadBoy", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/b", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
FAIL();
}
catch (...)
{
zookeeper->tryRemoveRecursive("/clickhouse_test_zkutil_multi");
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
String msg = getCurrentExceptionMessage(false);
bool msg_has_reqired_patterns = msg.find("/clickhouse_test_zkutil_multi/a") != std::string::npos && msg.find("#2") != std::string::npos;
bool msg_has_reqired_patterns = msg.find("/clickhouse_test/zkutil_multi/a") != std::string::npos && msg.find("#2") != std::string::npos;
EXPECT_TRUE(msg_has_reqired_patterns) << msg;
}
}
@ -69,7 +72,7 @@ TEST(zkutil, multi_async)
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
zookeeper->tryRemoveRecursive("/clickhouse_test_zkutil_multi");
zookeeper->tryRemoveRecursive("/clickhouse_test/zkutil_multi");
{
ops.clear();
@ -78,8 +81,8 @@ TEST(zkutil, multi_async)
{
ops.clear();
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/a", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "", acl, zkutil::CreateMode::Persistent));
auto fut = zookeeper->tryAsyncMulti(ops);
ops.clear();
@ -97,11 +100,11 @@ TEST(zkutil, multi_async)
for (size_t i = 0; i < 10000; ++i)
{
ops.clear();
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test_zkutil_multi", -1));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Check("/clickhouse_test_zkutil_multi", -1));
ops.emplace_back(new zkutil::Op::SetData("/clickhouse_test_zkutil_multi", "xxx", 42));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Remove("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", -1));
ops.emplace_back(new zkutil::Op::SetData("/clickhouse_test/zkutil_multi", "xxx", 42));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
futures.emplace_back(zookeeper->asyncMulti(ops));
}
@ -115,8 +118,8 @@ TEST(zkutil, multi_async)
{
ops.clear();
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test_zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi", "_", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(new zkutil::Op::Create("/clickhouse_test/zkutil_multi/a", "_", acl, zkutil::CreateMode::Persistent));
auto fut = zookeeper->tryAsyncMulti(ops);
ops.clear();
@ -128,31 +131,55 @@ TEST(zkutil, multi_async)
}
}
/// Run this test under sudo
TEST(zkutil, multi_async_libzookeeper_segfault)
{
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", 1000);
zkutil::Ops ops;
ops.emplace_back(new zkutil::Op::Check("/clickhouse_test/zkutil_multi", 0));
/// Uncomment to test
//auto cmd = ShellCommand::execute("sudo service zookeeper restart");
//cmd->wait();
auto future = zookeeper->asyncMulti(ops);
auto res = future.get();
EXPECT_TRUE(zkutil::isUnrecoverableErrorCode(res.code));
}
TEST(zkutil, multi_create_sequential)
{
try
{
/// Create chroot node firstly
auto zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181");
zookeeper->createAncestors("/clickhouse_test/");
zookeeper = std::make_unique<zkutil::ZooKeeper>("localhost:2181", "", zkutil::DEFAULT_SESSION_TIMEOUT, "/clickhouse_test");
auto acl = zookeeper->getDefaultACL();
zkutil::Ops ops;
String base_path = "/clickhouse_test/zkutil/multi_create_sequential";
String base_path = "/zkutil/multi_create_sequential";
zookeeper->tryRemoveRecursive(base_path);
zookeeper->createAncestors(base_path + "/");
String entry_path = base_path + "/queue-";
ops.emplace_back(new zkutil::Op::Create(entry_path, "", acl, zkutil::CreateMode::EphemeralSequential));
String sequential_node_prefix = base_path + "/queue-";
ops.emplace_back(new zkutil::Op::Create(sequential_node_prefix, "", acl, zkutil::CreateMode::EphemeralSequential));
zkutil::OpResultsPtr results = zookeeper->multi(ops);
zkutil::OpResult & result = results->at(0);
zkutil::OpResult & sequential_node_result_op = results->at(0);
EXPECT_TRUE(result.value != nullptr);
EXPECT_TRUE(startsWith(result.value, entry_path));
EXPECT_FALSE(sequential_node_result_op.value.empty());
EXPECT_GT(sequential_node_result_op.value.length(), sequential_node_prefix.length());
EXPECT_EQ(sequential_node_result_op.value.substr(0, sequential_node_prefix.length()), sequential_node_prefix);
}
catch (...)
{
std::cerr << getCurrentExceptionMessage(false);
throw;
}
}

View File

@ -34,11 +34,11 @@ int main(int argc, char ** argv)
{
{
zkutil::Ops ops;
ops.emplace_back(std::make_unique<zkutil::Op::Create>("/test/zk_expiration_test", "hello", zk.getDefaultACL(), zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_unique<zkutil::Op::Remove>("/test/zk_expiration_test", -1));
ops.emplace_back(std::make_shared<zkutil::Op::Create>("/test/zk_expiration_test", "hello", zk.getDefaultACL(), zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>("/test/zk_expiration_test", -1));
zkutil::MultiTransactionInfo info;
zk.tryMultiUnsafe(ops, info);
zk.tryMultiNoThrow(ops, nullptr, &info);
std::cout << time(nullptr) - time0 << "s: " << zkutil::ZooKeeper::error2string(info.code) << std::endl;
try

View File

@ -1,6 +1,6 @@
#include <Common/ZooKeeper/ZooKeeper.h>
#include <iostream>
#include <unistd.h>
#include <port/unistd.h>
using namespace zkutil;

View File

@ -12,7 +12,6 @@ namespace Poco
namespace DB
{
/** Lets you check if the address is similar to `localhost`.
* The purpose of this check is usually to make an assumption,
* that when we go to this address via the Internet, we'll get to ourselves.

View File

@ -1,13 +1,10 @@
#include <sys/stat.h>
#include <unistd.h>
#include <string>
#include <iostream>
#include <Poco/DirectoryIterator.h>
#include <Poco/File.h>
#include <Common/Exception.h>
#include <port/unistd.h>
namespace DB

View File

@ -1,12 +1,10 @@
#include <time.h>
#include <unistd.h>
#include <port/unistd.h>
#include <sys/types.h>
#include <Common/Exception.h>
#include <Common/randomSeed.h>
#include <Common/SipHash.h>
#include <Core/Types.h>
#ifdef __APPLE__
#include <common/apple_rt.h>
#endif

View File

@ -7,15 +7,13 @@
#include <Common/CompactArray.h>
#include <IO/WriteBufferFromFile.h>
#include <IO/ReadBufferFromFile.h>
#include <boost/filesystem.hpp>
#include <string>
#include <iostream>
#include <fstream>
#include <stdexcept>
#include <unistd.h>
#include <cstdlib>
#include <port/unistd.h>
namespace fs = boost::filesystem;

View File

@ -60,6 +60,7 @@
#define DBMS_MIN_REVISION_WITH_QUOTA_KEY_IN_CLIENT_INFO 54060
#define DBMS_MIN_REVISION_WITH_TABLES_STATUS 54226
#define DBMS_MIN_REVISION_WITH_TIME_ZONE_PARAMETER_IN_DATETIME_DATA_TYPE 54337
#define DBMS_MIN_REVISION_WITH_SERVER_DISPLAY_NAME 54372
/// Version of ClickHouse TCP protocol. Set to git tag with latest protocol change.
#define DBMS_TCP_PROTOCOL_VERSION 54226
@ -74,8 +75,18 @@
#define DEFAULT_HTTP_READ_BUFFER_TIMEOUT 1800
#define DEFAULT_HTTP_READ_BUFFER_CONNECTION_TIMEOUT 1
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))
// more aliases: https://mailman.videolan.org/pipermail/x264-devel/2014-May/010660.html
#if defined(_MSC_VER)
#define ALWAYS_INLINE __forceinline
#define NO_INLINE static __declspec(noinline)
#define MAY_ALIAS
#else
#define ALWAYS_INLINE __attribute__((__always_inline__))
#define NO_INLINE __attribute__((__noinline__))
#define MAY_ALIAS __attribute__((__may_alias__))
#endif
#define PLATFORM_NOT_SUPPORTED "The only supported platforms are x86_64 and AArch64 (work in progress)"

View File

@ -198,14 +198,14 @@ public:
template <typename T> T & get()
{
using TWithoutRef = std::remove_reference_t<T>;
TWithoutRef * __attribute__((__may_alias__)) ptr = reinterpret_cast<TWithoutRef*>(&storage);
TWithoutRef * MAY_ALIAS ptr = reinterpret_cast<TWithoutRef*>(&storage);
return *ptr;
};
template <typename T> const T & get() const
{
using TWithoutRef = std::remove_reference_t<T>;
const TWithoutRef * __attribute__((__may_alias__)) ptr = reinterpret_cast<const TWithoutRef*>(&storage);
const TWithoutRef * MAY_ALIAS ptr = reinterpret_cast<const TWithoutRef*>(&storage);
return *ptr;
};
@ -340,7 +340,7 @@ private:
void createConcrete(T && x)
{
using JustT = std::decay_t<T>;
JustT * __attribute__((__may_alias__)) ptr = reinterpret_cast<JustT *>(&storage);
JustT * MAY_ALIAS ptr = reinterpret_cast<JustT *>(&storage);
new (ptr) JustT(std::forward<T>(x));
which = TypeToEnum<JustT>::value;
}
@ -350,7 +350,7 @@ private:
void assignConcrete(T && x)
{
using JustT = std::decay_t<T>;
JustT * __attribute__((__may_alias__)) ptr = reinterpret_cast<JustT *>(&storage);
JustT * MAY_ALIAS ptr = reinterpret_cast<JustT *>(&storage);
*ptr = std::forward<T>(x);
}
@ -398,7 +398,7 @@ private:
void create(const char * data, size_t size)
{
String * __attribute__((__may_alias__)) ptr = reinterpret_cast<String*>(&storage);
String * MAY_ALIAS ptr = reinterpret_cast<String*>(&storage);
new (ptr) String(data, size);
which = Types::String;
}
@ -434,7 +434,7 @@ private:
template <typename T>
void destroy()
{
T * __attribute__((__may_alias__)) ptr = reinterpret_cast<T*>(&storage);
T * MAY_ALIAS ptr = reinterpret_cast<T*>(&storage);
ptr->~T();
}
};

View File

@ -138,4 +138,14 @@ NamesAndTypesList NamesAndTypesList::addTypes(const Names & names) const
return res;
}
bool NamesAndTypesList::contains(const String & name) const
{
for (const NameAndTypePair & column : *this)
{
if (column.name == name)
return true;
}
return false;
}
}

View File

@ -69,6 +69,8 @@ public:
/// Unlike `filter`, returns columns in the order in which they go in `names`.
NamesAndTypesList addTypes(const Names & names) const;
bool contains(const String & name) const;
};
}

View File

@ -44,7 +44,7 @@ std::ostream & operator<<(std::ostream & stream, const IDataType & what)
std::ostream & operator<<(std::ostream & stream, const IStorage & what)
{
stream << "IStorage(name = " << what.getName() << ", tableName = " << what.getTableName() << ") {"
<< what.getColumnsList().toString()
<< what.getColumns().getAllPhysical().toString()
<< "}";
// isRemote supportsSampling supportsFinal supportsPrewhere
return stream;

View File

@ -1,6 +1,5 @@
#include <sys/ioctl.h>
#include <unistd.h>
#include <port/unistd.h>
#include <DataStreams/PrettyBlockOutputStream.h>
#include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h>

View File

@ -138,7 +138,7 @@ void RemoteBlockInputStream::sendExternalTables()
{
StoragePtr cur = table.second;
QueryProcessingStage::Enum stage = QueryProcessingStage::Complete;
BlockInputStreams input = cur->read(cur->getColumnNamesList(), {}, context,
BlockInputStreams input = cur->read(cur->getColumns().getNamesOfPhysical(), {}, 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

@ -104,8 +104,8 @@ int main(int, char **)
/// create an object of an existing hit log table
StoragePtr table = StorageLog::create("./", "HitLog", names_and_types_list,
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
StoragePtr table = StorageLog::create(
"./", "HitLog", ColumnsDescription{names_and_types_list}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
table->startup();
/// read from it, apply the expression, filter, and write in tsv form to the console

View File

@ -95,8 +95,8 @@ try
/// create an object of an existing hit log table
StoragePtr table = StorageLog::create("./", "HitLog", names_and_types_list,
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
StoragePtr table = StorageLog::create(
"./", "HitLog", ColumnsDescription{names_and_types_list}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
table->startup();
/// read from it

View File

@ -107,8 +107,8 @@ try
/// create an object of an existing hit log table
StoragePtr table = StorageLog::create("./", "HitLog", names_and_types_list,
NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
StoragePtr table = StorageLog::create(
"./", "HitLog", ColumnsDescription{names_and_types_list}, DEFAULT_MAX_COMPRESS_BLOCK_SIZE);
table->startup();
/// read from it, sort it, and write it in tsv form to the console

View File

@ -41,8 +41,7 @@ Tables DatabaseDictionary::loadTables()
{
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
tables[name] = StorageDictionary::create(name,
columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, dictionary_structure, name);
tables[name] = StorageDictionary::create(name, ColumnsDescription{columns}, dictionary_structure, name);
}
}
@ -76,8 +75,7 @@ StoragePtr DatabaseDictionary::tryGetTable(
{
const DictionaryStructure & dictionary_structure = dict_ptr->getStructure();
auto columns = StorageDictionary::getNamesAndTypes(dictionary_structure);
return StorageDictionary::create(table_name,
columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{}, dictionary_structure, table_name);
return StorageDictionary::create(table_name, ColumnsDescription{columns}, dictionary_structure, table_name);
}
}
}
@ -142,10 +140,7 @@ void DatabaseDictionary::renameTable(
void DatabaseDictionary::alterTable(
const Context &,
const String &,
const NamesAndTypesList &,
const NamesAndTypesList &,
const NamesAndTypesList &,
const ColumnDefaults &,
const ColumnsDescription &,
const ASTModifier &)
{
throw Exception("DatabaseDictionary: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);

View File

@ -79,10 +79,7 @@ public:
void alterTable(
const Context & context,
const String & name,
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults,
const ColumnsDescription & columns,
const ASTModifier & engine_modifier) override;
time_t getTableMetadataModificationTime(

View File

@ -105,10 +105,7 @@ void DatabaseMemory::renameTable(
void DatabaseMemory::alterTable(
const Context &,
const String &,
const NamesAndTypesList &,
const NamesAndTypesList &,
const NamesAndTypesList &,
const ColumnDefaults &,
const ColumnsDescription &,
const ASTModifier &)
{
throw Exception("DatabaseMemory: alterTable() is not supported", ErrorCodes::NOT_IMPLEMENTED);

View File

@ -70,10 +70,7 @@ public:
void alterTable(
const Context & context,
const String & name,
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults,
const ColumnsDescription & columns,
const ASTModifier & engine_modifier) override;
time_t getTableMetadataModificationTime(

View File

@ -462,10 +462,7 @@ void DatabaseOrdinary::drop()
void DatabaseOrdinary::alterTable(
const Context & context,
const String & name,
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults,
const ColumnsDescription & columns,
const ASTModifier & storage_modifier)
{
/// Read the definition of the table and replace the necessary parts with new ones.
@ -486,7 +483,7 @@ void DatabaseOrdinary::alterTable(
ASTCreateQuery & ast_create_query = typeid_cast<ASTCreateQuery &>(*ast);
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns, materialized_columns, alias_columns, column_defaults);
ASTPtr new_columns = InterpreterCreateQuery::formatColumns(columns);
ast_create_query.replace(ast_create_query.columns, new_columns);
if (storage_modifier)

View File

@ -45,10 +45,7 @@ public:
void alterTable(
const Context & context,
const String & name,
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults,
const ColumnsDescription & columns,
const ASTModifier & engine_modifier) override;
time_t getTableMetadataModificationTime(

View File

@ -67,7 +67,7 @@ std::pair<String, StoragePtr> createTableFromDefinition(
if (!ast_create_query.columns)
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
InterpreterCreateQuery::ColumnsInfo columns_info = InterpreterCreateQuery::getColumnsInfo(*ast_create_query.columns, context);
ColumnsDescription columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns, context);
return
{
@ -75,7 +75,7 @@ std::pair<String, StoragePtr> createTableFromDefinition(
StorageFactory::instance().get(
ast_create_query,
database_data_path, ast_create_query.table, database_name, context, context.getGlobalContext(),
columns_info.columns, columns_info.materialized_columns, columns_info.alias_columns, columns_info.column_defaults,
columns,
true, has_force_restore_data_flag)
};
}

View File

@ -2,7 +2,7 @@
#include <Core/Types.h>
#include <Core/NamesAndTypes.h>
#include <Storages/ColumnDefault.h>
#include <Storages/ColumnsDescription.h>
#include <ctime>
#include <memory>
#include <functional>
@ -113,10 +113,7 @@ public:
virtual void alterTable(
const Context & context,
const String & name,
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults,
const ColumnsDescription & columns,
const ASTModifier & engine_modifier) = 0;
/// Returns time of table's metadata change, 0 if there is no corresponding metadata file.

View File

@ -8,7 +8,6 @@
#include <Common/ProfileEvents.h>
#include <Common/CurrentMetrics.h>
#include <ext/range.h>
#include <ext/scope_guard.h>
#include <ext/map.h>

View File

@ -6,6 +6,7 @@
#include <common/logger_useful.h>
#include <ext/bit_cast.h>
#include <ext/range.h>
#include <ext/scope_guard.h>
namespace DB
{
@ -70,11 +71,11 @@ namespace
Block dataToBlock(const Block & sample_block, const void * data)
{
if (!data)
return sample_block.cloneEmpty();
throw Exception("LibraryDictionarySource: No data returned", ErrorCodes::EXTERNAL_LIBRARY_ERROR);
auto columns_received = static_cast<const ClickHouseLibrary::Table *>(data);
if (columns_received->error_code)
throw Exception("Received error: " + std::to_string(columns_received->error_code) + " "
throw Exception("LibraryDictionarySource: Returned error: " + std::to_string(columns_received->error_code) + " "
+ (columns_received->error_string ? columns_received->error_string : ""),
ErrorCodes::EXTERNAL_LIBRARY_ERROR);
@ -85,7 +86,7 @@ namespace
for (size_t col_n = 0; col_n < columns_received->size; ++col_n)
{
if (columns.size() != columns_received->data[col_n].size)
throw Exception("Received unexpected number of columns: " + std::to_string(columns_received->data[col_n].size)
throw Exception("LibraryDictionarySource: Returned unexpected number of columns: " + std::to_string(columns_received->data[col_n].size)
+ ", must be " + std::to_string(columns.size()),
ErrorCodes::SIZES_OF_COLUMNS_DOESNT_MATCH);
@ -123,9 +124,8 @@ LibraryDictionarySource::LibraryDictionarySource(const DictionaryStructure & dic
description.init(sample_block);
library = std::make_shared<SharedLibrary>(path);
settings = std::make_shared<CStringsHolder>(getLibSettings(config, config_prefix + lib_config_settings));
auto fptr = library->tryGet<void * (*)()>("ClickHouseDictionary_v2_libNew");
if (fptr)
lib_data = fptr();
if (auto libNew = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings))>("ClickHouseDictionary_v2_libNew"))
lib_data = libNew(&settings->strings);
}
LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource & other)
@ -138,15 +138,17 @@ LibraryDictionarySource::LibraryDictionarySource(const LibraryDictionarySource &
, library{other.library}
, description{other.description}
, settings{other.settings}
, lib_data{other.lib_data}
{
if (auto libClone = library->tryGet<decltype(lib_data) (*)(decltype(other.lib_data))>("ClickHouseDictionary_v2_libClone"))
lib_data = libClone(other.lib_data);
else if (auto libNew = library->tryGet<decltype(lib_data) (*)(decltype(&settings->strings))>("ClickHouseDictionary_v2_libNew"))
lib_data = libNew(&settings->strings);
}
LibraryDictionarySource::~LibraryDictionarySource()
{
auto fptr = library->tryGet<void * (*)(void *)>("ClickHouseDictionary_v2_libDelete");
if (fptr)
fptr(lib_data);
if (auto libDelete = library->tryGet<void (*)(decltype(lib_data))>("ClickHouseDictionary_v2_libDelete"))
libDelete(lib_data);
}
BlockInputStreamPtr LibraryDictionarySource::loadAll()
@ -165,12 +167,12 @@ BlockInputStreamPtr LibraryDictionarySource::loadAll()
void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw.
auto fptr
auto func_loadAll
= library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&columns))>("ClickHouseDictionary_v2_loadAll");
data_ptr = library->get<void * (*)(void *)>("ClickHouseDictionary_v2_dataNew")(lib_data);
auto data = fptr(data_ptr, &settings->strings, &columns);
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v2_dataNew")(lib_data);
auto data = func_loadAll(data_ptr, &settings->strings, &columns);
auto block = dataToBlock(description.sample_block, data);
library->get<void (*)(void *, void *)>("ClickHouseDictionary_v2_dataDelete")(lib_data, data_ptr);
SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v2_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block);
}
@ -191,12 +193,13 @@ BlockInputStreamPtr LibraryDictionarySource::loadIds(const std::vector<UInt64> &
void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw.
auto fptr = library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&columns_pass), decltype(&ids_data))>(
"ClickHouseDictionary_v2_loadIds");
data_ptr = library->get<void * (*)(void *)>("ClickHouseDictionary_v2_dataNew")(lib_data);
auto data = fptr(data_ptr, &settings->strings, &columns_pass, &ids_data);
auto func_loadIds
= library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&columns_pass), decltype(&ids_data))>(
"ClickHouseDictionary_v2_loadIds");
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v2_dataNew")(lib_data);
auto data = func_loadIds(data_ptr, &settings->strings, &columns_pass, &ids_data);
auto block = dataToBlock(description.sample_block, data);
library->get<void (*)(void *, void *)>("ClickHouseDictionary_v2_dataDelete")(lib_data, data_ptr);
SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v2_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block);
}
@ -218,30 +221,29 @@ BlockInputStreamPtr LibraryDictionarySource::loadKeys(const Columns & key_column
void * data_ptr = nullptr;
/// Get function pointer before dataNew call because library->get may throw.
auto fptr
auto func_loadKeys
= library->get<void * (*)(decltype(data_ptr), decltype(&settings->strings), decltype(&columns_pass), decltype(&requested_rows_c))>(
"ClickHouseDictionary_v2_loadKeys");
data_ptr = library->get<void * (*)(void *)>("ClickHouseDictionary_v2_dataNew")(lib_data);
auto data = fptr(data_ptr, &settings->strings, &columns_pass, &requested_rows_c);
data_ptr = library->get<decltype(data_ptr) (*)(decltype(lib_data))>("ClickHouseDictionary_v2_dataNew")(lib_data);
auto data = func_loadKeys(data_ptr, &settings->strings, &columns_pass, &requested_rows_c);
auto block = dataToBlock(description.sample_block, data);
library->get<void (*)(void *, void *)>("ClickHouseDictionary_v2_dataDelete")(lib_data, data_ptr);
SCOPE_EXIT(library->get<void (*)(decltype(lib_data), decltype(data_ptr))>("ClickHouseDictionary_v2_dataDelete")(lib_data, data_ptr));
return std::make_shared<OneBlockInputStream>(block);
}
bool LibraryDictionarySource::isModified() const
{
auto fptr = library->tryGet<void * (*)(decltype(lib_data), decltype(&settings->strings))>("ClickHouseDictionary_v2_isModified");
if (fptr)
return fptr(lib_data, &settings->strings);
if (auto func_isModified
= library->tryGet<bool (*)(decltype(lib_data), decltype(&settings->strings))>("ClickHouseDictionary_v2_isModified"))
return func_isModified(lib_data, &settings->strings);
return true;
}
bool LibraryDictionarySource::supportsSelectiveLoad() const
{
auto fptr
= library->tryGet<void * (*)(decltype(lib_data), decltype(&settings->strings))>("ClickHouseDictionary_v2_supportsSelectiveLoad");
if (fptr)
return fptr(lib_data, &settings->strings);
if (auto func_supportsSelectiveLoad
= library->tryGet<bool (*)(decltype(lib_data), decltype(&settings->strings))>("ClickHouseDictionary_v2_supportsSelectiveLoad"))
return func_supportsSelectiveLoad(lib_data, &settings->strings);
return true;
}

View File

@ -1,26 +1,25 @@
#pragma once
#include <Common/SharedLibrary.h>
#include <Dictionaries/DictionaryStructure.h>
#include <Dictionaries/ExternalResultDescription.h>
#include <Dictionaries/IDictionarySource.h>
#include <Common/SharedLibrary.h>
#include <common/LocalDateTime.h>
namespace Poco
{
class Logger;
class Logger;
namespace Util
{
class AbstractConfiguration;
}
namespace Util
{
class AbstractConfiguration;
}
}
namespace DB
{
class CStringsHolder;
/// Allows loading dictionaries from dynamic libraries (.so)
@ -55,7 +54,10 @@ public:
bool supportsSelectiveLoad() const override;
///Not yet supported
bool hasUpdateField() const override { return false; }
bool hasUpdateField() const override
{
return false;
}
DictionarySourcePtr clone() const override;
@ -76,5 +78,4 @@ private:
std::shared_ptr<CStringsHolder> settings;
void * lib_data = nullptr;
};
}

View File

@ -76,20 +76,19 @@ public:
private:
size_t getNumberOfArguments() const override { return 2; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments[0]->isString())
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataTypeUInt64>(arguments[1].get()) &&
!checkDataType<DataTypeTuple>(arguments[1].get()))
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
}
@ -100,9 +99,21 @@ private:
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
/** Do not require existence of the dictionary if the function is called for empty block.
* This is needed to allow successful query analysis on a server,
* that is the initiator of a distributed query,
* in the case when the function will be invoked for real data only at the remote servers.
* This feature is controversial and implemented specially
* for backward compatibility with the case in Yandex Banner System.
*/
if (block.rows() == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
return;
}
auto dict = dictionaries.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
@ -113,9 +124,7 @@ private:
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -135,19 +144,8 @@ private:
dict->has(ids, out->getData());
block.getByPosition(result).column = std::move(out);
}
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
{
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
PaddedPODArray<UInt8> out(1);
dict->has(ids, out);
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(id_col->size(), toField(out.front()));
}
else
throw Exception{
"Second argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -161,11 +159,7 @@ private:
return false;
const ColumnWithTypeAndName & key_col_with_type = block.getByPosition(arguments[1]);
ColumnPtr key_col = key_col_with_type.column;
/// Functions in external dictionaries only support full-value (not constant) columns with keys.
if (ColumnPtr key_col_materialized = key_col_with_type.column->convertToFullColumnIfConst())
key_col = key_col_materialized;
const ColumnPtr & key_col = key_col_with_type.column;
if (checkColumn<ColumnTuple>(key_col.get()))
{
@ -177,9 +171,7 @@ private:
block.getByPosition(result).column = std::move(out);
}
else
throw Exception{
"Second argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};
throw Exception{"Second argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH};
return true;
}
@ -195,15 +187,11 @@ static bool isDictGetFunctionInjective(const ExternalDictionaries & dictionaries
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(0).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function dictGet... must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(sample_block.getByPosition(1).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function dictGet... must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function dictGet... must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
return dictionaries.getDictionary(dict_name_col->getValue<String>())->isInjective(attr_name_col->getValue<String>());
}
@ -227,6 +215,9 @@ private:
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
bool isInjective(const Block & sample_block) override
{
return isDictGetFunctionInjective(dictionaries, sample_block);
@ -235,42 +226,33 @@ private:
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments.size() != 3 && arguments.size() != 4)
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3 or 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3 or 4.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!arguments[0]->isString())
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!arguments[1]->isString())
{
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
!checkDataType<DataTypeTuple>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
/// This is for the case of range dictionaries.
if (arguments.size() == 4 && !checkDataType<DataTypeDate>(arguments[3].get()))
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return std::make_shared<DataTypeString>();
@ -282,9 +264,14 @@ private:
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
return;
}
auto dict = dictionaries.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
@ -296,9 +283,7 @@ private:
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -310,16 +295,12 @@ private:
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -330,19 +311,8 @@ private:
dict->getString(attr_name, id_col->getData(), out.get());
block.getByPosition(result).column = std::move(out);
}
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
{
const PaddedPODArray<UInt64> ids(1, id_col->getValue<UInt64>());
auto out = ColumnString::create();
dict->getString(attr_name, ids, out.get());
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
}
else
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -356,16 +326,12 @@ private:
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -386,9 +352,7 @@ private:
block.getByPosition(result).column = std::move(out);
}
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};
throw Exception{"Third argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH};
return true;
}
@ -402,16 +366,12 @@ private:
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -423,9 +383,7 @@ private:
executeRange(block, result, dict, attr_name, id_col, date_col_untyped);
else
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -451,9 +409,7 @@ private:
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be Date", ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -480,9 +436,7 @@ private:
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be Date", ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -507,34 +461,29 @@ public:
private:
size_t getNumberOfArguments() const override { return 4; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments[0]->isString())
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() +
", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() +
", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!arguments[1]->isString())
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() +
", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() +
", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
!checkDataType<DataTypeTuple>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!arguments[3]->isString())
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() +
", must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() +
", must be String.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeString>();
}
@ -545,9 +494,14 @@ private:
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
return;
}
auto dict = dictionaries.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
@ -558,9 +512,7 @@ private:
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -573,9 +525,7 @@ private:
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -585,9 +535,7 @@ private:
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -617,9 +565,7 @@ private:
block.getByPosition(result).column = std::move(out);
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -648,9 +594,7 @@ private:
block.getByPosition(result).column = DataTypeString().createColumnConst(id_col->size(), out->getDataAt(0).toString());
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -663,9 +607,7 @@ private:
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -692,9 +634,7 @@ private:
dict->getString(attr_name, key_columns, key_types, def, out.get());
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be String", ErrorCodes::ILLEGAL_COLUMN};
block.getByPosition(result).column = std::move(out);
return true;
@ -781,6 +721,9 @@ private:
bool isVariadic() const override { return true; }
size_t getNumberOfArguments() const override { return 0; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
bool isInjective(const Block & sample_block) override
{
return isDictGetFunctionInjective(dictionaries, sample_block);
@ -792,37 +735,21 @@ private:
throw Exception{"Function " + getName() + " takes 3 or 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!arguments[0]->isString())
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!arguments[1]->isString())
{
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
!checkDataType<DataTypeTuple>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (arguments.size() == 4 && !checkDataType<DataTypeDate>(arguments[3].get()))
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataType>();
}
@ -833,9 +760,14 @@ private:
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
return;
}
auto dict = dictionaries.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
@ -847,9 +779,7 @@ private:
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -861,16 +791,12 @@ private:
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -891,11 +817,7 @@ private:
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
}
else
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -909,16 +831,12 @@ private:
return false;
if (arguments.size() != 3)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -940,9 +858,7 @@ private:
block.getByPosition(result).column = std::move(out);
}
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH};
throw Exception{"Third argument of function " + getName() + " must be " + dict->getKeyDescription(), ErrorCodes::TYPE_MISMATCH};
return true;
}
@ -956,16 +872,12 @@ private:
return false;
if (arguments.size() != 4)
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -976,11 +888,7 @@ private:
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
executeRange(block, result, dict, attr_name, id_col, date_col_untyped);
else
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -1013,11 +921,7 @@ private:
block.getByPosition(result).column = std::move(out);
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Fourth argument of function " + getName() + " must be Date", ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -1045,11 +949,7 @@ private:
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
}
else
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Fourth argument of function " + getName() + " must be Date", ErrorCodes::ILLEGAL_COLUMN};
}
const ExternalDictionaries & dictionaries;
@ -1104,40 +1004,27 @@ public:
private:
size_t getNumberOfArguments() const override { return 4; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0, 1}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments[0]->isString())
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!arguments[1]->isString())
{
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataTypeUInt64>(arguments[2].get()) &&
!checkDataType<DataTypeTuple>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataType>(arguments[3].get()))
{
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be " + String(DataType{}.getFamilyName()) + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be " + String(DataType{}.getFamilyName()) + ".", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataType>();
}
@ -1148,9 +1035,14 @@ private:
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
return;
}
auto dict = dictionaries.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
@ -1161,9 +1053,7 @@ private:
!executeDispatchComplex<ComplexKeyHashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatchComplex<TrieDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -1176,9 +1066,7 @@ private:
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -1188,9 +1076,7 @@ private:
else if (const auto id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(id_col_untyped))
executeDispatch(block, arguments, result, dict, attr_name, id_col);
else
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Third argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -1223,9 +1109,7 @@ private:
block.getByPosition(result).column = std::move(out);
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()), ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -1257,9 +1141,7 @@ private:
block.getByPosition(result).column = DataTypeNumber<Type>().createColumnConst(id_col->size(), toField(data.front()));
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()), ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -1272,9 +1154,7 @@ private:
const auto attr_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Second argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
String attr_name = attr_name_col->getValue<String>();
@ -1308,9 +1188,7 @@ private:
DictGetTraits<DataType>::getOrDefault(dict, attr_name, key_columns, key_types, def, data);
}
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Fourth argument of function " + getName() + " must be " + String(DataType{}.getFamilyName()), ErrorCodes::ILLEGAL_COLUMN};
block.getByPosition(result).column = std::move(out);
return true;
@ -1368,23 +1246,18 @@ private:
size_t getNumberOfArguments() const override { return 2; }
bool isInjective(const Block & /*sample_block*/) override { return true; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments[0]->isString())
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataTypeUInt64>(arguments[1].get()))
{
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeArray>(std::make_shared<DataTypeUInt64>());
}
@ -1395,9 +1268,14 @@ private:
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
return;
}
auto dict = dictionaries.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
@ -1405,9 +1283,7 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr) &&
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -1419,9 +1295,7 @@ private:
return false;
if (!dict->hasHierarchy())
throw Exception{
"Dictionary does not have a hierarchy",
ErrorCodes::UNSUPPORTED_METHOD};
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
const auto get_hierarchies = [&] (const PaddedPODArray<UInt64> & in, PaddedPODArray<UInt64> & out, PaddedPODArray<UInt64> & offsets)
{
@ -1502,11 +1376,7 @@ private:
block.getByPosition(result).column = block.getByPosition(result).type->createColumnConst(id_col->size(), (*array)[0].get<Array>());
}
else
{
throw Exception{
"Second argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Second argument of function " + getName() + " must be UInt64", ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -1532,31 +1402,22 @@ public:
private:
size_t getNumberOfArguments() const override { return 3; }
bool useDefaultImplementationForConstants() const final { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return {0}; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (!arguments[0]->isString())
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataTypeUInt64>(arguments[1].get()))
{
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!checkDataType<DataTypeUInt64>(arguments[2].get()))
{
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
throw Exception{"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_shared<DataTypeUInt8>();
}
@ -1567,9 +1428,14 @@ private:
{
const auto dict_name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"First argument of function " + getName() + " must be a constant string", ErrorCodes::ILLEGAL_COLUMN};
if (block.rows() == 0)
{
auto & elem = block.getByPosition(result);
elem.column = elem.type->createColumn();
return;
}
auto dict = dictionaries.getDictionary(dict_name_col->getValue<String>());
const auto dict_ptr = dict.get();
@ -1577,9 +1443,7 @@ private:
if (!executeDispatch<FlatDictionary>(block, arguments, result, dict_ptr)
&& !executeDispatch<HashedDictionary>(block, arguments, result, dict_ptr)
&& !executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE};
throw Exception{"Unsupported dictionary type " + dict_ptr->getTypeName(), ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -1591,9 +1455,7 @@ private:
return false;
if (!dict->hasHierarchy())
throw Exception{
"Dictionary does not have a hierarchy",
ErrorCodes::UNSUPPORTED_METHOD};
throw Exception{"Dictionary does not have a hierarchy", ErrorCodes::UNSUPPORTED_METHOD};
const auto child_id_col_untyped = block.getByPosition(arguments[1]).column.get();
const auto ancestor_id_col_untyped = block.getByPosition(arguments[2]).column.get();
@ -1603,10 +1465,8 @@ private:
else if (const auto child_id_col = checkAndGetColumnConst<ColumnVector<UInt64>>(child_id_col_untyped))
execute(block, result, dict, child_id_col, ancestor_id_col_untyped);
else
throw Exception{
"Illegal column " + child_id_col_untyped->getName()
+ " of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Illegal column " + child_id_col_untyped->getName()
+ " of second argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -1643,10 +1503,8 @@ private:
}
else
{
throw Exception{
"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -1679,12 +1537,8 @@ private:
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(child_id_col->size(), UInt64(res));
}
else
{
throw Exception{
"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN};
}
throw Exception{"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(), ErrorCodes::ILLEGAL_COLUMN};
return true;
}

View File

@ -1750,9 +1750,8 @@ void FunctionHasColumnInTable::executeImpl(Block & block, const ColumnNumbers &
{
std::vector<std::vector<String>> host_names = {{ host_name }};
auto cluster = std::make_shared<Cluster>(global_context.getSettings(), host_names, !user_name.empty() ? user_name : "default", password, global_context.getTCPPort(), false);
auto names_and_types_list = getStructureOfRemoteTable(*cluster, database_name, table_name, global_context);
const auto & names = names_and_types_list.getNames();
has_column = std::find(names.begin(), names.end(), column_name) != names.end();
auto remote_columns = getStructureOfRemoteTable(*cluster, database_name, table_name, global_context);
has_column = remote_columns.hasPhysical(column_name);
}
block.getByPosition(result).column = DataTypeUInt8().createColumnConst(block.rows(), UInt64(has_column));

View File

@ -15,7 +15,8 @@ namespace DB
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ILLEGAL_COLUMN;
}
namespace
@ -127,7 +128,7 @@ NullPresence getNullPresense(const ColumnsWithTypeAndName & args)
bool allArgumentsAreConstants(const Block & block, const ColumnNumbers & args)
{
for (auto arg : args)
if (!typeid_cast<const ColumnConst *>(block.getByPosition(arg).column.get()))
if (!block.getByPosition(arg).column->isColumnConst())
return false;
return true;
}
@ -135,11 +136,16 @@ bool allArgumentsAreConstants(const Block & block, const ColumnNumbers & args)
bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result)
{
ColumnNumbers arguments_to_remain_constants = getArgumentsThatAreAlwaysConstant();
/// Check that these arguments are really constant.
for (auto arg_num : arguments_to_remain_constants)
if (arg_num < args.size() && !block.getByPosition(args[arg_num]).column->isColumnConst())
throw Exception("Argument at index " + toString(arg_num) + " for function " + getName() + " must be constant", ErrorCodes::ILLEGAL_COLUMN);
if (args.empty() || !useDefaultImplementationForConstants() || !allArgumentsAreConstants(block, args))
return false;
ColumnNumbers arguments_to_remain_constants = getArgumentsThatAreAlwaysConstant();
Block temporary_block;
bool have_converted_columns = false;
@ -162,7 +168,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
*/
if (!have_converted_columns)
throw Exception("Number of arguments for function " + getName() + " doesn't match: the function requires more arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
temporary_block.insert(block.getByPosition(result));

View File

@ -1,4 +1,4 @@
#if !(defined(__FreeBSD__) || defined(__APPLE__))
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#include <IO/ReadBufferAIO.h>
#include <Common/ProfileEvents.h>

View File

@ -1,6 +1,6 @@
#pragma once
#if !(defined(__FreeBSD__) || defined(__APPLE__))
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadBuffer.h>
@ -8,7 +8,6 @@
#include <Core/Defines.h>
#include <Common/AIO.h>
#include <Common/CurrentMetrics.h>
#include <string>
#include <limits>
#include <unistd.h>

View File

@ -1,14 +1,10 @@
#include <errno.h>
#include <time.h>
#include <optional>
#include <Common/ProfileEvents.h>
#include <Common/Stopwatch.h>
#include <Common/Exception.h>
#include <Common/CurrentMetrics.h>
#include <IO/ReadBufferFromFileDescriptor.h>
#include <IO/WriteHelpers.h>

View File

@ -1,7 +1,6 @@
#pragma once
#include <unistd.h>
#include <port/unistd.h>
#include <IO/ReadBufferFromFileBase.h>
#include <IO/ReadBuffer.h>

View File

@ -52,6 +52,8 @@ inline char parseEscapeSequence(char c)
return '\a';
case 'b':
return '\b';
case 'e':
return '\x1B'; /// \e escape sequence is non standard for C and C++ but supported by gcc and clang.
case 'f':
return '\f';
case 'n':

View File

@ -1,4 +1,4 @@
#if !(defined(__FreeBSD__) || defined(__APPLE__))
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#include <IO/WriteBufferAIO.h>
#include <Common/ProfileEvents.h>

View File

@ -1,6 +1,6 @@
#pragma once
#if !(defined(__FreeBSD__) || defined(__APPLE__))
#if !(defined(__FreeBSD__) || defined(__APPLE__) || defined(_MSC_VER))
#include <IO/WriteBufferFromFileBase.h>
#include <IO/WriteBuffer.h>

View File

@ -1,4 +1,4 @@
#include <unistd.h>
#include <port/unistd.h>
#include <errno.h>
#include <Common/Exception.h>

View File

@ -1,6 +1,6 @@
#include <IO/createReadBufferFromFileBase.h>
#include <IO/ReadBufferFromFile.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
#include <IO/ReadBufferAIO.h>
#endif
#include <Common/ProfileEvents.h>
@ -14,7 +14,7 @@ namespace ProfileEvents
namespace DB
{
#if defined(__APPLE__) || defined(__FreeBSD__)
#if defined(__APPLE__) || defined(__FreeBSD__) || defined(_MSC_VER)
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
@ -31,7 +31,7 @@ std::unique_ptr<ReadBufferFromFileBase> createReadBufferFromFileBase(const std::
}
else
{
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
ProfileEvents::increment(ProfileEvents::CreatedReadBufferAIO);
return std::make_unique<ReadBufferAIO>(filename_, buffer_size_, flags_, existing_memory_);
#else

View File

@ -1,6 +1,6 @@
#include <IO/createWriteBufferFromFileBase.h>
#include <IO/WriteBufferFromFile.h>
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
#include <IO/WriteBufferAIO.h>
#endif
#include <Common/ProfileEvents.h>
@ -15,7 +15,7 @@ namespace ProfileEvents
namespace DB
{
#if defined(__APPLE__) || defined(__FreeBSD__)
#if defined(__APPLE__) || defined(__FreeBSD__) || defined(_MSC_VER)
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
@ -33,7 +33,7 @@ WriteBufferFromFileBase * createWriteBufferFromFileBase(const std::string & file
}
else
{
#if !defined(__APPLE__) && !defined(__FreeBSD__)
#if !defined(__APPLE__) && !defined(__FreeBSD__) && !defined(_MSC_VER)
ProfileEvents::increment(ProfileEvents::CreatedWriteBufferAIO);
return new WriteBufferAIO(filename_, buffer_size_, flags_, mode, existing_memory_);
#else

View File

@ -1,14 +1,13 @@
#include <IO/ReadBufferAIO.h>
#include <Core/Defines.h>
#include <boost/filesystem.hpp>
#include <vector>
#include <iostream>
#include <fstream>
#include <functional>
#include <cstdlib>
#include <unistd.h>
#include <port/unistd.h>
namespace
{

View File

@ -1,5 +1,3 @@
#include <unistd.h>
#include <Interpreters/ClientInfo.h>
#include <IO/ReadBuffer.h>
#include <IO/WriteBuffer.h>
@ -8,6 +6,7 @@
#include <Core/Defines.h>
#include <Common/getFQDNOrHostName.h>
#include <Common/ClickHouseRevision.h>
#include <port/unistd.h>
namespace DB

View File

@ -130,7 +130,7 @@ struct ContextShared
ConfigurationPtr users_config; /// Config with the users, profiles and quotas sections.
InterserverIOHandler interserver_io_handler; /// Handler for interserver communication.
BackgroundProcessingPoolPtr background_pool; /// The thread pool for the background work performed by the tables.
Macros macros; /// Substitutions extracted from config.
MultiVersion<Macros> macros; /// Substitutions extracted from config.
std::unique_ptr<Compiler> compiler; /// Used for dynamic compilation of queries' parts if it necessary.
std::shared_ptr<DDLWorker> ddl_worker; /// Process ddl commands from zk.
/// Rules for selecting the compression settings, depending on the size of the part.
@ -185,6 +185,8 @@ struct ContextShared
pcg64 rng{randomSeed()};
Context::ConfigReloadCallback config_reload_callback;
ContextShared(std::shared_ptr<IRuntimeComponentsFactory> runtime_components_factory_)
: runtime_components_factory(std::move(runtime_components_factory_))
{
@ -1042,15 +1044,14 @@ void Context::setDefaultFormat(const String & name)
default_format = name;
}
const Macros & Context::getMacros() const
MultiVersion<Macros>::Version Context::getMacros() const
{
return shared->macros;
return shared->macros.get();
}
void Context::setMacros(Macros && macros)
void Context::setMacros(std::unique_ptr<Macros> && macros)
{
/// We assume that this assignment occurs once when the server starts. If this is not the case, you need to use a mutex.
shared->macros = macros;
shared->macros.set(std::move(macros));
}
const Context & Context::getQueryContext() const
@ -1622,6 +1623,22 @@ time_t Context::getUptimeSeconds() const
}
void Context::setConfigReloadCallback(ConfigReloadCallback && callback)
{
/// Is initialized at server startup, so lock isn't required. Otherwise use mutex.
shared->config_reload_callback = std::move(callback);
}
void Context::reloadConfig() const
{
/// Use mutex if callback may be changed after startup.
if (!shared->config_reload_callback)
throw Exception("Can't reload config beacuse config_reload_callback is not set.", ErrorCodes::LOGICAL_ERROR);
shared->config_reload_callback();
}
void Context::shutdown()
{
system_logs.reset();

View File

@ -7,6 +7,7 @@
#include <mutex>
#include <thread>
#include <common/MultiVersion.h>
#include <Core/Types.h>
#include <Core/NamesAndTypes.h>
#include <Interpreters/Settings.h>
@ -80,7 +81,6 @@ using Dependencies = std::vector<DatabaseAndTableName>;
using TableAndCreateAST = std::pair<StoragePtr, ASTPtr>;
using TableAndCreateASTs = std::map<String, TableAndCreateAST>;
/** A set of known objects that can be used in the query.
* Consists of a shared part (always common to all sessions and queries)
* and copied part (which can be its own for each session or query).
@ -206,8 +206,8 @@ public:
String getDefaultFormat() const; /// If default_format is not specified, some global default format is returned.
void setDefaultFormat(const String & name);
const Macros & getMacros() const;
void setMacros(Macros && macros);
MultiVersion<Macros>::Version getMacros() const;
void setMacros(std::unique_ptr<Macros> && macros);
Settings getSettings() const;
void setSettings(const Settings & settings_);
@ -360,6 +360,10 @@ public:
/// Get the server uptime in seconds.
time_t getUptimeSeconds() const;
using ConfigReloadCallback = std::function<void()>;
void setConfigReloadCallback(ConfigReloadCallback && callback);
void reloadConfig() const;
void shutdown();
enum class ApplicationType

View File

@ -600,8 +600,8 @@ void DDLWorker::processTask(DDLTask & task)
/// Delete active flag and create finish flag
zkutil::Ops ops;
ops.emplace_back(std::make_unique<zkutil::Op::Remove>(active_node_path, -1));
ops.emplace_back(std::make_unique<zkutil::Op::Create>(finished_node_path, task.execution_status.serializeText(),
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(active_node_path, -1));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(finished_node_path, task.execution_status.serializeText(),
zookeeper->getDefaultACL(), zkutil::CreateMode::Persistent));
zookeeper->multi(ops);
}
@ -780,8 +780,8 @@ void DDLWorker::cleanupQueue()
/// Remove the lock node and its parent atomically
zkutil::Ops ops;
ops.emplace_back(std::make_unique<zkutil::Op::Remove>(lock_path, -1));
ops.emplace_back(std::make_unique<zkutil::Op::Remove>(node_path, -1));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(lock_path, -1));
ops.emplace_back(std::make_shared<zkutil::Op::Remove>(node_path, -1));
zookeeper->multi(ops);
lock->unlockAssumeLockNodeRemovedManually();
@ -800,8 +800,8 @@ void DDLWorker::createStatusDirs(const std::string & node_path)
{
zkutil::Ops ops;
auto acl = zookeeper->getDefaultACL();
ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/active", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_unique<zkutil::Op::Create>(node_path + "/finished", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(node_path + "/active", "", acl, zkutil::CreateMode::Persistent));
ops.emplace_back(std::make_shared<zkutil::Op::Create>(node_path + "/finished", "", acl, zkutil::CreateMode::Persistent));
int code = zookeeper->tryMulti(ops);
if (code != ZOK && code != ZNODEEXISTS)
@ -1132,7 +1132,7 @@ BlockIO executeDDLQueryOnCluster(const ASTPtr & query_ptr_, const Context & cont
}
}
query->cluster = context.getMacros().expand(query->cluster);
query->cluster = context.getMacros()->expand(query->cluster);
ClusterPtr cluster = context.getCluster(query->cluster);
DDLWorker & ddl_worker = context.getDDLWorker();

View File

@ -49,7 +49,7 @@ bool EmbeddedDictionaries::reloadDictionary(
{
auto new_dictionary = reload_dictionary(config);
if (new_dictionary)
dictionary.set(new_dictionary.release());
dictionary.set(std::move(new_dictionary));
}
catch (...)
{

View File

@ -190,7 +190,7 @@ ExpressionAnalyzer::ExpressionAnalyzer(
}
if (storage && source_columns.empty())
source_columns = storage->getSampleBlock().getNamesAndTypesList();
source_columns = storage->getColumns().getAllPhysical();
else
removeDuplicateColumns(source_columns);
@ -712,7 +712,7 @@ static std::shared_ptr<InterpreterSelectWithUnionQuery> interpretSubquery(
/// 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();
const auto & columns = storage->getColumns().ordinary;
select_expression_list->children.reserve(columns.size());
/// manually substitute column names in place of asterisk
@ -826,7 +826,7 @@ void ExpressionAnalyzer::addExternalStorage(ASTPtr & subquery_or_table_name_or_t
Block sample = interpreter->getSampleBlock();
NamesAndTypesList columns = sample.getNamesAndTypesList();
StoragePtr external_storage = StorageMemory::create(external_table_name, columns, NamesAndTypesList{}, NamesAndTypesList{}, ColumnDefaults{});
StoragePtr external_storage = StorageMemory::create(external_table_name, ColumnsDescription{columns});
external_storage->startup();
/** We replace the subquery with the name of the temporary table.
@ -980,7 +980,8 @@ void ExpressionAnalyzer::normalizeTreeImpl(
/// `IN t` can be specified, where t is a table, which is equivalent to `IN (SELECT * FROM t)`.
if (functionIsInOrGlobalInOperator(func_node->name))
if (ASTIdentifier * right = typeid_cast<ASTIdentifier *>(func_node->arguments->children.at(1).get()))
right->kind = ASTIdentifier::Table;
if (!aliases.count(right->name))
right->kind = ASTIdentifier::Table;
/// Special cases for count function.
String func_name_lowercase = Poco::toLower(func_node->name);
@ -1050,7 +1051,7 @@ void ExpressionAnalyzer::normalizeTreeImpl(
if (storage)
{
/// If we select from a table, get only not MATERIALIZED, not ALIAS columns.
for (const auto & name_type : storage->getColumnsListNonMaterialized())
for (const auto & name_type : storage->getColumns().ordinary)
all_columns.emplace_back(std::make_shared<ASTIdentifier>(name_type.name));
}
else
@ -1147,7 +1148,8 @@ void ExpressionAnalyzer::addAliasColumns()
if (!storage)
return;
source_columns.insert(std::end(source_columns), std::begin(storage->alias_columns), std::end(storage->alias_columns));
const auto & aliases = storage->getColumns().aliases;
source_columns.insert(std::end(source_columns), std::begin(aliases), std::end(aliases));
}
@ -1456,8 +1458,13 @@ void ExpressionAnalyzer::optimizeLimitBy()
void ExpressionAnalyzer::makeSetsForIndex()
{
if (storage && ast && storage->supportsIndexForIn())
makeSetsForIndexImpl(ast, storage->getSampleBlock());
if (storage && select_query && storage->supportsIndexForIn())
{
if (select_query->where_expression)
makeSetsForIndexImpl(select_query->where_expression, storage->getSampleBlock());
if (select_query->prewhere_expression)
makeSetsForIndexImpl(select_query->prewhere_expression, storage->getSampleBlock());
}
}
@ -1482,40 +1489,42 @@ void ExpressionAnalyzer::makeSetsForIndexImpl(const ASTPtr & node, const Block &
{
for (auto & child : node->children)
{
/// Process expression only in current subquery
if (!typeid_cast<ASTSubquery *>(child.get()))
makeSetsForIndexImpl(child, sample_block);
/// Don't descent into subqueries.
if (typeid_cast<ASTSubquery *>(child.get()))
continue;
/// Don't dive into lambda functions
const ASTFunction * func = typeid_cast<const ASTFunction *>(child.get());
if (func && func->name == "lambda")
continue;
makeSetsForIndexImpl(child, sample_block);
}
const ASTFunction * func = typeid_cast<const ASTFunction *>(node.get());
if (func && functionIsInOperator(func->name))
{
const IAST & args = *func->arguments;
const ASTPtr & arg = args.children.at(1);
if (!prepared_sets.count(arg.get())) /// Not already prepared.
if (storage && storage->mayBenefitFromIndexForIn(args.children.at(0)))
{
if (typeid_cast<ASTSubquery *>(arg.get()) || typeid_cast<ASTIdentifier *>(arg.get()))
const ASTPtr & arg = args.children.at(1);
if (!prepared_sets.count(arg.get())) /// Not already prepared.
{
if (settings.use_index_for_in_with_subqueries && storage->mayBenefitFromIndexForIn(args.children.at(0)))
tryMakeSetFromSubquery(arg);
}
else
{
try
if (typeid_cast<ASTSubquery *>(arg.get()) || typeid_cast<ASTIdentifier *>(arg.get()))
{
if (settings.use_index_for_in_with_subqueries)
tryMakeSetFromSubquery(arg);
}
else
{
ExpressionActionsPtr temp_actions = std::make_shared<ExpressionActions>(source_columns, settings);
getRootActions(func->arguments->children.at(0), true, false, temp_actions);
makeExplicitSet(func, temp_actions->getSampleBlock(), true);
}
catch (const Exception & e)
{
/// in `sample_block` there are no columns that are added by `getActions`
if (e.code() != ErrorCodes::NOT_FOUND_COLUMN_IN_BLOCK && e.code() != ErrorCodes::UNKNOWN_IDENTIFIER)
throw;
/// TODO: Delete the catch in the next release
tryLogCurrentException(&Poco::Logger::get("ExpressionAnalyzer"));
Block sample_block_with_calculated_columns = temp_actions->getSampleBlock();
if (sample_block_with_calculated_columns.has(args.children.at(0)->getColumnName()))
makeExplicitSet(func, sample_block_with_calculated_columns, true);
}
}
}
@ -1578,7 +1587,7 @@ void ExpressionAnalyzer::makeSet(const ASTFunction * node, const Block & sample_
* in the subquery_for_set object, this subquery is set as source and the temporary table _data1 as the table.
* - this function shows the expression IN_data1.
*/
if (!subquery_for_set.source)
if (!subquery_for_set.source && (!storage || !storage->isRemote()))
{
auto interpreter = interpretSubquery(arg, context, subquery_depth, {});
subquery_for_set.source = std::make_shared<LazyBlockInputStream>(
@ -1645,15 +1654,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
if (left_arg_tuple && left_arg_tuple->name == "tuple")
{
for (const auto & arg : left_arg_tuple->arguments->children)
{
const auto & data_type = sample_block.getByName(arg->getColumnName()).type;
/// 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);
set_element_types.push_back(data_type);
}
set_element_types.push_back(sample_block.getByName(arg->getColumnName()).type);
}
else
{

View File

@ -110,7 +110,7 @@ void InterpreterAlterQuery::parseAlter(
}
if (ast_col_decl.default_expression)
{
command.default_type = columnDefaultTypeFromString(ast_col_decl.default_specifier);
command.default_kind = columnDefaultKindFromString(ast_col_decl.default_specifier);
command.default_expression = ast_col_decl.default_expression;
}
@ -157,7 +157,7 @@ void InterpreterAlterQuery::parseAlter(
if (ast_col_decl.default_expression)
{
command.default_type = columnDefaultTypeFromString(ast_col_decl.default_specifier);
command.default_kind = columnDefaultKindFromString(ast_col_decl.default_specifier);
command.default_expression = ast_col_decl.default_expression;
}
@ -200,7 +200,7 @@ void InterpreterAlterQuery::PartitionCommands::validate(const IStorage * table)
{
String column_name = command.column_name.safeGet<String>();
if (!table->hasRealColumn(column_name))
if (!table->getColumns().hasPhysical(column_name))
{
throw Exception("Wrong column name. Cannot find column " + column_name + " to clear it from partition",
DB::ErrorCodes::ILLEGAL_COLUMN);

View File

@ -246,7 +246,7 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast
explicit_type = block.getByName(column_name).type;
defaults.emplace(column_name, ColumnDefault{
columnDefaultTypeFromString(col_decl_ptr->default_specifier),
columnDefaultKindFromString(col_decl_ptr->default_specifier),
col_decl_ptr->default_expression
});
}
@ -256,7 +256,7 @@ static ColumnsAndDefaults parseColumns(const ASTExpressionList & column_list_ast
}
static NamesAndTypesList removeAndReturnColumns(ColumnsAndDefaults & columns_and_defaults, const ColumnDefaultType type)
static NamesAndTypesList removeAndReturnColumns(ColumnsAndDefaults & columns_and_defaults, const ColumnDefaultKind kind)
{
auto & columns = columns_and_defaults.first;
auto & defaults = columns_and_defaults.second;
@ -266,7 +266,7 @@ static NamesAndTypesList removeAndReturnColumns(ColumnsAndDefaults & columns_and
for (auto it = std::begin(columns); it != std::end(columns);)
{
const auto jt = defaults.find(it->name);
if (jt != std::end(defaults) && jt->second.type == type)
if (jt != std::end(defaults) && jt->second.kind == kind)
{
removed.push_back(*it);
it = columns.erase(it);
@ -301,15 +301,11 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
return columns_list;
}
ASTPtr InterpreterCreateQuery::formatColumns(
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults)
ASTPtr InterpreterCreateQuery::formatColumns(const ColumnsDescription & columns)
{
auto columns_list = std::make_shared<ASTExpressionList>();
for (const auto & column : boost::join(columns, boost::join(materialized_columns, alias_columns)))
for (const auto & column : columns.getAll())
{
const auto column_declaration = std::make_shared<ASTColumnDeclaration>();
ASTPtr column_declaration_ptr{column_declaration};
@ -324,10 +320,10 @@ ASTPtr InterpreterCreateQuery::formatColumns(
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->owned_string = type_name;
const auto it = column_defaults.find(column.name);
if (it != std::end(column_defaults))
const auto it = columns.defaults.find(column.name);
if (it != std::end(columns.defaults))
{
column_declaration->default_specifier = toString(it->second.type);
column_declaration->default_specifier = toString(it->second.kind);
column_declaration->default_expression = it->second.expression->clone();
}
@ -338,49 +334,46 @@ ASTPtr InterpreterCreateQuery::formatColumns(
}
InterpreterCreateQuery::ColumnsInfo InterpreterCreateQuery::getColumnsInfo(const ASTExpressionList & columns, const Context & context)
ColumnsDescription InterpreterCreateQuery::getColumnsDescription(const ASTExpressionList & columns, const Context & context)
{
ColumnsInfo res;
ColumnsDescription res;
auto && columns_and_defaults = parseColumns(columns, context);
res.materialized_columns = removeAndReturnColumns(columns_and_defaults, ColumnDefaultType::Materialized);
res.alias_columns = removeAndReturnColumns(columns_and_defaults, ColumnDefaultType::Alias);
res.columns = std::move(columns_and_defaults.first);
res.column_defaults = std::move(columns_and_defaults.second);
res.materialized = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Materialized);
res.aliases = removeAndReturnColumns(columns_and_defaults, ColumnDefaultKind::Alias);
res.ordinary = std::move(columns_and_defaults.first);
res.defaults = std::move(columns_and_defaults.second);
if (res.columns.size() + res.materialized_columns.size() == 0)
if (res.ordinary.size() + res.materialized.size() == 0)
throw Exception{"Cannot CREATE table without physical columns", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED};
return res;
}
InterpreterCreateQuery::ColumnsInfo InterpreterCreateQuery::setColumns(
ColumnsDescription InterpreterCreateQuery::setColumns(
ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const
{
ColumnsInfo res;
ColumnsDescription res;
if (create.columns)
{
res = getColumnsInfo(*create.columns, context);
res = getColumnsDescription(*create.columns, context);
}
else if (!create.as_table.empty())
{
res.columns = as_storage->getColumnsListNonMaterialized();
res.materialized_columns = as_storage->materialized_columns;
res.alias_columns = as_storage->alias_columns;
res.column_defaults = as_storage->column_defaults;
res = as_storage->getColumns();
}
else if (create.select)
{
for (size_t i = 0; i < as_select_sample.columns(); ++i)
res.columns.emplace_back(as_select_sample.safeGetByPosition(i).name, as_select_sample.safeGetByPosition(i).type);
res.ordinary.emplace_back(as_select_sample.safeGetByPosition(i).name, as_select_sample.safeGetByPosition(i).type);
}
else
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY);
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
ASTPtr new_columns = formatColumns(res.columns, res.materialized_columns, res.alias_columns, res.column_defaults);
ASTPtr new_columns = formatColumns(res);
if (create.columns)
create.replace(create.columns, new_columns);
else
@ -394,11 +387,11 @@ InterpreterCreateQuery::ColumnsInfo InterpreterCreateQuery::setColumns(
throw Exception("Column " + backQuoteIfNeed(column_name_and_type.name) + " already exists", ErrorCodes::DUPLICATE_COLUMN);
};
for (const auto & elem : res.columns)
for (const auto & elem : res.ordinary)
check_column_already_exists(elem);
for (const auto & elem : res.materialized_columns)
for (const auto & elem : res.materialized)
check_column_already_exists(elem);
for (const auto & elem : res.alias_columns)
for (const auto & elem : res.aliases)
check_column_already_exists(elem);
return res;
@ -489,7 +482,7 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
}
/// Set and retrieve list of columns.
ColumnsInfo columns = setColumns(create, as_select_sample, as_storage);
ColumnsDescription columns = setColumns(create, as_select_sample, as_storage);
/// Set the table engine if it was not specified explicitly.
setEngine(create);
@ -523,17 +516,16 @@ BlockIO InterpreterCreateQuery::createTable(ASTCreateQuery & create)
throw Exception("Table " + database_name + "." + table_name + " already exists.", ErrorCodes::TABLE_ALREADY_EXISTS);
}
}
else if (context.tryGetExternalTable(table_name) && create.if_not_exists)
return {};
res = StorageFactory::instance().get(create,
data_path,
table_name,
database_name,
context,
context.getGlobalContext(),
columns.columns,
columns.materialized_columns,
columns.alias_columns,
columns.column_defaults,
columns,
create.attach,
false);

View File

@ -1,7 +1,7 @@
#pragma once
#include <Interpreters/IInterpreter.h>
#include <Storages/ColumnDefault.h>
#include <Storages/ColumnsDescription.h>
class ThreadPool;
@ -27,11 +27,7 @@ public:
/// List of columns and their types in AST.
static ASTPtr formatColumns(const NamesAndTypesList & columns);
static ASTPtr formatColumns(
const NamesAndTypesList & columns,
const NamesAndTypesList & materialized_columns,
const NamesAndTypesList & alias_columns,
const ColumnDefaults & column_defaults);
static ASTPtr formatColumns(const ColumnsDescription & columns);
void setDatabaseLoadingThreadpool(ThreadPool & thread_pool_)
{
@ -48,23 +44,15 @@ public:
internal = internal_;
}
struct ColumnsInfo
{
NamesAndTypesList columns;
NamesAndTypesList materialized_columns;
NamesAndTypesList alias_columns;
ColumnDefaults column_defaults;
};
/// Obtain information about columns, their types and default values, for case when columns in CREATE query is specified explicitly.
static ColumnsInfo getColumnsInfo(const ASTExpressionList & columns, const Context & context);
static ColumnsDescription getColumnsDescription(const ASTExpressionList & columns, const Context & context);
private:
BlockIO createDatabase(ASTCreateQuery & create);
BlockIO createTable(ASTCreateQuery & create);
/// Calculate list of columns of table and return it.
ColumnsInfo setColumns(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const;
ColumnsDescription setColumns(ASTCreateQuery & create, const Block & as_select_sample, const StoragePtr & as_storage) const;
void setEngine(ASTCreateQuery & create) const;
void checkAccess(const ASTCreateQuery & create);

View File

@ -99,9 +99,8 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
}
auto table_lock = table->lockStructure(false, __PRETTY_FUNCTION__);
columns = table->getColumnsList();
columns.insert(std::end(columns), std::begin(table->alias_columns), std::end(table->alias_columns));
column_defaults = table->column_defaults;
columns = table->getColumns().getAll();
column_defaults = table->getColumns().defaults;
}
Block sample_block = getSampleBlock();
@ -120,7 +119,7 @@ BlockInputStreamPtr InterpreterDescribeQuery::executeImpl()
}
else
{
res_columns[2]->insert(toString(it->second.type));
res_columns[2]->insert(toString(it->second.kind));
res_columns[3]->insert(queryToString(it->second.expression));
}
}

View File

@ -95,7 +95,7 @@ BlockIO InterpreterInsertQuery::execute()
auto table_lock = table->lockStructure(true, __PRETTY_FUNCTION__);
NamesAndTypesList required_columns = table->getColumnsList();
NamesAndTypesList required_columns = table->getColumns().getAllPhysical();
/// We create a pipeline of several streams, into which we will write data.
BlockOutputStreamPtr out;
@ -103,7 +103,7 @@ BlockIO InterpreterInsertQuery::execute()
out = std::make_shared<PushingToViewsBlockOutputStream>(query.database, query.table, table, context, query_ptr, query.no_destination);
out = std::make_shared<AddingDefaultBlockOutputStream>(
out, getSampleBlock(query, table), required_columns, table->column_defaults, context);
out, getSampleBlock(query, table), required_columns, table->getColumns().defaults, context);
/// Do not squash blocks if it is a sync INSERT into Distributed, since it lead to double bufferization on client and server side.
/// Client-side bufferization might cause excessive timeouts (especially in case of big blocks).
@ -136,7 +136,7 @@ BlockIO InterpreterInsertQuery::execute()
if (!allow_materialized)
{
Block in_header = res.in->getHeader();
for (const auto & name_type : table->materialized_columns)
for (const auto & name_type : table->getColumns().materialized)
if (in_header.has(name_type.name))
throw Exception("Cannot insert column " + name_type.name + ", because it is MATERIALIZED column.", ErrorCodes::ILLEGAL_COLUMN);
}

View File

@ -63,6 +63,7 @@ namespace ErrorCodes
extern const int ILLEGAL_PREWHERE;
extern const int TOO_MANY_COLUMNS;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
}
InterpreterSelectQuery::InterpreterSelectQuery(
@ -338,6 +339,9 @@ void InterpreterSelectQuery::executeImpl(Pipeline & pipeline, const BlockInputSt
/** Read the data from Storage. from_stage - to what stage the request was completed in Storage. */
QueryProcessingStage::Enum from_stage = executeFetchColumns(pipeline, dry_run);
if (from_stage == QueryProcessingStage::WithMergeableState && to_stage == QueryProcessingStage::WithMergeableState)
throw Exception("Distributed on Distributed is not supported", ErrorCodes::NOT_IMPLEMENTED);
if (!dry_run)
LOG_TRACE(log, QueryProcessingStage::toString(from_stage) << " -> " << QueryProcessingStage::toString(to_stage));
@ -516,12 +520,13 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
/// Are ALIAS columns required for query execution?
auto alias_columns_required = false;
if (storage && !storage->alias_columns.empty())
if (storage && !storage->getColumns().aliases.empty())
{
const auto & column_defaults = storage->getColumns().defaults;
for (const auto & column : required_columns)
{
const auto default_it = storage->column_defaults.find(column);
if (default_it != std::end(storage->column_defaults) && default_it->second.type == ColumnDefaultType::Alias)
const auto default_it = column_defaults.find(column);
if (default_it != std::end(column_defaults) && default_it->second.kind == ColumnDefaultKind::Alias)
{
alias_columns_required = true;
break;
@ -535,8 +540,8 @@ QueryProcessingStage::Enum InterpreterSelectQuery::executeFetchColumns(Pipeline
for (const auto & column : required_columns)
{
const auto default_it = storage->column_defaults.find(column);
if (default_it != std::end(storage->column_defaults) && default_it->second.type == ColumnDefaultType::Alias)
const auto default_it = column_defaults.find(column);
if (default_it != std::end(column_defaults) && default_it->second.kind == ColumnDefaultKind::Alias)
required_columns_expr_list->children.emplace_back(setAlias(default_it->second.expression->clone(), column));
else
required_columns_expr_list->children.emplace_back(std::make_shared<ASTIdentifier>(column));

View File

@ -97,6 +97,9 @@ BlockIO InterpreterSystemQuery::execute()
throw Exception(status.message, status.code);
break;
}
case Type::RELOAD_CONFIG:
context.reloadConfig();
break;
case Type::STOP_LISTEN_QUERIES:
case Type::START_LISTEN_QUERIES:
case Type::RESTART_REPLICAS:

View File

@ -38,7 +38,6 @@ public:
std::string getRemoteTableName() const { return remote_table; }
std::string getTableName() const override { return ""; }
const DB::NamesAndTypesList & getColumnsListImpl() const override { return names_and_types; }
protected:
StorageDistributedFake(const std::string & remote_database_, const std::string & remote_table_, size_t shard_count_)
@ -50,7 +49,6 @@ private:
const std::string remote_database;
const std::string remote_table;
size_t shard_count;
DB::NamesAndTypesList names_and_types;
};

View File

@ -1,8 +1,6 @@
#include <Common/Config/ConfigProcessor.h>
#include <Interpreters/SecurityManager.h>
#include <boost/filesystem.hpp>
#include <vector>
#include <string>
#include <tuple>
@ -10,8 +8,8 @@
#include <fstream>
#include <sstream>
#include <stdexcept>
#include <unistd.h>
#include <cstdlib>
#include <port/unistd.h>
namespace
{

View File

@ -39,6 +39,8 @@ const char * ASTSystemQuery::typeToString(Type type)
return "RELOAD DICTIONARY";
case Type::RELOAD_DICTIONARIES:
return "RELOAD DICTIONARIES";
case Type::RELOAD_CONFIG:
return "RELOAD CONFIG";
case Type::STOP_MERGES:
return "STOP MERGES";
case Type::START_MERGES:

View File

@ -24,6 +24,7 @@ public:
SYNC_REPLICA,
RELOAD_DICTIONARY,
RELOAD_DICTIONARIES,
RELOAD_CONFIG,
STOP_MERGES,
START_MERGES,
STOP_REPLICATION_QUEUES,

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