Merge branch 'master' into fix-memory-tracker

This commit is contained in:
Alexey Milovidov 2018-10-05 18:15:50 +03:00
commit 5503822983
144 changed files with 2272 additions and 759 deletions

View File

@ -11,5 +11,4 @@ ClickHouse is an open-source column-oriented database management system that all
## Upcoming Meetups
* [Paris on October 2](http://bit.ly/clickhouse-paris-2-october-2018)
* [Beijing on October 28](http://www.clickhouse.com.cn/topic/5ba0e3f99d28dfde2ddc62a1)

View File

@ -1,18 +1,11 @@
# Freebsd: /usr/local/include/libcpuid/libcpuid_types.h:61:29: error: conflicting declaration 'typedef long long int int64_t'
# TODO: test new libcpuid - maybe already fixed
if (NOT ARCH_ARM)
if (OS_FREEBSD)
set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY 1)
else ()
set (DEFAULT_USE_INTERNAL_CPUID_LIBRARY ${NOT_UNBUNDLED})
endif ()
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${DEFAULT_USE_INTERNAL_CPUID_LIBRARY})
option (USE_INTERNAL_CPUID_LIBRARY "Set to FALSE to use system cpuid library instead of bundled" ${NOT_UNBUNDLED})
endif ()
#if (USE_INTERNAL_CPUID_LIBRARY AND NOT EXISTS "${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include/cpuid/libcpuid.h")
# message (WARNING "submodule contrib/libcpuid is missing. to fix try run: \n git submodule update --init --recursive")
# set (USE_INTERNAL_CPUID_LIBRARY 0)
# set (MISSING_INTERNAL_CPUID_LIBRARY 1)
#endif ()
if (NOT USE_INTERNAL_CPUID_LIBRARY)
@ -21,7 +14,13 @@ if (NOT USE_INTERNAL_CPUID_LIBRARY)
endif ()
if (CPUID_LIBRARY AND CPUID_INCLUDE_DIR)
else ()
if (OS_FREEBSD)
# need in /usr/local/include/libcpuid/libcpuid_types.h
# Freebsd: /usr/local/include/libcpuid/libcpuid_types.h:61:29: error: conflicting declaration 'typedef long long int int64_t'
add_definitions(-DHAVE_STDINT_H)
# TODO: make virtual target cpuid:cpuid with COMPILE_DEFINITIONS property
endif ()
elseif (NOT MISSING_INTERNAL_CPUID_LIBRARY)
set (CPUID_INCLUDE_DIR ${ClickHouse_SOURCE_DIR}/contrib/libcpuid/include)
set (USE_INTERNAL_CPUID_LIBRARY 1)
set (CPUID_LIBRARY cpuid)

View File

@ -1,4 +1,6 @@
option (USE_INTERNAL_ZLIB_LIBRARY "Set to FALSE to use system zlib library instead of bundled" ${NOT_UNBUNDLED})
if (NOT OS_FREEBSD)
option (USE_INTERNAL_ZLIB_LIBRARY "Set to FALSE to use system zlib library instead of bundled" ${NOT_UNBUNDLED})
endif ()
if (NOT USE_INTERNAL_ZLIB_LIBRARY)
find_package (ZLIB)

View File

@ -14,11 +14,12 @@ cmake_push_check_state ()
#define __SSE4_1__ 1
set (TEST_FLAG "-msse4.1")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0")
check_cxx_source_compiles("
#include <smmintrin.h>
int main() {
_mm_insert_epi8(__m128i(), 0, 0);
auto a = _mm_insert_epi8(__m128i(), 0, 0);
(void)a;
return 0;
}
" HAVE_SSE41)
@ -31,11 +32,12 @@ endif ()
#define __SSE4_2__ 1
set (TEST_FLAG "-msse4.2")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0")
check_cxx_source_compiles("
#include <nmmintrin.h>
int main() {
_mm_crc32_u64(0, 0);
auto a = _mm_crc32_u64(0, 0);
(void)a;
return 0;
}
" HAVE_SSE42)
@ -49,10 +51,11 @@ endif ()
set (TEST_FLAG "-mpopcnt")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG}")
set (CMAKE_REQUIRED_FLAGS "${TEST_FLAG} -O0")
check_cxx_source_compiles("
int main() {
__builtin_popcountll(0);
auto a = __builtin_popcountll(0);
(void)a;
return 0;
}
" HAVE_POPCNT)

View File

@ -142,6 +142,7 @@ if (USE_INTERNAL_POCO_LIBRARY)
set (save_CMAKE_C_FLAGS ${CMAKE_C_FLAGS})
set (_save ${ENABLE_TESTS})
set (ENABLE_TESTS 0)
set (POCO_ENABLE_TESTS 0)
set (CMAKE_DISABLE_FIND_PACKAGE_ZLIB 1)
if (MSVC)
set (ENABLE_DATA_ODBC 0 CACHE INTERNAL "") # TODO (build fail)

2
contrib/capnproto vendored

@ -1 +1 @@
Subproject commit 7173ab638fdf144032411dc69fb1082cd473e08f
Subproject commit a00ccd91b3746ef2ab51d40fe3265829949d1ace

View File

@ -155,13 +155,14 @@ if (CMAKE_BUILD_TYPE_UC STREQUAL "RELEASE" OR CMAKE_BUILD_TYPE_UC STREQUAL "RELW
PROPERTIES COMPILE_FLAGS -g0)
endif ()
if (NOT ARCH_ARM)
if (NOT ARCH_ARM AND CPUID_LIBRARY)
set (LINK_LIBRARIES_ONLY_ON_X86_64 ${CPUID_LIBRARY})
endif()
target_link_libraries (clickhouse_common_io
common
string_utils
widechar_width
${LINK_LIBRARIES_ONLY_ON_X86_64}
${LZ4_LIBRARY}
${ZSTD_LIBRARY}

View File

@ -2,10 +2,10 @@
set(VERSION_REVISION 54409 CACHE STRING "")
set(VERSION_MAJOR 18 CACHE STRING "")
set(VERSION_MINOR 14 CACHE STRING "")
set(VERSION_PATCH 0 CACHE STRING "")
set(VERSION_GITHASH f09970ff704a32c41ae207e3db152541800dc580 CACHE STRING "")
set(VERSION_DESCRIBE v18.14.0-testing CACHE STRING "")
set(VERSION_STRING 18.14.0 CACHE STRING "")
set(VERSION_PATCH 2 CACHE STRING "")
set(VERSION_GITHASH 147a2a13c256c72b7155e864e6a95024ca9ba31a CACHE STRING "")
set(VERSION_DESCRIBE v18.14.2-testing CACHE STRING "")
set(VERSION_STRING 18.14.2 CACHE STRING "")
# end of autochange
set(VERSION_EXTRA "" CACHE STRING "")

View File

@ -26,7 +26,7 @@ elseif (EXISTS ${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE})
set (COPY_HEADERS_COMPILER "${INTERNAL_COMPILER_BIN_ROOT}${INTERNAL_COMPILER_EXECUTABLE}")
endif ()
if (COPY_HEADERS_COMPILER AND NOT APPLE)
if (COPY_HEADERS_COMPILER AND OS_LINUX)
add_custom_target (copy-headers ALL env CLANG=${COPY_HEADERS_COMPILER} BUILD_PATH=${ClickHouse_BINARY_DIR} DESTDIR=${ClickHouse_SOURCE_DIR} ${ClickHouse_SOURCE_DIR}/copy_headers.sh ${ClickHouse_SOURCE_DIR} ${TMP_HEADERS_DIR} DEPENDS ${COPY_HEADERS_DEPENDS} WORKING_DIRECTORY ${ClickHouse_SOURCE_DIR} SOURCES ${ClickHouse_SOURCE_DIR}/copy_headers.sh)
if (USE_INTERNAL_LLVM_LIBRARY)

View File

@ -61,7 +61,7 @@
#include <AggregateFunctions/registerAggregateFunctions.h>
#if USE_READLINE
#include "Suggest.h"
#include "Suggest.h" // Y_IGNORE
#endif
#ifndef __clang__

View File

@ -26,7 +26,7 @@
-->
<prompt_by_server_display_name>
<default>{display_name} :) </default>
<test>{display_name} \x01\e[1;32m\x02:)\x01\e[0m\x02 </test> <!-- if it matched to the substring "test" in the server display name - -->
<test>{display_name} \x01\e[1;32m\x02:)\x01\e[0m\x02 </test> <!-- if it matched to the substring "test" in the server display name - -->
<production>{display_name} \x01\e[1;31m\x02:)\x01\e[0m\x02 </production> <!-- if it matched to the substring "production" in the server display name -->
</prompt_by_server_display_name>
</config>

View File

@ -2,9 +2,9 @@
#if USE_POCO_SQLODBC || USE_POCO_DATAODBC
#if USE_POCO_SQLODBC
#include <Poco/SQL/ODBC/ODBCException.h>
#include <Poco/SQL/ODBC/SessionImpl.h>
#include <Poco/SQL/ODBC/Utility.h>
#include <Poco/SQL/ODBC/ODBCException.h> // Y_IGNORE
#include <Poco/SQL/ODBC/SessionImpl.h> // Y_IGNORE
#include <Poco/SQL/ODBC/Utility.h> // Y_IGNORE
#define POCO_SQL_ODBC_CLASS Poco::SQL::ODBC
#endif
#if USE_POCO_DATAODBC

View File

@ -1159,11 +1159,8 @@ private:
StringToVector::iterator substitutions_first = substitutions.begin();
StringToVector::iterator substitutions_last = substitutions.end();
--substitutions_last;
std::map<String, String> substitutions_map;
runThroughAllOptionsAndPush(substitutions_first, substitutions_last, query, queries, substitutions_map);
runThroughAllOptionsAndPush(substitutions_first, substitutions_last, query, queries);
return queries;
}
@ -1173,44 +1170,37 @@ private:
void runThroughAllOptionsAndPush(StringToVector::iterator substitutions_left,
StringToVector::iterator substitutions_right,
const String & template_query,
std::vector<String> & queries,
const StringKeyValue & template_substitutions_map = StringKeyValue())
std::vector<String> & queries)
{
String name = substitutions_left->first;
std::vector<String> values = substitutions_left->second;
if (substitutions_left == substitutions_right)
{
queries.push_back(template_query); /// completely substituted query
return;
}
for (const String & value : values)
String substitution_mask = "{" + substitutions_left->first + "}";
if (template_query.find(substitution_mask) == String::npos) /// nothing to substitute here
{
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, template_query, queries);
return;
}
for (const String & value : substitutions_left->second)
{
/// Copy query string for each unique permutation
Query query = template_query;
StringKeyValue substitutions_map = template_substitutions_map;
size_t substr_pos = 0;
while (substr_pos != String::npos)
{
substr_pos = query.find("{" + name + "}");
substr_pos = query.find(substitution_mask);
if (substr_pos != String::npos)
{
query.replace(substr_pos, 1 + name.length() + 1, value);
}
query.replace(substr_pos, substitution_mask.length(), value);
}
substitutions_map[name] = value;
/// If we've reached the end of substitution chain
if (substitutions_left == substitutions_right)
{
queries.push_back(query);
substitutions_maps.push_back(substitutions_map);
}
else
{
StringToVector::iterator next_it = substitutions_left;
++next_it;
runThroughAllOptionsAndPush(next_it, substitutions_right, query, queries, substitutions_map);
}
runThroughAllOptionsAndPush(std::next(substitutions_left), substitutions_right, query, queries);
}
}

View File

@ -27,7 +27,7 @@ AggregateFunctionPtr createAggregateFunctionAvg(const std::string & name, const
AggregateFunctionPtr res;
DataTypePtr data_type = argument_types[0];
if (isDecimal(data_type))
res.reset(createWithDecimalType<AggregateFuncAvg>(*data_type));
res.reset(createWithDecimalType<AggregateFuncAvg>(*data_type, *data_type));
else
res.reset(createWithNumericType<AggregateFuncAvg>(*data_type));

View File

@ -13,7 +13,7 @@
#include <Common/typeid_cast.h>
#include <Poco/String.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace DB
@ -42,14 +42,14 @@ void AggregateFunctionFactory::registerFunction(const String & name, Creator cre
ErrorCodes::LOGICAL_ERROR);
}
static DataTypes convertTypesWithDictionaryToNested(const DataTypes & types)
static DataTypes convertLowCardinalityTypesToNested(const DataTypes & types)
{
DataTypes res_types;
res_types.reserve(types.size());
for (const auto & type : types)
{
if (auto * type_with_dict = typeid_cast<const DataTypeWithDictionary *>(type.get()))
res_types.push_back(type_with_dict->getDictionaryType());
if (auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(type.get()))
res_types.push_back(low_cardinality_type->getDictionaryType());
else
res_types.push_back(type);
}
@ -63,7 +63,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
const Array & parameters,
int recursion_level) const
{
auto type_without_dictionary = convertTypesWithDictionaryToNested(argument_types);
auto type_without_low_cardinality = convertLowCardinalityTypesToNested(argument_types);
/// If one of types is Nullable, we apply aggregate function combinator "Null".
@ -74,7 +74,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
if (!combinator)
throw Exception("Logical error: cannot find aggregate function combinator to apply a function to Nullable arguments.", ErrorCodes::LOGICAL_ERROR);
DataTypes nested_types = combinator->transformArguments(type_without_dictionary);
DataTypes nested_types = combinator->transformArguments(type_without_low_cardinality);
AggregateFunctionPtr nested_function;
@ -87,7 +87,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
return combinator->transformAggregateFunction(nested_function, argument_types, parameters);
}
auto res = getImpl(name, type_without_dictionary, parameters, recursion_level);
auto res = getImpl(name, type_without_low_cardinality, parameters, recursion_level);
if (!res)
throw Exception("Logical error: AggregateFunctionFactory returned nullptr", ErrorCodes::LOGICAL_ERROR);
return res;

View File

@ -20,7 +20,7 @@ namespace
{
template <typename T> using FuncQuantile = AggregateFunctionQuantile<T, QuantileReservoirSampler<T>, NameQuantile, false, Float64, false>;
template <typename T> using FuncQuantiles = AggregateFunctionQuantile<T, QuantileReservoirSampler<T>, NameQuantile, false, Float64, true>;
template <typename T> using FuncQuantiles = AggregateFunctionQuantile<T, QuantileReservoirSampler<T>, NameQuantiles, false, Float64, true>;
template <typename T> using FuncQuantileDeterministic = AggregateFunctionQuantile<T, QuantileReservoirSamplerDeterministic<T>, NameQuantileDeterministic, true, Float64, false>;
template <typename T> using FuncQuantilesDeterministic = AggregateFunctionQuantile<T, QuantileReservoirSamplerDeterministic<T>, NameQuantilesDeterministic, true, Float64, true>;

View File

@ -50,7 +50,7 @@ AggregateFunctionPtr createAggregateFunctionSum(const std::string & name, const
AggregateFunctionPtr res;
DataTypePtr data_type = argument_types[0];
if (isDecimal(data_type))
res.reset(createWithDecimalType<Function>(*data_type));
res.reset(createWithDecimalType<Function>(*data_type, *data_type));
else
res.reset(createWithNumericType<Function>(*data_type));

View File

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

View File

@ -8,6 +8,8 @@
#include <Columns/ColumnArray.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnDecimal.h>
#include <Common/FieldVisitors.h>
#include <AggregateFunctions/IAggregateFunction.h>
@ -53,6 +55,8 @@ class AggregateFunctionSumMap final : public IAggregateFunctionDataHelper<
AggregateFunctionSumMapData<typename NearestFieldType<T>::Type>, AggregateFunctionSumMap<T>>
{
private:
using ColVecType = std::conditional_t<IsDecimalNumber<T>, ColumnDecimal<T>, ColumnVector<T>>;
DataTypePtr keys_type;
DataTypes values_types;
@ -78,7 +82,7 @@ public:
// Column 0 contains array of keys of known type
const ColumnArray & array_column = static_cast<const ColumnArray &>(*columns[0]);
const IColumn::Offsets & offsets = array_column.getOffsets();
const auto & keys_vec = static_cast<const ColumnVector<T> &>(array_column.getData());
const auto & keys_vec = static_cast<const ColVecType &>(array_column.getData());
const size_t keys_vec_offset = row_num == 0 ? 0 : offsets[row_num - 1];
const size_t keys_vec_size = (offsets[row_num] - keys_vec_offset);
@ -99,9 +103,20 @@ public:
// Insert column values for all keys
for (size_t i = 0; i < keys_vec_size; ++i)
{
using MapType = std::decay_t<decltype(merged_maps)>;
using IteratorType = typename MapType::iterator;
array_column.getData().get(values_vec_offset + i, value);
const auto & key = keys_vec.getData()[keys_vec_offset + i];
const auto & it = merged_maps.find(key);
IteratorType it;
if constexpr (IsDecimalNumber<T>)
{
UInt32 scale = keys_vec.getData().getScale();
it = merged_maps.find(DecimalField<T>(key, scale));
}
else
it = merged_maps.find(key);
if (it != merged_maps.end())
applyVisitor(FieldVisitorSum(value), it->second[col]);
@ -113,7 +128,13 @@ public:
for (size_t k = 0; k < new_values.size(); ++k)
new_values[k] = (k == col) ? value : values_types[k]->getDefault();
merged_maps[key] = std::move(new_values);
if constexpr (IsDecimalNumber<T>)
{
UInt32 scale = keys_vec.getData().getScale();
merged_maps.emplace(DecimalField<T>(key, scale), std::move(new_values));
}
else
merged_maps.emplace(key, std::move(new_values));
}
}
}
@ -167,7 +188,10 @@ public:
for (size_t col = 0; col < values_types.size(); ++col)
values_types[col]->deserializeBinary(values[col], buf);
merged_maps[key.get<T>()] = values;
if constexpr (IsDecimalNumber<T>)
merged_maps[key.get<DecimalField<T>>()] = values;
else
merged_maps[key.get<T>()] = values;
}
}

View File

@ -24,9 +24,7 @@ AggregateFunctionPtr createAggregateFunctionStatisticsUnary(const std::string &
AggregateFunctionPtr res;
DataTypePtr data_type = argument_types[0];
if (isDecimal(data_type))
{
res.reset(createWithDecimalType<FunctionTemplate>(*data_type));
}
res.reset(createWithDecimalType<FunctionTemplate>(*data_type, *data_type));
else
res.reset(createWithNumericType<FunctionTemplate>(*data_type));

View File

@ -70,13 +70,28 @@ static IAggregateFunction * createWithUnsignedIntegerType(const IDataType & argu
return nullptr;
}
template <template <typename> class AggregateFunctionTemplate, typename ... TArgs>
static IAggregateFunction * createWithNumericBasedType(const IDataType & argument_type, TArgs && ... args)
{
IAggregateFunction * f = createWithNumericType<AggregateFunctionTemplate>(argument_type, std::forward<TArgs>(args)...);
if (f)
return f;
/// expects that DataTypeDate based on UInt16, DataTypeDateTime based on UInt32 and UUID based on UInt128
WhichDataType which(argument_type);
if (which.idx == TypeIndex::Date) return new AggregateFunctionTemplate<UInt16>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::DateTime) return new AggregateFunctionTemplate<UInt32>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::UUID) return new AggregateFunctionTemplate<UInt128>(std::forward<TArgs>(args)...);
return nullptr;
}
template <template <typename> class AggregateFunctionTemplate, typename ... TArgs>
static IAggregateFunction * createWithDecimalType(const IDataType & argument_type, TArgs && ... args)
{
WhichDataType which(argument_type);
if (which.idx == TypeIndex::Decimal32) return new AggregateFunctionTemplate<Decimal32>(argument_type, std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal64) return new AggregateFunctionTemplate<Decimal64>(argument_type, std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal128) return new AggregateFunctionTemplate<Decimal128>(argument_type, std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal32) return new AggregateFunctionTemplate<Decimal32>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal64) return new AggregateFunctionTemplate<Decimal64>(std::forward<TArgs>(args)...);
if (which.idx == TypeIndex::Decimal128) return new AggregateFunctionTemplate<Decimal128>(std::forward<TArgs>(args)...);
return nullptr;
}

View File

@ -0,0 +1,41 @@
#include "TimeoutSetter.h"
#include <common/logger_useful.h>
namespace DB
{
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_,
const Poco::Timespan & send_timeout_,
const Poco::Timespan & receive_timeout_,
bool limit_max_timeout)
: socket(socket_), send_timeout(send_timeout_), receive_timeout(receive_timeout_)
{
old_send_timeout = socket.getSendTimeout();
old_receive_timeout = socket.getReceiveTimeout();
if (!limit_max_timeout || old_send_timeout > send_timeout)
socket.setSendTimeout(send_timeout);
if (!limit_max_timeout || old_receive_timeout > receive_timeout)
socket.setReceiveTimeout(receive_timeout);
}
TimeoutSetter::TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout)
: TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout)
{
}
TimeoutSetter::~TimeoutSetter()
{
try
{
socket.setSendTimeout(old_send_timeout);
socket.setReceiveTimeout(old_receive_timeout);
}
catch (std::exception & e)
{
// Sometimes catched on macos
LOG_ERROR(&Logger::get("Client"), std::string{"TimeoutSetter: Can't reset timeouts: "} + e.what());
}
}
}

View File

@ -1,47 +1,30 @@
#pragma once
#include <Poco/Timespan.h>
#include <Poco/Net/StreamSocket.h>
#include <Poco/Timespan.h>
namespace DB
{
/// Temporarily overrides socket send/recieve timeouts and reset them back into destructor
/// Temporarily overrides socket send/receive timeouts and reset them back into destructor
/// If "limit_max_timeout" is true, timeouts could be only decreased (maxed by previous value).
struct TimeoutSetter
{
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & send_timeout_, const Poco::Timespan & recieve_timeout_,
bool limit_max_timeout = false)
: socket(socket_), send_timeout(send_timeout_), recieve_timeout(recieve_timeout_)
{
old_send_timeout = socket.getSendTimeout();
old_receive_timeout = socket.getReceiveTimeout();
TimeoutSetter(Poco::Net::StreamSocket & socket_,
const Poco::Timespan & send_timeout_,
const Poco::Timespan & receive_timeout_,
bool limit_max_timeout = false);
if (!limit_max_timeout || old_send_timeout > send_timeout)
socket.setSendTimeout(send_timeout);
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout = false);
if (!limit_max_timeout || old_receive_timeout > recieve_timeout)
socket.setReceiveTimeout(recieve_timeout);
}
TimeoutSetter(Poco::Net::StreamSocket & socket_, const Poco::Timespan & timeout_, bool limit_max_timeout = false)
: TimeoutSetter(socket_, timeout_, timeout_, limit_max_timeout) {}
~TimeoutSetter()
{
socket.setSendTimeout(old_send_timeout);
socket.setReceiveTimeout(old_receive_timeout);
}
~TimeoutSetter();
Poco::Net::StreamSocket & socket;
Poco::Timespan send_timeout;
Poco::Timespan recieve_timeout;
Poco::Timespan receive_timeout;
Poco::Timespan old_send_timeout;
Poco::Timespan old_receive_timeout;
};
}

View File

@ -32,7 +32,7 @@ ColumnPtr ColumnConst::convertToFullColumn() const
ColumnPtr ColumnConst::removeLowCardinality() const
{
return ColumnConst::create(data->convertToFullColumnIfWithDictionary(), s);
return ColumnConst::create(data->convertToFullColumnIfLowCardinality(), s);
}
ColumnPtr ColumnConst::filter(const Filter & filt, ssize_t /*result_size_hint*/) const

View File

@ -1,4 +1,4 @@
#include <Columns/ColumnWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnsNumber.h>
#include <DataStreams/ColumnGathererStream.h>
#include <DataTypes/NumberTraits.h>
@ -109,34 +109,34 @@ namespace
}
ColumnWithDictionary::ColumnWithDictionary(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_, bool is_shared)
ColumnLowCardinality::ColumnLowCardinality(MutableColumnPtr && column_unique_, MutableColumnPtr && indexes_, bool is_shared)
: dictionary(std::move(column_unique_), is_shared), idx(std::move(indexes_))
{
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insert(const Field & x)
void ColumnLowCardinality::insert(const Field & x)
{
compactIfSharedDictionary();
idx.insertPosition(dictionary.getColumnUnique().uniqueInsert(x));
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insertDefault()
void ColumnLowCardinality::insertDefault()
{
idx.insertPosition(getDictionary().getDefaultValueIndex());
}
void ColumnWithDictionary::insertFrom(const IColumn & src, size_t n)
void ColumnLowCardinality::insertFrom(const IColumn & src, size_t n)
{
auto * src_with_dict = typeid_cast<const ColumnWithDictionary *>(&src);
auto * low_cardinality_src = typeid_cast<const ColumnLowCardinality *>(&src);
if (!src_with_dict)
throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
if (!low_cardinality_src)
throw Exception("Expected ColumnLowCardinality, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
size_t position = src_with_dict->getIndexes().getUInt(n);
size_t position = low_cardinality_src->getIndexes().getUInt(n);
if (&src_with_dict->getDictionary() == &getDictionary())
if (&low_cardinality_src->getDictionary() == &getDictionary())
{
/// Dictionary is shared with src column. Insert only index.
idx.insertPosition(position);
@ -144,31 +144,31 @@ void ColumnWithDictionary::insertFrom(const IColumn & src, size_t n)
else
{
compactIfSharedDictionary();
const auto & nested = *src_with_dict->getDictionary().getNestedColumn();
const auto & nested = *low_cardinality_src->getDictionary().getNestedColumn();
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(nested, position));
}
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insertFromFullColumn(const IColumn & src, size_t n)
void ColumnLowCardinality::insertFromFullColumn(const IColumn & src, size_t n)
{
compactIfSharedDictionary();
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertFrom(src, n));
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, size_t length)
void ColumnLowCardinality::insertRangeFrom(const IColumn & src, size_t start, size_t length)
{
auto * src_with_dict = typeid_cast<const ColumnWithDictionary *>(&src);
auto * low_cardinality_src = typeid_cast<const ColumnLowCardinality *>(&src);
if (!src_with_dict)
throw Exception("Expected ColumnWithDictionary, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
if (!low_cardinality_src)
throw Exception("Expected ColumnLowCardinality, got" + src.getName(), ErrorCodes::ILLEGAL_COLUMN);
if (&src_with_dict->getDictionary() == &getDictionary())
if (&low_cardinality_src->getDictionary() == &getDictionary())
{
/// Dictionary is shared with src column. Insert only indexes.
idx.insertPositionsRange(src_with_dict->getIndexes(), start, length);
idx.insertPositionsRange(low_cardinality_src->getIndexes(), start, length);
}
else
{
@ -176,10 +176,10 @@ void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, si
/// TODO: Support native insertion from other unique column. It will help to avoid null map creation.
auto sub_idx = (*src_with_dict->getIndexes().cut(start, length)).mutate();
auto sub_idx = (*low_cardinality_src->getIndexes().cut(start, length)).mutate();
auto idx_map = mapUniqueIndex(*sub_idx);
auto src_nested = src_with_dict->getDictionary().getNestedColumn();
auto src_nested = low_cardinality_src->getDictionary().getNestedColumn();
auto used_keys = src_nested->index(*idx_map, 0);
auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(*used_keys, 0, used_keys->size());
@ -188,7 +188,7 @@ void ColumnWithDictionary::insertRangeFrom(const IColumn & src, size_t start, si
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length)
void ColumnLowCardinality::insertRangeFromFullColumn(const IColumn & src, size_t start, size_t length)
{
compactIfSharedDictionary();
auto inserted_indexes = dictionary.getColumnUnique().uniqueInsertRangeFrom(src, start, length);
@ -196,7 +196,7 @@ void ColumnWithDictionary::insertRangeFromFullColumn(const IColumn & src, size_t
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions)
void ColumnLowCardinality::insertRangeFromDictionaryEncodedColumn(const IColumn & keys, const IColumn & positions)
{
Index(positions.getPtr()).check(keys.size());
compactIfSharedDictionary();
@ -205,26 +205,26 @@ void ColumnWithDictionary::insertRangeFromDictionaryEncodedColumn(const IColumn
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insertData(const char * pos, size_t length)
void ColumnLowCardinality::insertData(const char * pos, size_t length)
{
compactIfSharedDictionary();
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertData(pos, length));
idx.check(getDictionary().size());
}
void ColumnWithDictionary::insertDataWithTerminatingZero(const char * pos, size_t length)
void ColumnLowCardinality::insertDataWithTerminatingZero(const char * pos, size_t length)
{
compactIfSharedDictionary();
idx.insertPosition(dictionary.getColumnUnique().uniqueInsertDataWithTerminatingZero(pos, length));
idx.check(getDictionary().size());
}
StringRef ColumnWithDictionary::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
StringRef ColumnLowCardinality::serializeValueIntoArena(size_t n, Arena & arena, char const *& begin) const
{
return getDictionary().serializeValueIntoArena(getIndexes().getUInt(n), arena, begin);
}
const char * ColumnWithDictionary::deserializeAndInsertFromArena(const char * pos)
const char * ColumnLowCardinality::deserializeAndInsertFromArena(const char * pos)
{
compactIfSharedDictionary();
@ -235,26 +235,26 @@ const char * ColumnWithDictionary::deserializeAndInsertFromArena(const char * po
return new_pos;
}
void ColumnWithDictionary::gather(ColumnGathererStream & gatherer)
void ColumnLowCardinality::gather(ColumnGathererStream & gatherer)
{
gatherer.gather(*this);
}
MutableColumnPtr ColumnWithDictionary::cloneResized(size_t size) const
MutableColumnPtr ColumnLowCardinality::cloneResized(size_t size) const
{
auto unique_ptr = dictionary.getColumnUniquePtr();
return ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size));
return ColumnLowCardinality::create((*std::move(unique_ptr)).mutate(), getIndexes().cloneResized(size));
}
int ColumnWithDictionary::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const
int ColumnLowCardinality::compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const
{
const auto & column_with_dictionary = static_cast<const ColumnWithDictionary &>(rhs);
const auto & low_cardinality_column = static_cast<const ColumnLowCardinality &>(rhs);
size_t n_index = getIndexes().getUInt(n);
size_t m_index = column_with_dictionary.getIndexes().getUInt(m);
return getDictionary().compareAt(n_index, m_index, column_with_dictionary.getDictionary(), nan_direction_hint);
size_t m_index = low_cardinality_column.getIndexes().getUInt(m);
return getDictionary().compareAt(n_index, m_index, low_cardinality_column.getDictionary(), nan_direction_hint);
}
void ColumnWithDictionary::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
void ColumnLowCardinality::getPermutation(bool reverse, size_t limit, int nan_direction_hint, Permutation & res) const
{
if (limit == 0)
limit = size();
@ -289,65 +289,65 @@ void ColumnWithDictionary::getPermutation(bool reverse, size_t limit, int nan_di
}
}
std::vector<MutableColumnPtr> ColumnWithDictionary::scatter(ColumnIndex num_columns, const Selector & selector) const
std::vector<MutableColumnPtr> ColumnLowCardinality::scatter(ColumnIndex num_columns, const Selector & selector) const
{
auto columns = getIndexes().scatter(num_columns, selector);
for (auto & column : columns)
{
auto unique_ptr = dictionary.getColumnUniquePtr();
column = ColumnWithDictionary::create((*std::move(unique_ptr)).mutate(), std::move(column));
column = ColumnLowCardinality::create((*std::move(unique_ptr)).mutate(), std::move(column));
}
return columns;
}
void ColumnWithDictionary::setSharedDictionary(const ColumnPtr & column_unique)
void ColumnLowCardinality::setSharedDictionary(const ColumnPtr & column_unique)
{
if (!empty())
throw Exception("Can't set ColumnUnique for ColumnWithDictionary because is't not empty.",
throw Exception("Can't set ColumnUnique for ColumnLowCardinality because is't not empty.",
ErrorCodes::LOGICAL_ERROR);
dictionary.setShared(column_unique);
}
ColumnWithDictionary::MutablePtr ColumnWithDictionary::compact()
ColumnLowCardinality::MutablePtr ColumnLowCardinality::compact()
{
auto positions = idx.getPositions();
/// Create column with new indexes and old dictionary.
auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), (*std::move(positions)).mutate());
auto column = ColumnLowCardinality::create(getDictionary().assumeMutable(), (*std::move(positions)).mutate());
/// Will create new dictionary.
column->compactInplace();
return column;
}
ColumnWithDictionary::MutablePtr ColumnWithDictionary::cutAndCompact(size_t start, size_t length) const
ColumnLowCardinality::MutablePtr ColumnLowCardinality::cutAndCompact(size_t start, size_t length) const
{
auto sub_positions = (*idx.getPositions()->cut(start, length)).mutate();
/// Create column with new indexes and old dictionary.
auto column = ColumnWithDictionary::create(getDictionary().assumeMutable(), std::move(sub_positions));
auto column = ColumnLowCardinality::create(getDictionary().assumeMutable(), std::move(sub_positions));
/// Will create new dictionary.
column->compactInplace();
return column;
}
void ColumnWithDictionary::compactInplace()
void ColumnLowCardinality::compactInplace()
{
auto positions = idx.detachPositions();
dictionary.compact(positions);
idx.attachPositions(std::move(positions));
}
void ColumnWithDictionary::compactIfSharedDictionary()
void ColumnLowCardinality::compactIfSharedDictionary()
{
if (dictionary.isShared())
compactInplace();
}
ColumnWithDictionary::DictionaryEncodedColumn
ColumnWithDictionary::getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const
ColumnLowCardinality::DictionaryEncodedColumn
ColumnLowCardinality::getMinimalDictionaryEncodedColumn(size_t offset, size_t limit) const
{
MutableColumnPtr sub_indexes = (*std::move(idx.getPositions()->cut(offset, limit))).mutate();
auto indexes_map = mapUniqueIndex(*sub_indexes);
@ -356,7 +356,7 @@ ColumnWithDictionary::getMinimalDictionaryEncodedColumn(size_t offset, size_t li
return {std::move(sub_keys), std::move(sub_indexes)};
}
ColumnPtr ColumnWithDictionary::countKeys() const
ColumnPtr ColumnLowCardinality::countKeys() const
{
const auto & nested_column = getDictionary().getNestedColumn();
size_t dict_size = nested_column->size();
@ -368,20 +368,20 @@ ColumnPtr ColumnWithDictionary::countKeys() const
ColumnWithDictionary::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {}
ColumnLowCardinality::Index::Index() : positions(ColumnUInt8::create()), size_of_type(sizeof(UInt8)) {}
ColumnWithDictionary::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions))
ColumnLowCardinality::Index::Index(MutableColumnPtr && positions) : positions(std::move(positions))
{
updateSizeOfType();
}
ColumnWithDictionary::Index::Index(ColumnPtr positions) : positions(std::move(positions))
ColumnLowCardinality::Index::Index(ColumnPtr positions) : positions(std::move(positions))
{
updateSizeOfType();
}
template <typename Callback>
void ColumnWithDictionary::Index::callForType(Callback && callback, size_t size_of_type)
void ColumnLowCardinality::Index::callForType(Callback && callback, size_t size_of_type)
{
switch (size_of_type)
{
@ -390,13 +390,13 @@ void ColumnWithDictionary::Index::callForType(Callback && callback, size_t size_
case sizeof(UInt32): { callback(UInt32()); break; }
case sizeof(UInt64): { callback(UInt64()); break; }
default: {
throw Exception("Unexpected size of index type for ColumnWithDictionary: " + toString(size_of_type),
throw Exception("Unexpected size of index type for ColumnLowCardinality: " + toString(size_of_type),
ErrorCodes::LOGICAL_ERROR);
}
}
}
size_t ColumnWithDictionary::Index::getSizeOfIndexType(const IColumn & column, size_t hint)
size_t ColumnLowCardinality::Index::getSizeOfIndexType(const IColumn & column, size_t hint)
{
auto checkFor = [&](auto type) { return typeid_cast<const ColumnVector<decltype(type)> *>(&column) != nullptr; };
auto tryGetSizeFor = [&](auto type) -> size_t { return checkFor(type) ? sizeof(decltype(type)) : 0; };
@ -419,22 +419,22 @@ size_t ColumnWithDictionary::Index::getSizeOfIndexType(const IColumn & column, s
if (auto size = tryGetSizeFor(UInt64()))
return size;
throw Exception("Unexpected indexes type for ColumnWithDictionary. Expected UInt, got " + column.getName(),
throw Exception("Unexpected indexes type for ColumnLowCardinality. Expected UInt, got " + column.getName(),
ErrorCodes::ILLEGAL_COLUMN);
}
void ColumnWithDictionary::Index::attachPositions(ColumnPtr positions_)
void ColumnLowCardinality::Index::attachPositions(ColumnPtr positions_)
{
positions = std::move(positions_);
updateSizeOfType();
}
template <typename IndexType>
typename ColumnVector<IndexType>::Container & ColumnWithDictionary::Index::getPositionsData()
typename ColumnVector<IndexType>::Container & ColumnLowCardinality::Index::getPositionsData()
{
auto * positions_ptr = typeid_cast<ColumnVector<IndexType> *>(positions->assumeMutable().get());
if (!positions_ptr)
throw Exception("Invalid indexes type for ColumnWithDictionary."
throw Exception("Invalid indexes type for ColumnLowCardinality."
" Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(),
ErrorCodes::LOGICAL_ERROR);
@ -442,11 +442,11 @@ typename ColumnVector<IndexType>::Container & ColumnWithDictionary::Index::getPo
}
template <typename IndexType>
const typename ColumnVector<IndexType>::Container & ColumnWithDictionary::Index::getPositionsData() const
const typename ColumnVector<IndexType>::Container & ColumnLowCardinality::Index::getPositionsData() const
{
const auto * positions_ptr = typeid_cast<const ColumnVector<IndexType> *>(positions.get());
if (!positions_ptr)
throw Exception("Invalid indexes type for ColumnWithDictionary."
throw Exception("Invalid indexes type for ColumnLowCardinality."
" Expected UInt" + toString(8 * sizeof(IndexType)) + ", got " + positions->getName(),
ErrorCodes::LOGICAL_ERROR);
@ -454,7 +454,7 @@ const typename ColumnVector<IndexType>::Container & ColumnWithDictionary::Index:
}
template <typename IndexType>
void ColumnWithDictionary::Index::convertPositions()
void ColumnLowCardinality::Index::convertPositions()
{
auto convert = [&](auto x)
{
@ -485,14 +485,14 @@ void ColumnWithDictionary::Index::convertPositions()
checkSizeOfType();
}
void ColumnWithDictionary::Index::expandType()
void ColumnLowCardinality::Index::expandType()
{
auto expand = [&](auto type)
{
using CurIndexType = decltype(type);
constexpr auto next_size = NumberTraits::nextSize(sizeof(CurIndexType));
if (next_size == sizeof(CurIndexType))
throw Exception("Can't expand indexes type for ColumnWithDictionary from type: "
throw Exception("Can't expand indexes type for ColumnLowCardinality from type: "
+ demangle(typeid(CurIndexType).name()), ErrorCodes::LOGICAL_ERROR);
using NewIndexType = typename NumberTraits::Construct<false, false, next_size>::Type;
@ -502,14 +502,14 @@ void ColumnWithDictionary::Index::expandType()
callForType(std::move(expand), size_of_type);
}
UInt64 ColumnWithDictionary::Index::getMaxPositionForCurrentType() const
UInt64 ColumnLowCardinality::Index::getMaxPositionForCurrentType() const
{
UInt64 value = 0;
callForType([&](auto type) { value = std::numeric_limits<decltype(type)>::max(); }, size_of_type);
return value;
}
size_t ColumnWithDictionary::Index::getPositionAt(size_t row) const
size_t ColumnLowCardinality::Index::getPositionAt(size_t row) const
{
size_t pos;
auto getPosition = [&](auto type)
@ -522,7 +522,7 @@ size_t ColumnWithDictionary::Index::getPositionAt(size_t row) const
return pos;
}
void ColumnWithDictionary::Index::insertPosition(UInt64 position)
void ColumnLowCardinality::Index::insertPosition(UInt64 position)
{
while (position > getMaxPositionForCurrentType())
expandType();
@ -531,7 +531,7 @@ void ColumnWithDictionary::Index::insertPosition(UInt64 position)
checkSizeOfType();
}
void ColumnWithDictionary::Index::insertPositionsRange(const IColumn & column, size_t offset, size_t limit)
void ColumnLowCardinality::Index::insertPositionsRange(const IColumn & column, size_t offset, size_t limit)
{
auto insertForType = [&](auto type)
{
@ -571,13 +571,13 @@ void ColumnWithDictionary::Index::insertPositionsRange(const IColumn & column, s
!insertForType(UInt16()) &&
!insertForType(UInt32()) &&
!insertForType(UInt64()))
throw Exception("Invalid column for ColumnWithDictionary index. Expected UInt, got " + column.getName(),
throw Exception("Invalid column for ColumnLowCardinality index. Expected UInt, got " + column.getName(),
ErrorCodes::ILLEGAL_COLUMN);
checkSizeOfType();
}
void ColumnWithDictionary::Index::check(size_t /*max_dictionary_size*/)
void ColumnLowCardinality::Index::check(size_t /*max_dictionary_size*/)
{
/// TODO: remove
/*
@ -601,14 +601,14 @@ void ColumnWithDictionary::Index::check(size_t /*max_dictionary_size*/)
*/
}
void ColumnWithDictionary::Index::checkSizeOfType()
void ColumnLowCardinality::Index::checkSizeOfType()
{
if (size_of_type != getSizeOfIndexType(*positions, size_of_type))
throw Exception("Invalid size of type. Expected " + toString(8 * size_of_type) +
", but positions are " + positions->getName(), ErrorCodes::LOGICAL_ERROR);
}
void ColumnWithDictionary::Index::countKeys(ColumnUInt64::Container & counts) const
void ColumnLowCardinality::Index::countKeys(ColumnUInt64::Container & counts) const
{
auto counter = [&](auto x)
{
@ -621,25 +621,25 @@ void ColumnWithDictionary::Index::countKeys(ColumnUInt64::Container & counts) co
}
ColumnWithDictionary::Dictionary::Dictionary(MutableColumnPtr && column_unique_, bool is_shared)
ColumnLowCardinality::Dictionary::Dictionary(MutableColumnPtr && column_unique_, bool is_shared)
: column_unique(std::move(column_unique_)), shared(is_shared)
{
checkColumn(*column_unique);
}
ColumnWithDictionary::Dictionary::Dictionary(ColumnPtr column_unique_, bool is_shared)
ColumnLowCardinality::Dictionary::Dictionary(ColumnPtr column_unique_, bool is_shared)
: column_unique(std::move(column_unique_)), shared(is_shared)
{
checkColumn(*column_unique);
}
void ColumnWithDictionary::Dictionary::checkColumn(const IColumn & column)
void ColumnLowCardinality::Dictionary::checkColumn(const IColumn & column)
{
if (!dynamic_cast<const IColumnUnique *>(&column))
throw Exception("ColumnUnique expected as an argument of ColumnWithDictionary.", ErrorCodes::ILLEGAL_COLUMN);
throw Exception("ColumnUnique expected as an argument of ColumnLowCardinality.", ErrorCodes::ILLEGAL_COLUMN);
}
void ColumnWithDictionary::Dictionary::setShared(const ColumnPtr & dictionary)
void ColumnLowCardinality::Dictionary::setShared(const ColumnPtr & dictionary)
{
checkColumn(*dictionary);
@ -647,7 +647,7 @@ void ColumnWithDictionary::Dictionary::setShared(const ColumnPtr & dictionary)
shared = true;
}
void ColumnWithDictionary::Dictionary::compact(ColumnPtr & positions)
void ColumnLowCardinality::Dictionary::compact(ColumnPtr & positions)
{
auto new_column_unique = column_unique->cloneEmpty();

View File

@ -13,21 +13,21 @@ namespace ErrorCodes
extern const int ILLEGAL_COLUMN;
}
class ColumnWithDictionary final : public COWPtrHelper<IColumn, ColumnWithDictionary>
class ColumnLowCardinality final : public COWPtrHelper<IColumn, ColumnLowCardinality>
{
friend class COWPtrHelper<IColumn, ColumnWithDictionary>;
friend class COWPtrHelper<IColumn, ColumnLowCardinality>;
ColumnWithDictionary(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false);
ColumnWithDictionary(const ColumnWithDictionary & other) = default;
ColumnLowCardinality(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false);
ColumnLowCardinality(const ColumnLowCardinality & other) = default;
public:
/** Create immutable column using immutable arguments. This arguments may be shared with other columns.
* Use IColumn::mutate in order to make mutable column and mutate shared nested columns.
*/
using Base = COWPtrHelper<IColumn, ColumnWithDictionary>;
using Base = COWPtrHelper<IColumn, ColumnLowCardinality>;
static Ptr create(const ColumnPtr & column_unique_, const ColumnPtr & indexes_, bool is_shared = false)
{
return ColumnWithDictionary::create(column_unique_->assumeMutable(), indexes_->assumeMutable(), is_shared);
return ColumnLowCardinality::create(column_unique_->assumeMutable(), indexes_->assumeMutable(), is_shared);
}
static MutablePtr create(MutableColumnPtr && column_unique, MutableColumnPtr && indexes, bool is_shared = false)
@ -35,11 +35,11 @@ public:
return Base::create(std::move(column_unique), std::move(indexes), is_shared);
}
std::string getName() const override { return "ColumnWithDictionary"; }
const char * getFamilyName() const override { return "ColumnWithDictionary"; }
std::string getName() const override { return "ColumnLowCardinality"; }
const char * getFamilyName() const override { return "ColumnLowCardinality"; }
ColumnPtr convertToFullColumn() const { return getDictionary().getNestedColumn()->index(getIndexes(), 0); }
ColumnPtr convertToFullColumnIfWithDictionary() const override { return convertToFullColumn(); }
ColumnPtr convertToFullColumnIfLowCardinality() const override { return convertToFullColumn(); }
MutableColumnPtr cloneResized(size_t size) const override;
size_t size() const override { return getIndexes().size(); }
@ -59,7 +59,7 @@ public:
bool isNullAt(size_t n) const override { return getDictionary().isNullAt(getIndexes().getUInt(n)); }
ColumnPtr cut(size_t start, size_t length) const override
{
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().cut(start, length));
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().cut(start, length));
}
void insert(const Field & x) override;
@ -89,17 +89,17 @@ public:
ColumnPtr filter(const Filter & filt, ssize_t result_size_hint) const override
{
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint));
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().filter(filt, result_size_hint));
}
ColumnPtr permute(const Permutation & perm, size_t limit) const override
{
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().permute(perm, limit));
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().permute(perm, limit));
}
ColumnPtr index(const IColumn & indexes_, size_t limit) const override
{
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit));
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().index(indexes_, limit));
}
int compareAt(size_t n, size_t m, const IColumn & rhs, int nan_direction_hint) const override;
@ -108,7 +108,7 @@ public:
ColumnPtr replicate(const Offsets & offsets) const override
{
return ColumnWithDictionary::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets));
return ColumnLowCardinality::create(dictionary.getColumnUniquePtr(), getIndexes().replicate(offsets));
}
std::vector<MutableColumnPtr> scatter(ColumnIndex num_columns, const Selector & selector) const override;
@ -138,7 +138,7 @@ public:
bool isFixedAndContiguous() const override { return getDictionary().isFixedAndContiguous(); }
size_t sizeOfValueIfFixed() const override { return getDictionary().sizeOfValueIfFixed(); }
bool isNumeric() const override { return getDictionary().isNumeric(); }
bool withDictionary() const override { return true; }
bool lowCardinality() const override { return true; }
const IColumnUnique & getDictionary() const { return dictionary.getColumnUnique(); }
const ColumnPtr & getDictionaryPtr() const { return dictionary.getColumnUniquePtr(); }
@ -166,7 +166,7 @@ public:
///void setIndexes(MutableColumnPtr && indexes_) { indexes = std::move(indexes_); }
/// Set shared ColumnUnique for empty column with dictionary.
/// Set shared ColumnUnique for empty low cardinality column.
void setSharedDictionary(const ColumnPtr & column_unique);
bool isSharedDictionary() const { return dictionary.isShared(); }

View File

@ -27,7 +27,7 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column)
if (column.isColumnConst())
{
const ColumnConst & column_const = static_cast<const ColumnConst &>(column);
ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfWithDictionary();
ColumnPtr column_nested = column_const.getDataColumnPtr()->convertToFullColumnIfLowCardinality();
if (!typeid_cast<const ColumnUInt8 *>(column_nested.get()))
{
@ -50,8 +50,8 @@ ConstantFilterDescription::ConstantFilterDescription(const IColumn & column)
FilterDescription::FilterDescription(const IColumn & column_)
{
if (column_.withDictionary())
data_holder = column_.convertToFullColumnIfWithDictionary();
if (column_.lowCardinality())
data_holder = column_.convertToFullColumnIfLowCardinality();
const auto & column = data_holder ? *data_holder : column_;

View File

@ -47,9 +47,9 @@ public:
*/
virtual Ptr convertToFullColumnIfConst() const { return {}; }
/// If column isn't ColumnWithDictionary, return itself.
/// If column is ColumnWithDictionary, transforms is to full column.
virtual Ptr convertToFullColumnIfWithDictionary() const { return getPtr(); }
/// If column isn't ColumnLowCardinality, return itself.
/// If column is ColumnLowCardinality, transforms is to full column.
virtual Ptr convertToFullColumnIfLowCardinality() const { return getPtr(); }
/// Creates empty column with the same type.
virtual MutablePtr cloneEmpty() const { return cloneResized(0); }
@ -333,7 +333,7 @@ public:
/// Can be inside ColumnNullable.
virtual bool canBeInsideNullable() const { return false; }
virtual bool withDictionary() const { return false; }
virtual bool lowCardinality() const { return false; }
virtual ~IColumn() {}

View File

@ -23,7 +23,7 @@ Macros::Macros(const Poco::Util::AbstractConfiguration & config, const String &
}
}
String Macros::expand(const String & s, size_t level) const
String Macros::expand(const String & s, size_t level, const String & database_name, const String & table_name) const
{
if (s.find('{') == String::npos)
return s;
@ -56,17 +56,27 @@ String Macros::expand(const String & s, size_t level) const
throw Exception("Unbalanced { and } in string with macros: '" + s + "'", ErrorCodes::SYNTAX_ERROR);
String macro_name = s.substr(begin, end - begin);
auto it = macros.find(macro_name);
if (it == macros.end())
throw Exception("No macro " + macro_name + " in config", ErrorCodes::SYNTAX_ERROR);
res += it->second;
/// Prefer explicit macros over implicit.
if (it != macros.end())
res += it->second;
else if (macro_name == "database" && !database_name.empty())
res += database_name;
else if (macro_name == "table" && !table_name.empty())
res += table_name;
else
throw Exception("No macro " + macro_name + " in config", ErrorCodes::SYNTAX_ERROR);
pos = end + 1;
}
return expand(res, level + 1);
return expand(res, level + 1, database_name, table_name);
}
String Macros::expand(const String & s, const String & database_name, const String & table_name) const
{
return expand(s, 0, database_name, table_name);
}
Names Macros::expand(const Names & source_names, size_t level) const

View File

@ -2,6 +2,7 @@
#include <Core/Types.h>
#include <Core/Names.h>
#include <map>
@ -27,9 +28,13 @@ public:
Macros(const Poco::Util::AbstractConfiguration & config, const String & key);
/** Replace the substring of the form {macro_name} with the value for macro_name, obtained from the config file.
* If {database} and {table} macros aren`t defined explicitly, expand them as database_name and table_name respectively.
* level - the level of recursion.
*/
String expand(const String & s, size_t level = 0) const;
String expand(const String & s, size_t level = 0, const String & database_name = "", const String & table_name = "") const;
String expand(const String & s, const String & database_name, const String & table_name) const;
/** Apply expand for the list.
*/

View File

@ -0,0 +1,134 @@
#include <Common/UTF8Helpers.h>
#include <widechar_width.h>
namespace DB
{
namespace UTF8
{
// based on https://bjoern.hoehrmann.de/utf-8/decoder/dfa/
// Copyright (c) 2008-2009 Bjoern Hoehrmann <bjoern@hoehrmann.de>
// Permission is hereby granted, free of charge, to any person obtaining a copy
// of this software and associated documentation files (the "Software"), to deal
// in the Software without restriction, including without limitation the rights
// to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
// copies of the Software, and to permit persons to whom the Software is
// furnished to do so, subject to the following conditions: The above copyright
// notice and this permission notice shall be included in all copies or
// substantial portions of the Software.
// THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
// IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
// FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
// AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
// LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
// OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE
// SOFTWARE.
static const UInt8 TABLE[] =
{
0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 00..1f
0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 20..3f
0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 40..5f
0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0,0, // 60..7f
1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,9,9,9,9,9,9,9,9,9,9,9,9,9,9,9,9, // 80..9f
7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7,7, // a0..bf
8,8,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2,2, // c0..df
0xa,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x3,0x4,0x3,0x3, // e0..ef
0xb,0x6,0x6,0x6,0x5,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8,0x8, // f0..ff
0x0,0x1,0x2,0x3,0x5,0x8,0x7,0x1,0x1,0x1,0x4,0x6,0x1,0x1,0x1,0x1, // s0..s0
1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,1,0,1,1,1,1,1,0,1,0,1,1,1,1,1,1, // s1..s2
1,2,1,1,1,1,1,2,1,2,1,1,1,1,1,1,1,1,1,1,1,1,1,2,1,1,1,1,1,1,1,1, // s3..s4
1,2,1,1,1,1,1,1,1,2,1,1,1,1,1,1,1,1,1,1,1,1,1,3,1,3,1,1,1,1,1,1, // s5..s6
1,3,1,1,1,1,1,3,1,3,1,1,1,1,1,1,1,3,1,1,1,1,1,1,1,1,1,1,1,1,1,1, // s7..s8
};
struct UTF8Decoder
{
enum
{
ACCEPT = 0,
REJECT = 1
};
UInt32 decode(UInt8 byte)
{
UInt32 type = TABLE[byte];
codepoint = (state != ACCEPT) ? (byte & 0x3fu) | (codepoint << 6) : (0xff >> type) & (byte);
state = TABLE[256 + state * 16 + type];
return state;
}
void reset()
{
state = ACCEPT;
codepoint = 0xfffdU;
}
UInt8 state {ACCEPT};
UInt32 codepoint {0};
};
static int wcwidth(wchar_t wc)
{
int width = widechar_wcwidth(wc);
switch (width)
{
case widechar_nonprint:
[[fallthrough]];
case widechar_combining:
[[fallthrough]];
case widechar_unassigned:
return 0;
case widechar_ambiguous:
[[fallthrough]];
case widechar_private_use:
[[fallthrough]];
case widechar_widened_in_9:
return 1;
default:
return width;
}
}
size_t computeWidth(const UInt8 * data, size_t size, size_t prefix) noexcept
{
UTF8Decoder decoder;
size_t width = 0;
size_t rollback = 0;
for (size_t i = 0; i < size; ++i)
{
switch (decoder.decode(data[i]))
{
case UTF8Decoder::REJECT:
decoder.reset();
// invalid sequences seem to have zero width in modern terminals
// tested in libvte-based, alacritty, urxvt and xterm
i -= rollback;
rollback = 0;
break;
case UTF8Decoder::ACCEPT:
// there are special control characters that manipulate the terminal output.
// (`0x08`, `0x09`, `0x0a`, `0x0b`, `0x0c`, `0x0d`, `0x1b`)
// Since we don't touch the original column data, there is no easy way to escape them.
// TODO: escape control characters
// TODO: multiline support for '\n'
// special treatment for '\t'
if (decoder.codepoint == '\t')
width += 8 - (prefix + width) % 8;
else
width += wcwidth(decoder.codepoint);
rollback = 0;
break;
// continue if we meet other values here
default:
++rollback;
}
}
// no need to handle trailing sequence as they have zero width
return width;
}
}
}

View File

@ -72,6 +72,11 @@ inline size_t countCodePoints(const UInt8 * data, size_t size)
return res;
}
/// returns UTF-8 wcswidth. Invalid sequence is treated as zero width character.
/// `prefix` is used to compute the `\t` width which extends the string before
/// and include `\t` to the nearest longer length with multiple of eight.
size_t computeWidth(const UInt8 * data, size_t size, size_t prefix = 0) noexcept;
}

View File

@ -1027,7 +1027,7 @@ void ZooKeeper::sendAuth(const String & scheme, const String & data)
read(err);
if (read_xid != auth_xid)
throw Exception("Unexpected event recieved in reply to auth request: " + toString(read_xid),
throw Exception("Unexpected event received in reply to auth request: " + toString(read_xid),
ZMARSHALLINGERROR);
int32_t actual_length = in->count() - count_before_event;

View File

@ -1,8 +1,8 @@
#pragma once
#include <DataStreams/IProfilingBlockInputStream.h>
#include <Columns/ColumnWithDictionary.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnConst.h>
namespace DB
@ -13,15 +13,15 @@ namespace DB
* Unlike UnionBlockInputStream, it does this sequentially.
* Blocks of different sources are not interleaved with each other.
*/
class ConvertColumnWithDictionaryToFullBlockInputStream : public IProfilingBlockInputStream
class ConvertColumnLowCardinalityToFullBlockInputStream : public IProfilingBlockInputStream
{
public:
explicit ConvertColumnWithDictionaryToFullBlockInputStream(const BlockInputStreamPtr & input)
explicit ConvertColumnLowCardinalityToFullBlockInputStream(const BlockInputStreamPtr & input)
{
children.push_back(input);
}
String getName() const override { return "ConvertColumnWithDictionaryToFull"; }
String getName() const override { return "ConvertColumnLowCardinalityToFull"; }
Block getHeader() const override { return convert(children.at(0)->getHeader()); }
@ -36,9 +36,9 @@ private:
if (auto * column_const = typeid_cast<const ColumnConst *>(column.column.get()))
column.column = column_const->removeLowCardinality();
else
column.column = column.column->convertToFullColumnIfWithDictionary();
column.column = column.column->convertToFullColumnIfLowCardinality();
if (auto * low_cardinality_type = typeid_cast<const DataTypeWithDictionary *>(column.type.get()))
if (auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(column.type.get()))
column.type = low_cardinality_type->getDictionaryType();
}

View File

@ -2,9 +2,12 @@
#include <DataStreams/SquashingBlockInputStream.h>
#include <DataTypes/NestedUtils.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Common/CurrentThread.h>
#include <Common/setThreadName.h>
#include <Common/getNumberOfPhysicalCPUCores.h>
#include <common/ThreadPool.h>
#include <Storages/MergeTree/ReplicatedMergeTreeBlockOutputStream.h>
namespace DB
{
@ -73,35 +76,30 @@ void PushingToViewsBlockOutputStream::write(const Block & block)
if (replicated_output && replicated_output->lastBlockIsDuplicate())
return;
/// Insert data into materialized views only after successful insert into main table
for (auto & view : views)
// Insert data into materialized views only after successful insert into main table
const Settings & settings = context.getSettingsRef();
if (settings.parallel_view_processing && views.size() > 1)
{
try
// Push to views concurrently if enabled, and more than one view is attached
ThreadPool pool(std::min(size_t(settings.max_threads), views.size()));
for (size_t view_num = 0; view_num < views.size(); ++view_num)
{
BlockInputStreamPtr from = std::make_shared<OneBlockInputStream>(block);
InterpreterSelectQuery select(view.query, *views_context, from);
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
/// Squashing is needed here because the materialized view query can generate a lot of blocks
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
/// and two-level aggregation is triggered).
in = std::make_shared<SquashingBlockInputStream>(
in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes);
in->readPrefix();
while (Block result_block = in->read())
auto thread_group = CurrentThread::getGroup();
pool.schedule([=]
{
Nested::validateArraySizes(result_block);
view.out->write(result_block);
}
in->readSuffix();
}
catch (Exception & ex)
{
ex.addMessage("while pushing to view " + view.database + "." + view.table);
throw;
setThreadName("PushingToViewsBlockOutputStream");
CurrentThread::attachToIfDetached(thread_group);
process(block, view_num);
});
}
// Wait for concurrent view processing
pool.wait();
}
else
{
// Process sequentially
for (size_t view_num = 0; view_num < views.size(); ++view_num)
process(block, view_num);
}
}
@ -152,4 +150,36 @@ void PushingToViewsBlockOutputStream::flush()
view.out->flush();
}
void PushingToViewsBlockOutputStream::process(const Block & block, size_t view_num)
{
auto & view = views[view_num];
try
{
BlockInputStreamPtr from = std::make_shared<OneBlockInputStream>(block);
InterpreterSelectQuery select(view.query, *views_context, from);
BlockInputStreamPtr in = std::make_shared<MaterializingBlockInputStream>(select.execute().in);
/// Squashing is needed here because the materialized view query can generate a lot of blocks
/// even when only one block is inserted into the parent table (e.g. if the query is a GROUP BY
/// and two-level aggregation is triggered).
in = std::make_shared<SquashingBlockInputStream>(
in, context.getSettingsRef().min_insert_block_size_rows, context.getSettingsRef().min_insert_block_size_bytes);
in->readPrefix();
while (Block result_block = in->read())
{
Nested::validateArraySizes(result_block);
view.out->write(result_block);
}
in->readSuffix();
}
catch (Exception & ex)
{
ex.addMessage("while pushing to view " + backQuoteIfNeed(view.database) + "." + backQuoteIfNeed(view.table));
throw;
}
}
}

View File

@ -47,6 +47,8 @@ private:
std::vector<ViewInfo> views;
std::unique_ptr<Context> views_context;
void process(const Block & block, size_t view_num);
};

View File

@ -143,7 +143,7 @@ void registerDataTypeUUID(DataTypeFactory & factory);
void registerDataTypeAggregateFunction(DataTypeFactory & factory);
void registerDataTypeNested(DataTypeFactory & factory);
void registerDataTypeInterval(DataTypeFactory & factory);
void registerDataTypeWithDictionary(DataTypeFactory & factory);
void registerDataTypeLowCardinality(DataTypeFactory & factory);
DataTypeFactory::DataTypeFactory()
@ -163,7 +163,7 @@ DataTypeFactory::DataTypeFactory()
registerDataTypeAggregateFunction(*this);
registerDataTypeNested(*this);
registerDataTypeInterval(*this);
registerDataTypeWithDictionary(*this);
registerDataTypeLowCardinality(*this);
}
}

View File

@ -1,4 +1,4 @@
#include <Columns/ColumnWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
#include <Columns/ColumnUnique.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnsCommon.h>
@ -6,7 +6,7 @@
#include <Common/typeid_cast.h>
#include <Core/TypeListNumber.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
@ -24,18 +24,18 @@ namespace ErrorCodes
namespace
{
const ColumnWithDictionary & getColumnWithDictionary(const IColumn & column)
const ColumnLowCardinality & getColumnLowCardinality(const IColumn & column)
{
return typeid_cast<const ColumnWithDictionary &>(column);
return typeid_cast<const ColumnLowCardinality &>(column);
}
ColumnWithDictionary & getColumnWithDictionary(IColumn & column)
ColumnLowCardinality & getColumnLowCardinality(IColumn & column)
{
return typeid_cast<ColumnWithDictionary &>(column);
return typeid_cast<ColumnLowCardinality &>(column);
}
}
DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_)
DataTypeLowCardinality::DataTypeLowCardinality(DataTypePtr dictionary_type_)
: dictionary_type(std::move(dictionary_type_))
{
auto inner_type = dictionary_type;
@ -45,11 +45,11 @@ DataTypeWithDictionary::DataTypeWithDictionary(DataTypePtr dictionary_type_)
if (!isStringOrFixedString(inner_type)
&& !isDateOrDateTime(inner_type)
&& !isNumber(inner_type))
throw Exception("DataTypeWithDictionary is supported only for numbers, strings, Date or DateTime, but got "
throw Exception("DataTypeLowCardinality is supported only for numbers, strings, Date or DateTime, but got "
+ dictionary_type->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
void DataTypeWithDictionary::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
void DataTypeLowCardinality::enumerateStreams(const StreamCallback & callback, SubstreamPath & path) const
{
path.push_back(Substream::DictionaryKeys);
dictionary_type->enumerateStreams(callback, path);
@ -74,7 +74,7 @@ struct KeysSerializationVersion
static void checkVersion(UInt64 version)
{
if (version != SharedDictionariesWithAdditionalKeys)
throw Exception("Invalid version for DataTypeWithDictionary key column.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Invalid version for DataTypeLowCardinality key column.", ErrorCodes::LOGICAL_ERROR);
}
KeysSerializationVersion(UInt64 version) : value(static_cast<Value>(version)) { checkVersion(version); }
@ -115,7 +115,7 @@ struct IndexesSerializationType
if (value <= TUInt64)
return;
throw Exception("Invalid type for DataTypeWithDictionary index column.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Invalid type for DataTypeLowCardinality index column.", ErrorCodes::LOGICAL_ERROR);
}
void serialize(WriteBuffer & buffer) const
@ -179,15 +179,15 @@ struct IndexesSerializationType
IndexesSerializationType() = default;
};
struct SerializeStateWithDictionary : public IDataType::SerializeBinaryBulkState
struct SerializeStateLowCardinality : public IDataType::SerializeBinaryBulkState
{
KeysSerializationVersion key_version;
MutableColumnUniquePtr shared_dictionary;
explicit SerializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {}
explicit SerializeStateLowCardinality(UInt64 key_version) : key_version(key_version) {}
};
struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkState
struct DeserializeStateLowCardinality : public IDataType::DeserializeBinaryBulkState
{
KeysSerializationVersion key_version;
ColumnUniquePtr global_dictionary;
@ -197,46 +197,46 @@ struct DeserializeStateWithDictionary : public IDataType::DeserializeBinaryBulkS
ColumnPtr null_map;
UInt64 num_pending_rows = 0;
explicit DeserializeStateWithDictionary(UInt64 key_version) : key_version(key_version) {}
explicit DeserializeStateLowCardinality(UInt64 key_version) : key_version(key_version) {}
};
static SerializeStateWithDictionary * checkAndGetWithDictionarySerializeState(
static SerializeStateLowCardinality * checkAndGetLowCardinalitySerializeState(
IDataType::SerializeBinaryBulkStatePtr & state)
{
if (!state)
throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Got empty state for DataTypeLowCardinality.", ErrorCodes::LOGICAL_ERROR);
auto * with_dictionary_state = typeid_cast<SerializeStateWithDictionary *>(state.get());
if (!with_dictionary_state)
auto * low_cardinality_state = typeid_cast<SerializeStateLowCardinality *>(state.get());
if (!low_cardinality_state)
{
auto & state_ref = *state;
throw Exception("Invalid SerializeBinaryBulkState for DataTypeWithDictionary. Expected: "
+ demangle(typeid(SerializeStateWithDictionary).name()) + ", got "
throw Exception("Invalid SerializeBinaryBulkState for DataTypeLowCardinality. Expected: "
+ demangle(typeid(SerializeStateLowCardinality).name()) + ", got "
+ demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR);
}
return with_dictionary_state;
return low_cardinality_state;
}
static DeserializeStateWithDictionary * checkAndGetWithDictionaryDeserializeState(
static DeserializeStateLowCardinality * checkAndGetLowCardinalityDeserializeState(
IDataType::DeserializeBinaryBulkStatePtr & state)
{
if (!state)
throw Exception("Got empty state for DataTypeWithDictionary.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Got empty state for DataTypeLowCardinality.", ErrorCodes::LOGICAL_ERROR);
auto * with_dictionary_state = typeid_cast<DeserializeStateWithDictionary *>(state.get());
if (!with_dictionary_state)
auto * low_cardinality_state = typeid_cast<DeserializeStateLowCardinality *>(state.get());
if (!low_cardinality_state)
{
auto & state_ref = *state;
throw Exception("Invalid DeserializeBinaryBulkState for DataTypeWithDictionary. Expected: "
+ demangle(typeid(DeserializeStateWithDictionary).name()) + ", got "
throw Exception("Invalid DeserializeBinaryBulkState for DataTypeLowCardinality. Expected: "
+ demangle(typeid(DeserializeStateLowCardinality).name()) + ", got "
+ demangle(typeid(state_ref).name()), ErrorCodes::LOGICAL_ERROR);
}
return with_dictionary_state;
return low_cardinality_state;
}
void DataTypeWithDictionary::serializeBinaryBulkStatePrefix(
void DataTypeLowCardinality::serializeBinaryBulkStatePrefix(
SerializeBinaryBulkSettings & settings,
SerializeBinaryBulkStatePtr & state) const
{
@ -245,7 +245,7 @@ void DataTypeWithDictionary::serializeBinaryBulkStatePrefix(
settings.path.pop_back();
if (!stream)
throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStatePrefix",
throw Exception("Got empty stream in DataTypeLowCardinality::serializeBinaryBulkStatePrefix",
ErrorCodes::LOGICAL_ERROR);
/// Write version and create SerializeBinaryBulkState.
@ -253,36 +253,36 @@ void DataTypeWithDictionary::serializeBinaryBulkStatePrefix(
writeIntBinary(key_version, *stream);
state = std::make_shared<SerializeStateWithDictionary>(key_version);
state = std::make_shared<SerializeStateLowCardinality>(key_version);
}
void DataTypeWithDictionary::serializeBinaryBulkStateSuffix(
void DataTypeLowCardinality::serializeBinaryBulkStateSuffix(
SerializeBinaryBulkSettings & settings,
SerializeBinaryBulkStatePtr & state) const
{
auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state);
KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value);
auto * low_cardinality_state = checkAndGetLowCardinalitySerializeState(state);
KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value);
if (state_with_dictionary->shared_dictionary && settings.low_cardinality_max_dictionary_size)
if (low_cardinality_state->shared_dictionary && settings.low_cardinality_max_dictionary_size)
{
auto nested_column = state_with_dictionary->shared_dictionary->getNestedNotNullableColumn();
auto nested_column = low_cardinality_state->shared_dictionary->getNestedNotNullableColumn();
settings.path.push_back(Substream::DictionaryKeys);
auto * stream = settings.getter(settings.path);
settings.path.pop_back();
if (!stream)
throw Exception("Got empty stream in DataTypeWithDictionary::serializeBinaryBulkStateSuffix",
throw Exception("Got empty stream in DataTypeLowCardinality::serializeBinaryBulkStateSuffix",
ErrorCodes::LOGICAL_ERROR);
UInt64 num_keys = nested_column->size();
writeIntBinary(num_keys, *stream);
removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *stream, 0, num_keys);
state_with_dictionary->shared_dictionary = nullptr;
low_cardinality_state->shared_dictionary = nullptr;
}
}
void DataTypeWithDictionary::deserializeBinaryBulkStatePrefix(
void DataTypeLowCardinality::deserializeBinaryBulkStatePrefix(
DeserializeBinaryBulkSettings & settings,
DeserializeBinaryBulkStatePtr & state) const
{
@ -296,7 +296,7 @@ void DataTypeWithDictionary::deserializeBinaryBulkStatePrefix(
UInt64 keys_version;
readIntBinary(keys_version, *stream);
state = std::make_shared<DeserializeStateWithDictionary>(keys_version);
state = std::make_shared<DeserializeStateLowCardinality>(keys_version);
}
namespace
@ -475,7 +475,7 @@ namespace
}
}
void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams(
void DataTypeLowCardinality::serializeBinaryBulkWithMultipleStreams(
const IColumn & column,
size_t offset,
size_t limit,
@ -492,16 +492,16 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams(
return;
if (!keys_stream)
throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Got empty stream for DataTypeLowCardinality keys.", ErrorCodes::LOGICAL_ERROR);
if (!indexes_stream)
throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Got empty stream for DataTypeLowCardinality indexes.", ErrorCodes::LOGICAL_ERROR);
const ColumnWithDictionary & column_with_dictionary = typeid_cast<const ColumnWithDictionary &>(column);
const ColumnLowCardinality & low_cardinality_column = typeid_cast<const ColumnLowCardinality &>(column);
auto * state_with_dictionary = checkAndGetWithDictionarySerializeState(state);
auto & global_dictionary = state_with_dictionary->shared_dictionary;
KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value);
auto * low_cardinality_state = checkAndGetLowCardinalitySerializeState(state);
auto & global_dictionary = low_cardinality_state->shared_dictionary;
KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value);
bool need_update_dictionary = global_dictionary == nullptr;
if (need_update_dictionary)
@ -510,7 +510,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams(
size_t max_limit = column.size() - offset;
limit = limit ? std::min(limit, max_limit) : max_limit;
auto sub_column = column_with_dictionary.cutAndCompact(offset, limit);
auto sub_column = low_cardinality_column.cutAndCompact(offset, limit);
ColumnPtr positions = sub_column->getIndexesPtr();
ColumnPtr keys = sub_column->getDictionary().getNestedColumn();
@ -520,7 +520,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams(
auto indexes_with_overflow = global_dictionary->uniqueInsertRangeWithOverflow(*keys, 0, keys->size(),
settings.low_cardinality_max_dictionary_size);
size_t max_size = settings.low_cardinality_max_dictionary_size + indexes_with_overflow.overflowed_keys->size();
ColumnWithDictionary::Index(indexes_with_overflow.indexes->getPtr()).check(max_size);
ColumnLowCardinality::Index(indexes_with_overflow.indexes->getPtr()).check(max_size);
if (global_dictionary->size() > settings.low_cardinality_max_dictionary_size)
throw Exception("Got dictionary with size " + toString(global_dictionary->size()) +
@ -553,7 +553,7 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams(
UInt64 num_keys = nested_column->size();
writeIntBinary(num_keys, *keys_stream);
removeNullable(dictionary_type)->serializeBinaryBulk(*nested_column, *keys_stream, 0, num_keys);
state_with_dictionary->shared_dictionary = nullptr;
low_cardinality_state->shared_dictionary = nullptr;
}
if (need_additional_keys)
@ -568,13 +568,13 @@ void DataTypeWithDictionary::serializeBinaryBulkWithMultipleStreams(
index_version.getDataType()->serializeBinaryBulk(*positions, *indexes_stream, 0, num_rows);
}
void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
void DataTypeLowCardinality::deserializeBinaryBulkWithMultipleStreams(
IColumn & column,
size_t limit,
DeserializeBinaryBulkSettings & settings,
DeserializeBinaryBulkStatePtr & state) const
{
ColumnWithDictionary & column_with_dictionary = typeid_cast<ColumnWithDictionary &>(column);
ColumnLowCardinality & low_cardinality_column = typeid_cast<ColumnLowCardinality &>(column);
settings.path.push_back(Substream::DictionaryKeys);
auto * keys_stream = settings.getter(settings.path);
@ -586,15 +586,15 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
return;
if (!keys_stream)
throw Exception("Got empty stream for DataTypeWithDictionary keys.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Got empty stream for DataTypeLowCardinality keys.", ErrorCodes::LOGICAL_ERROR);
if (!indexes_stream)
throw Exception("Got empty stream for DataTypeWithDictionary indexes.", ErrorCodes::LOGICAL_ERROR);
throw Exception("Got empty stream for DataTypeLowCardinality indexes.", ErrorCodes::LOGICAL_ERROR);
auto * state_with_dictionary = checkAndGetWithDictionaryDeserializeState(state);
KeysSerializationVersion::checkVersion(state_with_dictionary->key_version.value);
auto * low_cardinality_state = checkAndGetLowCardinalityDeserializeState(state);
KeysSerializationVersion::checkVersion(low_cardinality_state->key_version.value);
auto readDictionary = [this, state_with_dictionary, keys_stream]()
auto readDictionary = [this, low_cardinality_state, keys_stream]()
{
UInt64 num_keys;
readIntBinary(num_keys, *keys_stream);
@ -604,62 +604,62 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
keys_type->deserializeBinaryBulk(*global_dict_keys, *keys_stream, num_keys, 0);
auto column_unique = createColumnUnique(*dictionary_type, std::move(global_dict_keys));
state_with_dictionary->global_dictionary = std::move(column_unique);
low_cardinality_state->global_dictionary = std::move(column_unique);
};
auto readAdditionalKeys = [this, state_with_dictionary, indexes_stream]()
auto readAdditionalKeys = [this, low_cardinality_state, indexes_stream]()
{
UInt64 num_keys;
readIntBinary(num_keys, *indexes_stream);
auto keys_type = removeNullable(dictionary_type);
auto additional_keys = keys_type->createColumn();
keys_type->deserializeBinaryBulk(*additional_keys, *indexes_stream, num_keys, 0);
state_with_dictionary->additional_keys = std::move(additional_keys);
low_cardinality_state->additional_keys = std::move(additional_keys);
if (!state_with_dictionary->index_type.need_global_dictionary && dictionary_type->isNullable())
if (!low_cardinality_state->index_type.need_global_dictionary && dictionary_type->isNullable())
{
auto null_map = ColumnUInt8::create(num_keys, 0);
if (num_keys)
null_map->getElement(0) = 1;
state_with_dictionary->null_map = std::move(null_map);
low_cardinality_state->null_map = std::move(null_map);
}
};
auto readIndexes = [this, state_with_dictionary, indexes_stream, &column_with_dictionary](UInt64 num_rows)
auto readIndexes = [this, low_cardinality_state, indexes_stream, &low_cardinality_column](UInt64 num_rows)
{
auto indexes_type = state_with_dictionary->index_type.getDataType();
auto indexes_type = low_cardinality_state->index_type.getDataType();
MutableColumnPtr indexes_column = indexes_type->createColumn();
indexes_type->deserializeBinaryBulk(*indexes_column, *indexes_stream, num_rows, 0);
auto & global_dictionary = state_with_dictionary->global_dictionary;
const auto & additional_keys = state_with_dictionary->additional_keys;
auto & global_dictionary = low_cardinality_state->global_dictionary;
const auto & additional_keys = low_cardinality_state->additional_keys;
bool has_additional_keys = state_with_dictionary->index_type.has_additional_keys;
bool column_is_empty = column_with_dictionary.empty();
bool has_additional_keys = low_cardinality_state->index_type.has_additional_keys;
bool column_is_empty = low_cardinality_column.empty();
if (!state_with_dictionary->index_type.need_global_dictionary)
if (!low_cardinality_state->index_type.need_global_dictionary)
{
ColumnPtr keys_column = additional_keys;
if (state_with_dictionary->null_map)
keys_column = ColumnNullable::create(additional_keys, state_with_dictionary->null_map);
column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*keys_column, *indexes_column);
if (low_cardinality_state->null_map)
keys_column = ColumnNullable::create(additional_keys, low_cardinality_state->null_map);
low_cardinality_column.insertRangeFromDictionaryEncodedColumn(*keys_column, *indexes_column);
}
else if (!has_additional_keys)
{
if (column_is_empty)
column_with_dictionary.setSharedDictionary(global_dictionary);
low_cardinality_column.setSharedDictionary(global_dictionary);
auto local_column = ColumnWithDictionary::create(global_dictionary, std::move(indexes_column));
column_with_dictionary.insertRangeFrom(*local_column, 0, num_rows);
auto local_column = ColumnLowCardinality::create(global_dictionary, std::move(indexes_column));
low_cardinality_column.insertRangeFrom(*local_column, 0, num_rows);
}
else
{
auto maps = mapIndexWithAdditionalKeys(*indexes_column, global_dictionary->size());
ColumnWithDictionary::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size());
ColumnLowCardinality::Index(maps.additional_keys_map->getPtr()).check(additional_keys->size());
ColumnWithDictionary::Index(indexes_column->getPtr()).check(
ColumnLowCardinality::Index(indexes_column->getPtr()).check(
maps.dictionary_map->size() + maps.additional_keys_map->size());
auto used_keys = (*std::move(global_dictionary->getNestedColumn()->index(*maps.dictionary_map, 0))).mutate();
@ -677,23 +677,23 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
used_keys->insertRangeFrom(*used_add_keys, 0, used_add_keys->size());
}
column_with_dictionary.insertRangeFromDictionaryEncodedColumn(*used_keys, *indexes_column);
low_cardinality_column.insertRangeFromDictionaryEncodedColumn(*used_keys, *indexes_column);
}
};
if (!settings.continuous_reading)
state_with_dictionary->num_pending_rows = 0;
low_cardinality_state->num_pending_rows = 0;
bool first_dictionary = true;
while (limit)
{
if (state_with_dictionary->num_pending_rows == 0)
if (low_cardinality_state->num_pending_rows == 0)
{
if (indexes_stream->eof())
break;
auto & index_type = state_with_dictionary->index_type;
auto & global_dictionary = state_with_dictionary->global_dictionary;
auto & index_type = low_cardinality_state->index_type;
auto & global_dictionary = low_cardinality_state->global_dictionary;
index_type.deserialize(*indexes_stream);
@ -703,51 +703,51 @@ void DataTypeWithDictionary::deserializeBinaryBulkWithMultipleStreams(
first_dictionary = false;
}
if (state_with_dictionary->index_type.has_additional_keys)
if (low_cardinality_state->index_type.has_additional_keys)
readAdditionalKeys();
else
state_with_dictionary->additional_keys = nullptr;
low_cardinality_state->additional_keys = nullptr;
readIntBinary(state_with_dictionary->num_pending_rows, *indexes_stream);
readIntBinary(low_cardinality_state->num_pending_rows, *indexes_stream);
}
size_t num_rows_to_read = std::min(limit, state_with_dictionary->num_pending_rows);
size_t num_rows_to_read = std::min(limit, low_cardinality_state->num_pending_rows);
readIndexes(num_rows_to_read);
limit -= num_rows_to_read;
state_with_dictionary->num_pending_rows -= num_rows_to_read;
low_cardinality_state->num_pending_rows -= num_rows_to_read;
}
}
void DataTypeWithDictionary::serializeBinary(const Field & field, WriteBuffer & ostr) const
void DataTypeLowCardinality::serializeBinary(const Field & field, WriteBuffer & ostr) const
{
dictionary_type->serializeBinary(field, ostr);
}
void DataTypeWithDictionary::deserializeBinary(Field & field, ReadBuffer & istr) const
void DataTypeLowCardinality::deserializeBinary(Field & field, ReadBuffer & istr) const
{
dictionary_type->deserializeBinary(field, istr);
}
template <typename ... Args>
void DataTypeWithDictionary::serializeImpl(
void DataTypeLowCardinality::serializeImpl(
const IColumn & column, size_t row_num, WriteBuffer & ostr,
DataTypeWithDictionary::SerealizeFunctionPtr<Args ...> func, Args & ... args) const
DataTypeLowCardinality::SerealizeFunctionPtr<Args ...> func, Args & ... args) const
{
auto & column_with_dictionary = getColumnWithDictionary(column);
size_t unique_row_number = column_with_dictionary.getIndexes().getUInt(row_num);
(dictionary_type.get()->*func)(*column_with_dictionary.getDictionary().getNestedColumn(), unique_row_number, ostr, std::forward<Args>(args)...);
auto & low_cardinality_column = getColumnLowCardinality(column);
size_t unique_row_number = low_cardinality_column.getIndexes().getUInt(row_num);
(dictionary_type.get()->*func)(*low_cardinality_column.getDictionary().getNestedColumn(), unique_row_number, ostr, std::forward<Args>(args)...);
}
template <typename ... Args>
void DataTypeWithDictionary::deserializeImpl(
void DataTypeLowCardinality::deserializeImpl(
IColumn & column, ReadBuffer & istr,
DataTypeWithDictionary::DeserealizeFunctionPtr<Args ...> func, Args & ... args) const
DataTypeLowCardinality::DeserealizeFunctionPtr<Args ...> func, Args & ... args) const
{
auto & column_with_dictionary = getColumnWithDictionary(column);
auto temp_column = column_with_dictionary.getDictionary().getNestedColumn()->cloneEmpty();
auto & low_cardinality_column= getColumnLowCardinality(column);
auto temp_column = low_cardinality_column.getDictionary().getNestedColumn()->cloneEmpty();
(dictionary_type.get()->*func)(*temp_column, istr, std::forward<Args>(args)...);
column_with_dictionary.insertFromFullColumn(*temp_column, 0);
low_cardinality_column.insertFromFullColumn(*temp_column, 0);
}
namespace
@ -774,7 +774,7 @@ namespace
}
template <typename Creator>
MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDataType & keys_type,
MutableColumnUniquePtr DataTypeLowCardinality::createColumnUniqueImpl(const IDataType & keys_type,
const Creator & creator)
{
auto * type = &keys_type;
@ -800,12 +800,12 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUniqueImpl(const IDat
return column;
}
throw Exception("Unexpected dictionary type for DataTypeWithDictionary: " + type->getName(),
throw Exception("Unexpected dictionary type for DataTypeLowCardinality: " + type->getName(),
ErrorCodes::LOGICAL_ERROR);
}
MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type)
MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataType & keys_type)
{
auto creator = [&](auto x)
{
@ -815,7 +815,7 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataTyp
return createColumnUniqueImpl(keys_type, creator);
}
MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys)
MutableColumnUniquePtr DataTypeLowCardinality::createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys)
{
auto creator = [&](auto x)
{
@ -825,20 +825,20 @@ MutableColumnUniquePtr DataTypeWithDictionary::createColumnUnique(const IDataTyp
return createColumnUniqueImpl(keys_type, creator);
}
MutableColumnPtr DataTypeWithDictionary::createColumn() const
MutableColumnPtr DataTypeLowCardinality::createColumn() const
{
MutableColumnPtr indexes = DataTypeUInt8().createColumn();
MutableColumnPtr dictionary = createColumnUnique(*dictionary_type);
return ColumnWithDictionary::create(std::move(dictionary), std::move(indexes));
return ColumnLowCardinality::create(std::move(dictionary), std::move(indexes));
}
bool DataTypeWithDictionary::equals(const IDataType & rhs) const
bool DataTypeLowCardinality::equals(const IDataType & rhs) const
{
if (typeid(rhs) != typeid(*this))
return false;
auto & rhs_with_dictionary = static_cast<const DataTypeWithDictionary &>(rhs);
return dictionary_type->equals(*rhs_with_dictionary.dictionary_type);
auto & low_cardinality_rhs= static_cast<const DataTypeLowCardinality &>(rhs);
return dictionary_type->equals(*low_cardinality_rhs.dictionary_type);
}
@ -848,10 +848,10 @@ static DataTypePtr create(const ASTPtr & arguments)
throw Exception("LowCardinality data type family must have single argument - type of elements",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
return std::make_shared<DataTypeWithDictionary>(DataTypeFactory::instance().get(arguments->children[0]));
return std::make_shared<DataTypeLowCardinality>(DataTypeFactory::instance().get(arguments->children[0]));
}
void registerDataTypeWithDictionary(DataTypeFactory & factory)
void registerDataTypeLowCardinality(DataTypeFactory & factory)
{
factory.registerDataType("LowCardinality", create);
}
@ -859,8 +859,8 @@ void registerDataTypeWithDictionary(DataTypeFactory & factory)
DataTypePtr removeLowCardinality(const DataTypePtr & type)
{
if (auto * type_with_dictionary = typeid_cast<const DataTypeWithDictionary *>(type.get()))
return type_with_dictionary->getDictionaryType();
if (auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(type.get()))
return low_cardinality_type->getDictionaryType();
return type;
}

View File

@ -5,13 +5,13 @@
namespace DB
{
class DataTypeWithDictionary : public IDataType
class DataTypeLowCardinality : public IDataType
{
private:
DataTypePtr dictionary_type;
public:
DataTypeWithDictionary(DataTypePtr dictionary_type_);
DataTypeLowCardinality(DataTypePtr dictionary_type_);
const DataTypePtr & getDictionaryType() const { return dictionary_type; }
@ -136,7 +136,7 @@ public:
bool isCategorial() const override { return false; }
bool isNullable() const override { return false; }
bool onlyNull() const override { return false; }
bool withDictionary() const override { return true; }
bool lowCardinality() const override { return true; }
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type);
static MutableColumnUniquePtr createColumnUnique(const IDataType & keys_type, MutableColumnPtr && keys);
@ -161,7 +161,7 @@ private:
static MutableColumnUniquePtr createColumnUniqueImpl(const IDataType & keys_type, const Creator & creator);
};
/// Returns dictionary type if type is DataTypeWithDictionary, type otherwise.
/// Returns dictionary type if type is DataTypeLowCardinality, type otherwise.
DataTypePtr removeLowCardinality(const DataTypePtr & type);
}

View File

@ -396,7 +396,7 @@ public:
*/
virtual bool canBeInsideNullable() const { return false; }
virtual bool withDictionary() const { return false; }
virtual bool lowCardinality() const { return false; }
/// Updates avg_value_size_hint for newly read column. Uses to optimize deserialization. Zero expected for first column.
@ -411,6 +411,11 @@ struct WhichDataType
{
TypeIndex idx;
/// For late initialization.
WhichDataType()
: idx(TypeIndex::Nothing)
{}
WhichDataType(const IDataType & data_type)
: idx(data_type.getTypeId())
{}

View File

@ -68,13 +68,24 @@ Field convertNodeToField(capnp::DynamicValue::Reader value)
auto listValue = value.as<capnp::DynamicList>();
Array res(listValue.size());
for (auto i : kj::indices(listValue))
res[i] = convertNodeToField(listValue[i]);
res[i] = convertNodeToField(listValue[i]);
return res;
}
case capnp::DynamicValue::ENUM:
return UInt64(value.as<capnp::DynamicEnum>().getRaw());
case capnp::DynamicValue::STRUCT:
throw Exception("STRUCT type not supported, read individual fields instead");
{
auto structValue = value.as<capnp::DynamicStruct>();
const auto & fields = structValue.getSchema().getFields();
Field field = Tuple(TupleBackend(fields.size()));
TupleBackend & tuple = get<Tuple &>(field).toUnderType();
for (auto i : kj::indices(fields))
tuple[i] = convertNodeToField(structValue.get(fields[i]));
return field;
}
case capnp::DynamicValue::CAPABILITY:
throw Exception("CAPABILITY type not supported");
case capnp::DynamicValue::ANY_POINTER:
@ -88,7 +99,7 @@ capnp::StructSchema::Field getFieldOrThrow(capnp::StructSchema node, const std::
KJ_IF_MAYBE(child, node.findFieldByName(field))
return *child;
else
throw Exception("Field " + field + " doesn't exist in schema.");
throw Exception("Field " + field + " doesn't exist in schema " + node.getShortDisplayName().cStr());
}
void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader)
@ -110,13 +121,40 @@ void CapnProtoRowInputStream::createActions(const NestedFieldList & sortedFields
// Descend to a nested structure
for (; level < field.tokens.size() - 1; ++level)
{
last = field.tokens[level];
parent = getFieldOrThrow(reader, last);
reader = parent.getType().asStruct();
actions.push_back({Action::PUSH, parent});
auto node = getFieldOrThrow(reader, field.tokens[level]);
if (node.getType().isStruct())
{
// Descend to field structure
last = field.tokens[level];
parent = node;
reader = parent.getType().asStruct();
actions.push_back({Action::PUSH, parent});
}
else if (node.getType().isList())
{
break; // Collect list
}
else
throw Exception("Field " + field.tokens[level] + "is neither Struct nor List");
}
// Read field from the structure
actions.push_back({Action::READ, getFieldOrThrow(reader, field.tokens[level]), field.pos});
auto node = getFieldOrThrow(reader, field.tokens[level]);
if (node.getType().isList() && actions.size() > 0 && actions.back().field == node)
{
// The field list here flattens Nested elements into multiple arrays
// In order to map Nested types in Cap'nProto back, they need to be collected
// Since the field names are sorted, the order of field positions must be preserved
// For example, if the fields are { b @0 :Text, a @1 :Text }, the `a` would come first
// even though it's position is second.
auto & columns = actions.back().columns;
auto it = std::upper_bound(columns.cbegin(), columns.cend(), field.pos);
columns.insert(it, field.pos);
}
else
{
actions.push_back({Action::READ, node, {field.pos}});
}
}
}
@ -176,7 +214,7 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns)
array = heap_array.asPtr();
}
capnp::FlatArrayMessageReader msg(array);
capnp::UnalignedFlatArrayMessageReader msg(array);
std::vector<capnp::DynamicStruct::Reader> stack;
stack.push_back(msg.getRoot<capnp::DynamicStruct>(root));
@ -186,9 +224,33 @@ bool CapnProtoRowInputStream::read(MutableColumns & columns)
{
case Action::READ:
{
auto & col = columns[action.column];
Field value = convertNodeToField(stack.back().get(action.field));
col->insert(value);
if (action.columns.size() > 1)
{
// Nested columns must be flattened into several arrays
// e.g. Array(Tuple(x ..., y ...)) -> Array(x ...), Array(y ...)
const Array & collected = DB::get<const Array &>(value);
size_t size = collected.size();
// The flattened array contains an array of a part of the nested tuple
Array flattened(size);
for (size_t column_index = 0; column_index < action.columns.size(); ++column_index)
{
// Populate array with a single tuple elements
for (size_t off = 0; off < size; ++off)
{
const TupleBackend & tuple = DB::get<const Tuple &>(collected[off]).toUnderType();
flattened[off] = tuple[column_index];
}
auto & col = columns[action.columns[column_index]];
col->insert(flattened);
}
}
else
{
auto & col = columns[action.columns[0]];
col->insert(value);
}
break;
}
case Action::POP:

View File

@ -41,12 +41,13 @@ private:
void createActions(const NestedFieldList & sortedFields, capnp::StructSchema reader);
/* Action for state machine for traversing nested structures. */
using BlockPositionList = std::vector<size_t>;
struct Action
{
enum Type { POP, PUSH, READ };
Type type;
capnp::StructSchema::Field field = {};
size_t column = 0;
BlockPositionList columns = {};
};
// Wrapper for classes that could throw in destructor

View File

@ -47,6 +47,7 @@ void PrettyBlockOutputStream::calculateWidths(
/// Calculate widths of all values.
String serialized_value;
size_t prefix = 2; // Tab character adjustment
for (size_t i = 0; i < columns; ++i)
{
const ColumnWithTypeAndName & elem = block.getByPosition(i);
@ -61,16 +62,18 @@ void PrettyBlockOutputStream::calculateWidths(
}
widths[i][j] = std::min(format_settings.pretty.max_column_pad_width,
UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size()));
UTF8::computeWidth(reinterpret_cast<const UInt8 *>(serialized_value.data()), serialized_value.size(), prefix));
max_widths[i] = std::max(max_widths[i], widths[i][j]);
}
/// And also calculate widths for names of columns.
{
// name string doesn't contain Tab, no need to pass `prefix`
name_widths[i] = std::min(format_settings.pretty.max_column_pad_width,
UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(elem.name.data()), elem.name.size()));
UTF8::computeWidth(reinterpret_cast<const UInt8 *>(elem.name.data()), elem.name.size()));
max_widths[i] = std::max(max_widths[i], name_widths[i]);
}
prefix += max_widths[i] + 3;
}
}

View File

@ -28,7 +28,7 @@ VerticalRowOutputStream::VerticalRowOutputStream(
/// Note that number of code points is just a rough approximation of visible string width.
const String & name = sample.getByPosition(i).name;
name_widths[i] = UTF8::countCodePoints(reinterpret_cast<const UInt8 *>(name.data()), name.size());
name_widths[i] = UTF8::computeWidth(reinterpret_cast<const UInt8 *>(name.data()), name.size());
if (name_widths[i] > max_name_width)
max_name_width = name_widths[i];
@ -43,7 +43,10 @@ VerticalRowOutputStream::VerticalRowOutputStream(
}
for (size_t i = 0; i < columns; ++i)
names_and_paddings[i].resize(max_name_width + strlen(": "), ' ');
{
size_t new_size = max_name_width - name_widths[i] + names_and_paddings[i].size();
names_and_paddings[i].resize(new_size, ' ');
}
}

View File

@ -37,8 +37,8 @@
#include <Functions/FunctionsMiscellaneous.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/DateTimeTransforms.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <Columns/ColumnWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnLowCardinality.h>
namespace DB
@ -1374,7 +1374,7 @@ protected:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {1}; }
private:
@ -1750,10 +1750,10 @@ private:
WrapperType prepareUnpackDictionaries(const DataTypePtr & from_type, const DataTypePtr & to_type) const
{
const auto * from_with_dict = typeid_cast<const DataTypeWithDictionary *>(from_type.get());
const auto * to_with_dict = typeid_cast<const DataTypeWithDictionary *>(to_type.get());
const auto & from_nested = from_with_dict ? from_with_dict->getDictionaryType() : from_type;
const auto & to_nested = to_with_dict ? to_with_dict->getDictionaryType() : to_type;
const auto * from_low_cardinality = typeid_cast<const DataTypeLowCardinality *>(from_type.get());
const auto * to_low_cardinality = typeid_cast<const DataTypeLowCardinality *>(to_type.get());
const auto & from_nested = from_low_cardinality ? from_low_cardinality->getDictionaryType() : from_type;
const auto & to_nested = to_low_cardinality ? to_low_cardinality->getDictionaryType() : to_type;
if (from_type->onlyNull())
{
@ -1768,10 +1768,10 @@ private:
}
auto wrapper = prepareRemoveNullable(from_nested, to_nested);
if (!from_with_dict && !to_with_dict)
if (!from_low_cardinality && !to_low_cardinality)
return wrapper;
return [wrapper, from_with_dict, to_with_dict]
return [wrapper, from_low_cardinality, to_low_cardinality]
(Block & block, const ColumnNumbers & arguments, const size_t result, size_t input_rows_count)
{
auto & arg = block.getByPosition(arguments[0]);
@ -1790,21 +1790,21 @@ private:
auto tmp_rows_count = input_rows_count;
if (to_with_dict)
res.type = to_with_dict->getDictionaryType();
if (to_low_cardinality)
res.type = to_low_cardinality->getDictionaryType();
if (from_with_dict)
if (from_low_cardinality)
{
auto * col_with_dict = typeid_cast<const ColumnWithDictionary *>(prev_arg_col.get());
arg.column = col_with_dict->getDictionary().getNestedColumn();
arg.type = from_with_dict->getDictionaryType();
auto * col_low_cardinality = typeid_cast<const ColumnLowCardinality *>(prev_arg_col.get());
arg.column = col_low_cardinality->getDictionary().getNestedColumn();
arg.type = from_low_cardinality->getDictionaryType();
/// TODO: Make map with defaults conversion.
src_converted_to_full_column = !removeNullable(arg.type)->equals(*removeNullable(res.type));
if (src_converted_to_full_column)
arg.column = arg.column->index(col_with_dict->getIndexes(), 0);
arg.column = arg.column->index(col_low_cardinality->getIndexes(), 0);
else
res_indexes = col_with_dict->getIndexesPtr();
res_indexes = col_low_cardinality->getIndexesPtr();
tmp_rows_count = arg.column->size();
}
@ -1817,18 +1817,18 @@ private:
res.type = prev_res_type;
}
if (to_with_dict)
if (to_low_cardinality)
{
auto res_column = to_with_dict->createColumn();
auto * col_with_dict = typeid_cast<ColumnWithDictionary *>(res_column.get());
auto res_column = to_low_cardinality->createColumn();
auto * col_low_cardinality = typeid_cast<ColumnLowCardinality *>(res_column.get());
if (from_with_dict && !src_converted_to_full_column)
if (from_low_cardinality && !src_converted_to_full_column)
{
auto res_keys = std::move(res.column);
col_with_dict->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes);
col_low_cardinality->insertRangeFromDictionaryEncodedColumn(*res_keys, *res_indexes);
}
else
col_with_dict->insertRangeFromFullColumn(*res.column, 0, res.column->size());
col_low_cardinality->insertRangeFromFullColumn(*res.column, 0, res.column->size());
res.column = std::move(res_column);
}
@ -2026,7 +2026,7 @@ protected:
}
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
private:
template <typename DataType>

View File

@ -8,12 +8,12 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/Native.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/getLeastSupertype.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnConst.h>
#include <Columns/ColumnTuple.h>
#include <Columns/ColumnWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
#include <Functions/FunctionHelpers.h>
#include <Functions/IFunction.h>
#include <Interpreters/ExpressionActions.h>
@ -42,6 +42,9 @@ namespace ErrorCodes
}
/// Cache for functions result if it was executed on low cardinality column.
/// It's LRUCache which stores function result executed on dictionary and index mapping.
/// It's expected that cache_size is a number of reading streams (so, will store single cached value per thread).
class PreparedFunctionLowCardinalityResultCache
{
public:
@ -120,7 +123,7 @@ static DataTypePtr recursiveRemoveLowCardinality(const DataTypePtr & type)
return std::make_shared<DataTypeTuple>(elements);
}
if (const auto * low_cardinality_type = typeid_cast<const DataTypeWithDictionary *>(type.get()))
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(type.get()))
return low_cardinality_type->getDictionaryType();
return type;
@ -145,7 +148,7 @@ static ColumnPtr recursiveRemoveLowCardinality(const ColumnPtr & column)
return ColumnTuple::create(columns);
}
if (const auto * column_low_cardinality = typeid_cast<const ColumnWithDictionary *>(column.get()))
if (const auto * column_low_cardinality = typeid_cast<const ColumnLowCardinality *>(column.get()))
return column_low_cardinality->convertToFullColumn();
return column;
@ -306,7 +309,7 @@ bool PreparedFunctionImpl::defaultImplementationForConstantArguments(Block & blo
for (size_t i = 0; i < arguments_size; ++i)
temporary_argument_numbers[i] = i;
executeWithoutColumnsWithDictionary(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows());
executeWithoutLowCardinalityColumns(temporary_block, temporary_argument_numbers, arguments_size, temporary_block.rows());
block.getByPosition(result).column = ColumnConst::create(temporary_block.getByPosition(arguments_size).column, input_rows_count);
return true;
@ -330,7 +333,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co
if (null_presence.has_nullable)
{
Block temporary_block = createBlockWithNestedColumns(block, args, result);
executeWithoutColumnsWithDictionary(temporary_block, args, result, temporary_block.rows());
executeWithoutLowCardinalityColumns(temporary_block, args, result, temporary_block.rows());
block.getByPosition(result).column = wrapInNullable(temporary_block.getByPosition(result).column, block, args,
result, input_rows_count);
return true;
@ -339,7 +342,7 @@ bool PreparedFunctionImpl::defaultImplementationForNulls(Block & block, const Co
return false;
}
void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
void PreparedFunctionImpl::executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
if (defaultImplementationForConstantArguments(block, args, result, input_rows_count))
return;
@ -350,14 +353,14 @@ void PreparedFunctionImpl::executeWithoutColumnsWithDictionary(Block & block, co
executeImpl(block, args, result, input_rows_count);
}
static const ColumnWithDictionary * findLowCardinalityArgument(const Block & block, const ColumnNumbers & args)
static const ColumnLowCardinality * findLowCardinalityArgument(const Block & block, const ColumnNumbers & args)
{
const ColumnWithDictionary * result_column = nullptr;
const ColumnLowCardinality * result_column = nullptr;
for (auto arg : args)
{
const ColumnWithTypeAndName & column = block.getByPosition(arg);
if (auto * low_cardinality_column = checkAndGetColumn<ColumnWithDictionary>(column.column.get()))
if (auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
{
if (result_column)
throw Exception("Expected single dictionary argument for function.", ErrorCodes::LOGICAL_ERROR);
@ -369,7 +372,7 @@ static const ColumnWithDictionary * findLowCardinalityArgument(const Block & blo
return result_column;
}
static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(
static ColumnPtr replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
Block & block, const ColumnNumbers & args, bool can_be_executed_on_default_arguments)
{
size_t num_rows = 0;
@ -378,13 +381,13 @@ static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(
for (auto arg : args)
{
ColumnWithTypeAndName & column = block.getByPosition(arg);
if (auto * column_with_dict = checkAndGetColumn<ColumnWithDictionary>(column.column.get()))
if (auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
{
if (indexes)
throw Exception("Expected single dictionary argument for function.", ErrorCodes::LOGICAL_ERROR);
indexes = column_with_dict->getIndexesPtr();
num_rows = column_with_dict->getDictionary().size();
indexes = low_cardinality_column->getIndexesPtr();
num_rows = low_cardinality_column->getDictionary().size();
}
}
@ -393,30 +396,30 @@ static ColumnPtr replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(
ColumnWithTypeAndName & column = block.getByPosition(arg);
if (auto * column_const = checkAndGetColumn<ColumnConst>(column.column.get()))
column.column = column_const->removeLowCardinality()->cloneResized(num_rows);
else if (auto * column_with_dict = checkAndGetColumn<ColumnWithDictionary>(column.column.get()))
else if (auto * low_cardinality_column = checkAndGetColumn<ColumnLowCardinality>(column.column.get()))
{
auto * type_with_dict = checkAndGetDataType<DataTypeWithDictionary>(column.type.get());
auto * low_cardinality_type = checkAndGetDataType<DataTypeLowCardinality>(column.type.get());
if (!type_with_dict)
throw Exception("Incompatible type for column with dictionary: " + column.type->getName(),
if (!low_cardinality_type)
throw Exception("Incompatible type for low cardinality column: " + column.type->getName(),
ErrorCodes::LOGICAL_ERROR);
if (can_be_executed_on_default_arguments)
column.column = column_with_dict->getDictionary().getNestedColumn();
column.column = low_cardinality_column->getDictionary().getNestedColumn();
else
{
auto dict_encoded = column_with_dict->getMinimalDictionaryEncodedColumn(0, column_with_dict->size());
auto dict_encoded = low_cardinality_column->getMinimalDictionaryEncodedColumn(0, low_cardinality_column->size());
column.column = dict_encoded.dictionary;
indexes = dict_encoded.indexes;
}
column.type = type_with_dict->getDictionaryType();
column.type = low_cardinality_type->getDictionaryType();
}
}
return indexes;
}
static void convertColumnsWithDictionaryToFull(Block & block, const ColumnNumbers & args)
static void convertLowCardinalityColumnsToFull(Block & block, const ColumnNumbers & args)
{
for (auto arg : args)
{
@ -429,18 +432,19 @@ static void convertColumnsWithDictionaryToFull(Block & block, const ColumnNumber
void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, size_t result, size_t input_rows_count)
{
if (useDefaultImplementationForColumnsWithDictionary())
if (useDefaultImplementationForLowCardinalityColumns())
{
auto & res = block.safeGetByPosition(result);
Block block_without_dicts = block.cloneWithoutColumns();
Block block_without_low_cardinality = block.cloneWithoutColumns();
for (auto arg : args)
block_without_dicts.safeGetByPosition(arg).column = block.safeGetByPosition(arg).column;
block_without_low_cardinality.safeGetByPosition(arg).column = block.safeGetByPosition(arg).column;
if (auto * res_type_with_dict = typeid_cast<const DataTypeWithDictionary *>(res.type.get()))
if (auto * res_low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(res.type.get()))
{
const auto * low_cardinality_column = findLowCardinalityArgument(block, args);
bool use_cache = low_cardinality_result_cache
bool can_be_executed_on_default_arguments = canBeExecutedOnDefaultArguments();
bool use_cache = low_cardinality_result_cache && can_be_executed_on_default_arguments
&& low_cardinality_column && low_cardinality_column->isSharedDictionary();
PreparedFunctionLowCardinalityResultCache::DictionaryKey key;
@ -453,22 +457,22 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si
if (cached_values)
{
auto indexes = cached_values->index_mapping->index(low_cardinality_column->getIndexes(), 0);
res.column = ColumnWithDictionary::create(cached_values->function_result, indexes, true);
res.column = ColumnLowCardinality::create(cached_values->function_result, indexes, true);
return;
}
}
block_without_dicts.safeGetByPosition(result).type = res_type_with_dict->getDictionaryType();
ColumnPtr indexes = replaceColumnsWithDictionaryByNestedAndGetDictionaryIndexes(
block_without_dicts, args, canBeExecutedOnDefaultArguments());
block_without_low_cardinality.safeGetByPosition(result).type = res_low_cardinality_type->getDictionaryType();
ColumnPtr indexes = replaceLowCardinalityColumnsByNestedAndGetDictionaryIndexes(
block_without_low_cardinality, args, can_be_executed_on_default_arguments);
executeWithoutColumnsWithDictionary(block_without_dicts, args, result, block_without_dicts.rows());
executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, block_without_low_cardinality.rows());
auto & keys = block_without_dicts.safeGetByPosition(result).column;
auto & keys = block_without_low_cardinality.safeGetByPosition(result).column;
if (auto full_column = keys->convertToFullColumnIfConst())
keys = full_column;
auto res_mut_dictionary = DataTypeWithDictionary::createColumnUnique(*res_type_with_dict->getDictionaryType());
auto res_mut_dictionary = DataTypeLowCardinality::createColumnUnique(*res_low_cardinality_type->getDictionaryType());
ColumnPtr res_indexes = res_mut_dictionary->uniqueInsertRangeFrom(*keys, 0, keys->size());
ColumnUniquePtr res_dictionary = std::move(res_mut_dictionary);
@ -486,22 +490,22 @@ void PreparedFunctionImpl::execute(Block & block, const ColumnNumbers & args, si
res_indexes = cache_values->index_mapping;
}
res.column = ColumnWithDictionary::create(res_dictionary, res_indexes->index(*indexes, 0), use_cache);
res.column = ColumnLowCardinality::create(res_dictionary, res_indexes->index(*indexes, 0), use_cache);
}
else
{
res.column = ColumnWithDictionary::create(res_dictionary, res_indexes);
res.column = ColumnLowCardinality::create(res_dictionary, res_indexes);
}
}
else
{
convertColumnsWithDictionaryToFull(block_without_dicts, args);
executeWithoutColumnsWithDictionary(block_without_dicts, args, result, input_rows_count);
res.column = block_without_dicts.safeGetByPosition(result).column;
convertLowCardinalityColumnsToFull(block_without_low_cardinality, args);
executeWithoutLowCardinalityColumns(block_without_low_cardinality, args, result, input_rows_count);
res.column = block_without_low_cardinality.safeGetByPosition(result).column;
}
}
else
executeWithoutColumnsWithDictionary(block, args, result, input_rows_count);
executeWithoutLowCardinalityColumns(block, args, result, input_rows_count);
}
void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) const
@ -517,7 +521,7 @@ void FunctionBuilderImpl::checkNumberOfArguments(size_t number_of_arguments) con
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const
DataTypePtr FunctionBuilderImpl::getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const
{
checkNumberOfArguments(arguments.size());
@ -609,23 +613,23 @@ llvm::Value * IFunction::compile(llvm::IRBuilderBase & builder, const DataTypes
DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & arguments) const
{
if (useDefaultImplementationForColumnsWithDictionary())
if (useDefaultImplementationForLowCardinalityColumns())
{
bool has_low_cardinality = false;
size_t num_full_low_cardinality_columns = 0;
size_t num_full_ordinary_columns = 0;
ColumnsWithTypeAndName args_without_dictionary(arguments);
ColumnsWithTypeAndName args_without_low_cardinality(arguments);
for (ColumnWithTypeAndName & arg : args_without_dictionary)
for (ColumnWithTypeAndName & arg : args_without_low_cardinality)
{
bool is_const = arg.column && arg.column->isColumnConst();
if (is_const)
arg.column = static_cast<const ColumnConst &>(*arg.column).removeLowCardinality();
if (auto * type_with_dictionary = typeid_cast<const DataTypeWithDictionary *>(arg.type.get()))
if (auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(arg.type.get()))
{
arg.type = type_with_dictionary->getDictionaryType();
arg.type = low_cardinality_type->getDictionaryType();
has_low_cardinality = true;
if (!is_const)
@ -635,7 +639,7 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar
++num_full_ordinary_columns;
}
for (auto & arg : args_without_dictionary)
for (auto & arg : args_without_low_cardinality)
{
arg.column = recursiveRemoveLowCardinality(arg.column);
arg.type = recursiveRemoveLowCardinality(arg.type);
@ -643,11 +647,11 @@ DataTypePtr FunctionBuilderImpl::getReturnType(const ColumnsWithTypeAndName & ar
if (canBeExecutedOnLowCardinalityDictionary() && has_low_cardinality
&& num_full_low_cardinality_columns <= 1 && num_full_ordinary_columns == 0)
return std::make_shared<DataTypeWithDictionary>(getReturnTypeWithoutDictionary(args_without_dictionary));
return std::make_shared<DataTypeLowCardinality>(getReturnTypeWithoutLowCardinality(args_without_low_cardinality));
else
return getReturnTypeWithoutDictionary(args_without_dictionary);
return getReturnTypeWithoutLowCardinality(args_without_low_cardinality);
}
return getReturnTypeWithoutDictionary(arguments);
return getReturnTypeWithoutLowCardinality(arguments);
}
}

View File

@ -77,11 +77,11 @@ protected:
*/
virtual bool useDefaultImplementationForConstants() const { return false; }
/** If function arguments has single column with dictionary and all other arguments are constants, call function on nested column.
* Otherwise, convert all columns with dictionary to ordinary columns.
* Returns ColumnWithDictionary if at least one argument is ColumnWithDictionary.
/** If function arguments has single low cardinality column and all other arguments are constants, call function on nested column.
* Otherwise, convert all low cardinality columns to ordinary columns.
* Returns ColumnLowCardinality if at least one argument is ColumnLowCardinality.
*/
virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; }
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
/** Some arguments could remain constant during this implementation.
*/
@ -97,7 +97,7 @@ private:
size_t input_rows_count);
bool defaultImplementationForConstantArguments(Block & block, const ColumnNumbers & args, size_t result,
size_t input_rows_count);
void executeWithoutColumnsWithDictionary(Block & block, const ColumnNumbers & arguments, size_t result,
void executeWithoutLowCardinalityColumns(Block & block, const ColumnNumbers & arguments, size_t result,
size_t input_rows_count);
/// Cache is created by function createLowCardinalityResultCache()
@ -292,12 +292,12 @@ protected:
virtual bool useDefaultImplementationForNulls() const { return true; }
/** If useDefaultImplementationForNulls() is true, than change arguments for getReturnType() and buildImpl().
* If function arguments has types with dictionary, convert them to ordinary types.
* getReturnType returns ColumnWithDictionary if at least one argument type is ColumnWithDictionary.
* If function arguments has low cardinality types, convert them to ordinary types.
* getReturnType returns ColumnLowCardinality if at least one argument type is ColumnLowCardinality.
*/
virtual bool useDefaultImplementationForColumnsWithDictionary() const { return true; }
virtual bool useDefaultImplementationForLowCardinalityColumns() const { return true; }
/// If it isn't, will convert all ColumnWithDictionary arguments to full columns.
/// If it isn't, will convert all ColumnLowCardinality arguments to full columns.
virtual bool canBeExecutedOnLowCardinalityDictionary() const { return true; }
virtual FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const = 0;
@ -309,7 +309,7 @@ protected:
private:
DataTypePtr getReturnTypeWithoutDictionary(const ColumnsWithTypeAndName & arguments) const;
DataTypePtr getReturnTypeWithoutLowCardinality(const ColumnsWithTypeAndName & arguments) const;
};
/// Previous function interface.
@ -324,7 +324,7 @@ public:
/// Override this functions to change default implementation behavior. See details in IMyFunction.
bool useDefaultImplementationForNulls() const override { return true; }
bool useDefaultImplementationForConstants() const override { return false; }
bool useDefaultImplementationForColumnsWithDictionary() const override { return true; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return true; }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const override { return {}; }
bool canBeExecutedOnDefaultArguments() const override { return true; }
bool canBeExecutedOnLowCardinalityDictionary() const override { return isDeterministicInScopeOfQuery(); }
@ -406,7 +406,7 @@ protected:
}
bool useDefaultImplementationForNulls() const final { return function->useDefaultImplementationForNulls(); }
bool useDefaultImplementationForConstants() const final { return function->useDefaultImplementationForConstants(); }
bool useDefaultImplementationForColumnsWithDictionary() const final { return function->useDefaultImplementationForColumnsWithDictionary(); }
bool useDefaultImplementationForLowCardinalityColumns() const final { return function->useDefaultImplementationForLowCardinalityColumns(); }
ColumnNumbers getArgumentsThatAreAlwaysConstant() const final { return function->getArgumentsThatAreAlwaysConstant(); }
bool canBeExecutedOnDefaultArguments() const override { return function->canBeExecutedOnDefaultArguments(); }
@ -477,7 +477,7 @@ protected:
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override { return function->getReturnTypeImpl(arguments); }
bool useDefaultImplementationForNulls() const override { return function->useDefaultImplementationForNulls(); }
bool useDefaultImplementationForColumnsWithDictionary() const override { return function->useDefaultImplementationForColumnsWithDictionary(); }
bool useDefaultImplementationForLowCardinalityColumns() const override { return function->useDefaultImplementationForLowCardinalityColumns(); }
bool canBeExecutedOnLowCardinalityDictionary() const override { return function->canBeExecutedOnLowCardinalityDictionary(); }
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & return_type) const override

View File

@ -1,9 +1,9 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
#include <Common/typeid_cast.h>
@ -27,13 +27,13 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
auto * type = typeid_cast<const DataTypeWithDictionary *>(arguments[0].get());
auto * type = typeid_cast<const DataTypeLowCardinality *>(arguments[0].get());
if (!type)
throw Exception("First first argument of function lowCardinalityIndexes must be ColumnWithDictionary, but got"
throw Exception("First first argument of function lowCardinalityIndexes must be ColumnLowCardinality, but got"
+ arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return std::make_shared<DataTypeUInt64>();
@ -44,7 +44,7 @@ public:
auto arg_num = arguments[0];
const auto & arg = block.getByPosition(arg_num);
auto & res = block.getByPosition(result);
auto indexes_col = typeid_cast<const ColumnWithDictionary *>(arg.column.get())->getIndexesPtr();
auto indexes_col = typeid_cast<const ColumnLowCardinality *>(arg.column.get())->getIndexesPtr();
auto new_indexes_col = ColumnUInt64::create(indexes_col->size());
auto & data = new_indexes_col->getData();
for (size_t i = 0; i < data.size(); ++i)

View File

@ -1,7 +1,7 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <Columns/ColumnWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnLowCardinality.h>
#include <Common/typeid_cast.h>
@ -26,13 +26,13 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
auto * type = typeid_cast<const DataTypeWithDictionary *>(arguments[0].get());
auto * type = typeid_cast<const DataTypeLowCardinality *>(arguments[0].get());
if (!type)
throw Exception("First first argument of function lowCardinalityKeys must be ColumnWithDictionary, but got"
throw Exception("First first argument of function lowCardinalityKeys must be ColumnLowCardinality, but got"
+ arguments[0]->getName(), ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
return type->getDictionaryType();
@ -43,8 +43,8 @@ public:
auto arg_num = arguments[0];
const auto & arg = block.getByPosition(arg_num);
auto & res = block.getByPosition(result);
const auto * column_with_dictionary = typeid_cast<const ColumnWithDictionary *>(arg.column.get());
res.column = column_with_dictionary->getDictionary().getNestedColumn()->cloneResized(arg.column->size());
const auto * low_cardinality_column = typeid_cast<const ColumnLowCardinality *>(arg.column.get());
res.column = low_cardinality_column->getDictionary().getNestedColumn()->cloneResized(arg.column->size());
}
};

View File

@ -1,9 +1,9 @@
#include <Functions/IFunction.h>
#include <Functions/FunctionFactory.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
#include <Common/typeid_cast.h>
@ -22,14 +22,14 @@ public:
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForConstants() const override { return true; }
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
DataTypePtr getReturnTypeImpl(const DataTypes & arguments) const override
{
if (arguments[0]->withDictionary())
if (arguments[0]->lowCardinality())
return arguments[0];
return std::make_shared<DataTypeWithDictionary>(arguments[0]);
return std::make_shared<DataTypeLowCardinality>(arguments[0]);
}
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/) override
@ -38,12 +38,12 @@ public:
const auto & arg = block.getByPosition(arg_num);
auto & res = block.getByPosition(result);
if (arg.type->withDictionary())
if (arg.type->lowCardinality())
res.column = arg.column;
else
{
auto column = res.type->createColumn();
typeid_cast<ColumnWithDictionary &>(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size());
typeid_cast<ColumnLowCardinality &>(*column).insertRangeFromFullColumn(*arg.column, 0, arg.column->size());
res.column = std::move(column);
}
}

View File

@ -24,7 +24,7 @@ public:
}
bool useDefaultImplementationForNulls() const override { return false; }
bool useDefaultImplementationForColumnsWithDictionary() const override { return false; }
bool useDefaultImplementationForLowCardinalityColumns() const override { return false; }
size_t getNumberOfArguments() const override
{

View File

@ -17,7 +17,7 @@ namespace ErrorCodes
* (lazy_sources contains not pointers themself, but their delayed constructors)
*
* Firtly, CascadeWriteBuffer redirects data to first buffer of the sequence
* If current WriteBuffer cannot recieve data anymore, it throws special exception CURRENT_WRITE_BUFFER_IS_EXHAUSTED in nextImpl() body,
* If current WriteBuffer cannot receive data anymore, it throws special exception CURRENT_WRITE_BUFFER_IS_EXHAUSTED in nextImpl() body,
* CascadeWriteBuffer prepare next buffer and continuously redirects data to it.
* If there are no buffers anymore CascadeWriteBuffer throws an exception.
*

View File

@ -12,7 +12,7 @@
#include <Columns/IColumn.h>
#include <Columns/ColumnsNumber.h>
#include <Columns/ColumnFixedString.h>
#include <Columns/ColumnWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
template <>

View File

@ -24,8 +24,8 @@
#include <common/demangle.h>
#if __has_include(<Interpreters/config_compile.h>)
#include <Interpreters/config_compile.h>
#include <Columns/ColumnWithDictionary.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
#include <DataTypes/DataTypeLowCardinality.h>
#endif
@ -405,7 +405,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
{
DataTypePtr type = (params.src_header ? params.src_header : params.intermediate_header).safeGetByPosition(pos).type;
if (type->withDictionary())
if (type->lowCardinality())
{
has_low_cardinality = true;
type = removeLowCardinality(type);
@ -748,7 +748,6 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
* To make them work anyway, we materialize them.
*/
Columns materialized_columns;
// ColumnRawPtrs key_counts;
/// Remember the columns we will work with
for (size_t i = 0; i < params.keys_size; ++i)
@ -761,14 +760,13 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
key_columns[i] = materialized_columns.back().get();
}
if (const auto * column_with_dictionary = typeid_cast<const ColumnWithDictionary *>(key_columns[i]))
if (const auto * low_cardinality_column = typeid_cast<const ColumnLowCardinality *>(key_columns[i]))
{
if (!result.isLowCardinality())
{
materialized_columns.push_back(column_with_dictionary->convertToFullColumn());
materialized_columns.push_back(low_cardinality_column->convertToFullColumn());
key_columns[i] = materialized_columns.back().get();
}
//key_counts.push_back(materialized_columns.back().get());
}
}
@ -787,9 +785,9 @@ bool Aggregator::executeOnBlock(const Block & block, AggregatedDataVariants & re
aggregate_columns[i][j] = materialized_columns.back().get();
}
if (auto * col_with_dict = typeid_cast<const ColumnWithDictionary *>(aggregate_columns[i][j]))
if (auto * col_low_cardinality = typeid_cast<const ColumnLowCardinality *>(aggregate_columns[i][j]))
{
materialized_columns.push_back(col_with_dict->convertToFullColumn());
materialized_columns.push_back(col_low_cardinality->convertToFullColumn());
aggregate_columns[i][j] = materialized_columns.back().get();
}
}

View File

@ -28,7 +28,7 @@
#include <Columns/ColumnAggregateFunction.h>
#include <Columns/ColumnVector.h>
#include <Columns/ColumnNullable.h>
#include <Columns/ColumnWithDictionary.h>
#include <Columns/ColumnLowCardinality.h>
namespace DB
@ -413,7 +413,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
void init(ColumnRawPtrs & key_columns, const AggregationStateCachePtr & cache_ptr)
{
auto column = typeid_cast<const ColumnWithDictionary *>(key_columns[0]);
auto column = typeid_cast<const ColumnLowCardinality *>(key_columns[0]);
if (!column)
throw Exception("Invalid aggregation key type for AggregationMethodSingleLowCardinalityColumn method. "
"Excepted LowCardinality, got " + key_columns[0]->getName(), ErrorCodes::LOGICAL_ERROR);
@ -583,7 +583,7 @@ struct AggregationMethodSingleLowCardinalityColumn : public SingleColumnMethod
static void insertKeyIntoColumns(const typename Data::value_type & value, MutableColumns & key_columns, size_t /*keys_size*/, const Sizes & /*key_sizes*/)
{
auto ref = Base::getValueRef(value);
static_cast<ColumnWithDictionary *>(key_columns[0].get())->insertData(ref.data, ref.size);
static_cast<ColumnLowCardinality *>(key_columns[0].get())->insertData(ref.data, ref.size);
}
};
@ -732,7 +732,7 @@ struct AggregationMethodKeysFixed
low_cardinality_keys.position_sizes.resize(key_columns.size());
for (size_t i = 0; i < key_columns.size(); ++i)
{
if (auto * low_cardinality_col = typeid_cast<const ColumnWithDictionary *>(key_columns[i]))
if (auto * low_cardinality_col = typeid_cast<const ColumnLowCardinality *>(key_columns[i]))
{
low_cardinality_keys.nested_columns[i] = low_cardinality_col->getDictionary().getNestedColumn().get();
low_cardinality_keys.positions[i] = &low_cardinality_col->getIndexes();

View File

@ -424,7 +424,7 @@ void DDLWorker::parseQueryAndResolveHost(DDLTask & task)
}
if (!task.query || !(task.query_on_cluster = dynamic_cast<ASTQueryWithOnCluster *>(task.query.get())))
throw Exception("Recieved unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
throw Exception("Received unknown DDL query", ErrorCodes::UNKNOWN_TYPE_OF_QUERY);
task.cluster_name = task.query_on_cluster->cluster;
task.cluster = context.tryGetCluster(task.cluster_name);
@ -536,6 +536,20 @@ bool DDLWorker::tryExecuteQuery(const String & query, const DDLTask & task, Exec
return true;
}
void DDLWorker::attachToThreadGroup()
{
if (thread_group)
{
/// Put all threads to one thread pool
CurrentThread::attachToIfDetached(thread_group);
}
else
{
CurrentThread::initializeQuery();
thread_group = CurrentThread::getGroup();
}
}
void DDLWorker::processTask(DDLTask & task)
{
@ -881,6 +895,8 @@ void DDLWorker::run()
{
try
{
attachToThreadGroup();
processTasks();
LOG_DEBUG(log, "Waiting a watch");

View File

@ -2,6 +2,7 @@
#include <Interpreters/Context.h>
#include <Interpreters/Cluster.h>
#include <DataStreams/BlockIO.h>
#include <Common/CurrentThread.h>
#include <common/logger_useful.h>
#include <atomic>
@ -67,6 +68,8 @@ private:
void run();
void attachToThreadGroup();
private:
Context & context;
Logger * log;
@ -98,6 +101,8 @@ private:
/// How many tasks could be in the queue
size_t max_tasks_in_queue = 1000;
ThreadGroupStatusPtr thread_group;
friend class DDLQueryStatusInputSream;
friend struct DDLTask;
};

View File

@ -89,8 +89,11 @@ public:
ColumnPtr added_column;
/// For APPLY_FUNCTION and LEFT ARRAY JOIN.
/// FunctionBuilder is used before action was added to ExpressionActions (when we don't know types of arguments).
FunctionBuilderPtr function_builder;
/// Can be used after action was added to ExpressionActions if we want to get function signature or properties like monotonicity.
FunctionBasePtr function_base;
/// Prepared function which is used in function execution.
PreparedFunctionPtr function;
Names argument_names;
bool is_function_compiled = false;

View File

@ -69,7 +69,7 @@
#include <Interpreters/evaluateQualified.h>
#include <Interpreters/QueryNormalizer.h>
#include <Interpreters/getQueryAliases.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace DB
@ -1507,7 +1507,7 @@ void ExpressionAnalyzer::makeExplicitSet(const ASTFunction * node, const Block &
set_element_types = left_tuple_type->getElements();
for (auto & element_type : set_element_types)
if (const auto * low_cardinality_type = typeid_cast<const DataTypeWithDictionary *>(element_type.get()))
if (const auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(element_type.get()))
element_type = low_cardinality_type->getDictionaryType();
ASTPtr elements_ast = nullptr;
@ -2450,16 +2450,23 @@ bool ExpressionAnalyzer::appendJoin(ExpressionActionsChain & chain, bool only_ty
return true;
}
bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types, const ASTPtr & sampling_expression)
bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool only_types,
const ASTPtr & sampling_expression, const ASTPtr & primary_expression)
{
assertSelect();
if (!select_query->prewhere_expression)
return false;
Names required_sample_columns;
Names additional_required_mergetree_columns;
if (sampling_expression)
required_sample_columns = ExpressionAnalyzer(sampling_expression, context, storage).getRequiredSourceColumns();
additional_required_mergetree_columns = ExpressionAnalyzer(sampling_expression, context, storage).getRequiredSourceColumns();
if (primary_expression)
{
auto required_primary_columns = ExpressionAnalyzer(primary_expression, context, storage).getRequiredSourceColumns();
additional_required_mergetree_columns.insert(additional_required_mergetree_columns.end(),
required_primary_columns.begin(), required_primary_columns.end());
}
initChain(chain, source_columns);
auto & step = chain.getLastStep();
@ -2476,10 +2483,9 @@ bool ExpressionAnalyzer::appendPrewhere(ExpressionActionsChain & chain, bool onl
auto required_columns = tmp_actions->getRequiredColumns();
NameSet required_source_columns(required_columns.begin(), required_columns.end());
/// Add required columns for sample expression to required output in order not to remove them after
/// prewhere execution because sampling is executed after prewhere.
/// TODO: add sampling execution to common chain.
for (const auto & column : required_sample_columns)
/// Add required columns to required output in order not to remove them after prewhere execution.
/// TODO: add sampling and final execution to common chain.
for (const auto & column : additional_required_mergetree_columns)
{
if (required_source_columns.count(column))
{

View File

@ -142,8 +142,9 @@ public:
bool appendArrayJoin(ExpressionActionsChain & chain, bool only_types);
bool appendJoin(ExpressionActionsChain & chain, bool only_types);
/// remove_filter is set in ExpressionActionsChain::finalize();
/// sampling_expression is needed if sampling is used in order to not remove columns are used in it.
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types, const ASTPtr & sampling_expression);
/// sampling_expression and primary_expression are needed in order to not remove columns are used in it.
bool appendPrewhere(ExpressionActionsChain & chain, bool only_types,
const ASTPtr & sampling_expression, const ASTPtr & primary_expression);
bool appendWhere(ExpressionActionsChain & chain, bool only_types);
bool appendGroupBy(ExpressionActionsChain & chain, bool only_types);
void appendAggregateFunctionsArguments(ExpressionActionsChain & chain, bool only_types);

View File

@ -78,7 +78,9 @@ void ExternalLoader::init(bool throw_on_error)
ExternalLoader::~ExternalLoader()
{
destroy.set();
reloading_thread.join();
/// It can be partially initialized
if (reloading_thread.joinable())
reloading_thread.join();
}

View File

@ -373,7 +373,7 @@ void InterpreterCreateQuery::checkSupportedTypes(const ColumnsDescription & colu
{
for (const auto & column : list)
{
if (!allow_low_cardinality && column.type && column.type->withDictionary())
if (!allow_low_cardinality && column.type && column.type->lowCardinality())
{
String message = "Cannot create table with column '" + column.name + "' which type is '"
+ column.type->getName() + "' because LowCardinality type is not allowed. "

View File

@ -20,7 +20,7 @@
#include <DataStreams/ConcatBlockInputStream.h>
#include <DataStreams/RollupBlockInputStream.h>
#include <DataStreams/CubeBlockInputStream.h>
#include <DataStreams/ConvertColumnWithDictionaryToFullBlockInputStream.h>
#include <DataStreams/ConvertColumnLowCardinalityToFullBlockInputStream.h>
#include <Parsers/ASTSelectQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
@ -202,8 +202,14 @@ InterpreterSelectQuery::InterpreterSelectQuery(
context.addExternalTable(it.first, it.second);
if (query_analyzer->isRewriteSubqueriesPredicate())
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
table_expression, getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1, only_analyze);
{
/// remake interpreter_subquery when PredicateOptimizer is rewrite subqueries and main table is subquery
if (typeid_cast<ASTSelectWithUnionQuery *>(table_expression.get()))
interpreter_subquery = std::make_unique<InterpreterSelectWithUnionQuery>(
table_expression, getSubqueryContext(context), required_columns, QueryProcessingStage::Complete, subquery_depth + 1,
only_analyze);
}
}
if (interpreter_subquery)
@ -308,21 +314,21 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
const ExpressionActionsChain::Step & step = chain.steps.at(0);
res.prewhere_info->remove_prewhere_column = step.can_remove_required_output.at(0);
Names columns_to_remove_after_sampling;
Names columns_to_remove;
for (size_t i = 1; i < step.required_output.size(); ++i)
{
if (step.can_remove_required_output[i])
columns_to_remove_after_sampling.push_back(step.required_output[i]);
columns_to_remove.push_back(step.required_output[i]);
}
if (!columns_to_remove_after_sampling.empty())
if (!columns_to_remove.empty())
{
auto columns = res.prewhere_info->prewhere_actions->getSampleBlock().getNamesAndTypesList();
ExpressionActionsPtr actions = std::make_shared<ExpressionActions>(columns, context);
for (const auto & column : columns_to_remove_after_sampling)
for (const auto & column : columns_to_remove)
actions->add(ExpressionAction::removeColumn(column));
res.prewhere_info->after_sampling_actions = std::move(actions);
res.prewhere_info->remove_columns_actions = std::move(actions);
}
}
if (has_where)
@ -336,8 +342,9 @@ InterpreterSelectQuery::AnalysisResult InterpreterSelectQuery::analyzeExpression
{
ExpressionActionsChain chain(context);
ASTPtr sampling_expression = storage && query.sample_size() ? storage->getSamplingExpression() : nullptr;
if (query_analyzer->appendPrewhere(chain, !res.first_stage, sampling_expression))
ASTPtr sampling_expression = (storage && query.sample_size()) ? storage->getSamplingExpression() : nullptr;
ASTPtr primary_expression = (storage && query.final()) ? storage->getPrimaryExpression() : nullptr;
if (query_analyzer->appendPrewhere(chain, !res.first_stage, sampling_expression, primary_expression))
{
has_prewhere = true;
@ -962,8 +969,7 @@ void InterpreterSelectQuery::executeAggregation(Pipeline & pipeline, const Expre
{
pipeline.transform([&](auto & stream)
{
stream = //std::make_shared<ConvertColumnWithDictionaryToFullBlockInputStream>(
std::make_shared<ExpressionBlockInputStream>(stream, expression); //);
stream = std::make_shared<ExpressionBlockInputStream>(stream, expression);
});
Names key_names;

View File

@ -222,6 +222,8 @@ void PredicateExpressionsOptimizer::cloneOuterPredicateForInnerPredicate(
{
inner_predicate = outer_predicate->clone();
/// clears the alias name contained in the outer predicate
cleanExpressionAlias(inner_predicate);
IdentifiersWithQualifiedNameSet new_expression_requires;
getDependenciesAndQualifiedOfExpression(inner_predicate, new_expression_requires, tables);
@ -419,4 +421,14 @@ std::vector<ASTTableExpression *> PredicateExpressionsOptimizer::getSelectTables
return tables_expression;
}
void PredicateExpressionsOptimizer::cleanExpressionAlias(ASTPtr & expression)
{
const auto my_alias = expression->tryGetAlias();
if (!my_alias.empty())
expression->setAlias("");
for (auto & child : expression->children)
cleanExpressionAlias(child);
}
}

View File

@ -88,6 +88,8 @@ private:
std::vector<ASTTableExpression *> getSelectTablesExpression(ASTSelectQuery * select_query);
ASTs evaluateAsterisk(ASTSelectQuery * select_query, const ASTPtr & asterisk);
void cleanExpressionAlias(ASTPtr & expression);
};
}

View File

@ -6,7 +6,7 @@
namespace ProfileEvents
{
/// Dumps profile events to two column Array(String) and Array(UInt64)
/// Dumps profile events to two columns Array(String) and Array(UInt64)
void dumpToArrayColumns(const Counters & counters, DB::IColumn * column_names, DB::IColumn * column_value, bool nonzero_only = true);
}

View File

@ -27,7 +27,7 @@
#include <Storages/MergeTree/KeyCondition.h>
#include <ext/range.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
namespace DB
@ -131,10 +131,10 @@ void Set::setHeader(const Block & block)
}
/// Convert low cardinality column to full.
if (auto * low_cardinality_type = typeid_cast<const DataTypeWithDictionary *>(data_types.back().get()))
if (auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(data_types.back().get()))
{
data_types.back() = low_cardinality_type->getDictionaryType();
materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfWithDictionary());
materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality());
key_columns.back() = materialized_columns.back().get();
}
}
@ -184,9 +184,9 @@ bool Set::insertFromBlock(const Block & block)
}
/// Convert low cardinality column to full.
if (key_columns.back()->withDictionary())
if (key_columns.back()->lowCardinality())
{
materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfWithDictionary());
materialized_columns.emplace_back(key_columns.back()->convertToFullColumnIfLowCardinality());
key_columns.back() = materialized_columns.back().get();
}
}

View File

@ -275,7 +275,7 @@ struct Settings
M(SettingDateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.") \
M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.") \
M(SettingBool, log_query_settings, true, "Log query settings into the query_log.") \
M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table.") \
M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.") \
M(SettingString, send_logs_level, "none", "Send server text logs with specified minumum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'") \
M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.") \
\
@ -291,6 +291,7 @@ struct Settings
M(SettingUInt64, http_max_multipart_form_data_size, 1024 * 1024 * 1024, "Limit on size of multipart/form-data content. This setting cannot be parsed from URL parameters and should be set in user profile. Note that content is parsed and external tables are created in memory before start of query execution. And this is the only limit that has effect on that stage (limits on max memory usage and max execution time have no effect while reading HTTP form data).") \
M(SettingBool, calculate_text_stack_trace, 1, "Calculate text stack trace in case of exceptions during query execution. This is the default. It requires symbol lookups that may slow down fuzzing tests when huge amount of wrong queries are executed. In normal cases you should not disable this option.") \
M(SettingBool, allow_ddl, true, "If it is set to true, then a user is allowed to executed DDL queries.") \
M(SettingBool, parallel_view_processing, false, "Enables pushing to attached views concurrently instead of sequentially.") \
#define DECLARE(TYPE, NAME, DEFAULT, DESCRIPTION) \
@ -332,7 +333,7 @@ struct Settings
/// Write changed settings to buffer. (For example, to be sent to remote server.)
void serialize(WriteBuffer & buf) const;
/// Dumps profile events to two column Array(String) and Array(UInt64)
/// Dumps profile events to two columns of type Array(String)
void dumpToArrayColumns(IColumn * column_names, IColumn * column_values, bool changed_only = true);
};

View File

@ -105,10 +105,13 @@ void ThreadStatus::finalizePerformanceCounters()
try
{
bool log_to_query_thread_log = global_context && query_context && query_context->getSettingsRef().log_query_threads.value != 0;
if (log_to_query_thread_log)
if (auto thread_log = global_context->getQueryThreadLog())
logToQueryThreadLog(*thread_log);
if (global_context && query_context)
{
auto & settings = query_context->getSettingsRef();
if (settings.log_queries && settings.log_query_threads)
if (auto thread_log = global_context->getQueryThreadLog())
logToQueryThreadLog(*thread_log);
}
}
catch (...)
{

View File

@ -19,7 +19,9 @@
#include <Common/typeid_cast.h>
#include <Common/NaNUtils.h>
#include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeWithDictionary.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <common/DateLUT.h>
namespace DB
@ -138,63 +140,69 @@ UInt64 stringToDateTime(const String & s)
return UInt64(date_time);
}
Field convertFieldToTypeImpl(const Field & src, const IDataType & type)
Field convertFieldToTypeImpl(const Field & src, const IDataType & type, const IDataType * from_type_hint)
{
if (type.isValueRepresentedByNumber())
WhichDataType which_type(type);
WhichDataType which_from_type;
if (from_type_hint)
which_from_type = WhichDataType(*from_type_hint);
/// Conversion between Date and DateTime and vice versa.
if (which_type.isDate() && which_from_type.isDateTime())
{
if (typeid_cast<const DataTypeUInt8 *>(&type)) return convertNumericType<UInt8>(src, type);
if (typeid_cast<const DataTypeUInt16 *>(&type)) return convertNumericType<UInt16>(src, type);
if (typeid_cast<const DataTypeUInt32 *>(&type)) return convertNumericType<UInt32>(src, type);
if (typeid_cast<const DataTypeUInt64 *>(&type)) return convertNumericType<UInt64>(src, type);
if (typeid_cast<const DataTypeInt8 *>(&type)) return convertNumericType<Int8>(src, type);
if (typeid_cast<const DataTypeInt16 *>(&type)) return convertNumericType<Int16>(src, type);
if (typeid_cast<const DataTypeInt32 *>(&type)) return convertNumericType<Int32>(src, type);
if (typeid_cast<const DataTypeInt64 *>(&type)) return convertNumericType<Int64>(src, type);
if (typeid_cast<const DataTypeFloat32 *>(&type)) return convertNumericType<Float32>(src, type);
if (typeid_cast<const DataTypeFloat64 *>(&type)) return convertNumericType<Float64>(src, type);
return UInt64(static_cast<const DataTypeDateTime &>(*from_type_hint).getTimeZone().toDayNum(src.get<UInt64>()));
}
else if (which_type.isDateTime() && which_from_type.isDate())
{
return UInt64(static_cast<const DataTypeDateTime &>(type).getTimeZone().fromDayNum(DayNum(src.get<UInt64>())));
}
else if (type.isValueRepresentedByNumber())
{
if (which_type.isUInt8()) return convertNumericType<UInt8>(src, type);
if (which_type.isUInt16()) return convertNumericType<UInt16>(src, type);
if (which_type.isUInt32()) return convertNumericType<UInt32>(src, type);
if (which_type.isUInt64()) return convertNumericType<UInt64>(src, type);
if (which_type.isInt8()) return convertNumericType<Int8>(src, type);
if (which_type.isInt16()) return convertNumericType<Int16>(src, type);
if (which_type.isInt32()) return convertNumericType<Int32>(src, type);
if (which_type.isInt64()) return convertNumericType<Int64>(src, type);
if (which_type.isFloat32()) return convertNumericType<Float32>(src, type);
if (which_type.isFloat64()) return convertNumericType<Float64>(src, type);
if (auto * ptype = typeid_cast<const DataTypeDecimal<Decimal32> *>(&type)) return convertDecimalType(src, *ptype);
if (auto * ptype = typeid_cast<const DataTypeDecimal<Decimal64> *>(&type)) return convertDecimalType(src, *ptype);
if (auto * ptype = typeid_cast<const DataTypeDecimal<Decimal128> *>(&type)) return convertDecimalType(src, *ptype);
const bool is_date = typeid_cast<const DataTypeDate *>(&type);
bool is_datetime = false;
bool is_enum = false;
bool is_uuid = false;
if (!is_date)
if (!(is_datetime = typeid_cast<const DataTypeDateTime *>(&type)))
if (!(is_uuid = typeid_cast<const DataTypeUUID *>(&type)))
if (!(is_enum = dynamic_cast<const IDataTypeEnum *>(&type)))
throw Exception{"Logical error: unknown numeric type " + type.getName(), ErrorCodes::LOGICAL_ERROR};
if (!which_type.isDateOrDateTime() && !which_type.isUUID() && !which_type.isEnum())
throw Exception{"Logical error: unknown numeric type " + type.getName(), ErrorCodes::LOGICAL_ERROR};
/// Numeric values for Enums should not be used directly in IN section
if (src.getType() == Field::Types::UInt64 && !is_enum)
if (src.getType() == Field::Types::UInt64 && !which_type.isEnum())
return src;
if (src.getType() == Field::Types::String)
{
if (is_date)
if (which_type.isDate())
{
/// Convert 'YYYY-MM-DD' Strings to Date
return UInt64(stringToDate(src.get<const String &>()));
}
else if (is_datetime)
else if (which_type.isDateTime())
{
/// Convert 'YYYY-MM-DD hh:mm:ss' Strings to DateTime
return stringToDateTime(src.get<const String &>());
}
else if (is_uuid)
else if (which_type.isUUID())
{
return stringToUUID(src.get<const String &>());
}
else if (is_enum)
else if (which_type.isEnum())
{
/// Convert String to Enum's value
return dynamic_cast<const IDataTypeEnum &>(type).castToValue(src);
}
}
}
else if (isStringOrFixedString(type))
else if (which_type.isStringOrFixedString())
{
if (src.getType() == Field::Types::String)
return src;
@ -249,12 +257,12 @@ Field convertFieldToType(const Field & from_value, const IDataType & to_type, co
if (from_type_hint && from_type_hint->equals(to_type))
return from_value;
if (auto * with_dict_type = typeid_cast<const DataTypeWithDictionary *>(&to_type))
return convertFieldToType(from_value, *with_dict_type->getDictionaryType(), from_type_hint);
if (auto * low_cardinality_type = typeid_cast<const DataTypeLowCardinality *>(&to_type))
return convertFieldToType(from_value, *low_cardinality_type->getDictionaryType(), from_type_hint);
else if (auto * nullable_type = typeid_cast<const DataTypeNullable *>(&to_type))
return convertFieldToTypeImpl(from_value, *nullable_type->getNestedType());
return convertFieldToTypeImpl(from_value, *nullable_type->getNestedType(), from_type_hint);
else
return convertFieldToTypeImpl(from_value, to_type);
return convertFieldToTypeImpl(from_value, to_type, from_type_hint);
}

View File

@ -346,6 +346,9 @@ public:
/// Returns sampling expression for storage or nullptr if there is no.
virtual ASTPtr getSamplingExpression() const { return nullptr; }
/// Returns primary expression for storage or nullptr if there is no.
virtual ASTPtr getPrimaryExpression() const { return nullptr; }
using ITableDeclaration::ITableDeclaration;
using std::enable_shared_from_this<IStorage>::shared_from_this;

View File

@ -2,7 +2,6 @@
#include <Common/config_version.h>
#if USE_RDKAFKA
#include <thread>
#include <boost/algorithm/string/replace.hpp>
#include <boost/algorithm/string/split.hpp>
#include <boost/algorithm/string/trim.hpp>
@ -25,7 +24,7 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTCreateQuery.h>
#include <Storages/Kafka/KafkaSettings.h>
#include <Storages/Kafka/StorageKafka.h> // Y_IGNORE
#include <Storages/Kafka/StorageKafka.h>
#include <Storages/StorageFactory.h>
#include <IO/ReadBuffer.h>
#include <common/logger_useful.h>
@ -33,7 +32,7 @@
#if __has_include(<rdkafka.h>) // maybe bundled
#include <rdkafka.h> // Y_IGNORE
#else // system
#include <librdkafka/rdkafka.h> // Y_IGNORE
#include <librdkafka/rdkafka.h>
#endif
@ -49,6 +48,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int TIMEOUT_EXCEEDED;
}
using namespace Poco::Util;
@ -132,7 +132,8 @@ public:
: ReadBuffer(nullptr, 0), consumer(consumer_), current(nullptr),
current_pending(false), log(log_), read_messages(0), row_delimiter(row_delimiter_)
{
LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
if (row_delimiter != '\0')
LOG_TRACE(log, "Row delimiter is: " << row_delimiter);
}
~ReadBufferFromKafkaConsumer() override { reset(); }
@ -156,23 +157,20 @@ class KafkaBlockInputStream : public IProfilingBlockInputStream
{
public:
KafkaBlockInputStream(StorageKafka & storage_, StorageKafka::ConsumerPtr consumer_, const Context & context_, const String & schema, size_t max_block_size)
: storage(storage_), consumer(consumer_)
KafkaBlockInputStream(StorageKafka & storage_, const Context & context_, const String & schema, size_t max_block_size_)
: storage(storage_), consumer(nullptr), context(context_), max_block_size(max_block_size_)
{
// Always skip unknown fields regardless of the context (JSON or TSKV)
Context context = context_;
context.setSetting("input_format_skip_unknown_fields", UInt64(1));
if (schema.size() > 0)
context.setSetting("format_schema", schema);
// Create a formatted reader on Kafka messages
LOG_TRACE(storage.log, "Creating formatted reader");
read_buf = std::make_unique<ReadBufferFromKafkaConsumer>(consumer->stream, storage.log, storage.row_delimiter);
reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size);
}
~KafkaBlockInputStream() override
{
if (!hasClaimed())
return;
// An error was thrown during the stream or it did not finish successfully
// The read offsets weren't comitted, so consumer must rejoin the group from the original starting point
if (!finalized)
@ -184,6 +182,7 @@ public:
// Return consumer for another reader
storage.pushConsumer(consumer);
consumer = nullptr;
}
String getName() const override
@ -193,16 +192,28 @@ public:
Block readImpl() override
{
if (isCancelledOrThrowIfKilled())
if (isCancelledOrThrowIfKilled() || !hasClaimed())
return {};
return reader->read();
}
Block getHeader() const override { return reader->getHeader(); }
Block getHeader() const override { return storage.getSampleBlock(); }
void readPrefixImpl() override
{
if (!hasClaimed())
{
// Create a formatted reader on Kafka messages
LOG_TRACE(storage.log, "Creating formatted reader");
consumer = storage.tryClaimConsumer(context.getSettingsRef().queue_max_wait_ms.totalMilliseconds());
if (consumer == nullptr)
throw Exception("Failed to claim consumer: ", ErrorCodes::TIMEOUT_EXCEEDED);
read_buf = std::make_unique<ReadBufferFromKafkaConsumer>(consumer->stream, storage.log, storage.row_delimiter);
reader = FormatFactory::instance().getInput(storage.format_name, *read_buf, storage.getSampleBlock(), context, max_block_size);
}
// Start reading data
finalized = false;
reader->readPrefix();
@ -210,10 +221,12 @@ public:
void readSuffixImpl() override
{
reader->readSuffix();
// Store offsets read in this stream
read_buf->commit();
if (hasClaimed())
{
reader->readSuffix();
// Store offsets read in this stream
read_buf->commit();
}
// Mark as successfully finished
finalized = true;
@ -222,10 +235,15 @@ public:
private:
StorageKafka & storage;
StorageKafka::ConsumerPtr consumer;
Context context;
size_t max_block_size;
Block sample_block;
std::unique_ptr<ReadBufferFromKafkaConsumer> read_buf;
BlockInputStreamPtr reader;
BlockInputStreamPtr reader = nullptr;
bool finalized = false;
// Return true if consumer has been claimed by the stream
bool hasClaimed() { return consumer != nullptr; }
};
static void loadFromConfig(struct rd_kafka_conf_s * conf, const AbstractConfiguration & config, const std::string & path)
@ -260,8 +278,10 @@ StorageKafka::StorageKafka(
row_delimiter(row_delimiter_),
schema_name(context.getMacros()->expand(schema_name_)),
num_consumers(num_consumers_), log(&Logger::get("StorageKafka (" + table_name_ + ")")),
semaphore(0, num_consumers_), mutex(), consumers(), event_update()
semaphore(0, num_consumers_), mutex(), consumers()
{
task = context.getSchedulePool().createTask(log->name(), [this]{ streamThread(); });
task->deactivate();
}
@ -286,12 +306,8 @@ BlockInputStreams StorageKafka::read(
// Claim as many consumers as requested, but don't block
for (size_t i = 0; i < stream_count; ++i)
{
auto consumer = tryClaimConsumer(0);
if (consumer == nullptr)
break;
// Use block size of 1, otherwise LIMIT won't work properly as it will buffer excess messages in the last block
streams.push_back(std::make_shared<KafkaBlockInputStream>(*this, consumer, context, schema_name, 1));
streams.emplace_back(std::make_shared<KafkaBlockInputStream>(*this, context, schema_name, 1));
}
LOG_DEBUG(log, "Starting reading " << streams.size() << " streams, " << max_block_size << " block size");
@ -326,7 +342,7 @@ void StorageKafka::startup()
}
// Start the reader thread
stream_thread = std::thread(&StorageKafka::streamThread, this);
task->activateAndSchedule();
}
@ -334,27 +350,24 @@ void StorageKafka::shutdown()
{
// Interrupt streaming thread
stream_cancelled = true;
event_update.set();
// Unsubscribe from assignments
LOG_TRACE(log, "Unsubscribing from assignments");
// Close all consumers
for (size_t i = 0; i < num_created_consumers; ++i)
{
auto consumer = claimConsumer();
consumer->unsubscribe();
consumer->close();
}
// Wait for stream thread to finish
if (stream_thread.joinable())
stream_thread.join();
LOG_TRACE(log, "Waiting for cleanup");
rd_kafka_wait_destroyed(CLEANUP_TIMEOUT_MS);
task->deactivate();
}
void StorageKafka::updateDependencies()
{
event_update.set();
task->activateAndSchedule();
}
@ -423,49 +436,45 @@ void StorageKafka::pushConsumer(StorageKafka::ConsumerPtr c)
void StorageKafka::streamThread()
{
setThreadName("KafkaStreamThr");
CurrentThread::initializeQuery();
while (!stream_cancelled)
try
{
try
// Keep streaming as long as there are attached views and streaming is not cancelled
while (!stream_cancelled)
{
// Keep streaming as long as there are attached views and streaming is not cancelled
while (!stream_cancelled)
// Check if all dependencies are attached
auto dependencies = context.getDependencies(database_name, table_name);
if (dependencies.size() == 0)
break;
// Check the dependencies are ready?
bool ready = true;
for (const auto & db_tab : dependencies)
{
// Check if all dependencies are attached
auto dependencies = context.getDependencies(database_name, table_name);
if (dependencies.size() == 0)
break;
// Check the dependencies are ready?
bool ready = true;
for (const auto & db_tab : dependencies)
{
if (!context.tryGetTable(db_tab.first, db_tab.second))
ready = false;
}
if (!ready)
break;
LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views");
streamToViews();
LOG_DEBUG(log, "Stopped streaming to views");
if (!context.tryGetTable(db_tab.first, db_tab.second))
ready = false;
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
if (!ready)
break;
// Wait for attached views
event_update.tryWait(READ_POLL_MS);
LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views");
// Reschedule if not limited
if (!streamToViews())
break;
}
}
catch (...)
{
tryLogCurrentException(__PRETTY_FUNCTION__);
}
LOG_DEBUG(log, "Stream thread finished");
// Wait for attached views
if (!stream_cancelled)
task->scheduleAfter(READ_POLL_MS);
}
void StorageKafka::streamToViews()
bool StorageKafka::streamToViews()
{
auto table = context.getTable(database_name, table_name);
if (!table)
@ -486,9 +495,8 @@ void StorageKafka::streamToViews()
streams.reserve(num_consumers);
for (size_t i = 0; i < num_consumers; ++i)
{
auto consumer = claimConsumer();
auto stream = std::make_shared<KafkaBlockInputStream>(*this, consumer, context, schema_name, block_size);
streams.push_back(stream);
auto stream = std::make_shared<KafkaBlockInputStream>(*this, context, schema_name, block_size);
streams.emplace_back(stream);
// Limit read batch to maximum block size to allow DDL
IProfilingBlockInputStream::LocalLimits limits;
@ -498,12 +506,27 @@ void StorageKafka::streamToViews()
p_stream->setLimits(limits);
}
auto in = std::make_shared<UnionBlockInputStream<>>(streams, nullptr, num_consumers);
// Join multiple streams if necessary
BlockInputStreamPtr in;
if (streams.size() > 1)
in = std::make_shared<UnionBlockInputStream<>>(streams, nullptr, num_consumers);
else
in = streams[0];
// Execute the query
InterpreterInsertQuery interpreter{insert, context};
auto block_io = interpreter.execute();
copyData(*in, *block_io.out, &stream_cancelled);
// Check whether the limits were applied during query execution
bool limits_applied = false;
if (IProfilingBlockInputStream * p_stream = dynamic_cast<IProfilingBlockInputStream *>(in.get()))
{
const BlockStreamProfileInfo & info = p_stream->getProfileInfo();
limits_applied = info.hasAppliedLimit();
}
return limits_applied;
}
@ -523,12 +546,7 @@ StorageKafka::Consumer::Consumer(struct rd_kafka_conf_s * conf)
StorageKafka::Consumer::~Consumer()
{
if (stream != nullptr)
{
rd_kafka_consumer_close(stream);
rd_kafka_destroy(stream);
stream = nullptr;
}
close();
}
@ -562,6 +580,15 @@ void StorageKafka::Consumer::unsubscribe()
rd_kafka_unsubscribe(stream);
}
void StorageKafka::Consumer::close()
{
if (stream != nullptr)
{
rd_kafka_consumer_close(stream);
rd_kafka_destroy(stream);
stream = nullptr;
}
}
void registerStorageKafka(StorageFactory & factory)
{

View File

@ -6,6 +6,7 @@
#include <ext/shared_ptr_helper.h>
#include <Core/NamesAndTypes.h>
#include <Core/BackgroundSchedulePool.h>
#include <Storages/IStorage.h>
#include <DataStreams/IBlockOutputStream.h>
#include <Poco/Event.h>
@ -62,6 +63,7 @@ private:
void subscribe(const Names & topics);
void unsubscribe();
void close();
struct rd_kafka_s * stream = nullptr;
};
@ -93,8 +95,7 @@ private:
std::vector<ConsumerPtr> consumers; /// Available consumers
// Stream thread
Poco::Event event_update;
std::thread stream_thread;
BackgroundSchedulePool::TaskHolder task;
std::atomic<bool> stream_cancelled{false};
void consumerConfiguration(struct rd_kafka_conf_s * conf);
@ -103,7 +104,7 @@ private:
void pushConsumer(ConsumerPtr c);
void streamThread();
void streamToViews();
bool streamToViews();
protected:
StorageKafka(

View File

@ -88,7 +88,7 @@ struct MergeTreeDataPartChecksums
/// A kind of MergeTreeDataPartChecksums intended to be stored in ZooKeeper (to save its RAM)
/// MinimalisticDataPartChecksums and MergeTreeDataPartChecksums hasve the same serialization format
/// MinimalisticDataPartChecksums and MergeTreeDataPartChecksums have the same serialization format
/// for versions less than MINIMAL_VERSION_WITH_MINIMALISTIC_CHECKSUMS.
struct MinimalisticDataPartChecksums
{

View File

@ -597,9 +597,9 @@ BlockInputStreams MergeTreeDataSelectExecutor::readFromParts(
stream = std::make_shared<AddingConstColumnBlockInputStream<Float64>>(
stream, std::make_shared<DataTypeFloat64>(), used_sample_factor, "_sample_factor");
if (query_info.prewhere_info && query_info.prewhere_info->after_sampling_actions)
if (query_info.prewhere_info && query_info.prewhere_info->remove_columns_actions)
for (auto & stream : res)
stream = std::make_shared<ExpressionBlockInputStream>(stream, query_info.prewhere_info->after_sampling_actions);
stream = std::make_shared<ExpressionBlockInputStream>(stream, query_info.prewhere_info->remove_columns_actions);
return res;
}

View File

@ -423,14 +423,31 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
/// block.rows() <= read_result.block. We must filter block before adding columns to read_result.block
/// Fill missing columns before filtering because some arrays from Nested may have empty data.
merge_tree_reader->fillMissingColumns(block, should_reorder, should_evaluate_missing_defaults);
merge_tree_reader->fillMissingColumns(block, should_reorder, should_evaluate_missing_defaults, block.rows());
if (read_result.getFilter())
filterBlock(block, read_result.getFilter()->getData());
for (auto i : ext::range(0, block.columns()))
read_result.block.insert(std::move(block.getByPosition(i)));
}
else
{
size_t num_rows = read_result.block.rows();
if (!read_result.block)
{
if (auto * filter = read_result.getFilter())
num_rows = countBytesInFilter(filter->getData()); /// All columns were removed and filter is not always true.
else if (read_result.totalRowsPerGranule())
num_rows = read_result.numReadRows(); /// All columns were removed and filter is always true.
/// else filter is always false.
}
/// If block is empty, we still may need to add missing columns.
/// In that case use number of rows in result block and don't filter block.
if (num_rows)
merge_tree_reader->fillMissingColumns(block, should_reorder, should_evaluate_missing_defaults, num_rows);
}
for (auto i : ext::range(0, block.columns()))
read_result.block.insert(std::move(block.getByPosition(i)));
if (read_result.block)
{
@ -444,7 +461,8 @@ MergeTreeRangeReader::ReadResult MergeTreeRangeReader::read(size_t max_rows, Mar
if (read_result.block)
{
bool should_evaluate_missing_defaults;
merge_tree_reader->fillMissingColumns(read_result.block, should_reorder, should_evaluate_missing_defaults);
merge_tree_reader->fillMissingColumns(read_result.block, should_reorder, should_evaluate_missing_defaults,
read_result.block.rows());
if (should_evaluate_missing_defaults)
merge_tree_reader->evaluateMissingDefaults(read_result.block);

View File

@ -24,6 +24,7 @@ MergeTreeReadPool::MergeTreeReadPool(
column_names{column_names}, do_not_steal_tasks{do_not_steal_tasks},
predict_block_size_bytes{preferred_block_size_bytes > 0}, prewhere_info{prewhere_info}
{
/// parts don't contain duplicate MergeTreeDataPart's.
const auto per_part_sum_marks = fillPerPartInfo(parts, prewhere_info, check_columns);
fillPerThreadInfo(threads, sum_marks, per_part_sum_marks, parts, min_marks_for_concurrent_read);
}
@ -44,6 +45,7 @@ MergeTreeReadTaskPtr MergeTreeReadPool::getTask(const size_t min_marks_to_read,
if (!tasks_remaining_for_this_thread && do_not_steal_tasks)
return nullptr;
/// Steal task if nothing to do and it's not prohibited
const auto thread_idx = tasks_remaining_for_this_thread ? thread : *std::begin(remaining_thread_tasks);
auto & thread_tasks = threads_tasks[thread_idx];

View File

@ -200,7 +200,7 @@ MergeTreeReader::Stream::Stream(
getMark(right).offset_in_compressed_file - getMark(all_mark_ranges[i].begin).offset_in_compressed_file);
}
/// Avoid empty buffer. May happen while reading dictionary for DataTypeWithDictionary.
/// Avoid empty buffer. May happen while reading dictionary for DataTypeLowCardinality.
/// For example: part has single dictionary and all marks point to the same position.
if (max_mark_range == 0)
max_mark_range = max_read_buffer_size;
@ -454,11 +454,8 @@ static bool arrayHasNoElementsRead(const IColumn & column)
}
void MergeTreeReader::fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults)
void MergeTreeReader::fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults, size_t num_rows)
{
if (!res)
throw Exception("Empty block passed to fillMissingColumns", ErrorCodes::LOGICAL_ERROR);
try
{
/// For a missing column of a nested data structure we must create not a column of empty
@ -528,7 +525,7 @@ void MergeTreeReader::fillMissingColumns(Block & res, bool & should_reorder, boo
{
/// We must turn a constant column into a full column because the interpreter could infer that it is constant everywhere
/// but in some blocks (from other parts) it can be a full column.
column_to_add.column = column_to_add.type->createColumnConstWithDefaultValue(res.rows())->convertToFullColumnIfConst();
column_to_add.column = column_to_add.type->createColumnConstWithDefaultValue(num_rows)->convertToFullColumnIfConst();
}
res.insert(std::move(column_to_add));

View File

@ -43,7 +43,8 @@ public:
/// Add columns from ordered_names that are not present in the block.
/// Missing columns are added in the order specified by ordered_names.
/// If at least one column was added, reorders all columns in the block according to ordered_names.
void fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults);
/// num_rows is needed in case block is empty.
void fillMissingColumns(Block & res, bool & should_reorder, bool & should_evaluate_missing_defaults, size_t num_rows);
/// Sort columns to ensure consistent order among all blocks.
/// If filter_name is not nullptr and block has filter column, move it to the end of block.
void reorderColumns(Block & res, const Names & ordered_names, const String * filter_name);
@ -96,6 +97,7 @@ private:
ValueSizeMap avg_value_size_hints;
/// Stores states for IDataType::deserializeBinaryBulk
DeserializeBinaryBulkStateMap deserialize_binary_bulk_state_map;
/// Path to the directory containing the part
String path;
MergeTreeData::DataPartPtr data_part;

View File

@ -65,6 +65,8 @@ bool MergeTreeThreadBlockInputStream::getNewTask()
}
const std::string path = task->data_part->getFullPath();
size_t current_task_first_mark = task->mark_ranges[0].begin;
size_t current_task_end_mark = task->mark_ranges.back().end;
/// Allows pool to reduce number of threads in case of too slow reads.
auto profile_callback = [this](ReadBufferFromFileBase::ProfileInfo info) { pool->profileFeedback(info); };
@ -80,6 +82,7 @@ bool MergeTreeThreadBlockInputStream::getNewTask()
path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache,
storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size, MergeTreeReader::ValueSizeMap{}, profile_callback);
if (prewhere_info)
pre_reader = std::make_unique<MergeTreeReader>(
path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache,
@ -88,18 +91,24 @@ bool MergeTreeThreadBlockInputStream::getNewTask()
}
else
{
/// retain avg_value_size_hints
reader = std::make_unique<MergeTreeReader>(
path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache,
storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size,
reader->getAvgValueSizeHints(), profile_callback);
/// in other case we can reuse readers, they stopped exactly at required position
if (last_task_end_mark != current_task_first_mark || path != last_readed_part_path)
{
/// retain avg_value_size_hints
reader = std::make_unique<MergeTreeReader>(
path, task->data_part, task->columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache,
storage, task->mark_ranges, min_bytes_to_use_direct_io, max_read_buffer_size,
reader->getAvgValueSizeHints(), profile_callback);
if (prewhere_info)
pre_reader = std::make_unique<MergeTreeReader>(
path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache,
storage, task->mark_ranges, min_bytes_to_use_direct_io,
max_read_buffer_size, pre_reader->getAvgValueSizeHints(), profile_callback);
if (prewhere_info)
pre_reader = std::make_unique<MergeTreeReader>(
path, task->data_part, task->pre_columns, owned_uncompressed_cache.get(), owned_mark_cache.get(), save_marks_in_cache,
storage, task->mark_ranges, min_bytes_to_use_direct_io,
max_read_buffer_size, pre_reader->getAvgValueSizeHints(), profile_callback);
}
}
last_readed_part_path = path;
last_task_end_mark = current_task_end_mark;
return true;
}

View File

@ -44,6 +44,10 @@ private:
std::shared_ptr<MergeTreeReadPool> pool;
size_t min_marks_to_read;
/// Last readed mark in task for this thread
size_t last_task_end_mark;
/// Last part readed in this thread
std::string last_readed_part_path;
/// Names from header. Used in order to order columns in read blocks.
Names ordered_names;
};

View File

@ -25,8 +25,8 @@ struct PrewhereInfo
ExpressionActionsPtr alias_actions;
/// Actions which are executed on block in order to get filter column for prewhere step.
ExpressionActionsPtr prewhere_actions;
/// Actions which are executed after sampling in order to remove unused columns.
ExpressionActionsPtr after_sampling_actions;
/// Actions which are executed after reading from storage in order to remove unused columns.
ExpressionActionsPtr remove_columns_actions;
String prewhere_column_name;
bool remove_prewhere_column = false;

View File

@ -96,6 +96,8 @@ public:
ASTPtr getSamplingExpression() const override { return data.sampling_expression; }
ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; }
private:
String path;
String database_name;

View File

@ -207,8 +207,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
: context(context_),
database_name(database_name_),
table_name(name_), full_path(path_ + escapeForFileName(table_name) + '/'),
zookeeper_path(context.getMacros()->expand(zookeeper_path_)),
replica_name(context.getMacros()->expand(replica_name_)),
zookeeper_path(context.getMacros()->expand(zookeeper_path_, database_name, table_name)),
replica_name(context.getMacros()->expand(replica_name_, database_name, table_name)),
data(database_name, table_name,
full_path, columns_,
context_, primary_expr_ast_, secondary_sorting_expr_list_, date_column_name, partition_expr_ast_,

View File

@ -195,6 +195,8 @@ public:
ASTPtr getSamplingExpression() const override { return data.sampling_expression; }
ASTPtr getPrimaryExpression() const override { return data.primary_expr_ast; }
private:
/// Delete old parts from disk and from ZooKeeper.
void clearOldPartsAndRemoveFromZK();

View File

@ -1,2 +1,10 @@
if (CLICKHOUSE_SPLIT_BINARY)
set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse-server CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse-client)
else()
set (TEST_USE_BINARIES CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/clickhouse)
endif()
add_test(NAME integration WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env CLICKHOUSE_TESTS_SERVER_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/server/clickhouse-server CLICKHOUSE_TESTS_CLIENT_BIN_PATH=${ClickHouse_BINARY_DIR}/dbms/programs/server/clickhouse-client "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=\${CLICKHOUSE_TESTS_BASE_CONFIG_DIR:=${ClickHouse_SOURCE_DIR}/dbms/programs/server/}" ${PYTEST_STARTER} pytest ${PYTEST_OPT})
# will mount only one binary to docker container - build with .so cant work
if (MAKE_STATIC_LIBRARIES)
add_test (NAME integration WORKING_DIRECTORY ${CMAKE_CURRENT_SOURCE_DIR} COMMAND env ${TEST_USE_BINARIES} "CLICKHOUSE_TESTS_BASE_CONFIG_DIR=${ClickHouse_SOURCE_DIR}/dbms/programs/server/" ${PYTEST_STARTER} pytest ${PYTEST_OPT})
endif()

View File

@ -30,6 +30,16 @@ def _create_env_file(path, variables, fname=DEFAULT_ENV_NAME):
f.write("=".join([var, value]) + "\n")
return full_path
def subprocess_check_call(args):
# Uncomment for debugging
# print('run:', ' ' . join(args))
subprocess.check_call(args)
def subprocess_call(args):
# Uncomment for debugging
# print('run:', ' ' . join(args))
subprocess.call(args)
class ClickHouseCluster:
"""ClickHouse cluster with several instances and (possibly) ZooKeeper.
@ -45,8 +55,8 @@ class ClickHouseCluster:
self.name = name if name is not None else ''
self.base_configs_dir = base_configs_dir or os.environ.get('CLICKHOUSE_TESTS_BASE_CONFIG_DIR', '/etc/clickhouse-server/')
self.server_bin_path = server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse')
self.client_bin_path = client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client')
self.server_bin_path = p.realpath(server_bin_path or os.environ.get('CLICKHOUSE_TESTS_SERVER_BIN_PATH', '/usr/bin/clickhouse'))
self.client_bin_path = p.realpath(client_bin_path or os.environ.get('CLICKHOUSE_TESTS_CLIENT_BIN_PATH', '/usr/bin/clickhouse-client'))
self.zookeeper_config_path = p.join(self.base_dir, zookeeper_config_path) if zookeeper_config_path else p.join(HELPERS_DIR, 'zookeeper_config.xml')
self.project_name = pwd.getpwuid(os.getuid()).pw_name + p.basename(self.base_dir) + self.name
@ -179,8 +189,8 @@ class ClickHouseCluster:
# Just in case kill unstopped containers from previous launch
try:
if not subprocess.call(['docker-compose', 'kill']):
subprocess.call(['docker-compose', 'down', '--volumes'])
if not subprocess_call(['docker-compose', 'kill']):
subprocess_call(['docker-compose', 'down', '--volumes'])
except:
pass
@ -194,23 +204,20 @@ class ClickHouseCluster:
self.docker_client = docker.from_env(version=self.docker_api_version)
if self.with_zookeeper and self.base_zookeeper_cmd:
subprocess.check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_zookeeper_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
for command in self.pre_zookeeper_commands:
self.run_kazoo_commands_with_retries(command, repeats=5)
self.wait_zookeeper_to_start(120)
if self.with_mysql and self.base_mysql_cmd:
subprocess.check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_mysql_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
self.wait_mysql_to_start(120)
if self.with_kafka and self.base_kafka_cmd:
subprocess.check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_kafka_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
self.kafka_docker_id = self.get_instance_docker_id('kafka1')
# Uncomment for debugging
#print ' '.join(self.base_cmd + ['up', '--no-recreate'])
subprocess.check_call(self.base_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
subprocess_check_call(self.base_cmd + ['up', '-d', '--force-recreate', '--remove-orphans'])
start_deadline = time.time() + 20.0 # seconds
for instance in self.instances.itervalues():
@ -226,8 +233,8 @@ class ClickHouseCluster:
def shutdown(self, kill=True):
if kill:
subprocess.check_call(self.base_cmd + ['kill'])
subprocess.check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
subprocess_check_call(self.base_cmd + ['kill'])
subprocess_check_call(self.base_cmd + ['down', '--volumes', '--remove-orphans'])
self.is_up = False
self.docker_client = None
@ -468,8 +475,12 @@ class ClickHouseInstance:
shutil.copy(p.join(self.base_configs_dir, 'config.xml'), configs_dir)
shutil.copy(p.join(self.base_configs_dir, 'users.xml'), configs_dir)
# used by all utils with any config
conf_d_dir = p.abspath(p.join(configs_dir, 'conf.d'))
# used by server with main config.xml
config_d_dir = p.abspath(p.join(configs_dir, 'config.d'))
users_d_dir = p.abspath(p.join(configs_dir, 'users.d'))
os.mkdir(conf_d_dir)
os.mkdir(config_d_dir)
os.mkdir(users_d_dir)
@ -483,7 +494,7 @@ class ClickHouseInstance:
# Put ZooKeeper config
if self.with_zookeeper:
shutil.copy(self.zookeeper_config_path, config_d_dir)
shutil.copy(self.zookeeper_config_path, conf_d_dir)
# Copy config dir
if self.custom_config_dir:

View File

@ -58,4 +58,4 @@
</shard_0_0>
</remote_servers>
</yandex>
</yandex>

View File

@ -8,4 +8,4 @@
<stderr>/var/log/clickhouse-server/copier/stderr.log</stderr>
<stdout>/var/log/clickhouse-server/copier/stdout.log</stdout>
</logger>
</yandex>
</yandex>

View File

@ -318,6 +318,27 @@ def test_macro(started_cluster):
ddl_check_query(instance, "DROP TABLE IF EXISTS tab ON CLUSTER '{cluster}'")
def test_implicit_macros(started_cluster):
# Temporarily disable random ZK packet drops, they might broke creation if ReplicatedMergeTree replicas
firewall_drops_rules = cluster.pm_random_drops.pop_rules()
instance = cluster.instances['ch2']
ddl_check_query(instance, "DROP DATABASE IF EXISTS test_db ON CLUSTER '{cluster}'")
ddl_check_query(instance, "CREATE DATABASE IF NOT EXISTS test_db ON CLUSTER '{cluster}'")
ddl_check_query(instance, """
CREATE TABLE IF NOT EXISTS test_db.test_macro ON CLUSTER '{cluster}' (p Date, i Int32)
ENGINE = ReplicatedMergeTree('/clickhouse/tables/{database}/{layer}-{shard}/{table}', '{replica}', p, p, 1)
""")
# Check that table was created at correct path in zookeeper
assert cluster.get_kazoo_client('zoo1').exists('/clickhouse/tables/test_db/0-1/test_macro') is not None
# Enable random ZK packet drops
cluster.pm_random_drops.push_rules(firewall_drops_rules)
def test_allowed_databases(started_cluster):
instance = cluster.instances['ch2']
instance.query("CREATE DATABASE IF NOT EXISTS db1 ON CLUSTER cluster")
@ -355,6 +376,14 @@ def test_optimize_query(started_cluster):
ddl_check_query(instance, "CREATE TABLE test_optimize ON CLUSTER cluster (p Date, i Int32) ENGINE = MergeTree(p, p, 8192)")
ddl_check_query(instance, "OPTIMIZE TABLE test_optimize ON CLUSTER cluster FORMAT TSV")
def test_create_as_select(started_cluster):
instance = cluster.instances['ch2']
ddl_check_query(instance, "CREATE TABLE test_as_select ON CLUSTER cluster ENGINE = Memory AS (SELECT 1 AS x UNION ALL SELECT 2 AS x)")
assert TSV(instance.query("SELECT x FROM test_as_select ORDER BY x")) == TSV("1\n2\n")
ddl_check_query(instance, "DROP TABLE IF EXISTS test_as_select ON CLUSTER cluster")
if __name__ == '__main__':
with contextmanager(started_cluster)() as cluster:
for name, instance in cluster.instances.items():

View File

@ -1,8 +1,10 @@
[500]
[500]
[500]
[500]
[0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1000]
[0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1000]
[0,0.50100005,9.51,49.55,99.6,199.7,299.8,399.9,500,600.1,700.2,800.3,900.4,950.45,990.49,999.499,1000]
[0,1,10,50,100,200,300,400,500,600,700,800,900,950,990,999,1000]
1 333334 [699140.3,835642,967430.8] [699999,833333,966666]
2 266667 [426549.5,536255.5,638957.6] [426665,533332,639999]

View File

@ -1,9 +1,11 @@
SELECT quantiles(0.5)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesExact(0.5)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesTDigest(0.5)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesDeterministic(0.5)(x, x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantiles(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesExact(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesTDigest(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT quantilesDeterministic(0, 0.001, 0.01, 0.05, 0.1, 0.2, 0.3, 0.4, 0.5, 0.6, 0.7, 0.8, 0.9, 0.95, 0.99, 0.999, 1)(x, x) FROM (SELECT number AS x FROM system.numbers LIMIT 1001);
SELECT round(1000000 / (number + 1)) AS k, count() AS c, quantilesDeterministic(0.1, 0.5, 0.9)(number, intHash64(number)) AS q1, quantilesExact(0.1, 0.5, 0.9)(number) AS q2 FROM (SELECT number FROM system.numbers LIMIT 1000000) GROUP BY k ORDER BY k;

View File

@ -5,13 +5,13 @@
│ Hello │ 0 │
│ \ │ 0 │
└───────┴───┘
┌─x─────┬─y─┐
│ Hello │ 0 │
│ \ │ 0 │
│ \t │ 0 │
└───────┴───┘
┌─x─────┬─y─┬─toInt8(x)─┬─s─────┬─casted─┐
│ Hello │ 0 │ -100 │ Hello │ Hello │
│ \ │ 0 │ 0 │ \ │ \ │
│ \t │ 0 │ 111 │ \t │ \t
└───────┴───┴───────────┴───────┴────────┘
┌─x────────┬─y─┐
│ Hello │ 0 │
│ \ │ 0 │
│ \t │ 0 │
└──────────┴───┘
┌─x────────┬─y─┬─toInt8(x)─┬─s─────┬─casted─┐
│ Hello │ 0 │ -100 │ Hello │ Hello │
│ \ │ 0 │ 0 │ \ │ \ │
│ \t │ 0 │ 111 │ \t │ \t │
└──────────┴───┴───────────┴───────┴────────┘

View File

@ -8,3 +8,10 @@
2000-01-01 00:01:00 ([4,5,6,7,8],[10,10,20,10,10])
2000-01-01 00:00:00 [1,2,3,4,5] [10,10,20,10,10]
2000-01-01 00:01:00 [4,5,6,7,8] [10,10,20,10,10]
([1],[1])
([1],[1])
(['a'],[1])
(['1970-01-01 03:00:01'],[1])
(['1970-01-02'],[1])
(['01234567-89ab-cdef-0123-456789abcdef'],[1])
([1.01],[1])

View File

@ -1,3 +1,6 @@
SET send_logs_level = 'none';
CREATE DATABASE IF NOT EXISTS test;
DROP TABLE IF EXISTS test.sum_map;
CREATE TABLE test.sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt16, requests UInt64)) ENGINE = Log;
@ -10,3 +13,15 @@ SELECT timeslot, sumMap(statusMap.status, statusMap.requests) FROM test.sum_map
SELECT timeslot, sumMap(statusMap.status, statusMap.requests).1, sumMap(statusMap.status, statusMap.requests).2 FROM test.sum_map GROUP BY timeslot ORDER BY timeslot;
DROP TABLE test.sum_map;
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'UInt64') ] as val, [1] as cnt );
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Float64') ] as val, [1] as cnt );
select sumMap(val, cnt) from ( SELECT [ CAST('a', 'Enum16(\'a\'=1)') ] as val, [1] as cnt );
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'DateTime') ] as val, [1] as cnt );
select sumMap(val, cnt) from ( SELECT [ CAST(1, 'Date') ] as val, [1] as cnt );
select sumMap(val, cnt) from ( SELECT [ CAST('01234567-89ab-cdef-0123-456789abcdef', 'UUID') ] as val, [1] as cnt );
select sumMap(val, cnt) from ( SELECT [ CAST(1.01, 'Decimal(10,2)') ] as val, [1] as cnt );
select sumMap(val, cnt) from ( SELECT [ CAST('a', 'FixedString(1)') ] as val, [1] as cnt ); -- { serverError 43 }
select sumMap(val, cnt) from ( SELECT [ CAST('a', 'String') ] as val, [1] as cnt ); -- { serverError 43 }

View File

@ -3,6 +3,8 @@
1
1
-------Need push down-------
0 0
1
1
1
1

View File

@ -17,6 +17,8 @@ SELECT 1 AS id WHERE id = 1;
SELECT arrayJoin([1,2,3]) AS id WHERE id = 1;
SELECT '-------Need push down-------';
SELECT * FROM system.one ANY LEFT JOIN (SELECT 0 AS dummy) USING dummy WHERE 1;
SELECT toString(value) AS value FROM (SELECT 1 AS value) WHERE value = '1';
SELECT * FROM (SELECT 1 AS id UNION ALL SELECT 2) WHERE id = 1;
SELECT * FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1;
SELECT id FROM (SELECT arrayJoin([1, 2, 3]) AS id) WHERE id = 1;

View File

@ -0,0 +1,13 @@
drop table if exists test.trepl;
create table test.trepl
(
d Date,
a Int32,
b Int32
) engine = ReplacingMergeTree(d, (a,b), 8192);
insert into test.trepl values ('2018-09-19', 1, 1);
select b from test.trepl FINAL prewhere a < 1000;
drop table test.trepl;

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